SQE — Sovereign Query Engine
SQE is a Rust-based distributed SQL query engine for Apache Iceberg tables. It replaces a patched Trino fork with a purpose-built engine based on Apache DataFusion and iceberg-rust.
graph LR
Client["JDBC / Flight SQL Client"] --> Coordinator
Coordinator --> Worker1["Worker 1"]
Coordinator --> Worker2["Worker 2"]
Coordinator --> WorkerN["Worker N"]
Worker1 --> S3["S3 / MinIO"]
Worker2 --> S3
WorkerN --> S3
Coordinator --> Polaris["Polaris Catalog"]
Coordinator --> Keycloak["Keycloak OIDC"]
Key Properties
- No service account — every query runs as the authenticated user. Bearer tokens pass through from client to Polaris catalog and S3 storage.
- Arrow-native — columnar data flows from Parquet files through the entire query pipeline to the client. No row-based serialization anywhere.
- Iceberg-native — built on iceberg-rust, not a connector bolted onto a generic engine. Partition pruning, metadata caching, and Iceberg v3 support are first-class.
- Fine-grained security — row filters and column masks enforced at the logical plan level, before the optimizer runs. Invisible columns, transparent row filtering, no information leakage.
- Rust performance — single binary, no JVM, no GC pauses, predictable memory usage, fast startup.
Quick Start (embedded, no server)
cargo install --path crates/sqe-cli
sqe-cli --embedded # ~/.sqe/warehouse persistent Iceberg catalog
sqe> SELECT * FROM '/data/sales.parquet' LIMIT 5;
sqe> SELECT * FROM read_csv('s3://bucket/orders.tsv.gz');
sqe> SELECT * FROM 'hf://datasets/squad/plain_text/train-00000-of-00001.parquet';
sqe> SELECT * FROM read_delta('/data/delta/sales', version => '5');
Full embedded reference: cli-embedded.md. DuckDB comparison: duckdb-comparision.md.
Quick Start (cluster mode)
# Build
cargo build --release --bin sqe-coordinator --bin sqe-cli
# Start coordinator
SQE_CONFIG=sqe.toml ./target/release/sqe-coordinator
# Connect
./target/release/sqe-cli --host localhost --port 50051
Project Status
SQE is production-ready against Apache Iceberg. The cluster mode runs distributed (coordinator + stateless workers) with OIDC bearer-token passthrough, Polaris / Nessie / Glue / HMS / S3 Tables / JDBC / Hadoop catalogs, and 167/189 (88.4%) on the public Iceberg matrix scoreboard. The embedded mode (V8 through V12.1) adds DuckDB-style file-format TVFs (read_csv, read_json, read_delta), HuggingFace hf:// URLs, and a single-binary CLI for laptop analytics.
Why SQE
The Problem
Our data platform runs on Apache Iceberg tables stored in S3, cataloged by Apache Polaris (Iceberg REST Catalog), with authentication through Keycloak OIDC. We need a SQL query engine that can:
- Authenticate users through Keycloak
- Pass the user’s bearer token through to Polaris and S3 (no service account)
- Enforce fine-grained security (row filters, column masks) per user
- Support full SQL for analytics, dbt transformations, and ad-hoc queries
- Run on Kubernetes with minimal operational overhead
Why Not Trino?
We started with Trino — the industry-standard SQL engine for data lakehouses. It works, but:
| Challenge | Detail |
|---|---|
| Service account model | Trino authenticates to the catalog and storage with a single service identity. It can’t pass per-user bearer tokens through to Polaris. We forked Trino (DCAF branch) to add token passthrough, but maintaining a JVM fork is expensive. |
| Security enforcement | Trino’s security model (system/catalog access control) doesn’t natively support Iceberg-level row filters and column masks applied at the query plan level. |
| JVM overhead | Trino requires significant heap memory, has GC pauses, and takes 10-30 seconds to start. Not ideal for auto-scaling Kubernetes pods. |
| Maintenance burden | Our Trino fork drifts from upstream with every release. Rebasing is a multi-week effort. Security patches are delayed. |
Why DataFusion + Rust?
graph TB
subgraph "Old: Trino Fork"
T[Trino JVM] -->|service account| P1[Polaris]
T -->|service account| S1[S3]
T -.- Fork[DCAF Fork<br/>maintenance burden]
end
subgraph "New: SQE"
C[SQE Coordinator<br/>Rust binary] -->|user bearer token| P2[Polaris]
W[SQE Workers] -->|user credentials| S2[S3]
C --> W
end
style Fork fill:#f96,stroke:#333
style C fill:#6f9,stroke:#333
style W fill:#6f9,stroke:#333
Apache DataFusion is a Rust-native query engine that gives us:
- Extensible query planning — we can inject security filters into the
LogicalPlanbefore optimization, which is exactly where row filters and column masks need to go - iceberg-rust integration — native Rust Iceberg library, no JNI bridge, no serialization overhead
- Per-query context — each query gets its own
SessionContextwith the user’s bearer token. No shared service account. - Single binary — the coordinator and worker ship as one ~50MB binary. Starts in milliseconds.
- No GC — predictable latency, no stop-the-world pauses during large scans
The Name
Sovereign — because every query runs with the identity and permissions of the user who submitted it. No service account intermediary. No privilege escalation. The user’s token is sovereign.
From Trino to DataFusion
Architecture Comparison
graph LR
subgraph Trino
TC[Coordinator<br/>JVM ~2GB heap] --> TW1[Worker<br/>JVM ~8GB heap]
TC --> TW2[Worker<br/>JVM ~8GB heap]
TC -->|Hive Metastore<br/>protocol| HMS[Hive Metastore<br/>or Polaris]
TW1 -->|service account| TS3[S3]
TW2 -->|service account| TS3
end
subgraph SQE
SC[sqe-server<br/>coordinator<br/>~50MB binary] --> SW1[sqe-server<br/>worker]
SC --> SW2[sqe-server<br/>worker]
SC -->|user bearer token<br/>Iceberg REST| POL[Polaris]
SW1 -->|user credentials| SS3[S3]
SW2 -->|user credentials| SS3
end
What Changes
| Aspect | Trino (DCAF fork) | SQE |
|---|---|---|
| Language | Java 21 | Rust |
| Binary size | ~1.2GB (with plugins) | ~50MB |
| Startup time | 10-30 seconds | < 1 second |
| Memory model | JVM heap + GC | Direct allocation, no GC |
| Catalog protocol | Hive Metastore / Iceberg REST | Iceberg REST (native) |
| Auth to catalog | Service account | User bearer token passthrough |
| Auth to storage | Service account IAM role | User credentials from catalog vending |
| Wire protocol | Trino HTTP (custom) | Arrow Flight SQL (gRPC) |
| Data format in-flight | Row-based JSON pages | Arrow columnar batches |
| Security model | System/catalog access control | LogicalPlan rewriting (row filters, column masks) |
| Query engine | Custom cost-based optimizer | Apache DataFusion |
| Table format | Iceberg connector | iceberg-rust (native) |
| Maintenance | Fork of 2M+ LOC Java project | Purpose-built ~5K LOC Rust |
What Stays the Same
- Apache Iceberg as the table format
- Apache Polaris as the REST catalog
- Keycloak as the identity provider
- S3 as the storage layer
- dbt as the transformation framework (new native adapter instead of Trino adapter)
- JDBC connectivity (via Arrow Flight SQL JDBC driver instead of Trino JDBC)
Migration Path
SQE includes an optional Trino-compatible HTTP endpoint (/v1/statement) that speaks enough of the Trino wire protocol to support existing dashboards and tools during the migration period. This is not a full Trino emulation — it covers SELECT, SHOW, and basic DDL, enough to keep things running while teams migrate to Flight SQL.
timeline
title Migration Timeline
Phase 1 : SQE single-node : Flight SQL : CLI
Phase 2 : Write path : dbt-sqe adapter : Views
Phase 3 : Distributed execution : Workers
Phase 4 : Trino compat layer : Dashboard migration
Phase 5 : Security policies : Row filters : Column masks
Phase 6 : Decommission Trino fork
The Auth Challenge
The core design constraint that drove us to build SQE: no service account.
The Problem with Service Accounts
In a typical data platform, the query engine authenticates to the catalog and storage with a service account — a single identity with broad permissions. The engine then enforces per-user access control internally.
sequenceDiagram
participant User
participant Trino
participant Polaris
participant S3
User->>Trino: Query (user token)
Note over Trino: Validates user token
Trino->>Polaris: List tables (SERVICE ACCOUNT)
Polaris-->>Trino: Table metadata
Trino->>S3: Read Parquet (SERVICE ACCOUNT IAM role)
S3-->>Trino: Data
Trino-->>User: Results
This means:
- Polaris sees one identity for all queries — audit logs show the service account, not the actual user
- S3 access is all-or-nothing — the service account can read everything, security depends entirely on the engine enforcing it correctly
- Credential rotation is a blast-radius event — rotating the service account key affects all users simultaneously
- Compliance gap — auditors want to see that Alice read table X, not that sqe-service-account did
SQE’s Approach: Bearer Token Passthrough
SQE never stores or uses a service account for data access. Instead, the user’s Keycloak bearer token flows through the entire stack:
sequenceDiagram
participant User
participant SQE as SQE Coordinator
participant KC as Keycloak
participant Polaris
participant S3
User->>SQE: Handshake (username, password)
SQE->>KC: OIDC Password Grant
KC-->>SQE: Access token + refresh token
SQE-->>User: Session (bearer token)
User->>SQE: Query (bearer token)
SQE->>Polaris: List tables (USER's bearer token)
Polaris-->>SQE: Table metadata + S3 credentials
Note over Polaris: Polaris vends scoped S3<br/>credentials for THIS user
SQE->>S3: Read Parquet (user-scoped credentials)
S3-->>SQE: Data
SQE-->>User: Arrow Flight results
Key Implications
| Property | Service Account Model | SQE Token Passthrough |
|---|---|---|
| Polaris audit trail | Service account | Actual user |
| S3 access scope | Everything | User-scoped (credential vending) |
| Credential rotation | Blast radius: all users | Per-user: transparent refresh |
| Security enforcement | Engine-internal only | Catalog + storage + engine |
| Compliance | Requires mapping logs | Native user identity |
Per-Session Catalog
Each user session gets its own SessionCatalog instance, initialized with the user’s bearer token:
graph TB
subgraph "Session: alice"
SC1[SessionCatalog<br/>token: alice_jwt] --> P[Polaris REST]
end
subgraph "Session: bob"
SC2[SessionCatalog<br/>token: bob_jwt] --> P
end
P -->|alice's token| S3A[S3: alice sees<br/>tables A, B, C]
P -->|bob's token| S3B[S3: bob sees<br/>tables A, B only]
Polaris enforces catalog-level access control based on the token. If Alice has access to tables A, B, C but Bob only has access to A and B, this is enforced at the catalog level — SQE doesn’t need to duplicate this logic.
Token Lifecycle
SQE manages token refresh transparently. A background task checks all active sessions every 10 seconds and refreshes tokens that are about to expire:
stateDiagram-v2
[*] --> Active: Handshake (ROPC grant)
Active --> Refreshing: Token expiry - 60s buffer
Refreshing --> Active: New token from Keycloak
Refreshing --> Expired: Refresh fails
Active --> Expired: Session timeout
Expired --> [*]: Session removed
note right of Active: Queries use current token
note right of Refreshing: Background task (10s interval)
The token fingerprint (last 8 characters of the access token) is used to invalidate iceberg-rust’s internal catalog session cache when a token is refreshed, ensuring the catalog client always uses the current token.
System Overview
Components
graph TB
subgraph Clients
JDBC["JDBC / ODBC<br/>(Flight SQL driver)"]
CLI["sqe-cli"]
DBT["dbt-sqe adapter"]
DASH["Dashboards<br/>(Trino compat)"]
end
subgraph "SQE Cluster"
subgraph "Coordinator (sqe-server --mode coordinator)"
FLS["Flight SQL Server<br/>:50051"]
TH["Trino HTTP<br/>:8080"]
SM["Session Manager"]
QH["Query Handler"]
PE["Policy Enforcer"]
SCHED["Scheduler"]
end
subgraph "Workers (sqe-server --mode worker)"
W1["Worker 1<br/>DataFusion executor"]
W2["Worker 2<br/>DataFusion executor"]
WN["Worker N<br/>DataFusion executor"]
end
end
subgraph "External Services"
KC["OIDC provider<br/>(Keycloak / Auth0 / Entra)"]
CAT["Catalog backend<br/>(Polaris / Nessie / Glue REST /<br/>S3 Tables / Unity / HMS / JDBC)"]
S3["S3-compatible storage<br/>(AWS / Ceph / R2 / rustfs)"]
end
JDBC --> FLS
CLI --> FLS
DBT --> FLS
DASH --> TH
SM --> KC
QH --> PE
QH --> SCHED
SCHED --> W1
SCHED --> W2
SCHED --> WN
QH --> CAT
W1 --> S3
W2 --> S3
WN --> S3
The catalog backend is selectable at runtime. Polaris is the primary target and the only one verified end-to-end for production write paths today. Nessie, AWS Glue, AWS S3 Tables, Unity Catalog OSS, Hive Metastore, JDBC (Postgres), and Hadoop storage-only are all reachable through the same iceberg::Catalog trait, with live integration tests in sqe-catalog/tests/backends_integration.rs. AWS endpoints share the OSS Iceberg REST code path through the aws-sigv4 cargo feature on the vendored iceberg-catalog-rest crate. See features/iceberg.md for the catalog-by-catalog state.
Request Flow
A query flows through SQE in these stages:
sequenceDiagram
participant C as Client
participant F as Flight SQL Server
participant SM as Session Manager
participant QH as Query Handler
participant PE as Policy Enforcer
participant DF as DataFusion
participant CAT as Polaris Catalog
participant S3 as S3 Storage
C->>F: do_handshake(user, pass)
F->>SM: authenticate(user, pass)
SM->>SM: Keycloak OIDC → session
F-->>C: bearer token
C->>F: execute(SQL, token)
F->>SM: get_session(token)
F->>QH: execute(session, SQL)
QH->>QH: parse & classify SQL
QH->>CAT: create SessionCatalog(user_token)
QH->>DF: plan SQL → LogicalPlan
QH->>PE: enforce(user, plan)
PE-->>QH: secured plan (row filters, column masks)
QH->>DF: optimize & execute
DF->>S3: read Parquet files
S3-->>DF: Arrow RecordBatches
DF-->>QH: results
QH-->>F: RecordBatches
F-->>C: Arrow Flight stream
Single-Node vs Distributed
SQE starts in single-node mode by default — the coordinator executes queries locally using DataFusion. No workers needed.
For larger deployments, enable workers:
graph LR
subgraph "Single-Node (default)"
C1[sqe-server] -->|local DataFusion| S1[S3]
end
subgraph "Distributed"
C2[Coordinator] -->|plan fragments| W1[Worker 1]
C2 --> W2[Worker 2]
W1 --> S2[S3]
W2 --> S2
end
| Mode | When to use | Config |
|---|---|---|
| Single-node | Dev, small datasets, < 100GB | sqe-server (default) |
| Distributed | Production, large scans, parallel I/O | worker.enabled=true in Helm |
Ports
| Port | Protocol | Purpose |
|---|---|---|
| 50051 | gRPC (Flight SQL) | Primary query interface |
| 50052 | gRPC (Flight) | Worker data exchange |
| 8080 | HTTP | Trino-compatible endpoint |
| 9090 | HTTP | Prometheus metrics |
| 9091 | HTTP | Health probes (/healthz, /readyz) |
Coordinator
The coordinator is the brain of SQE. It handles SQL parsing, query planning, security enforcement, and result delivery. In single-node mode, it also executes queries directly.
Responsibilities
graph TB
subgraph Coordinator
FLS["Flight SQL Server"] --> SM["Session Manager"]
FLS --> QH["Query Handler"]
QH --> PARSE["SQL Parser<br/>(sqlparser-rs)"]
QH --> CLASS["Statement Classifier"]
QH --> PLAN["Query Planner<br/>(DataFusion)"]
QH --> PE["Policy Enforcer"]
QH --> CAT["Catalog Ops<br/>(DDL)"]
QH --> WH["Write Handler<br/>(CTAS, INSERT,<br/>DELETE, UPDATE,<br/>MERGE — CoW)"]
SM --> AUTH["Authenticator<br/>(Keycloak OIDC)"]
SM --> TC["Token Cache"]
PLAN --> DF["DataFusion<br/>SessionContext"]
DF --> SC["SessionCatalog<br/>(per-user token)"]
end
Statement Routing
The coordinator classifies every SQL statement and routes it to the appropriate handler:
| Statement | Handler | Description |
|---|---|---|
SELECT | execute_query | Plan → policy enforce → execute → stream results |
SHOW CATALOGS | handle_show_catalogs | Returns warehouse name |
SHOW SCHEMAS | handle_show_schemas | Lists namespaces from Polaris |
SHOW TABLES | handle_show_tables | Lists tables in namespace(s) |
CREATE TABLE AS SELECT | handle_ctas | Execute SELECT → write Parquet → commit to Iceberg |
INSERT INTO | handle_insert | Execute SELECT → append Parquet → commit |
CREATE VIEW | handle_create_view | Plan SELECT for schema validation → store in catalog |
DROP TABLE | catalog_ops.drop_table | Forward to Polaris REST |
CREATE SCHEMA | catalog_ops.create_schema | Create namespace in Polaris |
DROP SCHEMA | catalog_ops.drop_schema | Drop namespace from Polaris |
EXPLAIN | handle_explain | Show query plan |
DELETE FROM | handle_delete | CoW: scan affected files, filter, rewrite via rewrite_files() |
UPDATE | handle_update | CoW: scan affected files, apply SET, rewrite via rewrite_files() |
MERGE INTO | handle_merge | CoW: full outer join, classify rows, rewrite via rewrite_files() |
GRANT/REVOKE | Policy (Phase 5) | Not yet implemented |
Session Context
Each query gets a fresh DataFusion SessionContext with the user’s catalog:
sequenceDiagram
participant QH as Query Handler
participant SC as SessionCatalog
participant POL as Polaris
participant DF as DataFusion
QH->>SC: new(catalog_url, warehouse, user_token)
SC->>POL: list_namespaces() [user_token]
POL-->>SC: [ns1, ns2, ns3]
QH->>DF: register_catalog(warehouse, CatalogProvider)
QH->>DF: sql("SELECT * FROM ns1.table1")
DF->>SC: get_table("ns1", "table1")
SC->>POL: load_table("ns1.table1") [user_token]
POL-->>SC: table metadata + S3 location
SC-->>DF: TableProvider (Iceberg scan)
This means two users running the same query may see different tables, schemas, or data — depending on what Polaris grants them.
Worker
Workers are stateless DataFusion executors. They receive plan fragments (scan tasks) from the coordinator, read Parquet files from S3, and stream Arrow results back.
Architecture
graph TB
subgraph Worker["sqe-server --mode worker"]
FS["Flight Service<br/>:50052"]
EX["Executor"]
PR["Parquet Reader"]
end
COORD["Coordinator"] -->|ScanTask ticket| FS
FS --> EX
EX --> PR
PR -->|read| S3["S3 / MinIO"]
PR -->|Arrow RecordBatches| FS
FS -->|Flight stream| COORD
Scan Task
The coordinator sends workers a ScanTask — a lightweight JSON message containing everything the worker needs:
{
"fragment_id": "frag-001",
"data_file_paths": [
"s3://warehouse/ns/table/data/00001.parquet",
"s3://warehouse/ns/table/data/00002.parquet"
],
"projected_columns": ["id", "name", "amount"],
"s3_endpoint": "http://s3:9000",
"s3_region": "us-east-1",
"s3_access_key": "...",
"s3_secret_key": "...",
"s3_path_style": true
}
Workers don’t need access to Polaris or Keycloak. The coordinator resolves table metadata, applies security, and provides the worker with direct S3 credentials and file paths.
Health Checking
The coordinator monitors workers with a background health check task:
stateDiagram-v2
[*] --> Unhealthy: Registered
Unhealthy --> Healthy: Health check OK
Healthy --> Healthy: Health check OK
Healthy --> Degraded: 1-2 consecutive failures
Degraded --> Unhealthy: 3 consecutive failures
Degraded --> Healthy: Health check OK
Unhealthy --> Healthy: Health check OK
- Health checks run every 5 seconds via Flight
Action("health_check") - A worker is marked unhealthy after 3 consecutive failures
- Unhealthy workers are excluded from query scheduling
- Recovery is automatic — a healthy response resets the failure counter
Scaling
Workers are stateless, so scaling is just changing the replica count:
# Helm
helm upgrade sqe deploy/helm/sqe/ --set worker.replicas=5
# kubectl
kubectl scale deployment sqe-worker --replicas=5
The coordinator discovers workers from the config or service discovery. No re-registration needed.
Authentication Flow
SQE supports two OAuth2 flows for initial authentication, then manages token lifecycle transparently:
- OIDC Password Grant (ROPC) – for user-interactive authentication where Flight SQL sends username and password. Works with Keycloak or any OIDC provider that supports the Resource Owner Password Credentials grant.
- OAuth2 Client Credentials – for service-to-service auth, test environments, or OIDC providers that do not support ROPC. Configured by setting
token_endpointdirectly instead ofkeycloak_url.
The mode is selected automatically based on configuration: if keycloak_url is set, SQE uses ROPC; if token_endpoint is set (and keycloak_url is empty), SQE uses client credentials.
Why ROPC?
Flight SQL’s handshake sends username and password directly. There’s no browser redirect flow possible over gRPC. ROPC is the standard mechanism for non-interactive clients (JDBC drivers, CLI tools, dbt adapters).
Client Credentials Mode
When token_endpoint is set (and keycloak_url is empty), SQE uses the client_credentials grant instead of ROPC. In this mode:
- The coordinator obtains a service token using
client_id+client_secretposted directly to the configured token endpoint. - The username from the Flight SQL handshake is informational only – it is used for session labeling and audit logs, but is not sent to the token endpoint.
- There is no
refresh_tokenin client credentials responses. When a token nears expiry, SQE re-fetches a new token via anotherclient_credentialsrequest. - This is the mode used by the lightweight test stack (Polaris built-in OAuth), where Polaris itself acts as the token issuer.
Example Configuration
[auth]
token_endpoint = "http://polaris:8181/api/catalog/v1/oauth/tokens"
client_id = "root"
client_secret = "s3cr3t"
Client Credentials Sequence
sequenceDiagram
participant Client
participant SQE as SQE Coordinator
participant TE as Token Endpoint
participant POL as Polaris
participant S3
Note over Client,TE: Authentication
Client->>SQE: Flight Handshake<br/>Basic auth (user:pass)
SQE->>TE: POST /token<br/>grant_type=client_credentials<br/>client_id, client_secret
TE-->>SQE: access_token, expires_in
SQE->>SQE: Create Session<br/>(username from handshake,<br/>token from endpoint)
SQE-->>Client: Bearer token (session_id)
Note over Client,S3: Query Execution
Client->>SQE: execute(SQL)<br/>Authorization: Bearer session_id
SQE->>SQE: Lookup session → get access_token
SQE->>POL: GET /namespaces<br/>Authorization: Bearer access_token
POL-->>SQE: [namespace list]
SQE->>POL: POST /tables/load<br/>Authorization: Bearer access_token
POL-->>SQE: table metadata + S3 credentials
SQE->>S3: GetObject (vended credentials)
S3-->>SQE: Parquet data
SQE-->>Client: Arrow Flight stream
Note over SQE,TE: Background Token Re-fetch
SQE->>TE: POST /token<br/>grant_type=client_credentials<br/>client_id, client_secret
TE-->>SQE: new access_token, expires_in
SQE->>SQE: Update session token
ROPC Flow
The following sections describe the ROPC (password grant) flow in detail.
Complete Flow
sequenceDiagram
participant Client
participant SQE as SQE Coordinator
participant KC as Keycloak
participant POL as Polaris
participant S3
Note over Client,KC: Authentication
Client->>SQE: Flight Handshake<br/>Basic auth (user:pass)
SQE->>KC: POST /token<br/>grant_type=password<br/>username, password, client_id
KC-->>SQE: access_token, refresh_token, expires_in
SQE->>SQE: Create Session<br/>(id, user, roles, tokens)
SQE-->>Client: Bearer token (session_id)
Note over Client,S3: Query Execution
Client->>SQE: execute(SQL)<br/>Authorization: Bearer session_id
SQE->>SQE: Lookup session → get access_token
SQE->>POL: GET /namespaces<br/>Authorization: Bearer access_token
POL-->>SQE: [namespace list]
SQE->>POL: POST /tables/load<br/>Authorization: Bearer access_token
POL-->>SQE: table metadata + S3 credentials
Note over POL: Polaris vends S3<br/>credentials scoped to<br/>this user + table
SQE->>S3: GetObject (vended credentials)
S3-->>SQE: Parquet data
SQE-->>Client: Arrow Flight stream
Note over SQE,KC: Background Token Refresh
SQE->>KC: POST /token<br/>grant_type=refresh_token
KC-->>SQE: new access_token, new refresh_token
SQE->>SQE: Update session tokens
Token Refresh
A background task runs every 10 seconds, scanning all active sessions:
#![allow(unused)]
fn main() {
// Pseudocode
loop {
sleep(10 seconds);
for session in sessions_expiring_within(60 seconds) {
match keycloak.refresh_token(session.refresh_token) {
Ok(new_tokens) => session.update(new_tokens),
Err(_) => session.mark_expired(),
}
}
}
}
The 60-second buffer ensures tokens are refreshed well before expiry, avoiding mid-query auth failures.
Client Credentials mode: There is no
refresh_tokenin client credentials responses. The background task detects this and re-fetches a fresh token via a newclient_credentialsrequest to the token endpoint when the current token is near expiry. The same 60-second buffer applies.
Token Fingerprinting
When a token is refreshed, the iceberg-rust catalog client’s internal HTTP session cache still holds the old token. SQE uses a token fingerprint (last 8 characters of the access token) as part of the catalog session key. When the fingerprint changes, a new catalog session is created with the fresh token.
graph LR
T1["Token: ...abc12345<br/>fingerprint: abc12345"] -->|refresh| T2["Token: ...xyz98765<br/>fingerprint: xyz98765"]
T1 --> CS1["CatalogSession 1"]
T2 --> CS2["CatalogSession 2<br/>(new, fresh token)"]
Role Extraction
SQE extracts user roles from the JWT realm_access.roles claim. These roles are stored in the session and used for policy evaluation:
{
"realm_access": {
"roles": ["data-analyst", "finance-reader", "admin"]
}
}
Roles flow through to the Policy Enforcer, which uses them to determine row filters and column masks for each query.
Client Credentials mode: Role extraction only applies in OIDC (ROPC) mode, where the JWT contains user-specific claims. In client credentials mode, the token represents the service itself and typically does not carry
realm_access.roles. The session’s role list is empty, and all authorization decisions are delegated to Polaris (which enforces access based on the service principal’s catalog grants).
Security & Policy
SQE enforces fine-grained security through LogicalPlan rewriting — injecting row filters and column masks into the query plan before DataFusion’s optimizer runs.
Status: The policy enforcement framework is designed and stubbed (Phase 5). Currently, a
PassthroughEnforceris active, which returns plans unmodified.
Design Principle
Security enforcement happens at the logical plan level, not at the data level:
graph TB
SQL["SQL: SELECT * FROM sales"] --> PARSE["Parse"]
PARSE --> PLAN["LogicalPlan<br/>Projection → TableScan(sales)"]
PLAN --> POLICY["Policy Enforcer<br/>inject row filter + column mask"]
POLICY --> SECURED["Secured LogicalPlan<br/>Projection → Filter(region='EU') → TableScan(sales)<br/>+ mask(ssn)"]
SECURED --> OPT["DataFusion Optimizer"]
OPT --> EXEC["Execute"]
style POLICY fill:#f96,stroke:#333
This approach means:
- Row filters are transparent — the user doesn’t know they exist
- Column masks block predicate pushdown on raw values — you can’t
WHERE ssn = '123-45-6789'to probe masked data - Denied columns are invisible — they don’t appear in
SELECT *, not as errors - The optimizer can push user predicates through row filters but not through column masks
Policy Enforcer Trait
#![allow(unused)]
fn main() {
#[async_trait]
pub trait PolicyEnforcer: Send + Sync {
async fn evaluate(
&self,
user: &SessionUser,
plan: LogicalPlan,
) -> Result<LogicalPlan>;
}
}
Implementations:
- PassthroughEnforcer — returns plan unchanged (current default)
- OPA Enforcer — queries Open Policy Agent for policies (planned)
- Cedar Enforcer — evaluates AWS Cedar policies locally (planned)
Planned SQL Extensions
-- Grant row filter
GRANT SELECT ON sales TO ROLE analyst
ROWS WHERE region = 'EU';
-- Grant column mask
GRANT SELECT ON customers TO ROLE support
MASKED WITH (ssn AS '***-**-' || RIGHT(ssn, 4));
-- View effective policies
SHOW EFFECTIVE POLICY ON sales FOR ROLE analyst;
-- View grants
SHOW GRANTS ON sales;
No Information Leakage
Following the PostgreSQL RLS model:
| Scenario | Behavior |
|---|---|
| User queries a denied column | Column is invisible in SELECT *, error on explicit reference |
| User queries filtered rows | Rows silently excluded, no indication they exist |
| User applies predicate on masked column | Predicate evaluated on masked value, not raw value |
User runs EXPLAIN | Shows secured plan (filters visible, mask functions visible) |
User runs SHOW TABLES | Only shows tables the user has access to (Polaris enforced) |
Runtime Security Controls
SQE includes several runtime security mechanisms that are active by default or can be enabled via configuration.
Rate Limiting
Throttles query submission to prevent abuse or runaway clients. Uses a token-bucket algorithm (via the governor crate).
[rate_limit]
enabled = true
per_user_queries_per_minute = 60
global_queries_per_minute = 1000
When a limit is exceeded, the client receives a RESOURCE_EXHAUSTED Flight error. Rate limiting is disabled by default.
Query Timeouts
Every query is subject to an execution timeout. If the query exceeds the limit, it is cancelled and the client receives an error.
[query]
timeout_secs = 300 # Default: 5 minutes
[query.role_overrides]
admin = 3600 # Admins get 1 hour
analyst = 600 # Analysts get 10 minutes
Role overrides allow different timeout limits per role. The user’s longest-matching role timeout wins.
Session Lifecycle
Sessions have both idle and absolute timeouts. A background sweeper runs every 60 seconds to clean up expired sessions.
[session]
idle_timeout_secs = 900 # 15 min idle timeout
absolute_timeout_secs = 28800 # 8 hour hard cap
- Idle timeout: sessions with no query activity for this long are expired
- Absolute timeout: sessions older than this are expired regardless of activity
Query Cancellation
SQE supports Arrow Flight’s native cancellation mechanism. When a client cancels a query (or disconnects), the CancellationToken is triggered and propagated to workers, stopping execution promptly.
Error Sanitization
In production mode (debug = false, the default), error messages returned to clients are sanitized:
- Internal details (stack traces, file paths, internal error types) are stripped
- Clients receive a short error message and a request ID for correlation
- Full details are logged server-side for debugging
Enable debug = true during development to see full error details:
[coordinator]
debug = true
TLS Encryption
Flight SQL connections can be encrypted with TLS. Optional mTLS adds client certificate verification.
[coordinator.tls]
cert_file = "/etc/sqe/server.crt"
key_file = "/etc/sqe/server.key"
ca_file = "/etc/sqe/ca.crt" # Optional: mTLS
See Configuration for details.
Streaming Execution
SQE’s streaming execution engine enables 1TB-scale queries on memory-constrained servers. The implementation is split into two phases: Phase A (safe) handles single-node memory management and scan optimization, while Phase B (fast) distributes computation across workers via Arrow Flight DoExchange.
The Problem
A coordinator-centric query engine hits a hard wall: every intermediate result flows through one process. An ORDER BY on 1TB of data requires 1TB of memory (or spill space) on the coordinator, regardless of how many workers scanned the data. A four-way hash join between large tables can exhaust coordinator memory long before the result set is assembled.
The fundamental tension is between sovereignty (run on your own hardware, which may be small) and scale (query datasets that don’t fit in memory). SQE solves this in two stages: first, make the coordinator survive large queries through spill-to-disk and scan optimization (Phase A); then, push computation to workers so the coordinator handles only final aggregation (Phase B).
Phase A: Safe (Single-Node)
Phase A ensures that a single coordinator with limited memory (e.g., 512MB) can execute large analytical queries without OOM kills.
Coordinator Spill-to-Disk
DataFusion’s FairSpillPool divides available memory across all active operators. When an operator (sort, hash aggregate, hash join) exceeds its share, it spills intermediate results to disk as sorted runs.
Key components:
- FairSpillPool – configured via
memory_limitinsqe.toml. Divides memory equally among registeredMemoryConsumerinstances. Triggers spill when any consumer exceeds its fair share. - Watermark system – four levels (green/yellow/orange/red) based on pool utilization percentage. Green (<60%) allows normal execution. Yellow (60-75%) triggers advisory warnings. Orange (75-90%) forces spillable operators to spill. Red (>90%) activates admission control, queueing new queries until memory drops below the orange threshold.
- Admission control – when the pool is in the red zone, new queries wait in a bounded queue rather than competing for memory. This prevents cascade failures where N concurrent queries each grab 1/N of memory and all spill simultaneously.
- External merge sort – when a
SortExecspills, it writes sorted runs tospill_dir(default:/tmp/sqe-spill). On final output, a k-way merge reads all runs simultaneously, producing a globally sorted stream with constant memory overhead.
Configuration in sqe.toml:
[coordinator]
memory_limit = "512MB"
spill_dir = "/tmp/sqe-spill"
spill_compression = "zstd" # lz4, zstd, or none
Late Materialization
Standard Parquet scans read all projected columns from every row group. Late materialization splits this into two phases:
- Predicate phase – read only the columns referenced in
WHEREclauses. Apply filters. Produce a set of surviving row indices. - Projection phase – for surviving rows only, read the remaining projected columns.
This is implemented as a two-phase RowFilter scan in the Iceberg scan planning layer. For queries with selective predicates (e.g., WHERE status = 'CLOSED' on a table where 5% of rows match), late materialization reduces I/O by up to 95% on the non-predicate columns.
The optimization is transparent to the rest of the plan – the TableScan still produces the same Arrow schema. The difference is entirely in how many bytes are read from Parquet.
Iceberg Scan Planning
Three optimizations happen before any Parquet data is read:
- File-level min/max pruning – Iceberg manifest files contain per-column min/max statistics for each data file. SQE reads these statistics and skips files where the predicate cannot match. For example,
WHERE order_date > '2025-01-01'skips any file whoseorder_datemax is before 2025. - Sort-order detection – Iceberg metadata records the sort order of each data file. When a query includes
ORDER BYon the sort column, SQE can skip the sort operator entirely and produce output directly from the pre-sorted scan. When multiple sorted files need merging, a merge-sort is cheaper than a full re-sort. - PageIndex pruning – for Parquet files with page-level statistics (column index), SQE prunes individual pages within a row group, further reducing I/O for selective predicates.
- TopK optimization –
ORDER BY ... LIMIT Nqueries use a heap-based TopK operator that maintains only N rows in memory, avoiding a full sort and spill.
S3 I/O Pipeline
Reading Parquet files from S3 involves many small HTTP GET requests (one per column chunk per row group). SQE optimizes this with:
- Request coalescing – adjacent byte ranges within
coalesce_threshold(default: 1MB) are merged into a single GET request. This reduces the number of HTTP round-trips, which dominate latency on high-latency S3 endpoints. - Footer cache – Parquet file footers (schema, row group metadata, column chunk offsets) are cached in a
footer_cache_size-bounded LRU cache. Repeated queries against the same table skip the footer read entirely. - Prefetch – while the executor processes the current row group, the next row group’s column chunks are fetched in the background, hiding S3 latency behind compute.
Configuration:
[storage]
coalesce_threshold = "1MB"
footer_cache_size = 256 # number of footers
SortMergeJoin Fallback
DataFusion’s default join strategy is hash join, which builds a hash table from the build side in memory. For large joins, this hash table can exceed the memory limit. DataFusion does not yet support hash join spill-to-disk upstream.
SQE registers a SortMergeJoin fallback: when the estimated build-side size exceeds hash_join_memory_threshold, the optimizer rewrites the join as a sort-merge join. Both sides are sorted (spilling to disk if needed via the external merge sort) and then merged with constant memory. This is slower than an in-memory hash join but avoids OOM on large joins.
[optimizer]
hash_join_memory_threshold = "256MB"
Phase B: Fast (Distributed)
Phase B pushes computation past the scan boundary. Instead of workers sending raw Arrow batches to the coordinator for all processing, workers perform filters, partial aggregations, partial sorts, and join probes locally.
DoExchange Shuffle
Arrow Flight’s DoExchange RPC enables bidirectional streaming between workers. SQE uses this to implement a hash-partitioned shuffle:
- The coordinator decomposes the physical plan into stages separated by shuffle boundaries (e.g., a hash join requires both sides to be hash-partitioned on the join key).
- Each stage runs on a set of workers. When a stage completes, its output is hash-partitioned by the shuffle key and streamed to the next stage’s workers via
DoExchange. - The partitioning function uses the same
hash(key) % num_partitionsscheme as DataFusion’sRepartitionExec, ensuring compatibility with the existing hash join and hash aggregate operators.
Distributed Sort (Range-Partition)
A distributed ORDER BY proceeds in three steps:
- Sample – each worker samples its local partition and sends the sample to the coordinator.
- Range boundaries – the coordinator computes quantile boundaries from the samples, producing N-1 split points for N workers.
- Range-partition and merge – each worker range-partitions its data and sends each range to the designated worker. Each receiving worker sorts its range locally (spilling if needed). The coordinator merges the sorted ranges via a k-way merge.
This distributes both the memory cost and the CPU cost of sorting. A 1TB ORDER BY with 8 workers requires roughly 125GB of spill per worker instead of 1TB on the coordinator.
Two-Phase Aggregation
Aggregation queries (GROUP BY) use a two-phase approach:
- Partial aggregation – each worker computes partial aggregates on its local data. For
SUM(amount) GROUP BY region, each worker produces a partial sum per region from its partition. - Final aggregation – partial results are shuffled by the grouping key to a set of finalizer workers (or the coordinator for small result sets). Each finalizer merges the partial aggregates into the final result.
This solves the q18 problem: TPC-H query 18 has a high-cardinality GROUP BY that produces millions of groups. On a single coordinator with 512MB, the GroupedHashAggregate exceeds memory. With two-phase aggregation, each worker handles a fraction of the groups, and memory pressure is distributed.
Distributed Joins
SQE supports four join strategies in distributed mode:
- Broadcast join – when one side of the join is small (below
broadcast_threshold, default 10MB), it is broadcast to all workers. Each worker probes its local partition of the large side against the broadcast table. No shuffle required. - Shuffle hash join – both sides are hash-partitioned on the join key and shuffled to matching workers. Each worker performs a local hash join on its partition.
- Pre-sorted merge join – when both sides are already sorted on the join key (detected via Iceberg sort-order metadata), workers perform a merge join without re-sorting. This avoids the sort cost entirely.
- Predicate transfer – before executing a join, the build side’s distinct join keys are collected and pushed as an
IN-list filter to the probe side’s scan. This skips probe-side files that contain no matching keys, reducing I/O by 90%+ for selective joins. Based on the predicate transfer technique from Yang et al. (SIGMOD 2025).
[optimizer]
broadcast_threshold = "10MB"
Multi-Endpoint Flight SQL
In Phase A, all results flow through the coordinator’s single Flight SQL endpoint. Phase B adds multi-endpoint support: get_flight_info can return multiple FlightEndpoint entries, each pointing to a different worker. The client fetches results directly from workers, bypassing the coordinator for data transfer.
This eliminates the coordinator NIC bottleneck for large result sets. A query returning 4GB across 4 workers streams 1GB directly from each worker to the client, achieving 4x the effective bandwidth.
Stage Decomposition
The coordinator decomposes the physical plan into stages:
- Scan stage – workers read Parquet files, apply predicates and projections.
- Shuffle stage – workers hash-partition or range-partition output for the next stage.
- Join/Aggregate stage – workers perform local joins or aggregations on shuffled data.
- Final stage – coordinator (or a designated worker) performs final aggregation, sort, or limit.
Each stage boundary is a shuffle point. The coordinator tracks stage completion and triggers the next stage when all workers in the current stage have finished.
Memory Model
SQE uses a four-level watermark system to manage memory pressure:
| Level | Pool Utilization | Behavior |
|---|---|---|
| Green | < 60% | Normal execution, no restrictions |
| Yellow | 60-75% | Advisory: log warnings, increment metrics |
| Orange | 75-90% | Spillable operators forced to spill |
| Red | > 90% | Admission control: new queries queued |
The FairSpillPool divides the total memory_limit equally among all registered MemoryConsumer instances. When a consumer’s try_grow call would push the pool past the orange threshold, the pool asks other spillable consumers to spill first. If spilling frees enough memory, the allocation succeeds. If not, the allocation fails with ResourceExhausted.
Per-operator behavior:
- SortExec – spills sorted runs to disk, later merged via k-way merge.
- HashAggregateExec – spills partition groups to disk (when supported by DataFusion).
- HashJoinExec – not spillable upstream; SQE rewrites to SortMergeJoin when estimated size exceeds threshold.
- SortMergeJoinExec – both sides sort-and-spill independently, then merge with constant memory.
Configuration Reference
| Field | Section | Default | Description |
|---|---|---|---|
memory_limit | [coordinator] / [worker] | 8GB | Maximum memory for the DataFusion runtime |
spill_dir | [coordinator] / [worker] | /tmp/sqe-spill | Directory for spill files |
spill_compression | [coordinator] / [worker] | zstd | Compression for spill files (lz4, zstd, none) |
hash_join_memory_threshold | [optimizer] | 256MB | Build-side size above which hash join is rewritten to sort-merge join |
broadcast_threshold | [optimizer] | 10MB | Join side size below which broadcast join is used |
coalesce_threshold | [storage] | 1MB | Maximum gap between byte ranges to coalesce into one S3 GET |
footer_cache_size | [storage] | 256 | Number of Parquet footers to cache |
Benchmark Results
TPC-H at scale factor 1 (approximately 1GB of data) on a coordinator with 512MB memory and spill-to-disk enabled:
- 21 of 22 queries pass. All queries produce correct results within the memory budget.
- 1 failure: q18. TPC-H query 18 uses a high-cardinality
GROUP BYwithHAVINGthat produces millions of intermediate groups. DataFusion’sGroupedHashAggregatedoes not yet support spill-to-disk for hash aggregation, so the operator exceeds the 512MB limit. This is a known upstream limitation. With Phase B’s two-phase aggregation (distributing the groups across workers), q18 passes.
These results demonstrate that SQE can run analytical workloads on hardware that would be considered undersized for traditional query engines. The combination of spill-to-disk, late materialization, and scan planning keeps memory usage bounded regardless of data size.
Research Papers
The streaming execution engine draws on decades of database systems research. This chapter lists the papers that most directly influenced SQE’s design, with notes on how each idea is implemented.
Papers and Their Influence
| Paper | Venue | How It Influenced SQE |
|---|---|---|
| Graefe, “Volcano – An Extensible and Parallel Query Evaluation System” | IEEE TKDE 1994 | Exchange operator model – foundation for DoExchange shuffle |
| Shapiro, “Join Processing in Database Systems with Large Main Memories” | VLDB 1986 | Partition-based spill – why SortMergeJoin is the safe fallback until hash join spill lands |
| Sethi et al., “Presto: SQL on Everything” | ICDE 2019 | Coordinator/worker architecture SQE mirrors; no-spill-then-spill evolution |
| Leis et al., “Morsel-Driven Parallelism: A NUMA-Aware Query Evaluation Framework” | SIGMOD 2014 | NUMA-aware execution; informs batch sizing and DataFusion’s pull model |
| Pedreira et al., “Velox: Meta’s Unified Execution Engine” | VLDB 2022 | Cooperative memory arbitration – future direction for SQE |
| Raasveldt & Muhleisen, “DuckDB: An Embeddable Analytical Database” | SIGMOD 2019 | Single-node out-of-core proof that 1TB works on 16GB with proper buffer management |
| Pang et al., “Memory-Adaptive External Sorting” | VLDB 1993 | Dynamic sort splitting – how FairSpillPool divides memory |
| Abadi et al., “Materialization Strategies in a Column-Oriented DBMS” | ICDE 2007 | Late materialization – predicate columns first, projection for survivors |
| Yang et al., “Predicate Transfer: Efficient Pre-Filtering for Joins” | SIGMOD 2025 | Push join keys as IN-list to probe side – skip 90%+ of probe files |
Detailed Notes
Graefe, “Volcano” (1994)
Volcano introduced the exchange operator as the universal mechanism for parallelism in query evaluation. An exchange operator sits between two plan tree segments and handles data redistribution – hash partitioning, round-robin, or broadcast – without the operators above or below knowing about parallelism. SQE’s DoExchange shuffle is a direct implementation of this model over Arrow Flight’s bidirectional streaming RPC. Each shuffle boundary in the stage decomposition corresponds to an exchange operator. The key insight from Volcano that SQE preserves: the operators themselves are single-threaded and unaware of distribution; all parallelism is encapsulated in the exchange.
Shapiro, “Hybrid Hash Join” (1986)
Shapiro’s hybrid hash join partitions the build side into buckets, keeping one bucket in memory and spilling the rest to disk. During the probe phase, the in-memory bucket is probed immediately; the spilled buckets are read back and probed sequentially. This is the standard approach for hash join spill in systems like PostgreSQL and Trino. DataFusion does not yet implement hash join spill upstream, so SQE cannot use this technique directly. Instead, SQE falls back to SortMergeJoin for large joins – both sides are sorted (spilling via external merge sort) and then merged with constant memory. The SortMergeJoin fallback is the safe path; when DataFusion adds hash join spill, SQE can adopt the hybrid approach from Shapiro for better performance on unsorted inputs.
Sethi et al., “Presto: SQL on Everything” (2019)
Presto’s architecture – a stateless coordinator that plans and schedules, stateless workers that execute, no shared storage between them – is the direct model for SQE’s coordinator/worker split. The paper describes Presto’s evolution from a no-spill engine (all intermediate data in memory) to one that supports spill-to-disk under memory pressure. SQE followed the same evolution: Phase A added coordinator spill (the “safe” path), and Phase B added distributed computation (pushing work to workers so the coordinator handles less data). The paper’s observation that “most queries fit in memory; spill is for the tail” matches SQE’s experience – 20 of 22 TPC-H queries run without spill on 512MB; only the largest sorts and aggregations trigger it.
Leis et al., “Morsel-Driven Parallelism” (2014)
The morsel-driven model assigns small, fixed-size chunks of work (morsels) to worker threads, enabling NUMA-aware scheduling without explicit thread pinning. DataFusion’s pull-based execution model, where each operator produces batches on demand, is conceptually similar. SQE’s batch sizing (default 8192 rows per RecordBatch) is informed by this paper’s finding that small, uniform work units lead to better load balancing and cache utilization. The paper also highlights the importance of avoiding global synchronization in the hot path – a principle SQE follows by using per-operator memory consumers and tokio::sync::watch channels for credential refresh rather than shared mutexes.
Pedreira et al., “Velox” (2022)
Velox introduces cooperative memory arbitration: operators register with a central arbitrator and respond to memory pressure by spilling or shrinking their buffers. This is more sophisticated than DataFusion’s FairSpillPool, which divides memory equally and triggers spill when any consumer exceeds its share. Velox’s arbitrator can make global decisions – asking a low-priority operator to spill so a high-priority one can proceed. SQE does not implement priority-based arbitration today, but the FairSpillPool’s watermark system (green/yellow/orange/red) provides a simpler version of the same concept. The Velox paper’s cooperative model is the planned future direction for SQE’s memory management, particularly for mixed workloads where interactive queries should preempt batch jobs.
Raasveldt & Muhleisen, “DuckDB” (2019)
DuckDB proves that a single-node engine with proper buffer management can process datasets far larger than available memory. Its out-of-core hash join and sort implementations use disk-backed buffers that transparently page data in and out. SQE’s Phase A is built on the same principle: spill-to-disk is not an error path, it is the normal execution path for large queries on small machines. The difference is that SQE operates over remote storage (S3) rather than local files, so the I/O pipeline (coalescing, footer cache, prefetch) is more critical. DuckDB’s benchmark results – 1TB queries on 16GB machines – provided the confidence that SQE’s 512MB target was achievable with the right memory management.
Pang et al., “Memory-Adaptive External Sorting” (1993)
This paper addresses the problem of external sorting when available memory fluctuates during execution (due to other concurrent operators). The key technique is dynamic run splitting: instead of committing to a fixed run size at the start of the sort, the algorithm adapts the run size based on currently available memory. SQE’s FairSpillPool implements a version of this: as other operators allocate and release memory, the pool available to a SortExec changes. When memory shrinks (another query starts), the sort produces smaller runs and spills more frequently. When memory grows (another query finishes), the sort can produce larger runs. The k-way merge at the end adapts to whatever set of runs was produced.
Abadi et al., “Materialization Strategies” (2007)
Abadi’s paper compares early materialization (read all columns, then filter) with late materialization (read predicate columns, filter, then read remaining columns for survivors) in column-oriented databases. Late materialization wins when predicates are selective because it avoids reading non-predicate columns for filtered-out rows. SQE implements late materialization in the Iceberg scan layer: the scan planner splits the column set into predicate columns and projection-only columns, reads the predicate columns first, applies the RowFilter, and then reads the projection columns only for rows that survived the filter. For a query like SELECT * FROM orders WHERE status = 'CLOSED' on a table where 5% of rows match, this reduces column-chunk reads by up to 19x (for a 20-column table where status is one column).
Yang et al., “Predicate Transfer” (2025)
Predicate transfer pushes join key values from the build side of a join to the probe side’s scan, filtering probe-side data before it enters the join operator. In the simplest form, the distinct join keys from the build side are collected into an IN-list and injected as a predicate on the probe side’s table scan. SQE implements this for distributed joins: after the build side is scanned and its distinct keys are known, the coordinator pushes the key set to probe-side workers as a scan predicate. Combined with Iceberg’s file-level min/max statistics, this can skip entire data files on the probe side. For selective joins (e.g., a dimension table join where only 100 of 10,000 distinct key values appear), predicate transfer skips 90%+ of probe-side files, dramatically reducing I/O. This is particularly effective for star-schema queries common in analytical workloads.
SQL Support
SQE inherits DataFusion’s comprehensive SQL support and adds Iceberg-specific operations.
Query Language
SELECT & Expressions
-- Full ANSI SQL
SELECT customer_id, SUM(amount) AS total
FROM orders
WHERE order_date >= '2024-01-01'
GROUP BY customer_id
HAVING SUM(amount) > 1000
ORDER BY total DESC
LIMIT 10;
-- CTEs
WITH monthly AS (
SELECT DATE_TRUNC('month', order_date) AS month, SUM(amount) AS total
FROM orders GROUP BY 1
)
SELECT month, total, LAG(total) OVER (ORDER BY month) AS prev_month
FROM monthly;
-- Subqueries, EXISTS, IN
SELECT * FROM customers
WHERE customer_id IN (SELECT customer_id FROM orders WHERE amount > 500);
Window Functions
SELECT
employee_id,
department,
salary,
ROW_NUMBER() OVER (PARTITION BY department ORDER BY salary DESC) AS rank,
AVG(salary) OVER (PARTITION BY department) AS dept_avg,
salary - LAG(salary) OVER (ORDER BY hire_date) AS salary_diff
FROM employees;
Supported: ROW_NUMBER, RANK, DENSE_RANK, NTILE, LAG, LEAD, FIRST_VALUE, LAST_VALUE, NTH_VALUE, CUME_DIST, PERCENT_RANK, with PARTITION BY, ORDER BY, and frame clauses (ROWS BETWEEN, RANGE BETWEEN).
Joins
-- All join types
SELECT * FROM a INNER JOIN b ON a.id = b.id;
SELECT * FROM a LEFT JOIN b ON a.id = b.id;
SELECT * FROM a RIGHT JOIN b ON a.id = b.id;
SELECT * FROM a FULL OUTER JOIN b ON a.id = b.id;
SELECT * FROM a CROSS JOIN b;
-- Anti and semi joins (via EXISTS/NOT EXISTS)
SELECT * FROM a WHERE NOT EXISTS (SELECT 1 FROM b WHERE b.id = a.id);
Set Operations
SELECT id FROM a UNION ALL SELECT id FROM b;
SELECT id FROM a INTERSECT SELECT id FROM b;
SELECT id FROM a EXCEPT SELECT id FROM b;
DDL
-- Schemas
CREATE SCHEMA analytics;
DROP SCHEMA staging;
-- Tables (via CTAS)
CREATE TABLE analytics.summary AS
SELECT region, SUM(revenue) AS total FROM sales GROUP BY region;
-- CREATE OR REPLACE
CREATE OR REPLACE TABLE analytics.summary AS
SELECT region, SUM(revenue) AS total FROM sales GROUP BY region;
-- Views
CREATE VIEW active_customers AS
SELECT * FROM customers WHERE status = 'active';
DROP VIEW active_customers;
-- Drop
DROP TABLE analytics.summary;
DROP TABLE IF EXISTS analytics.summary;
DML
-- Insert from query
INSERT INTO target_table
SELECT * FROM source_table WHERE condition;
-- CTAS
CREATE TABLE new_table AS SELECT * FROM existing_table;
-- DELETE (Copy-on-Write by default)
DELETE FROM orders WHERE status = 'cancelled';
DELETE FROM orders WHERE customer_id IN (SELECT id FROM blacklist);
-- DELETE (Merge-on-Read; opt in via table property)
ALTER TABLE orders SET TBLPROPERTIES ('write.delete.mode' = 'merge-on-read');
DELETE FROM orders WHERE status = 'cancelled'; -- writes a position delete file
-- UPDATE (Copy-on-Write)
UPDATE orders SET status = 'shipped' WHERE tracking_id IS NOT NULL;
UPDATE orders SET amount = CASE WHEN amount > 1000 THEN amount * 0.9 ELSE amount END;
-- MERGE INTO (Copy-on-Write)
MERGE INTO target USING source ON target.id = source.id
WHEN MATCHED THEN UPDATE SET value = source.value
WHEN NOT MATCHED THEN INSERT (id, value) VALUES (source.id, source.value);
All row-level write operations (DELETE, UPDATE, MERGE INTO) default to Copy-on-Write via the RisingWave iceberg-rust fork’s rewrite_files() transaction API. Affected data files are read, filtered/transformed, and rewritten as new files in a single atomic commit.
DELETE also supports Merge-on-Read when write.delete.mode = 'merge-on-read' is set on the table. SQE writes a position-delete file (or an equality-delete file when the table declares an identifier-field-id) and commits via FastAppendAction / RowDeltaAction. MoR avoids rewriting whole data files for small deletes against large tables.
Data Types
SQE accepts the standard ANSI SQL type set plus a few Iceberg-specific extensions:
CREATE TABLE events (
id BIGINT,
payload JSON, -- Aliases to Utf8 underneath
occurred_at TIMESTAMP(6),
occurred_time TIME(6), -- Time-of-day, microseconds
occurred_at_tz TIMESTAMP(6) WITH TIME ZONE,
occurred_ns TIMESTAMP_NS, -- V3-only: nanosecond precision
region_id INTEGER,
amount DECIMAL(18, 2)
);
JSONcolumns store asUtf8.CAST(json_col AS BIGINT|VARCHAR|DOUBLE)rides DataFusion’s built-in coercion. JSON-shaped extraction works throughjson_extract,json_extract_scalar,json_array_length,json_parse,json_get_str,json_get_int,json_get_float,json_get_bool.TIME/TIME(p)maps to ArrowTime64(Microsecond)since Iceberg’stimeprimitive is microsecond-only across V2 and V3. Precisions 0..=6 collapse to microsecond.TIME(p > 6)rejects with a clear NotImplemented; useTIMESTAMP(9)for sub-microsecond resolution.TIME WITH TIME ZONErejects at CREATE TABLE: Arrow has no equivalent. UseTIMESTAMP WITH TIME ZONEinstead.TIMESTAMP_NS(andTIMESTAMP_NS WITH TIME ZONE) is a V3-only nanosecond timestamp. SQE auto-upgrades the table to format-version 3 when one of these types appears in a CREATE.localtime()returns Time64.EXTRACT(HOUR|MINUTE|SECOND FROM time_col)works through the Trino-aliasedhour()/minute()/second()UDFs.year()/month()/day()on a TIME column raise a clear plan error per Trino spec.
Metadata Queries
SHOW CATALOGS;
SHOW SCHEMAS;
SHOW TABLES;
SHOW TABLES IN schema_name;
-- information_schema
SELECT * FROM information_schema.tables;
SELECT * FROM information_schema.schemata;
SELECT * FROM information_schema.columns WHERE table_name = 'orders';
-- Query plan (logical + physical)
EXPLAIN SELECT * FROM orders WHERE amount > 100;
-- With actual execution metrics
EXPLAIN ANALYZE SELECT * FROM orders WHERE amount > 100;
-- With Iceberg file/row estimates (no execution)
EXPLAIN FULL SELECT * FROM orders WHERE amount > 100;
Feature Comparison
| Category | SQE | Trino | Spark SQL |
|---|---|---|---|
| Window functions | Full | Full | Full |
| CTEs | Full | Full | Full |
| Joins (all types) | Full | Full | Full |
| Set operations | Full | Full | Full |
| CTAS | Yes | Yes | Yes |
| INSERT INTO SELECT | Yes | Yes | Yes |
| MERGE INTO | Yes (CoW) | Yes | Yes |
| DELETE FROM | Yes (CoW) | Yes | Yes |
| UPDATE | Yes (CoW) | Yes | Yes |
| Views | Yes | Yes | Yes |
| Arrow-native wire format | Yes | No (JSON) | No (Thrift) |
| Row-level security | Planned | Plugin | Ranger |
| Bearer token passthrough | Yes | No | No |
Query Plan Inspection (EXPLAIN)
SQE provides three variants of EXPLAIN for inspecting how queries are planned and executed.
EXPLAIN
Returns the logical and physical query plan without executing the query.
EXPLAIN SELECT * FROM orders WHERE amount > 100;
Output: Two rows — logical_plan and physical_plan — each containing a
text representation of the plan tree. The plan shown is the policy-enforced
plan: any row filters or column masks applied by the security layer are visible.
EXPLAIN ANALYZE
Executes the query and returns per-operator timing and row counts.
EXPLAIN ANALYZE
SELECT dept_id, COUNT(*), AVG(salary)
FROM employees
GROUP BY dept_id;
Output columns: step, operation, output_rows, elapsed_ms
Rows are ordered leaf-to-root (execution order). output_rows and elapsed_ms
are NULL for operators that do not expose DataFusion metrics.
EXPLAIN FULL
Returns the plan enriched with Iceberg table statistics — without executing the query.
EXPLAIN FULL SELECT * FROM large_table WHERE region = 'EU';
Output columns: step, operation, estimated_rows, estimated_bytes,
files_scanned, files_total
For IcebergScanExec nodes, statistics come from the Iceberg snapshot summary
(fast, no data file reads). estimated_rows reflects the total rows in the
snapshot at plan time. files_scanned equals files_total because
predicate-pushdown to file level is not yet implemented.
For other operators (Filter, Aggregate, Sort) estimated_rows comes from
DataFusion’s cardinality analysis where available; file columns are NULL.
Notes
- All three variants apply policy enforcement — the plan reflects what will actually execute for the authenticated user.
EXPLAIN FULLon non-Iceberg tables (e.g.,information_schema) returns NULL for all statistics columns without error.
Custom SQL Extensions
SQE extends standard SQL with statements for security policy management. These are parsed by wrapping sqlparser-rs — we don’t fork the parser.
Status: Parser extensions are designed (Phase 5). The parser currently recognizes
GRANTandREVOKEas standard SQL but does not yet handle the customROWS WHEREandMASKED WITHclauses.
Policy Statements
GRANT with Row Filter
GRANT SELECT ON schema.table TO ROLE role_name
ROWS WHERE condition;
Example:
-- Analysts can only see European data
GRANT SELECT ON sales.orders TO ROLE eu_analyst
ROWS WHERE region = 'EU';
-- Finance team sees only their cost center
GRANT SELECT ON hr.expenses TO ROLE finance
ROWS WHERE cost_center = current_user_attr('cost_center');
GRANT with Column Mask
GRANT SELECT ON schema.table TO ROLE role_name
MASKED WITH (column AS expression);
Example:
-- Support sees masked SSN (last 4 digits only)
GRANT SELECT ON customers TO ROLE support
MASKED WITH (ssn AS '***-**-' || RIGHT(ssn, 4));
-- Partial email masking
GRANT SELECT ON users TO ROLE viewer
MASKED WITH (email AS CONCAT(LEFT(email, 2), '***@', SPLIT_PART(email, '@', 2)));
REVOKE
REVOKE SELECT ON schema.table FROM ROLE role_name;
SHOW Statements
-- All grants on a table
SHOW GRANTS ON schema.table;
-- Effective policy for a role (combines all grants, resolves conflicts)
SHOW EFFECTIVE POLICY ON schema.table FOR ROLE analyst;
Parser Strategy
SQE wraps sqlparser-rs rather than forking it:
graph LR
SQL["SQL Input"] --> SP["sqlparser-rs<br/>(standard parse)"]
SP --> AST["Standard AST"]
AST --> PP["Post-Parse Transform"]
PP -->|Standard SQL| STD["Standard Statement"]
PP -->|GRANT with ROWS WHERE| PS["PolicyStatement::GrantRowFilter"]
PP -->|GRANT with MASKED WITH| PS2["PolicyStatement::GrantColumnMask"]
PP -->|SHOW GRANTS| PS3["PolicyStatement::ShowGrants"]
The post-parse transform detects GRANT/REVOKE statements with the custom extensions and converts them to PolicyStatement AST nodes. Standard GRANT/REVOKE (without extensions) passes through unchanged.
Statement Classification
Every SQL statement is classified for routing, metrics, and audit:
#![allow(unused)]
fn main() {
pub enum StatementKind {
Query, // SELECT
Ctas, // CREATE TABLE AS SELECT
Insert, // INSERT INTO
Merge, // MERGE INTO (planned)
Delete, // DELETE FROM (planned)
Drop, // DROP TABLE/VIEW
Rename, // ALTER TABLE RENAME
CreateView, // CREATE VIEW
DropView, // DROP VIEW
CreateSchema, // CREATE SCHEMA
DropSchema, // DROP SCHEMA
ShowCatalogs, // SHOW CATALOGS
ShowSchemas, // SHOW SCHEMAS
ShowTables, // SHOW TABLES
Policy, // GRANT, REVOKE
Utility, // EXPLAIN, SET, etc.
}
}
Each kind maps to a stable lowercase label ("query", "ctas", "insert") used in Prometheus metrics and audit logs.
Iceberg Integration
SQE is built on the iceberg-rust library (vendored fork) and speaks the Iceberg REST Catalog protocol natively. Iceberg is the only table format SQE supports.
Iceberg version
- iceberg-rust: SQE-rebased fork of risingwavelabs/iceberg-rust
dev_rebase_main_20260303at commit645f02a4b533, vendored atvendor/iceberg-rust/. ProvidesRewriteFilesAction/OverwriteFilesAction(Copy-on-Write DELETE/UPDATE),PositionDeleteFileWriter(Merge-on-Read position deletes), andDeletionVectorWriter(Iceberg V3) on top of upstream v0.9.0. - DataFusion: 53.0
- Arrow: 58
- Parquet: 58
- Iceberg table format: V2 and V3. V3 features verified end-to-end (TIMESTAMP_NS, column defaults, equality-delete UPDATE on identifier-fields, partition evolution).
The matrix score against icebergmatrix.org is 167/189 (88.4%), placing SQE fifth on the public scoreboard behind only Spark distributions (EMR, AWS Glue, OSS Spark, Dataproc). See iceberg-matrix.md for the per-cell breakdown and iceberg-matrix-compare.md for the side-by-side V2/V3 comparison against every other engine on the public scoreboard.
Architecture
graph TB
subgraph SQE
SC["SessionCatalog<br/>per-user token"]
TP["IcebergTableProvider<br/>DataFusion TableProvider"]
WR["Writer<br/>Parquet output"]
end
SC -->|REST API + bearer or SigV4| CAT["Catalog backend<br/>Polaris, Nessie, Glue REST,<br/>S3 Tables, Unity, HMS, JDBC, Hadoop"]
CAT -->|table metadata| SC
CAT -->|S3 credentials<br/>credential vending| SC
TP -->|read Parquet| OS["Object storage<br/>AWS S3, GCS, ADLS Gen2, R2,<br/>Ceph, MinIO, rustfs"]
WR -->|write Parquet| OS
SC -->|commit| CAT
Supported catalogs
SQE keeps the catalog choice as a runtime configuration concern. Every catalog below ships compiled in by default; pick one in [catalog.backend]. See Catalog backends for the full per-backend recipe with TOML examples and verification queries. For where the data files live (S3, R2, GCS, ADLS Gen2, HTTPS, hf://), see Storage backends.
| Catalog | type | Transport | Auth | Status |
|---|---|---|---|---|
| Apache Polaris | rest | Iceberg REST | OIDC bearer + credential vending | primary |
| Project Nessie 0.107+ | rest | Iceberg REST | bearer / anonymous | live verified |
| AWS Glue (Iceberg REST) | rest | Iceberg REST + AWS SigV4 | AWS provider chain | live verified |
| AWS S3 Tables (REST) | rest | Iceberg REST + AWS SigV4 (s3tables signing) | AWS provider chain | live verified |
| Unity Catalog OSS | rest | Iceberg REST | bearer (Databricks) / anonymous (OSS) | live verified, read-only on OSS |
| AWS Glue (native SDK) | glue | aws-sdk-glue | AWS provider chain | live verified eu-example-1 |
| AWS S3 Tables (native SDK) | s3tables | aws-sdk-s3tables | AWS provider chain | live verified eu-example-2 |
| Hive Metastore | hms | Thrift | none / Kerberos | live verified |
| JDBC (Postgres / MySQL / SQLite) | jdbc | iceberg-catalog-sql | DB credentials | live verified (Postgres) |
| Hadoop / storage-only | hadoop | object_store path scan | none | live verified, read-only |
There are two ways into AWS-managed Iceberg. The REST path (type = "rest" with the Glue REST or S3 Tables REST endpoint as catalog_url) speaks the Iceberg REST protocol that AWS exposes on top of these services. The native SDK path (type = "glue" or type = "s3tables") goes directly through aws-sdk-glue or aws-sdk-s3tables without REST. Both work; the SDK path is currently more maintained upstream and avoids the SigV4 signing patches.
The five non-Hadoop backends share one dispatch path through the upstream iceberg-catalog-loader crate. SQE’s for_session_other_backend translates the typed CatalogBackend enum into a uniform (catalog_type, name, props) tuple and hands it to iceberg_catalog_loader::load(type). The loader picks the right CatalogBuilder (REST / Glue / S3 Tables / HMS / SQL) and returns an Arc<dyn iceberg::Catalog>. Hadoop is the lone outlier because it is filesystem-only; its dispatch stays in sqe-catalog/src/backends/hadoop.rs.
Two SQE-only patches sit on top of the vendored loader. The first feature-gates each backend so a slim build does not transitively pull every backend’s AWS SDK / Thrift / sqlx weight. The second adds Send + Sync to BoxedCatalogBuilder so the boxed builder can cross await points in async contexts. Both are documented in vendor/iceberg-rust/README.md and forward-compatible with upstream.
Live integration tests for HMS, Nessie, JDBC Postgres, AWS Glue, AWS S3 Tables, and Unity OSS live in sqe-catalog/tests/backends_integration.rs. Each is #[ignore] and runs against a docker-compose overlay or a real cloud account configured via .env. Glue and S3 Tables are also live-verified (2026-05-05) against AWS account ACCOUNT_ID in eu-example-1 and eu-example-2.
The AWS REST endpoints share the OSS Iceberg REST code path. Phase P added an aws-sigv4 cargo feature to the vendored iceberg-catalog-rest crate that swaps the OAuth/Bearer authenticator for an AWS SigV4 signer when rest.sigv4-enabled=true lands in the catalog properties (or in the server’s /v1/config defaults). The signer reads credentials from the standard AWS provider chain.
Catalog REST surface
For Polaris, Nessie, Unity OSS, AWS Glue, and AWS S3 Tables, SQE talks to the catalog via the Iceberg REST API. Key interactions:
| Operation | REST endpoint | SQE use |
|---|---|---|
| List namespaces | GET /v1/{prefix}/namespaces | SHOW SCHEMAS |
| List tables | GET /v1/{prefix}/namespaces/{ns}/tables | SHOW TABLES |
| Load table | GET /v1/{prefix}/namespaces/{ns}/tables/{t} | Query planning |
| Create table | POST /v1/{prefix}/namespaces/{ns}/tables | CREATE TABLE |
| Drop table | DELETE /v1/{prefix}/namespaces/{ns}/tables/{t} | DROP TABLE |
| Create namespace | POST /v1/{prefix}/namespaces | CREATE SCHEMA |
| Drop namespace | DELETE /v1/{prefix}/namespaces/{ns} | DROP SCHEMA |
| Commit table | POST /v1/{prefix}/namespaces/{ns}/tables/{t} | After write |
| Server config | GET /v1/config?warehouse=... | Discovery / signing hints |
Every request includes the user’s bearer token (Polaris, Unity, Nessie, anything OIDC) or is signed with AWS SigV4 (Glue, S3 Tables). The catalog enforces access control.
Credential vending
When SQE loads a table, the catalog returns the table metadata and scoped storage credentials for accessing the data files:
sequenceDiagram
participant SQE
participant Catalog
participant Storage as S3 / Object store
SQE->>Catalog: Load table (user token / SigV4)
Catalog-->>SQE: Table metadata<br/>+ scoped storage credentials<br/>(STS / table-scoped key)
Note over SQE: Credentials are per-user,<br/>per-table, time-limited
SQE->>Storage: Read Parquet (scoped credentials)
Storage-->>SQE: Data (allowed prefix only)
This means:
- No service account with broad storage access.
- Each user’s storage access is scoped to exactly the tables they are querying.
- Credentials are short-lived (STS or equivalent).
Read path
graph LR
PLAN["LogicalPlan<br/>TableScan"] --> META["Load table metadata<br/>from catalog"]
META --> PRUNE["Partition pruning<br/>(manifest filtering)"]
PRUNE --> FILES["Data file list"]
FILES --> READ["Read Parquet<br/>(columnar, predicate pushdown,<br/>row-group skipping, RowFilter)"]
READ --> BATCH["Arrow RecordBatches"]
Read-side optimizations:
- Partition pruning: Iceberg manifest stats skip whole partitions that cannot match the query predicate.
- Column projection: only requested columns leave Parquet.
- Predicate pushdown: filters land at the row group level, the page-index level, and the Parquet
RowFilter. - Runtime filter pushdown: Phase P shipped a
DynamicPredicateAPI that absorbs DataFusion 53 hash-join build-side runtime filters into the same pruning surface. SF10 TPC-H lineitem-heavy queries sawq06 -51%,q07 -31%,q14 -33%. Engineering log atdocs/features/runtime-filter-pushdown.md. - Bloom filter consultation:
write.parquet.bloom-filter-columnslands bloom offsets in the file footer; DataFusion consults them automatically for literal equality predicates at scan time. - 5-layer caching: REST catalog cache, table metadata cache, manifest cache, SessionContext cache, OAuth token cache. Warm queries hit sub-millisecond planning.
Write path
graph LR
SQL["CTAS / INSERT / DELETE / UPDATE / MERGE"] --> EXEC["Execute SELECT"]
EXEC --> BATCH["RecordBatches"]
BATCH --> WRITE["Write Parquet<br/>(WriterProperties from table props)"]
WRITE --> COMMIT["Commit to Iceberg<br/>(append / row-delta / rewrite)"]
Supported DML, both V2 and V3 verified:
- CREATE TABLE AS SELECT: Apache Iceberg V2 and V3, including TIMESTAMP_NS columns and DEFAULT literals.
- INSERT INTO: streaming, with proper schema validation against the catalog.
- DELETE FROM: Copy-on-Write via
RewriteFilesAction, or Merge-on-Read viaPositionDeleteFileWriterwhenwrite.delete.mode=merge-on-read. - UPDATE: CoW or MoR (equality deletes when the table declares an identifier-field-id).
- MERGE INTO: full WHEN MATCHED / WHEN NOT MATCHED semantics, dispatching to CoW or MoR based on the table’s
write.update.mode. - ALTER TABLE:
ADD/DROP/RENAME COLUMN,SET/DROP NOT NULL, type promotion,ADD/DROP/REPLACE PARTITION FIELD(partition evolution),CREATE/DROP BRANCH/TAG,SET WRITE BRANCH.
The writer respects write.parquet.bloom-filter-columns and write.parquet.bloom-filter-fpp for any column the schema knows about. The footer-inspection test in sqe-catalog/src/parquet_writer_config.rs proves bloom offsets land in the resulting Parquet file.
V3 features verified
- TIMESTAMP_NS / TIMESTAMPTZ_NS: V3 nanosecond timestamps round-trip end-to-end.
- Column defaults:
CREATE TABLE ... DEFAULT <literal>applieswrite_default;ALTER TABLE ADD COLUMN ... DEFAULTappliesinitial_default. - Position deletes (V3): MoR DELETE on a V3 table writes position-delete files.
- Equality deletes (V3): UPDATE with a declared identifier-field-id commits a single RowDelta with new data file plus equality-delete row.
- Partition evolution (V3):
ALTER TABLE ADD/DROP/REPLACE PARTITION FIELDevolves the spec on V3 tables, including with day(ts) on TIMESTAMP_NS columns. - Time travel (V3):
FOR SYSTEM_TIME AS OFandFOR VERSION AS OFwork against V3 tables through the same snapshot walk as V2. - Schema evolution (V3):
ADD COLUMN,DROP COLUMN,RENAME COLUMN,SET DATA TYPEall work on V3.
V3 features still blocked upstream:
- Variant: pending iceberg-rust #2188.
- Geometry: pending DataFusion UDT #12644.
- Vector / Embedding: V3 spec not finalised.
The deferred list is tracked in docs/iceberg-matrix-state.json under caveats for each cell.
Write Path
SQE supports writing data to Iceberg tables through SQL. Writes go through the coordinator, which executes the SELECT portion, writes Parquet files to S3, and commits to the Iceberg catalog.
Supported Operations
CREATE TABLE AS SELECT (CTAS)
CREATE TABLE analytics.monthly_sales AS
SELECT
DATE_TRUNC('month', order_date) AS month,
region,
SUM(amount) AS total
FROM raw.orders
GROUP BY 1, 2;
Flow:
- Parse SQL, extract target table name and SELECT query
- Execute SELECT → get Arrow RecordBatches
- Convert Arrow schema to Iceberg schema
- Create table in Polaris catalog
- Write RecordBatches as Parquet files to S3
- Commit data files to Iceberg via AppendAction
CREATE OR REPLACE TABLE
CREATE OR REPLACE TABLE analytics.monthly_sales AS
SELECT ... ;
Drops the existing table (if it exists) and creates a new one. Useful for dbt table materializations.
INSERT INTO
INSERT INTO analytics.monthly_sales
SELECT
DATE_TRUNC('month', order_date) AS month,
region,
SUM(amount) AS total
FROM raw.orders
WHERE order_date >= '2024-06-01'
GROUP BY 1, 2;
Flow:
- Parse SQL, extract target table and SELECT query
- Execute SELECT → get Arrow RecordBatches
- Write RecordBatches as Parquet files to S3
- Commit data files to Iceberg via AppendAction (new snapshot)
Write Architecture
sequenceDiagram
participant QH as Query Handler
participant DF as DataFusion
participant WH as Write Handler
participant S3
participant POL as Polaris
QH->>DF: Execute SELECT query
DF-->>QH: RecordBatches
QH->>WH: handle_ctas(table_name, schema, batches)
WH->>POL: Create table (schema)
POL-->>WH: Table created
WH->>S3: Write Parquet data files
S3-->>WH: Written (paths + sizes)
WH->>POL: Commit AppendAction<br/>(data file list → new snapshot)
POL-->>WH: Committed
WH-->>QH: Success
Row-Level Operations (Copy-on-Write)
Row-level write operations are implemented via Copy-on-Write using the RisingWave iceberg-rust fork’s rewrite_files() transaction API. Affected data files are read, filtered/transformed, and rewritten as new files in a single atomic Iceberg commit.
DELETE FROM
DELETE FROM sales.orders WHERE status = 'cancelled';
-- Cross-table subqueries in WHERE
DELETE FROM sales.orders
WHERE customer_id IN (SELECT id FROM blacklist);
-- DELETE without WHERE = truncate
DELETE FROM sales.orders;
Flow:
- Scan table metadata to identify affected data files
- Read each affected file, apply the WHERE filter
- If all rows match: mark file for removal
- If partial match: rewrite file without matching rows
- Commit via
rewrite_files()(remove old files, add rewritten files)
UPDATE
UPDATE sales.orders SET status = 'shipped' WHERE tracking_id IS NOT NULL;
-- CASE WHEN transformations
UPDATE sales.orders SET amount = CASE
WHEN amount > 1000 THEN amount * 0.9
ELSE amount
END;
Flow:
- Scan table metadata to identify affected data files
- Read each affected file, apply the WHERE filter
- For matching rows: apply SET expressions
- Rewrite file with modified rows
- Commit via
rewrite_files()
MERGE INTO
MERGE INTO target USING source ON target.id = source.id
WHEN MATCHED THEN UPDATE SET value = source.value
WHEN NOT MATCHED THEN INSERT (id, value) VALUES (source.id, source.value);
Flow:
- Execute a full outer join of source and target via DataFusion
- Classify each result row: matched (UPDATE/DELETE) or not matched (INSERT)
- Rewrite affected target data files with modifications applied
- Add new data files for INSERT rows
- Commit via
rewrite_files()(remove old files, add new + rewritten files)
Iceberg Dependency
Row-level writes depend on the risingwavelabs/iceberg-rust fork (rev 1978911ec4) which provides the rewrite_files() transaction support not yet available in upstream iceberg-rust. When upstream ships OverwriteAction, the dependency can be migrated back to the official crate.
dbt Compatibility
The write path is designed to support dbt Core via a native dbt-sqe adapter:
| dbt Materialization | SQL | Status |
|---|---|---|
table | CREATE OR REPLACE TABLE AS SELECT | Supported |
incremental (append) | INSERT INTO SELECT | Supported |
incremental (merge) | MERGE INTO | Supported (CoW) |
view | CREATE VIEW AS SELECT | Supported |
seed | INSERT INTO (from CSV) | Supported |
read_parquet TVF
read_parquet() is a table-valued function registered on every SQE SessionContext. It reads Parquet files from local disk or S3-compatible storage and returns them as a DataFusion table scan, making Parquet files directly queryable without first loading data into Iceberg.
Syntax
SELECT * FROM read_parquet(
'<path>',
[access_key => '<key>',]
[secret_key => '<secret>',]
[endpoint => '<url>',]
[region => '<region>']
)
The first argument is the file path or glob pattern. All other arguments are named (keyword) parameters for S3 credentials. Named parameters are optional and fall back to the engine’s configured storage defaults when omitted.
Local files
Absolute paths and glob patterns both work:
-- Single file
SELECT * FROM read_parquet('/data/tpch/sf1/lineitem/part-0000.parquet');
-- All files in a directory
SELECT * FROM read_parquet('/data/tpch/sf1/lineitem/*.parquet');
-- Recursive glob
SELECT * FROM read_parquet('/data/tpch/sf1/lineitem/**/*.parquet');
The schema is inferred from the Parquet metadata of the first matched file. All matched files must share the same schema.
S3 with inline credentials
Pass credentials directly in the SQL statement. This is the primary mechanism used by sqe-bench load to inject credentials at load time without relying on environment variables or configuration files.
SELECT * FROM read_parquet(
's3://bench-data/tpch/sf1/lineitem/*.parquet',
access_key => 'AKIAIOSFODNN7EXAMPLE',
secret_key => 'wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY',
endpoint => 'http://localhost:9000',
region => 'us-east-1'
);
All four named parameters are optional independently. Omit endpoint for AWS S3 (uses the default AWS endpoint for the given region). Omit region to default to us-east-1.
S3 with default credentials
When no inline credentials are provided, read_parquet() falls back to the storage configuration in sqe.toml:
-- Uses [storage] section from sqe.toml
SELECT * FROM read_parquet('s3://bench-data/tpch/sf1/lineitem/*.parquet');
This is convenient for internal workloads where the engine already has ambient S3 credentials configured.
Glob patterns
read_parquet() supports the same glob syntax as object_store:
| Pattern | Matches |
|---|---|
*.parquet | All .parquet files in the named directory |
**/*.parquet | All .parquet files in any subdirectory |
part-00[0-9][0-9].parquet | Files matching the character class |
For S3 paths, globbing is applied to the key prefix after the bucket name.
Using with CTAS for data loading
The primary use case for read_parquet() is ingesting external Parquet data into Iceberg tables via CTAS. This avoids an intermediate format conversion step — the Parquet files are read directly and written as Iceberg data files in one operation.
-- Load TPC-H lineitem from local disk
CREATE TABLE tpch_sf1.lineitem AS
SELECT * FROM read_parquet('/data/tpch/sf1/lineitem/*.parquet');
-- Load from S3 with inline credentials
CREATE TABLE tpch_sf1.lineitem AS
SELECT * FROM read_parquet(
's3://bench-data/tpch/sf1/lineitem/*.parquet',
access_key => 'AKIA...',
secret_key => '...',
endpoint => 'http://localhost:9000',
region => 'us-east-1'
);
-- Transform during load
CREATE TABLE analytics.orders_summary AS
SELECT
o_orderdate,
o_orderstatus,
COUNT(*) AS order_count,
SUM(o_totalprice) AS total_revenue
FROM read_parquet('/data/tpch/sf1/orders/*.parquet')
GROUP BY o_orderdate, o_orderstatus;
Because read_parquet() returns a standard DataFusion table scan, it participates in the full optimizer pipeline: predicate pushdown, projection pruning, and partition pruning all apply.
Querying without loading
read_parquet() can also be used as a one-off query target, without creating an Iceberg table:
-- Inspect schema
DESCRIBE SELECT * FROM read_parquet('/data/tpch/sf1/orders/*.parquet') LIMIT 0;
-- Quick aggregation over raw Parquet
SELECT o_orderstatus, COUNT(*) AS cnt
FROM read_parquet('/data/tpch/sf1/orders/*.parquet')
GROUP BY o_orderstatus;
-- Join Parquet with an Iceberg table
SELECT p.p_name, l.l_quantity
FROM read_parquet('/data/tpch/sf1/lineitem/*.parquet') AS l
JOIN warehouse.tpch_sf1.part AS p ON l.l_partkey = p.p_partkey
LIMIT 20;
Implementation
read_parquet() is registered in sqe-catalog (or sqe-functions) as a DataFusion TableFunction. On each invocation:
- The path argument is parsed to detect
s3://vs local (/orfile://) paths. - For S3: an
AmazonS3Builderis constructed from the inline named parameters, with fallback to theStorageConfigfromsqe-corefor any omitted fields. - For local paths: the built-in DataFusion local filesystem
ObjectStoreis used. - Glob patterns are expanded against the chosen
ObjectStore. - A
ListingTableis constructed over the matched files and returned as the table scan node.
The function is registered on every SessionContext at startup, so it is always available without any special configuration.
Limitations
- All matched Parquet files must share an identical Arrow schema. Schema evolution across files in the same glob is not supported.
read_parquet()is read-only. It cannot be used as the target of an INSERT INTO.- Credential parameters are passed as SQL literals. Avoid logging or displaying these queries in audit logs without redaction. SQE’s audit logger redacts named parameter values that match
access_key,secret_key, andsession_tokenpatterns. - Very large numbers of matched files (>10,000) may cause slow planning due to the object listing step.
File-format TVFs
The four TVFs read_parquet, read_csv, read_json, and read_delta query files directly without registering an external table. They share a uniform calling convention and a uniform path-resolution layer (local filesystem, S3, HTTPS, HuggingFace hf://).
This chapter covers read_csv, read_json, and read_delta. The dedicated read_parquet chapter covers Parquet specifically.
Common path forms
All four TVFs accept the same path shapes:
-- Local
SELECT * FROM read_csv('/data/sales.csv');
-- S3 (anywhere object_store understands)
SELECT * FROM read_csv('s3://bucket/key.csv',
access_key => 'AKIA...', secret_key => '...',
endpoint => 'http://localhost:9000', region => 'us-east-1');
-- HTTP / HTTPS (V10)
SELECT * FROM read_csv('https://raw.githubusercontent.com/.../data.csv');
-- HuggingFace (V10)
SELECT * FROM read_csv('hf://datasets/squad/plain_text/train.csv');
-- HuggingFace with revision (V12.1)
SELECT * FROM read_parquet('hf://datasets/foo/[email protected]/data.parquet');
-- HuggingFace auto-generated parquet view
SELECT * FROM read_parquet('hf://datasets/foo/bar@~parquet/default/train/0.parquet');
S3 credentials default to the engine’s [storage] block when not supplied inline. HTTPS and hf:// paths flow through V10’s LazyHttpObjectStoreRegistry, which constructs an HttpStore for the host on first request.
Quoted-string auto-detect
V8 introduced a shortcut. With the embedded CLI, the engine recognises a quoted string in a FROM clause as a file URL and dispatches to the right TVF based on extension:
SELECT * FROM '/data/sales.parquet';
SELECT * FROM 's3://bucket/orders.csv';
SELECT * FROM 'hf://datasets/foo/bar/data.csv';
Format dispatch happens by extension. .parquet -> read_parquet, .csv / .tsv / .psv / .ssv -> read_csv, .json / .jsonl / .ndjson -> read_json, .avro -> the Avro reader. Compressed extensions are recognised: .csv.gz, .tsv.zst, .json.bz2 all dispatch to the right reader with the right codec.
read_csv
SELECT * FROM read_csv(
'<path>',
[delimiter | delim | sep => '<byte>',]
[has_header | header => '<bool>',]
[quote => '<byte>',]
[escape => '<byte>',]
[comment => '<byte>',]
[null_regex | nullstr => '<regex>',]
[compression | compress => 'auto|none|gzip|bz2|xz|zstd',]
[file_extension => '<.ext>']
);
Smart defaults:
- Delimiter detected from the path extension.
.csvis,,.tsvis tab,.psvis|,.ssvis;. Compression suffixes are stripped first, so.tsv.gzstill picks tab. - Compression detected from the path extension.
.gz,.bz2,.xz,.zstare recognised. has_headerdefaults to true (DataFusion default).
DuckDB-style aliases: sep, delim for delimiter; header for has_header; nullstr for null_regex; compress for compression.
-- All three are equivalent
SELECT * FROM read_csv('events.tsv');
SELECT * FROM read_csv('events.tsv', sep => '\t');
SELECT * FROM read_csv('events.tsv', delimiter => '\t', has_header => 'true');
-- Compressed, with explicit override
SELECT * FROM read_csv('events.tsv.zst', compression => 'auto');
-- Semicolon-separated file
SELECT * FROM read_csv('financial.ssv', sep => ';');
read_json
SELECT * FROM read_json(
'<path>',
[access_key | secret_key | endpoint | region | file_extension]
);
Reads NDJSON (one JSON document per line). Schema inference samples the first batch.
SELECT * FROM read_json('/var/log/events.jsonl');
SELECT * FROM read_json('s3://logs/2026-05-07/events.json.gz');
SELECT * FROM read_json('hf://datasets/nyu-mll/glue/cola/train.jsonl');
read_delta
SELECT * FROM read_delta(
'<path>',
[access_key | secret_key | endpoint | region,]
[version => '<u64>',]
[timestamp => '<RFC3339>']
);
Read-only Delta Lake reader, V11. Wraps deltalake-core 0.32.1. Time travel via version (snapshot id) or timestamp (RFC3339); the two are mutually exclusive.
SELECT * FROM read_delta('/data/delta/sales');
SELECT * FROM read_delta('s3://bucket/delta/orders',
access_key => 'AKIA...');
-- Time travel
SELECT * FROM read_delta('/data/delta/sales', version => '5');
SELECT * FROM read_delta('/data/delta/sales',
timestamp => '2026-04-01T00:00:00Z');
Writes are not exposed. The Delta transaction pipeline is substantial; the read path covers the most common ad-hoc query case.
HuggingFace specifics
The hf:// path uses a slightly different shape than S3 or HTTPS because HuggingFace expects a revision in the URL.
Two revision spellings work:
-
Inline
@<rev>(DuckDB-style):SELECT * FROM read_parquet('hf://datasets/foo/[email protected]/train.parquet'); -
Query parameter
?revision=<rev>:SELECT * FROM read_parquet('hf://datasets/foo/bar/train.parquet?revision=v1.0');
Default is main when neither is specified. Specifying both rejects with a clear error.
@~parquet is special. HuggingFace auto-generates a Parquet conversion of every dataset on the refs/convert/parquet branch. The TVF translates this:
-- Equivalent to https://huggingface.co/datasets/foo/bar/resolve/refs%2Fconvert%2Fparquet/data.parquet
SELECT * FROM read_parquet('hf://datasets/foo/bar@~parquet/default/train/0.parquet');
Glob expansion (**/*.parquet) is on the V12.2 roadmap; today the path must point to a specific file.
When to use which
read_parquet: ad-hoc queries against Parquet on disk, S3, HTTPS, or hf://. Anything Iceberg-aware that does not need the catalog.read_csv: ETL ingestion, log analysis, dataset preview before deciding to load into Iceberg.read_json: NDJSON logs, HuggingFacetrain.jsonlstyle splits.read_delta: query a Delta Lake table without converting to Iceberg.
For tables with metadata that you want to write back to, register them in a catalog. The TVFs are reads only.
Implementation references
sqe-catalog/src/read_parquet.rssqe-catalog/src/read_csv.rssqe-catalog/src/read_json.rssqe-catalog/src/read_delta.rssqe-catalog/src/file_tvf_common.rs: shared parsing + S3 / HTTPS / hf:// resolversqe-catalog/src/lazy_object_store.rs: V10’s lazy HTTPS object-store registry
Observability
SQE provides comprehensive observability through Prometheus metrics, OpenTelemetry traces/logs, and structured audit logging.
Metrics (Prometheus)
Available at http://coordinator:9090/metrics in Prometheus text format.
| Metric | Type | Labels | Description |
|---|---|---|---|
sqe_query_count_total | Counter | status, statement_type | Total queries by status and type |
sqe_query_duration_seconds | Histogram | statement_type | Query duration distribution |
sqe_rows_returned_total | Counter | — | Cumulative rows returned |
sqe_active_sessions | Gauge | — | Current active sessions |
sqe_healthy_workers | Gauge | — | Workers passing health checks |
Histogram buckets: 10ms, 25ms, 50ms, 100ms, 250ms, 500ms, 1s, 2.5s, 5s, 10s, 30s, 60s.
Statement types: query, ctas, insert, merge, delete, drop, create_view, create_schema, show_catalogs, show_schemas, show_tables, policy, utility.
Example Queries (PromQL)
# Query rate (queries per second)
rate(sqe_query_count_total[5m])
# Error rate
rate(sqe_query_count_total{status="error"}[5m])
# P99 query duration
histogram_quantile(0.99, rate(sqe_query_duration_seconds_bucket[5m]))
# Active sessions
sqe_active_sessions
Health Endpoints
Available on port 9091 (metrics port + 1) for both coordinator and workers.
Kubernetes Probes
| Endpoint | Purpose | Response |
|---|---|---|
GET /healthz | Liveness probe | Always returns 200 ok |
GET /readyz | Readiness probe | 200 when ready, 503 during init |
Cluster Status (Ballista/DataFusion-style)
GET /api/v1/status returns a JSON snapshot of the node and cluster:
{
"status": "ACTIVE",
"node": {
"role": "coordinator",
"version": "0.1.0",
"datafusionVersion": "51",
"uptimeSeconds": 3600
},
"workers": {
"total": 2,
"healthy": 2,
"healthyUrls": ["http://worker-0:50052", "http://worker-1:50052"]
}
}
For worker nodes, the workers field is null.
Trino-Compatible Info (port 8080)
When the Trino compat layer is enabled, standard Trino info endpoints are available on the Trino HTTP port:
| Endpoint | Response |
|---|---|
GET /v1/info | JSON: nodeVersion, environment, coordinator, starting, uptime |
GET /v1/info/state | Plain text: ACTIVE or STARTING |
These endpoints are compatible with Trino JDBC drivers, DBeaver, and other Trino-aware tools for auto-detecting node state.
OpenTelemetry
When otlp_endpoint is configured, SQE exports traces, metrics, and logs via OTLP/gRPC:
graph LR
SQE["sqe-server"] -->|OTLP gRPC| COLL["OTel Collector"]
COLL --> JAEGER["Jaeger<br/>(traces)"]
COLL --> PROM["Prometheus<br/>(metrics)"]
COLL --> LOKI["Loki<br/>(logs)"]
Configuration:
[metrics]
otlp_endpoint = "http://otel-collector:4317"
When the endpoint is empty (default), SQE falls back to structured JSON logs on stdout — no external dependency required.
Trace Spans
Key spans emitted:
sqe.query.execute— full query lifecyclesqe.query.plan— SQL parsing and planningsqe.policy.evaluate— policy enforcementsqe.flight.do_get— result streamingsqe.auth.handshake— authenticationsqe.worker.scan— worker scan execution
Audit Log
SQE writes a JSONL audit log capturing every query:
{
"timestamp": "2025-03-15T10:30:00Z",
"username": "alice",
"session_id": "a1b2c3d4-e5f6-7890-abcd-ef1234567890",
"query_text": "SELECT * FROM sales.orders WHERE region = 'EU'",
"query_hash": "sha256:e3b0c44298fc1c149afb...",
"statement_type": "query",
"client_ip": "10.0.1.42",
"duration_ms": 142,
"rows_returned": 1583,
"status": "ok"
}
The query_hash field is a SHA-256 hash of the SQL text, useful for correlating repeated queries without storing the full text. When audit logging is enabled, all fields are always present.
Configuration:
[metrics]
audit_log_path = "/var/log/sqe/audit.jsonl"
When the path is empty (default), audit logging is disabled (no-op).
Structured Logging
All SQE components use tracing with JSON output:
{
"timestamp": "2025-03-15T10:30:00.142Z",
"level": "INFO",
"target": "sqe_coordinator::query_handler",
"message": "Query executed",
"user": "alice",
"statement_type": "query",
"duration_ms": 142,
"rows": 1583
}
Log level controlled via RUST_LOG environment variable:
RUST_LOG=info # Default
RUST_LOG=sqe=debug # Debug SQE crates only
RUST_LOG=sqe=trace # Everything
Kubernetes Integration
The Helm chart includes optional ServiceMonitor for Prometheus Operator:
serviceMonitor:
enabled: true
interval: 30s
labels:
release: prometheus
Trino Compatibility
SQE includes a Trino-compatible HTTP endpoint that allows existing Trino clients (JDBC drivers, DBeaver, CLI tools) to connect without modification.
Enabling
The Trino HTTP endpoint is enabled by default on port 8080. Set port to 0 to disable:
[coordinator]
trino_http_port = 8080 # 0 to disable
Endpoints
| Endpoint | Method | Description |
|---|---|---|
/v1/info | GET | Node info (version, uptime, coordinator status) |
/v1/info/state | GET | Plain text: ACTIVE or STARTING |
/v1/statement | POST | Submit a SQL query |
/v1/statement/{id}/{token} | GET | Fetch paginated results |
/v1/statement/{id} | DELETE | Cancel a running query |
Authentication
The Trino endpoint supports two authentication methods:
Bearer Token
Pass an existing access token directly:
curl -H "Authorization: Bearer eyJhbG..." \
-H "X-Trino-User: alice" \
-d "SELECT 1" \
http://localhost:8080/v1/statement
Basic Auth
Username and password are exchanged for a token via the configured OIDC/OAuth2 backend:
curl -u alice:password \
-d "SELECT 1" \
http://localhost:8080/v1/statement
Client Headers
SQE respects standard Trino client headers:
| Header | Purpose |
|---|---|
X-Trino-User | Override username (used with Bearer auth) |
X-Trino-Catalog | Set default catalog for the session |
X-Trino-Schema | Set default schema for the session |
X-Trino-Source | Client identifier (logged for audit) |
Result Pagination
Query results are paginated. The initial response includes a nextUri field. Follow nextUri links to retrieve subsequent pages:
{
"id": "query-uuid",
"stats": { "state": "FINISHED" },
"columns": [{"name": "result", "type": "integer"}],
"data": [[1]],
"nextUri": "http://localhost:8080/v1/statement/query-uuid/1"
}
When nextUri is absent, all results have been consumed.
Using with the CLI
sqe-cli --protocol http --host localhost --port 8080 --user alice
Connecting External Tools
DBeaver
- Create a new Trino connection
- Host:
localhost, Port:8080 - Authentication: Username/Password
- Driver properties: no special settings needed
JDBC (Java)
String url = "jdbc:trino://localhost:8080";
Properties props = new Properties();
props.setProperty("user", "alice");
props.setProperty("password", "secret");
Connection conn = DriverManager.getConnection(url, props);
Limitations
- The Trino endpoint returns results as JSON (Trino wire format), not Arrow. For maximum performance, use Flight SQL.
- Prepared statements are not supported via the Trino protocol.
- Transaction control (
START TRANSACTION,COMMIT) is not supported — queries execute in auto-commit mode. - Type mapping covers common types; complex nested types may differ from native Trino behavior.
Flight SQL vs Trino HTTP
| Aspect | Flight SQL (default) | Trino HTTP |
|---|---|---|
| Port | 50051 | 8080 |
| Wire format | Arrow IPC (binary, columnar) | JSON |
| Performance | High (zero-copy) | Lower (serialization overhead) |
| Client support | ADBC, JDBC (Flight SQL), dbt | Trino JDBC, DBeaver, Tableau |
| Pagination | Arrow Flight streaming | nextUri polling |
Use Flight SQL for performance-sensitive workloads. Use Trino HTTP for compatibility with existing tools.
Benchmark Suite
SQE ships with sqe-bench, a Rust CLI tool that generates benchmark data, loads it into SQE via the read_parquet() TVF, and runs query suites to validate SQL correctness and measure query performance.
For the longitudinal view (every benchmark JSON in benchmarks/results/ plotted across time, per-suite, per-scale, per-query heatmaps), see docs/benchmark/. Charts auto-regenerate from the committed JSONs via make benchmark-charts.
Available Benchmarks
| Benchmark | Queries | Tables | Focus |
|---|---|---|---|
tpch | 22 | 8 | Star/snowflake schema, pure analytical reads |
tpcds | 99 | 24 | Complex SQL, correlated subqueries, window functions |
ssb | 13 | 5 | Denormalized star schema, fast smoke testing |
tpcc | 17 | 9 | OLTP read + write queries (DELETE/UPDATE via CoW) |
tpce | 11 | 33 | Brokerage OLTP, complex demographics and trade schema |
tpcbb | 10 | ~25 | SQL-only subset over TPC-DS data + web logs |
Why these benchmarks? Each covers a different slice of SQL correctness:
- TPC-H and SSB validate the analytical core: joins, aggregates, GROUP BY, ORDER BY, date arithmetic. TPC-H is the standard first check for any SQL engine.
- TPC-DS is the hardest. Its 99 queries exercise correlated subqueries, CTEs, window functions, GROUPING SETS, and complex multi-table joins. Passing TPC-DS well means the engine handles real analytical workloads.
- TPC-C and TPC-E cover OLTP patterns: point lookups, small aggregates, indexed access by key ranges, plus write operations (DELETE, UPDATE) exercised via Copy-on-Write.
- TPC-BB exercises semi-structured data alongside the TPC-DS schema — useful for validating string functions and JSON handling.
Generating Data
The generate command produces Parquet files on local disk or S3. Data is deterministic (seeded) so results are reproducible.
# Generate TPC-H at scale factor 1 (~1 GB, 8 tables)
cargo run -p sqe-bench -- generate tpch --scale 1 --output ./data
# Scale factor 10 (~10 GB)
cargo run -p sqe-bench -- generate tpch --scale 10 --output ./data
# Write directly to S3
cargo run -p sqe-bench -- generate tpch --scale 1 \
--output s3://bench-data/ \
--s3-access-key AKIA... \
--s3-secret-key ... \
--s3-endpoint http://localhost:9000 \
--s3-region us-east-1
# Generate all benchmarks at once
./scripts/benchmark-generate-all.sh
Scale factors explained
The scale factor controls dataset size. Scale factor 1 produces roughly 1 GB for TPC-H and TPC-DS; SSB is ~600 MB at SF1.
| Scale factor | TPC-H size | TPC-DS size | Use case |
|---|---|---|---|
| 1 | ~1 GB | ~1 GB | Development, CI, correctness checks |
| 10 | ~10 GB | ~10 GB | Performance testing |
| 100 | ~100 GB | ~100 GB | Near-production load |
| 1000 | ~1 TB | ~1 TB | Full-scale benchmarking |
Files are split at 128 MB for parallelism. Output is structured as:
./data/
└── tpch/
└── sf1/
├── lineitem/
│ ├── part-0000.parquet
│ └── part-0001.parquet
├── orders/
│ └── part-0000.parquet
└── ... (8 tables total)
Loading Data
The load command connects to SQE and creates Iceberg tables using read_parquet() + CTAS. No intermediate format conversion is needed — Parquet files are read directly and written as Iceberg.
# Load TPC-H from local disk
cargo run -p sqe-bench -- load tpch \
--scale 1 \
--data ./data \
--host localhost \
--port 60051 \
--username root \
--password ""
# Load from S3
cargo run -p sqe-bench -- load tpch \
--scale 1 \
--data s3://bench-data/ \
--s3-access-key AKIA... \
--s3-secret-key ... \
--s3-endpoint http://localhost:9000 \
--s3-region us-east-1 \
--host localhost \
--port 60051 \
--username root \
--password ""
# Drop and recreate tables before loading
cargo run -p sqe-bench -- load tpch --scale 1 --data ./data --clean \
--host localhost --port 60051 --username root --password ""
# Use the Trino HTTP protocol instead of Flight SQL
cargo run -p sqe-bench -- load tpch --scale 1 --data ./data \
--protocol trino \
--host localhost \
--port 8080 \
--username root \
--password ""
The loader creates a namespace named <benchmark>_sf<N> (e.g., tpch_sf1) and sends one CTAS statement per table:
CREATE TABLE tpch_sf1.lineitem AS
SELECT * FROM read_parquet('/data/tpch/sf1/lineitem/*.parquet');
For S3 sources, inline credentials are injected:
CREATE TABLE tpch_sf1.lineitem AS
SELECT * FROM read_parquet(
's3://bench-data/tpch/sf1/lineitem/*.parquet',
access_key => 'AKIA...',
secret_key => '...',
endpoint => 'http://localhost:9000',
region => 'us-east-1'
);
See read_parquet TVF for full syntax documentation.
Running Tests
The test command executes all queries in the benchmark suite against the loaded data and reports correctness and timing.
# Run all TPC-H queries (Flight SQL, default)
cargo run -p sqe-bench -- test tpch \
--scale 1 \
--host localhost \
--port 60051 \
--username root \
--password ""
# Run a single query
cargo run -p sqe-bench -- test tpch --scale 1 --query q03 \
--host localhost --port 60051 --username root --password ""
# Use Trino HTTP protocol
cargo run -p sqe-bench -- test tpch --scale 1 \
--protocol trino \
--host localhost \
--port 8080 \
--username root \
--password ""
# Run all benchmarks end-to-end
./scripts/benchmark-test.sh tpch
./scripts/benchmark-test.sh tpcds
./scripts/benchmark-test.sh ssb
Query result statuses
| Status | Meaning |
|---|---|
PASS | Result matches expected output exactly (within numeric tolerance) |
DIFF | Result matches in shape but has minor differences (e.g., decimal precision) |
FAIL | Result is wrong — wrong rows, wrong values, wrong schema |
SKIP | Query requires an unimplemented feature; counted but not failed |
ERROR | Query failed to execute (engine error, timeout, crash) |
DIFF is not treated as a failure in CI — it is a signal for investigation. Decimal precision differences are expected when comparing float-heavy aggregates across different engines.
Queries can declare their requirements in a header comment:
-- name: Revenue by nation
-- requires: delete, merge
-- timeout: 30s
SELECT ...
Any query with -- requires: will be SKIPped if SQE does not support that feature, rather than FAILing the suite.
Understanding Results
Terminal output
TPC-H SF1 — Flight SQL (localhost:50051)
─────────────────────────────────────────
q01 PASS 1.23s 6001215 rows
q02 PASS 0.45s 460 rows
q03 PASS 0.89s 11620 rows
...
q17 DIFF 2.10s 1 rows (decimal precision)
q22 PASS 0.33s 7 rows
Results: 20/22 PASS, 1 DIFF, 1 SKIP
Total time: 28.4s
Report: benchmarks/results/tpch-sf1-flight-2026-03-24T14:30:00.json
JSON report format
Reports are written to benchmarks/results/<benchmark>-sf<N>-<protocol>-<timestamp>.json:
{
"benchmark": "tpch",
"scale_factor": 1,
"protocol": "flight",
"host": "localhost:50051",
"timestamp": "2026-03-24T14:30:00Z",
"summary": {
"total": 22,
"pass": 20,
"fail": 0,
"diff": 1,
"skip": 1,
"error": 0,
"total_duration_ms": 28400
},
"queries": [
{
"id": "q01",
"status": "pass",
"duration_ms": 1230,
"rows": 6001215
},
{
"id": "q17",
"status": "diff",
"duration_ms": 2100,
"rows": 1,
"diff_detail": "decimal precision mismatch: expected 1.0000, got 0.9999"
}
]
}
JSON reports are machine-readable and suitable for tracking regressions over time in CI.
Historical Performance Tracking
Benchmark JSON results are committed to benchmarks/results/ for historical comparison. This enables tracking performance regressions and improvements across releases.
TPC-H SF1 — Historical Comparison (Apr 2 baseline vs. Apr 6 streaming execution)
After implementing the streaming execution engine (coordinator spill-to-disk, late materialization, file-level pruning, S3 I/O pipeline, distributed execution), TPC-H SF1 improved 3.1x on a distributed cluster (coordinator + 2 workers) compared to the single-node baseline:
Query Apr 2 (single-node) Apr 6 (distributed) Speedup
────────────────────────────────────────────────────────────
q01 3.21s 1.29s 2.5x
q02 0.89s 0.27s 3.3x
q03 2.23s 0.94s 2.4x
q04 1.14s 0.32s 3.6x
q05 1.89s 0.55s 3.4x
q06 1.13s 0.30s 3.7x
q07 2.07s 0.85s 2.4x
q08 1.81s 0.54s 3.4x
q09 1.78s 0.60s 3.0x
q10 2.47s 0.63s 3.9x
q11 0.74s 0.11s 6.8x
q12 1.71s 0.57s 3.0x
q13 1.10s 0.18s 6.1x
q14 1.46s 0.55s 2.7x
q15 2.24s 0.72s 3.1x
q16 0.75s 0.10s 7.4x
q17 1.89s 0.63s 3.0x
q18 3.19s 0.74s 4.3x
q19 1.68s 0.79s 2.1x
q20 1.39s 0.53s 2.6x
q21 2.11s 0.68s 3.1x
q22 0.67s 0.09s 7.7x
────────────────────────────────────────────────────────────
TOTAL 37.5s 12.0s 3.1x
Key observations:
- Metadata-light queries (q11, q13, q16, q22) see 6-8x speedup — footer cache, file pruning, and scan distribution eliminate I/O overhead
- Scan-heavy queries (q01, q03, q07) see 2-2.5x speedup — proportional to worker count (2 workers)
- q18 (the hardest TPC-H query) improved from 3.19s to 0.74s (4.3x) — benefits from distributed aggregation across workers
- Single-node with 512MB spill: 21/22 pass — only q18 fails due to DataFusion hash aggregate memory limitation (DF#17334). With 1GB+ memory or with workers, all 22 pass.
Full Benchmark Matrix (Apr 7, 2026 — SF1)
| Suite (queries) | single-512mb | single-8gb | distributed-2w |
|---|---|---|---|
| TPC-H (22) | 21/22 (29.6s) | 22/22 (28.6s) | 22/22 (13.5s) |
| TPC-DS (99) | 92/99 (94.1s) | 99/99 (99.4s) | 98/99 (36.1s) |
| SSB (13) | 4/13 (14.4s) | 13/13 (14.3s) | 13/13 (5.3s) |
| TPC-C (17) | 17/17 (21.5s) | 17/17 (22.0s) | 17/17 (8.6s) |
| TPC-E (18) | 12/18 (8.4s) | 13/18 (127.4s) | 10/18 (56.0s) |
| Total (169) | 146 (86%) | 164 (97%) | 162 (96%) |
Spill behavior across configs
| Config | Sort Spills | Bytes Spilled | Analysis |
|---|---|---|---|
| single-512mb | 30 | 1.1 GB | TPC-DS complex sorts spill to disk. 92/99 pass — spill works. |
| single-8gb | 128 | 27.7 GB | Mostly TPC-E (33-table joins). More spills because more queries run to completion. |
| distributed-2w | 3 | 49 MB | Near-zero spill. Workers absorb scan/aggregation work. |
The counterintuitive finding: 8GB spills more than 512MB. This is because 8GB successfully runs TPC-E queries that 512MB cannot — those TPC-E queries involve massive multi-table joins that produce 27GB of intermediate sorted data. With 512MB, the same queries OOM before reaching the spill point.
With distribution (2 workers), spill drops to 49MB. Workers handle scan and partial aggregation; the coordinator only merges small result sets.
Scheduler observations
At SF1, all distributed queries ran locally on the coordinator (scheduler_decisions{local}=120+). This is correct — SF1 tables have 1-2 data files each, below the distribution threshold (default: 4 files). The 2.5x speedup comes from streaming execution improvements (spill, scan planning), not from worker distribution. To observe actual worker distribution, run at SF10+ where tables have 10+ files.
TPC-E: the outlier
TPC-E has the lowest pass rate (56-72%) across all configs:
- 5 queries blocked by DataFusion’s IN(subquery) limitation (cannot decorrelate)
- Deep join chains across 33 tables produce massive intermediate results
- Some queries timeout at 600s after spilling 27GB
Metrics gaps
Several Phase A/B metrics show 0 because the increment calls are not yet wired into the execution path (the infrastructure exists but metric.inc() calls are missing):
- Footer cache hits/misses —
FooterCachenot wired intoIcebergScanExec - File pruning counts —
PruningPredicatebuilt but counter not incremented - Late materialization bytes — RowFilter wired but byte tracking not connected
- Time to first row — histogram registered but not observed
These are wiring tasks for the next iteration.
Comparing against Trino
The benchmark harness can run the same suite against a real Trino on the same data, so you can compare SQE and Trino directly. There are two modes:
- Correctness parity —
--compare-trinodiffs SQE’s results against Trino’s row-for-row. This is how SQL correctness is validated at scale, not just timing. Small decimal differences on float-heavy aggregates are expected and flagged for investigation rather than treated as failures. - Timing — the same run records per-query wall-clock for both engines, so a head-to-head speed comparison falls out of the parity run.
SQE’s own distributed execution path (coordinator + workers, spill-to-disk, late materialization, file-level pruning) gives a measured ~3.1× speedup over single-node on TPC-H SF1, with metadata-light queries seeing more.
Run a comparison yourself and see the captured numbers in the benchmark
quickstart: Benchmarks: TPC-H / TPC-DS / SSB, or
in the repo under
benchmarks/.
CI/CD Integration
All three scripts support automated use without a TTY:
# Generate data once (idempotent — skip if files exist)
./scripts/benchmark-generate-all.sh
# Load all benchmarks
./scripts/benchmark-load.sh
# Run and report
./scripts/benchmark-test.sh tpch
./scripts/benchmark-test.sh tpcds
./scripts/benchmark-test.sh ssb
# Exit code is 0 if all queries are PASS or SKIP
# Exit code is 1 if any query is FAIL or ERROR
A typical CI pipeline runs TPC-H at SF1 as a smoke test on every PR, and the full suite (TPC-H + TPC-DS + SSB) nightly.
Query Files
Query SQL files are stored under benchmarks/queries/<benchmark>/:
benchmarks/
├── queries/
│ ├── tpch/ q01.sql ... q22.sql
│ ├── tpcds/ q01.sql ... q99.sql
│ ├── ssb/ q1.1.sql ... q4.3.sql
│ ├── tpcc/ order_status.sql, stock_level.sql, ...
│ ├── tpce/ trade_lookup.sql, customer_position.sql, ...
│ └── tpcbb/ q01.sql ... q10.sql
├── expected/
│ ├── tpch/sf1/ q01.csv ... q22.csv
│ └── ...
└── schemas/
├── tpch.sql
├── tpcds.sql
└── ...
Expected results under benchmarks/expected/ are CSV files containing the correct output at the given scale factor. They are committed to the repository and used for regression checking.
Adding New Benchmarks
Implement the BenchmarkGenerator trait in sqe-bench/src/generate/:
#![allow(unused)]
fn main() {
pub trait BenchmarkGenerator {
fn name(&self) -> &str;
fn tables(&self) -> Vec<TableDef>;
fn generate_table(
&self,
table: &str,
scale: f64,
output: &dyn ObjectStore,
prefix: &str,
) -> Result<GenerateStats>;
}
pub struct TableDef {
pub name: String,
pub schema: Arc<Schema>, // Arrow schema
pub row_count_fn: fn(f64) -> usize, // scale factor → row count
}
}
Steps to add a benchmark:
- Create
sqe-bench/src/generate/<name>.rsimplementingBenchmarkGenerator. - Add SQL query files under
benchmarks/queries/<name>/. - Add expected result CSVs under
benchmarks/expected/<name>/sf1/. - Register the generator in
sqe-bench/src/generate/mod.rs. - Add the benchmark name to the CLI subcommand list in
sqe-bench/src/cli.rs. - Add a schema DDL file under
benchmarks/schemas/<name>.sqlfor reference.
Supported catalog backends
SQE supports multiple Iceberg catalog backends and wire-protocol client
adapters. The same binary works with every option below; choose by setting the
[catalog.backend] block in your TOML. Catalog weight is opt-in through cargo
features: a default REST-only build pulls no AWS SDK, no Thrift, no sqlx. Add
glue, s3tables, hms, or sql as needed; rest and hadoop are always
available.
For per-backend TOML configuration, credential setup, and troubleshooting, see Catalog backends.
Polaris (and any Iceberg REST catalog)
Apache Polaris exposes the Iceberg REST catalog specification. SQE uses this as its default backend. Any Iceberg REST-compatible service (Polaris, Lakeformation REST, custom) works with the same config block.
[catalog]
polaris_url = "http://localhost:18181/api/catalog"
warehouse = "test_warehouse"
See the quickstart: Polaris + Keycloak (client credentials).
AWS Glue Data Catalog
Native AWS SDK integration against the regional Glue Data Catalog. Credentials
come from the standard provider chain (AWS_PROFILE, instance profile, SSO).
[catalog.backend]
type = "glue"
region = "eu-example-1"
warehouse = "s3://my-bucket/warehouse"
See the quickstart: AWS Glue Data Catalog.
AWS S3 Tables
Managed Iceberg via the federated Glue Iceberg REST endpoint with AWS SigV4
authentication on every request. Backed by the vendored iceberg-catalog-rest
crate with the aws-sigv4 feature.
[catalog.backend]
type = "s3tables"
table_bucket_arn = "arn:aws:s3tables:eu-example-1:ACCOUNT:bucket/NAME"
See the quickstart: AWS S3 Tables (managed Iceberg).
Unity Catalog OSS
Unity Catalog OSS exposes an Iceberg REST adapter at
/api/2.1/unity-catalog/iceberg/. The OSS image is read-only on create/drop;
use for query workloads.
docker compose -f docker-compose.unity.yml up -d
set -a; source .env; set +a
cargo test -p sqe-catalog --test backends_integration -- --ignored unity_catalog::
See the quickstart: Unity Catalog OSS (Iceberg REST, read-only).
Hive Metastore
Thrift metastore protocol. Requires the hms cargo feature.
docker compose -f docker-compose.hms.yml up -d
set -a; source .env; set +a
cargo test -p sqe-catalog --features hms --test backends_integration -- --ignored hms::
Covered by the suite (hms::live_hms_namespace_round_trip, a create / list /
drop round-trip against the Thrift metastore).
Project Nessie
Git-like Iceberg REST catalog with branch/tag semantics.
docker compose -f docker-compose.nessie.yml up -d
set -a; source .env; set +a
cargo test -p sqe-catalog --test backends_integration -- --ignored nessie::
See the quickstart: Project Nessie (Iceberg REST catalog).
Hadoop (filesystem warehouse, no catalog service)
No metadata service. SQE walks the warehouse prefix for metadata.json files.
See Embedded mode for the catalog-free embedded flow.
[catalog.backend]
type = "hadoop"
warehouse = "s3://my-bucket/warehouse"
Quack (DuckDB wire protocol)
Quack is not an Iceberg catalog; it is a client/server wire protocol that lets DuckDB clients (and other Quack-compatible tools) issue SQL to SQE and receive Arrow-serialised results. It sits alongside the Iceberg catalog backends, not in competition with them.
See Quack.
How catalog backends are tested
sqe-catalog/tests/backends_integration.rs: live round-trips per backend (create / list / drop namespace, or read smoke), gated on#[ignore]plus the.envwarehouse variables.sqe-catalog/tests/mount_*_test.rs: mount-time validation per backend (rejects bad secrets, requires a warehouse, and so on).
Embedded mode
SQE can run the full query engine in-process, with no server and no network.
sqe-cli in embedded mode starts DataFusion, the Iceberg reader, and the same
SQL planner locally — inside the CLI process. This is the fastest path for
querying a warehouse from a laptop, a CI job, or a script.
Four warehouse modes:
- In-memory (
--memory): a transient DataFusion catalog. Nothing is persisted. Good for ad-hoc SQL and testing SQL functions. - Filesystem warehouse (
--warehouse PATH): an Iceberg warehouse on local disk or object storage with no catalog service. SQE walks the path formetadata.jsonfiles and treats the prefix as the catalog. The “Iceberg without a catalog” case. - Persistent SQLite catalog (
--catalog-backend sqlite): a durable single-node catalog backed by a local SQLite file. Survives restarts. - Cloud catalogs embedded: Glue and S3 Tables can be attached directly, with no coordinator, using the standard AWS credential chain.
See the quickstarts:
- Embedded: query local and remote files
- Embedded: persistent local catalog (SQLite)
- Embedded: attach multiple catalogs
In-memory
sqe-cli --embedded --memory -e "SELECT 1 AS one"
+-----+
| one |
+-----+
| 1 |
+-----+
Filesystem warehouse (no catalog service)
Point at a directory; SQE reads the Iceberg metadata directly. No Polaris, no Glue, no metastore.
sqe-cli --embedded --warehouse /data/warehouse \
-e "SELECT COUNT(*) FROM sales.orders"
This is the catalog-free Hadoop mode. Writes need atomic rename, which object
stores do not all provide, so this mode is read-oriented; for writes use a real
catalog. The same backend powers the [catalog.backend] type = "hadoop" server
config.
Cloud catalogs embedded
The embedded engine can attach a Glue or S3 Tables catalog directly, with no
coordinator. Pass --catalog-backend plus the cloud warehouse; credentials come
from the standard AWS provider chain (AWS_PROFILE, instance profile, SSO).
These catalogs attach read-only (query, not write); use the server for writes.
Requires the aws cargo feature (default-on).
# AWS Glue Data Catalog (warehouse is an s3:// prefix)
AWS_PROFILE=analytics sqe-cli --embedded \
--catalog-backend glue \
--catalog-warehouse s3://my-bucket/warehouse --region eu-example-1 \
-e "SELECT * FROM glue.analytics.events LIMIT 10"
# AWS S3 Tables (warehouse is the table-bucket ARN)
AWS_PROFILE=analytics sqe-cli --embedded \
--catalog-backend s3tables \
--catalog-warehouse arn:aws:s3tables:eu-example-1:ACCOUNT:bucket/NAME \
--region eu-example-1 \
-e "SHOW SCHEMAS"
The catalog mounts under the backend name by default (glue. / s3tables.);
override with --catalog-name.
How it is tested
crates/sqe-cli/tests/cli_smoke.rs: binary-level flag parsing, exit codes, mutually-exclusive flag rejection, and the--embedded --memoryhappy path.- The catalog spec parser (
NAME=PATH) is validated for empty names, missing separators, and dotted names.
Notes
--memoryand--warehouseare mutually exclusive.- Local-path TVFs (
read_csvand friends) work in embedded mode; the embedded engine enablesallow_local_pathsso a laptop user can read local files. - Embedded mode authenticates the OS user against the configured catalog’s credential source, not OIDC; there is no server to pass tokens through.
Flight SQL
Arrow Flight SQL is SQE’s primary wire protocol. It is columnar end to end:
results come back as Arrow record batches over gRPC, with no row-by-row
serialization tax. Every official client (the sqe-cli, ADBC drivers, the
dbt-sqe adapter) speaks it. This page covers both topologies against Apache
Polaris.
Single server
One coordinator parses, plans, and executes. Good for development, small deployments, and the default Docker image.
Prerequisites
docker compose -f docker-compose.test.yml up -d
./scripts/bootstrap-test.sh
This brings up Polaris (http://localhost:18181) and an S3-compatible store,
then creates the test_warehouse warehouse and the default / test_ns
namespaces.
Configuration
The coordinator reads one TOML file. The Flight SQL listener is on 50051:
[auth]
token_endpoint = "http://localhost:18181/api/catalog/v1/oauth/tokens"
client_id = "root"
[catalog]
polaris_url = "http://localhost:18181/api/catalog"
warehouse = "test_warehouse"
[storage]
s3_endpoint = "http://localhost:19000"
s3_region = "us-east-1"
s3_path_style = true
Run
# Start the coordinator (Flight SQL on 50051).
./target/release/sqe-server --config sqe.toml
# Connect with the CLI over Flight.
./target/release/sqe-cli --protocol flight --host localhost --port 50051 \
-u root -e "SELECT 1 AS one"
Expected output
one
-----
1
(1 row)
The in-process equivalent is exercised by integration_test.rs:
test_authentication (OIDC client-credentials against Polaris),
test_simple_select, and the file-format TVF tests all run the full
Flight SQL query path against this stack.
How it is tested
crates/sqe-coordinator/tests/integration_test.rs(run with./scripts/integration-test.sh): authentication, SELECT, CTAS round-trip, information_schema, andread_parquet/read_csv/read_json.
Distributed (coordinator + workers)
The coordinator parses and plans, then ships secured plan fragments to stateless workers over Arrow Flight. Workers hold no catalog state; they receive the plan and the user’s bearer token and execute.
Prerequisites
docker compose -f docker-compose.test.yml -f docker-compose.distributed.yml up --build -d
./scripts/bootstrap-test.sh
This adds a coordinator (Flight SQL on 60051) and two workers (internal
50052, exposed as 60061 / 60062) on the shared Polaris and storage.
Run
./scripts/distributed-test.sh
The script builds sqe-cli, runs SQL over Flight on 60051, and verifies
worker dispatch through system.runtime.tasks (proving fragments actually
reach the workers rather than silently falling back to local execution).
Expected output
The script runs a sequence of SQL statements over Flight on 60051 and prints
a pass line per check, ending with the worker-dispatch verification. The
single-node Flight path (auth, CTAS, SELECT against live Polaris) was re-run
this round through the TVF integration tests (3/3, 23.6s); the distributed
harness is covered by the suite and the committed benchmark baselines
(TPC-H SF1 distributed 22/22 in 12.0s, 3.1x over single node).
How it is tested
crates/sqe-coordinator/tests/integration_test.rs::test_distributed_select(ignored by default; needs workers listening on:50052).scripts/distributed-test.sh: full coordinator + worker harness.scripts/concurrent-test.sh: N parallel Flight clients, cache behaviour.
Notes
- Auth is bearer-token passthrough: the CLI authenticates the user via OIDC, and the token rides to Polaris and S3. There is no service account.
- The internal Flight port is
50051; compose maps it to60051to avoid colliding with a local coordinator. - Workers are stateless. Scale by adding worker replicas; the coordinator load-balances fragments across registered workers.
Trino HTTP compatibility
SQE speaks enough of the Trino HTTP protocol that Trino clients, JDBC drivers,
and BI tools can point at it unchanged. The coordinator exposes the
/v1/statement endpoint with nextUri pagination, /v1/info, and
/v1/info/state. This is a compatibility surface, not a re-implementation of
Trino; it covers the query-submission path that clients actually use.
Single server
Prerequisites
docker compose -f docker-compose.test.yml up -d
./scripts/bootstrap-test.sh
Configuration
The Trino HTTP listener runs alongside Flight SQL. The default port is 8080
(the test/distributed compose files map it to 28080):
[trino_compat]
enabled = true
port = 8080
Run
# Submit a query. Basic auth carries the user; the password is the OIDC secret
# (empty for the local root client).
curl -s -u root: \
-H "X-Trino-User: root" \
-d "SELECT 1 AS one" \
http://localhost:28080/v1/statement
Trino clients follow the nextUri field until results are exhausted. A JDBC
client connects with the Trino driver against http://localhost:28080.
Expected output
The first response carries a nextUri; following it returns the data:
{
"columns": [{"name": "one", "type": "bigint"}],
"data": [[1]],
"stats": {"state": "FINISHED"}
}
How it is tested
crates/sqe-coordinator/tests/integration_test.rs::test_trino_http_query: server startup, Basic auth,/v1/statementPOST, pagination.test_trino_type_mapping,test_trino_batches_to_json: Arrow to Trino JSON.scripts/trino-parity-test.shandscripts/trino-compat-test.sh: run the same SQL against SQE and a real Trino and diff the results.
Distributed
The Trino HTTP endpoint lives on the coordinator. Distribution across workers is identical to the Flight path: the coordinator plans, workers execute. The client sees a single Trino-compatible endpoint regardless of worker count.
Prerequisites
docker compose -f docker-compose.test.yml -f docker-compose.distributed.yml up --build -d
./scripts/bootstrap-test.sh
Run
./scripts/distributed-test.sh
The script exercises the Trino HTTP endpoint on 28080 (test 11) alongside the
Flight path and confirms worker dispatch.
Expected output
distributed-test.sh test 11 submits a query to the Trino HTTP endpoint on the
coordinator (28080) and follows nextUri to completion, alongside the Flight
path on the same cluster. This is covered by the suite; the docker-dependent
re-run was not repeated this round (see the validation matrix note on local
Docker capacity).
Trino SQL parity
SQE adds a Trino-compatibility function layer (date/time helpers like year(),
month(), day_of_week(), JSON casts, and more) so dbt models and Trino SQL
run with fewer rewrites. The current parity surface is tracked in
Trino Compatibility. The parity scripts
above are the regression guard.
Notes
- Authentication needs Basic auth (
-u user:password) to populate the session, not just theX-Trino-Userheader. For the local root client the password is empty (-u root:). - The Trino layer is optional; enable it with
[trino_compat] enabled = true. Flight SQL is the recommended protocol for SQE-native clients.
SQL Reference
A function-by-function and statement-by-statement reference for everything SQE can parse and execute. Every entry lists where the implementation lives so you can jump from “what does this do” to “where do I read the code”.
The reference focuses on what ships in the running engine, not the SQL standard in the abstract. If a function is not listed here, it is not registered in our SessionContext.
How to read these tables
Every page uses the same column shape. The first three columns describe the function in SQE; the four right columns describe how the same idea looks in other engines.
| Column | Meaning |
|---|---|
| Function | The name SQE accepts in SQL. Case-insensitive on the surface, lower-case canonical name. |
| Origin | Where the implementation comes from. See origins below. |
| Notes | One-line summary, return type, gotchas, link to source line. |
| Trino | The Trino-equivalent function name, or - if Trino has none. |
| Snowflake | The Snowflake-equivalent function name, or - if Snowflake has none. |
| Spark SQL | The Spark-equivalent function name, or - if Spark has none. |
| DuckDB | The DuckDB-equivalent function name, or - if DuckDB has none. |
Origins
Every function in SQE has exactly one origin. Eight values appear:
| Origin tag | What it means | Where it lives |
|---|---|---|
datafusion-builtin | Shipped automatically with SessionContext::new(). No SQE registration. | Upstream datafusion-functions-* crates. |
datafusion-functions-json | DataFusion JSON helper crate, registered explicitly. | datafusion_functions_json::register_all() in session_context.rs:361. |
sqe-trino-functions | Our Trino-compatibility crate. Adds Trino names for things DataFusion calls differently. | sqe-trino-functions/src/trino_functions.rs and trino_functions_ext.rs. |
sqe-trino-functions (ext) | Extended Trino aliases. Same crate, separate registration call. | register_extended_trino_functions() in the same crate. |
sqe-policy | Security crate. Currently exposes one UDF (sha256) used by column masks. | sqe-policy/src/sha256_udf.rs. |
sqe-catalog | Iceberg catalog and TVF crate. Provides read_* and table_* table functions. | sqe-catalog/src/. |
sqe-sql | Parser extension. Statements pre-parsed before DataFusion sees them. | sqe-sql/src/. |
sqe-coordinator | Statement router. Handles statements that need catalog calls or auth before execution. | crates/sqe-coordinator/src/query_handler.rs, catalog_ops.rs. |
The two registration entry points are crates/sqe-coordinator/src/session_context.rs (cluster mode) and crates/sqe-cli/src/embedded.rs (single-binary mode). Both register the same UDFs / UDTFs in the same order, so a function works the same way in both personas.
Pages
Scalar functions
- Conditional and null-handling:
if,iff,case,coalesce,nullif,greatest,least,nvl,nvl2,typeof,try. - String:
concat,substring,trim,lower,upper, regex, normalisation,split,format, hash digests. - Math: trig, rounding, logs, exponents, sign, modular, base conversion.
- Date and time: timestamp construction, extraction, formatting, parsing, arithmetic, time-zone handling.
- Array, map, struct: the 40+ functions from
datafusion-functions-nestedplus Trino aggregate constructors (map_agg,histogram). - JSON: two layered surfaces: Trino-named (
json_extract,json_parse) and thedatafusion-functions-jsonjson_get_*family. - Encoding, hashing, URL: base64, hex,
md5,sha224..512,url_extract_*,url_encode,url_decode.
Aggregate and window
- Aggregate functions:
count,sum,avg, statistical, regression,array_agg,string_agg/listagg,histogram,map_agg, approximation. - Window functions:
row_number,rank,lag,lead,first_value, frame syntax (ROWS BETWEEN,RANGE BETWEEN,GROUPS BETWEEN).
Table-valued functions
- Table-valued functions: file format (
read_parquet,read_csv,read_json,read_delta), Iceberg metadata (table_snapshots,table_history,table_files,table_partitions,table_manifests,table_refs), generators (generate_series,unnest).
Statements
- DDL:
CREATE,ALTER,DROPfor tables, schemas, views; partition evolution; branches and tags; column defaults. - DML:
SELECT,INSERT,UPDATE,DELETE,MERGE,COPY TO,TRUNCATE, time travel (FOR VERSION AS OF,FOR SYSTEM_TIME AS OF,FOR INCREMENTAL BETWEEN),SET WRITE_BRANCH. - CALL procedures:
system.rewrite_data_files,expire_snapshots,remove_orphan_files,rewrite_manifests,suggest_bloom_filter_columns. - GRANT and REVOKE: SQE-specific security extensions.
GRANT MASKED WITH,GRANT ROWS WHERE,SHOW GRANTS,SHOW EFFECTIVE GRANTS,CHECK ACCESS. - SHOW and EXPLAIN: metadata queries and plan inspection.
SHOW CATALOGS,SHOW STATS,EXPLAIN FULL. - Operators: arithmetic, string, comparison, null tests, casting (
::), set membership.
Embedded CLI
- Dot-commands:
.help,.tables,.schema,.describe,.summarize,.timer,.read,.format. Embedded CLI only.
What is intentionally not in SQE
Some functions appear in the dialect comparison columns as missing. The reasoning:
PIVOT,UNPIVOT,QUALIFY,ASOF JOIN, FROM-first syntax: DataFusion’s parser does not accept them. Tracked upstream.- Lambda expressions, list comprehensions: DataFusion has no AST node for closures.
- Oracle / Snowflake
DECODE: name collides with DataFusion’s binarydecode(input, encoding)helper.CASE WHENcovers the use case. IIF(T-SQL): covered byif(Trino) andiff(Snowflake), both registered.postgres_table_scanner,mysql_table_scanner,sqlite_scanner: out of scope. SQE is Iceberg-first; if you need a non-Iceberg engine, query it where it lives.spatial,vss,fts,excel: niche. Use a tool built for the job (PostGIS, a vector DB, an FTS engine).
The full DuckDB-comparison audit lives at duckdb-comparision.md. The Trino-comparison audit lives at trino-compatibility.md.
Conditional and null-handling
Functions for choosing between values, replacing nulls, and inspecting types. Most are scalar UDFs; CASE WHEN is a SQL expression handled by the planner.
Function table
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
if(cond, then, else) | sqe-trino-functions | 3-arg conditional. NULL condition returns else. Result type is the type of then. trino_functions.rs:955 | if | - | if | if |
iff(cond, then, else) | sqe-trino-functions | Identical semantics to if. Snowflake spelling. NULL condition returns else. trino_functions.rs:1039 | - | iff | - | - |
case when ... then ... [else ...] end | datafusion-builtin | Searched form. Walks branches in order; first true when wins. | case | case | case | case |
case <expr> when <val> then ... end | datafusion-builtin | Simple form. Compares expr to each when val; first equal wins. NULL never matches. | case | case | case | case |
coalesce(a, b, ...) | datafusion-builtin | First non-NULL argument. Variadic. Returns NULL if every arg is NULL. | coalesce | coalesce | coalesce | coalesce |
nullif(a, b) | datafusion-builtin | Returns NULL when a = b, else returns a. Inverse of coalesce(nullif(...), default) for “blank to null” patterns. | nullif | nullif | nullif | nullif |
nvl(a, b) | datafusion-builtin | Two-arg coalesce shape. Returns a if non-NULL, else b. | - | nvl | nvl | - |
nvl2(expr, when_not_null, when_null) | datafusion-builtin | Three-arg form: branches on whether expr IS NULL. | - | nvl2 | nvl2 | - |
greatest(a, b, ...) | datafusion-builtin | Max of the arguments. NULLs ignored unless every argument is NULL. Variadic. | greatest | greatest | greatest | greatest |
least(a, b, ...) | datafusion-builtin | Min of the arguments. NULLs ignored unless every argument is NULL. Variadic. | least | least | least | least |
typeof(expr) | sqe-trino-functions | Returns the Arrow type as text ("Int64", "Utf8", "Timestamp(Microsecond, None)"). Trino spells the same way; result string differs by engine. trino_functions.rs:1031 | typeof | - | - | typeof |
try(expr) | sqe-trino-functions (ext) | Catches errors from expr and returns NULL on failure. Handy for casting strings of unknown shape. trino_functions_ext.rs:76 | try | try_cast (different shape) | - | - |
arbitrary(col) | sqe-trino-functions (ext) | Aggregate that returns one non-deterministic non-NULL value. Trino name. Equivalent to any_value. trino_functions_ext.rs:68 | arbitrary | any_value | any_value | any_value |
Patterns
Replace NULL with a default
SELECT coalesce(comment, 'no comment') FROM orders;
SELECT nvl(comment, 'no comment') FROM orders; -- two-arg shorthand
Treat empty strings as NULL
SELECT coalesce(nullif(name, ''), 'unknown') FROM users;
nullif(name, '') returns NULL when name is the empty string, then coalesce substitutes the default.
Branch on a boolean
SELECT
iff(amount > 1000, 'large', 'small') AS bucket, -- Snowflake
if(amount > 1000, 'large', 'small') AS bucket_t -- Trino
FROM orders;
Both calls produce the same result. Use whichever matches your team’s existing dbt models. dbt-snowflake projects ported to SQE keep iff() working unmodified.
Complex branching: prefer CASE
SELECT
CASE
WHEN amount < 100 THEN 'small'
WHEN amount < 1000 THEN 'medium'
ELSE 'large'
END AS bucket
FROM orders;
Reach for CASE when there are more than two branches or the condition is not a single boolean expression.
Take the safer cast
SELECT try(CAST(payload AS BIGINT)) AS amount FROM events;
try() swallows the conversion error and returns NULL for rows that fail. Without it, one bad row aborts the query.
Type promotion
coalesce, greatest, least, if, iff all coerce arguments to a common supertype. The rules follow SQL standard widening: integer + decimal -> decimal; integer + double -> double; date + timestamp -> timestamp. If the arguments have no common supertype the planner returns an error before execution.
case is stricter: every branch must produce the same type, or the planner adds explicit casts when it can. Mixed types without an obvious supertype fail at plan time.
NULL handling cheat sheet
| Construct | NULL input | Result |
|---|---|---|
if(NULL, x, y) | NULL condition | y (NULL treated as false) |
iff(NULL, x, y) | NULL condition | y (NULL treated as false) |
case when NULL then x else y end | NULL condition | y |
coalesce(NULL, NULL, x) | All but x are NULL | x |
nullif(NULL, x) | First arg NULL | NULL |
nullif(x, NULL) | Second arg NULL | x (NULL is not equal to anything) |
greatest(NULL, 1, 2) | One NULL | 2 (NULLs skipped) |
greatest(NULL, NULL) | All NULL | NULL |
Why no IIF (T-SQL)
T-SQL’s IIF(cond, then, else) is the same shape as iff. SQE registers iff (Snowflake) and if (Trino), both pointing at the same implementation, so a T-SQL IIF rename is the only change needed. We deliberately did not register a third name to keep the function table tight.
Why no Oracle / Snowflake DECODE
Snowflake’s DECODE(expr, search1, result1, ..., default) is a multi-way conditional with NULL = NULL match semantics. Two reasons it is not in SQE:
- The name collides with DataFusion’s built-in
decode(input, encoding), which decodes base64 / hex strings to binary. Registering a Snowflake-style DECODE under the same name would shadow the encoding helper and break any existing callsite. CASE WHEN expr IS NOT DISTINCT FROM s1 THEN r1 ... ENDcovers the same ground in standard SQL. (IS NOT DISTINCT FROMtreats NULL = NULL as true.)
The audit row lives in features.md so the conflict is visible.
String functions
DataFusion contributes ~35 string functions plus a unicode submodule and a regex submodule. SQE adds Trino-name aliases and a few extras.
Concatenation
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
concat(a, b, ...) | datafusion-builtin | Variadic. NULL inputs become empty string. | yes | yes | yes | yes |
concat_ws(sep, a, b, ...) | datafusion-builtin | Concat with separator; skips NULL args. | yes | yes | yes | yes |
a || b | datafusion-builtin | SQL standard concat. NULL propagates. | yes | yes | yes | yes |
Length and offsets
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
length(s) / char_length(s) / character_length(s) | datafusion-builtin | Number of characters. UTF-8-aware. | yes | yes | yes | yes |
octet_length(s) | datafusion-builtin | Number of bytes. | yes | yes | yes | yes |
bit_length(s) | datafusion-builtin | octet_length * 8. | yes | yes | yes | yes |
position(needle in haystack) / strpos(haystack, needle) | datafusion-builtin | 1-based offset of needle. 0 if not found. | yes | yes | yes | yes |
find_in_set(needle, comma_list) | datafusion-builtin | 1-based offset in a comma-separated list. | - | - | yes | - |
codepoint(s) | sqe-trino-functions | Unicode codepoint of the first character. trino_functions.rs:101 | yes | - | - | - |
chr(n) / char(n) | datafusion-builtin | Codepoint -> character. | yes | yes | yes | yes |
ascii(s) | datafusion-builtin | Codepoint of the first character. | yes | yes | yes | yes |
Slicing
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
substring(s from start [for length]) | datafusion-builtin | SQL standard. 1-based. | yes | yes | yes | yes |
substr(s, start [, length]) | datafusion-builtin | Function form. | yes | yes | yes | yes |
left(s, n) | datafusion-builtin | Leftmost n chars. | yes | yes | yes | yes |
right(s, n) | datafusion-builtin | Rightmost n chars. | yes | yes | yes | yes |
split_part(s, delim, n) | datafusion-builtin | Nth part after splitting by delim. | yes | yes | yes | yes |
split(s, delim) | sqe-trino-functions | Returns array of parts. trino_functions.rs:118 | yes | yes | yes | yes |
split_part(s, delim, n) | datafusion-builtin | Single part by index. | yes | yes | yes | yes |
Trimming
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
trim(s) / trim(both ' ' from s) | datafusion-builtin | Default trims whitespace. | yes | yes | yes | yes |
ltrim(s [, chars]) | datafusion-builtin | Trim from left. | yes | yes | yes | yes |
rtrim(s [, chars]) | datafusion-builtin | Trim from right. | yes | yes | yes | yes |
btrim(s [, chars]) | datafusion-builtin | Trim both sides; explicit alias of trim. | - | yes | yes | yes |
Case and padding
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
lower(s) / upper(s) | datafusion-builtin | Case conversion. | yes | yes | yes | yes |
initcap(s) | datafusion-builtin | Title-case each word. | yes | yes | yes | yes |
lpad(s, n [, fill]) | datafusion-builtin | Pad on left to length n. | yes | yes | yes | yes |
rpad(s, n [, fill]) | datafusion-builtin | Pad on right. | yes | yes | yes | yes |
repeat(s, n) | datafusion-builtin | Repeat n times. | yes | yes | yes | yes |
reverse(s) | datafusion-builtin | Reverse the string. | yes | yes | yes | yes |
replace(s, from, to) | datafusion-builtin | Replace all occurrences. | yes | yes | yes | yes |
translate(s, from, to) | datafusion-builtin | Per-character substitution. | yes | yes | yes | yes |
overlay(s placing rep from start [for length]) | datafusion-builtin | Splice substring. | yes | yes | yes | yes |
format(pattern, args...) | sqe-trino-functions (ext) | C-style printf. trino_functions_ext.rs:79 | yes | - | yes | - |
Predicates
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
starts_with(s, prefix) | datafusion-builtin | True if s begins with prefix. | yes | yes | yes | yes |
ends_with(s, suffix) | datafusion-builtin | True if s ends with suffix. | yes | yes | yes | yes |
contains(s, sub) | datafusion-builtin | True if s contains sub. | - | yes | yes | yes |
s LIKE pattern | datafusion-builtin | SQL standard pattern. _ and %. | yes | yes | yes | yes |
s ILIKE pattern | datafusion-builtin | Case-insensitive LIKE. | yes | yes | partial | yes |
s SIMILAR TO pattern | datafusion-builtin | SQL/POSIX-light regex. | yes | yes | - | yes |
Regular expressions
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
regexp_like(s, pattern) | datafusion-builtin | True if pattern matches anywhere. | yes | yes | yes | yes |
regexp_match(s, pattern) | datafusion-builtin | Returns array of capture groups. | yes | yes | yes | yes |
regexp_replace(s, pattern, repl [, flags]) | datafusion-builtin | Replace matches. Flags g (global), i (insensitive). | yes | yes | yes | yes |
regexp_count(s, pattern) | datafusion-builtin | Count of non-overlapping matches. | yes | yes | yes | yes |
regexp_extract(s, pattern [, group]) | sqe-trino-functions (ext) | Extract first match (or capture group N). trino_functions_ext.rs:46 | yes | yes | yes | yes |
regexp_extract_all(s, pattern [, group]) | sqe-trino-functions (ext) | All matches as array. trino_functions_ext.rs:47 | yes | - | partial | - |
regexp_split(s, pattern) | sqe-trino-functions (ext) | Split by regex. trino_functions_ext.rs:48 | yes | - | - | - |
Hashing
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
md5(s) | datafusion-builtin | 128-bit hex. | yes | yes | yes | yes |
sha224(s) | datafusion-builtin | SHA-224 hex. | yes | - | yes | - |
sha256(s) | datafusion-builtin + sqe-policy | SHA-256 hex. SQE registers an alias used by column masks. | yes | yes | yes | yes |
sha384(s) | datafusion-builtin | SHA-384 hex. | yes | - | yes | - |
sha512(s) | datafusion-builtin | SHA-512 hex. | yes | yes | yes | yes |
digest(s, algorithm) | datafusion-builtin | Generic digest. Algos: md5, sha224, sha256, sha384, sha512, blake2s, blake2b, blake3. | - | - | - | - |
Distance / phonetic
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
levenshtein(a, b) | datafusion-builtin | Edit distance. | yes | yes | yes | yes |
hamming_distance(a, b) | sqe-trino-functions (ext) | Hamming distance for equal-length strings. trino_functions_ext.rs:35 | yes | - | - | - |
soundex(s) | sqe-trino-functions (ext) | Soundex code. trino_functions_ext.rs:34 | yes | yes | yes | - |
word_stem(s [, lang]) | sqe-trino-functions (ext) | Stemmer. Default English. trino_functions_ext.rs:61 | yes | - | - | - |
Encoding
See Encoding, hashing, URL for from_base64, to_base64, from_hex, to_hex, encode, decode.
Unicode normalization
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
normalize(s [, form]) | sqe-trino-functions (ext) | Unicode normalization. Forms: NFC (default), NFD, NFKC, NFKD. trino_functions_ext.rs:49 | yes | - | - | - |
from_utf8(bytes) | sqe-trino-functions | Convert binary to UTF-8 string. trino_functions.rs:85 | yes | - | - | - |
to_utf8(s) | sqe-trino-functions | Convert string to UTF-8 binary. trino_functions.rs:86 | yes | - | - | - |
Examples
Cleanse user input
SELECT
initcap(trim(lower(name))) AS name_clean,
regexp_replace(email, '\\s+', '') AS email_clean,
coalesce(nullif(trim(comment), ''), 'no comment') AS comment_clean
FROM users;
Extract domain from URL
SELECT
url,
regexp_extract(url, 'https?://([^/]+)', 1) AS host_via_regex,
url_extract_host(url) AS host_via_helper
FROM access_logs;
url_extract_host is dramatically faster than the regex version. Prefer it whenever the input is well-formed URLs.
Tokenise a sentence
SELECT id, token
FROM articles, UNNEST(split(body, ' ')) AS t(token)
WHERE length(token) > 3;
Mask sensitive data
SELECT
user_id,
sha256(email) AS email_hash,
concat('***', right(phone, 4)) AS phone_masked
FROM users;
(For declarative masking via grants, see GRANT and REVOKE.)
What is NOT supported
SOUNDEX_DIFFERENCE(T-SQL). uselevenshtein(soundex(a), soundex(b)).PARSE_URLfamily with named parts. Use theurl_extract_*family in Encoding, hashing, URL.STRING_SPLIT_TO_ARRAY. usesplit(s, delim).
Math functions
DataFusion contributes ~30 math functions. SQE adds a small set of Trino-named extras (e(), mod(), truncate(), sign()) plus base conversion and IEEE specials (infinity, nan).
Sign and rounding
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
abs(x) | datafusion-builtin | Absolute value. | yes | yes | yes | yes |
sign(x) | sqe-trino-functions | -1 / 0 / +1 (and NaN -> NaN). trino_functions.rs:100 | yes | yes | yes | yes |
ceil(x) / ceiling(x) | datafusion-builtin | Round up to integer. | yes | yes | yes | yes |
floor(x) | datafusion-builtin | Round down. | yes | yes | yes | yes |
round(x [, n]) | datafusion-builtin | Round to N decimal places. Banker’s rounding by default. | yes | yes | yes | yes |
trunc(x [, n]) | datafusion-builtin | Round toward zero. | yes | yes | yes | yes |
truncate(x [, n]) | sqe-trino-functions | Trino-named alias of trunc. trino_functions.rs:99 | yes | yes | partial | - |
Powers, logs, roots
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
pow(x, y) / power(x, y) | datafusion-builtin | x to the y. | yes | yes | yes | yes |
sqrt(x) | datafusion-builtin | Square root. | yes | yes | yes | yes |
cbrt(x) | datafusion-builtin | Cube root. | yes | yes | yes | yes |
exp(x) | datafusion-builtin | e^x. | yes | yes | yes | yes |
ln(x) | datafusion-builtin | Natural log. | yes | yes | yes | yes |
log(x [, base]) | datafusion-builtin | Log base 10 by default; or specified base. | yes | yes | yes | yes |
log2(x) / log10(x) | datafusion-builtin | Specific bases. | yes | yes | yes | yes |
e() | sqe-trino-functions | Euler’s number. trino_functions.rs:97 | yes | - | - | - |
pi() | datafusion-builtin | Pi as a constant. | yes | yes | yes | yes |
Trigonometry
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
sin(x) / cos(x) / tan(x) | datafusion-builtin | Standard trig. Radians. | yes | yes | yes | yes |
asin(x) / acos(x) / atan(x) | datafusion-builtin | Inverse trig. | yes | yes | yes | yes |
atan2(y, x) | datafusion-builtin | Two-arg arctangent, full quadrant. | yes | yes | yes | yes |
sinh(x) / cosh(x) / tanh(x) | datafusion-builtin | Hyperbolic. | yes | yes | partial | yes |
asinh(x) / acosh(x) / atanh(x) | datafusion-builtin | Inverse hyperbolic. | yes | - | - | yes |
degrees(x) | datafusion-builtin | Radians -> degrees. | yes | yes | yes | yes |
radians(x) | datafusion-builtin | Degrees -> radians. | yes | yes | yes | yes |
Modular and bit / base
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
mod(n, m) | sqe-trino-functions | Modulo. Trino-named alias. trino_functions.rs:98 | yes | yes | yes | yes |
n % m | datafusion-builtin | SQL operator form. | yes | yes | yes | yes |
gcd(a, b) | datafusion-builtin | Greatest common divisor. | yes | - | - | yes |
lcm(a, b) | datafusion-builtin | Least common multiple. | yes | - | - | yes |
factorial(n) | datafusion-builtin | n!. | - | - | yes | yes |
from_base(s, radix) | sqe-trino-functions (ext) | Parse a base-N string to integer. trino_functions_ext.rs:36 | yes | - | - | - |
to_base(n, radix) | sqe-trino-functions (ext) | Convert integer to base-N string. trino_functions_ext.rs:37 | yes | - | - | - |
SELECT to_base(255, 16); -- 'ff'
SELECT from_base('ff', 16); -- 255
SELECT to_base(8, 2); -- '1000'
Random
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
random() | datafusion-builtin | Uniform [0, 1). Volatile (re-evaluated per call). | yes | yes | yes | yes |
uuid() | datafusion-builtin | RFC 4122 v4 random UUID. | yes | yes | yes | yes |
IEEE specials
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
nanvl(x, y) | datafusion-builtin | If x is NaN, return y; else x. | - | - | yes | - |
isnan(x) | datafusion-builtin | True if x is NaN. | yes | - | yes | yes |
isinf(x) | datafusion-builtin | True if x is infinite. | - | - | - | yes |
iszero(x) | datafusion-builtin | True if x is exactly zero. | - | - | - | - |
infinity() | sqe-trino-functions (ext) | Positive infinity (Double). trino_functions_ext.rs:28 | yes | - | - | - |
nan() | sqe-trino-functions (ext) | NaN (Double). trino_functions_ext.rs:29 | yes | - | - | - |
Statistical helpers
For aggregates (stddev, variance, corr, covar_*, regr_*), see Aggregate functions. The math page covers scalars only.
Examples
Bucketing and binning
SELECT
floor(amount / 100) * 100 AS bucket,
count(*)
FROM orders
GROUP BY 1
ORDER BY 1;
Geometric mean via logs
SELECT exp(avg(ln(price))) AS geo_mean FROM products WHERE price > 0;
DataFusion has no built-in geo_mean; the log identity is the standard workaround.
Distance from a reference point (Pythagorean)
SELECT
name,
sqrt(pow(x - 100, 2) + pow(y - 200, 2)) AS distance
FROM points
ORDER BY distance
LIMIT 10;
Hex and binary representations
SELECT
n,
to_base(n, 16) AS hex,
to_base(n, 2) AS bin,
to_base(n, 8) AS oct
FROM generate_series(0, 255) AS t(n);
Numeric type promotion
pow, log, exp always return Double regardless of input. abs, floor, ceil, round preserve the input type. +, -, * follow SQL standard widening: integer + decimal -> decimal; integer + double -> double; decimal + decimal -> decimal with combined precision.
/ between two integers in DataFusion returns Double, not integer. For integer division use floor(a / b) or div(a, b).
Decimal precision
DECIMAL(p, s) arithmetic widens precision per SQL standard. Two DECIMAL(18, 2) values multiplied produce DECIMAL(36, 4). Going beyond DECIMAL(38, ...) overflows; CAST or use Double.
Date and time
The largest single category in SQE. Two layers stack:
- DataFusion native functions:
date_part,date_trunc,date_bin,extract,now, theto_timestamp_*family,make_date. Powerful but uses DataFusion-specific names. - Trino aliases registered by
sqe-trino-functions:year(),month(),day(),date_add(),date_diff(),format_datetime(). These cover every Trino date function so dbt-trino models work unmodified.
Snowflake and Spark also map well via the Trino layer.
Construction and current value
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
now() | sqe-trino-functions | Returns TIMESTAMP(6). Stable within one query. trino_functions.rs:55 | now | current_timestamp | now / current_timestamp | now / current_timestamp |
current_timestamp | datafusion-builtin | Same value as now(). SQL keyword, no parens. | current_timestamp | current_timestamp | current_timestamp | current_timestamp |
current_date | datafusion-builtin | Today’s date in session timezone. | current_date | current_date | current_date | current_date |
current_time | datafusion-builtin | Current wall-clock time. | current_time | current_time | current_time | current_time |
localtime() | sqe-trino-functions | Local time-of-day. Returns TIME(6). trino_functions.rs:64 | localtime | - | - | - |
localtimestamp() | sqe-trino-functions | Local timestamp without offset. Returns TIMESTAMP(6). trino_functions.rs:65 | localtimestamp | - | - | - |
current_timezone() | sqe-trino-functions (ext) | Returns the session timezone string. trino_functions_ext.rs:41 | current_timezone | - | current_timezone | - |
make_date(year, month, day) | datafusion-builtin | Construct a DATE. | - | date_from_parts | make_date | make_date |
Extraction (year, month, day, …)
These return integer parts. SQE registers Trino names; DataFusion’s extract and date_part are also available for SQL standard syntax.
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
year(d) | sqe-trino-functions | trino_functions.rs:28 | year | year | year | year |
month(d) | sqe-trino-functions | trino_functions.rs:29 | month | month | month | month |
day(d) | sqe-trino-functions | trino_functions.rs:30 | day / day_of_month | day | day | day |
hour(d) | sqe-trino-functions | trino_functions.rs:31 | hour | hour | hour | hour |
minute(d) | sqe-trino-functions | trino_functions.rs:32 | minute | minute | minute | minute |
second(d) | sqe-trino-functions | trino_functions.rs:33 | second | second | second | second |
millisecond(d) | sqe-trino-functions (ext) | Sub-second component. trino_functions_ext.rs:27 | millisecond | - | - | - |
day_of_week(d) | sqe-trino-functions | ISO: 1=Mon..7=Sun. trino_functions.rs:34 | day_of_week / dow | dayofweek | dayofweek | dayofweek |
day_of_year(d) | sqe-trino-functions | 1..366. trino_functions.rs:35 | day_of_year / doy | dayofyear | dayofyear | dayofyear |
quarter(d) | sqe-trino-functions | 1..4. trino_functions.rs:36 | quarter | quarter | quarter | quarter |
week(d) | sqe-trino-functions | ISO week number. trino_functions.rs:37 | week / week_of_year | weekofyear | weekofyear | weekofyear |
extract(<part> from d) | datafusion-builtin | SQL standard. Parts: YEAR, MONTH, DAY, HOUR, MINUTE, SECOND, DOW, DOY, EPOCH, QUARTER, WEEK. | extract | extract | extract | extract |
date_part('year', d) | datafusion-builtin | Function form of extract. | date_part | date_part | - | date_part |
Truncation and binning
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
date_trunc('month', d) | datafusion-builtin | Round down to a calendar boundary. Parts: year, quarter, month, week, day, hour, minute, second, millisecond, microsecond. | date_trunc | date_trunc | date_trunc | date_trunc |
date_bin(stride, d) | datafusion-builtin | Bin into fixed-width buckets, e.g. INTERVAL '15' minutes. SQE-relevant for time-series rollups. | - | time_slice | - | time_bucket |
last_day_of_month(d) | sqe-trino-functions (ext) | Last calendar day of d’s month. trino_functions_ext.rs:43 | last_day_of_month | last_day | last_day | last_day |
Arithmetic
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
date_add(unit, n, d) | sqe-trino-functions | unit is a string: year, quarter, month, week, day, hour, minute, second, millisecond. trino_functions.rs:40 | date_add | dateadd | date_add (different shape) | date_add |
date_diff(unit, d1, d2) | sqe-trino-functions | Difference d2 - d1 in units. trino_functions.rs:41 | date_diff | datediff | datediff | date_diff |
d + INTERVAL '5' DAY | datafusion-builtin | SQL standard interval arithmetic. | yes | yes | yes | yes |
d - INTERVAL '1' MONTH | datafusion-builtin | Subtraction works the same way. | yes | yes | yes | yes |
date_add example:
SELECT date_add('day', 7, DATE '2026-05-08'); -- 2026-05-15
SELECT date_add('month', -1, DATE '2026-05-08'); -- 2026-04-08
SELECT date_add('hour', 36, TIMESTAMP '2026-05-08 09:00:00'); -- 2026-05-09 21:00:00
date_diff example:
SELECT date_diff('day', DATE '2026-01-01', DATE '2026-01-06'); -- 5
SELECT date_diff('month', DATE '2026-01-01', DATE '2026-04-01'); -- 3
SELECT date_diff('year', DATE '2020-01-01', DATE '2026-01-01'); -- 6
Formatting and parsing
Two parallel formatting families:
- Trino / MySQL
%Y-%m-%dstyle viadate_format/date_parse. Familiar to anyone who’s usedstrftime. - Java / Joda
yyyy-MM-ddstyle viaformat_datetime/parse_datetime. The Trino default for new code.
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
date_format(d, fmt) | sqe-trino-functions | Trino / MySQL specifiers (%Y, %m, %d, %H, %i for minutes, %s for seconds). trino_functions.rs:51 | date_format | to_char | date_format | strftime |
date_parse(str, fmt) | sqe-trino-functions | Inverse of date_format. Returns TIMESTAMP(6). trino_functions.rs:52 | date_parse | to_timestamp | to_timestamp | strptime |
format_datetime(d, fmt) | sqe-trino-functions (ext) | Java / Joda specifiers (yyyy-MM-dd HH:mm:ss). trino_functions_ext.rs:54 | format_datetime | - | date_format | - |
parse_datetime(str, fmt) | sqe-trino-functions (ext) | Inverse of format_datetime. trino_functions_ext.rs:55 | parse_datetime | - | to_timestamp | - |
to_iso8601(d) | sqe-trino-functions (ext) | RFC 3339 / ISO 8601 string. trino_functions_ext.rs:40 | to_iso8601 | - | - | - |
from_iso8601_date(str) | sqe-trino-functions (ext) | ISO 8601 -> DATE. trino_functions_ext.rs:38 | from_iso8601_date | - | - | - |
from_iso8601_timestamp(str) | sqe-trino-functions (ext) | ISO 8601 -> TIMESTAMP(6). trino_functions_ext.rs:39 | from_iso8601_timestamp | - | - | - |
to_char(d, fmt) | datafusion-builtin | Postgres-style. Specifiers differ from Trino’s date_format. | - | to_char | date_format | strftime |
to_date(str [, fmt]) | datafusion-builtin | Parse to DATE. | - | to_date | to_date | strptime |
to_timestamp(str [, fmt]) | datafusion-builtin | Parse to TIMESTAMP. | parse_datetime | to_timestamp | to_timestamp | strptime |
to_timestamp_seconds(epoch) | datafusion-builtin | Cast a unix epoch to TIMESTAMP(0). | - | to_timestamp | - | to_timestamp |
to_timestamp_millis(ms) | datafusion-builtin | Cast millis to TIMESTAMP(3). | - | - | - | - |
to_timestamp_micros(us) | datafusion-builtin | Cast micros to TIMESTAMP(6). | - | - | - | - |
to_timestamp_nanos(ns) | datafusion-builtin | Cast nanos to TIMESTAMP(9). V3 Iceberg ns timestamps. | - | - | - | - |
Trino-style example:
SELECT date_format(TIMESTAMP '2026-05-08 14:30:00', '%Y-%m-%d %H:%i:%s');
-- 2026-05-08 14:30:00
SELECT date_parse('2026-05-08 14:30:00', '%Y-%m-%d %H:%i:%s');
-- 2026-05-08 14:30:00 (TIMESTAMP)
Java-style example (preferred for new code):
SELECT format_datetime(TIMESTAMP '2026-05-08 14:30:00', 'yyyy-MM-dd HH:mm:ss');
-- 2026-05-08 14:30:00
Unix epoch
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
from_unixtime(epoch) | sqe-trino-functions | Seconds since epoch -> TIMESTAMP(6). trino_functions.rs:42 | from_unixtime | - | from_unixtime | to_timestamp |
to_unixtime(d) | sqe-trino-functions | TIMESTAMP -> seconds since epoch (Double). trino_functions.rs:43 | to_unixtime | - | unix_timestamp | epoch |
extract(epoch from d) | datafusion-builtin | SQL-standard alternative for to_unixtime. | yes | - | yes | yes |
Time zones
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
with_timezone(d, tz) | sqe-trino-functions (ext) | Attach a timezone offset to a naive timestamp. trino_functions_ext.rs:50 | with_timezone | convert_timezone | from_utc_timestamp | - |
at_timezone(d, tz) | sqe-trino-functions (ext) | Convert a TIMESTAMP WITH TIME ZONE to another zone. trino_functions_ext.rs:51 | at_timezone | convert_timezone | - | - |
timezone_hour(d) | sqe-trino-functions (ext) | Hour component of the offset. trino_functions_ext.rs:70 | timezone_hour | - | - | - |
timezone_minute(d) | sqe-trino-functions (ext) | Minute component of the offset. trino_functions_ext.rs:71 | timezone_minute | - | - | - |
expr AT TIME ZONE 'Europe/Amsterdam' | datafusion-builtin | SQL-standard syntax. | yes | yes | yes | yes |
Misc
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
human_readable_seconds(n) | sqe-trino-functions (ext) | Format seconds as "1d 4h 30m". Useful in monitoring queries. trino_functions_ext.rs:42 | human_readable_seconds | - | - | - |
Worked example
A daily revenue rollup, time-zone aware, formatted for human reports:
SELECT
date_format(
with_timezone(date_trunc('day', order_ts), 'Europe/Amsterdam'),
'%Y-%m-%d'
) AS day_local,
quarter(order_ts) AS qtr,
day_of_week(order_ts) AS dow,
SUM(amount) AS revenue
FROM orders
WHERE order_ts >= TIMESTAMP '2026-01-01 00:00:00' AT TIME ZONE 'UTC'
GROUP BY 1, 2, 3
ORDER BY 1;
Iceberg V3 nanosecond timestamps
TIMESTAMP_NS and TIMESTAMP_NS WITH TIME ZONE only exist in Iceberg format-version 3. Adding such a column to a CREATE TABLE auto-bumps the table format version. All datetime functions operate on these the same way they operate on TIMESTAMP(6); the underlying Arrow type is Timestamp(Nanosecond, ...) instead of Timestamp(Microsecond, ...).
When you query a V3 ns column from a Trino client, Trino downscales to microseconds. SQE keeps the full precision when serving Arrow Flight SQL clients.
Array, map, struct
DataFusion’s datafusion-functions-nested crate ships ~40 array and map helpers. SQE adds Trino-named aggregate constructors (map_agg, histogram, multimap_agg).
Array construction
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
[1, 2, 3] (literal) | datafusion-builtin | Element type from common supertype. | yes | yes | yes | yes |
make_array(a, b, ...) | datafusion-builtin | Function form of literal. | - | yes | yes | yes |
array(...) | datafusion-builtin | Alias for make_array. | yes | yes | yes | yes |
range(start, stop) | datafusion-builtin | Half-open integer array. | - | - | - | yes |
range(start, stop, step) | datafusion-builtin | With step. | - | - | - | yes |
array_repeat(elem, n) | datafusion-builtin | Array of n copies. | yes | - | - | yes |
Array inspection
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
array_length(a) / cardinality(a) | datafusion-builtin | Number of elements. | yes | yes | yes | yes |
array_dims(a) | datafusion-builtin | Array of per-dimension sizes (for nested arrays). | - | - | - | yes |
array_ndims(a) | datafusion-builtin | Nesting depth. | - | - | - | yes |
array_position(a, elem) | datafusion-builtin | 1-based offset of first match; 0 if missing. | yes | yes | yes | yes |
array_positions(a, elem) | datafusion-builtin | Array of all matching offsets. | - | - | - | yes |
array_contains(a, elem) / array_has(a, elem) | datafusion-builtin | Boolean membership. | yes | yes | yes | yes |
array_has_all(a, sub) | datafusion-builtin | All of sub are in a. | - | - | - | yes |
array_has_any(a, sub) | datafusion-builtin | Any of sub is in a. | yes | - | - | yes |
Array transformation
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
array_append(a, elem) | datafusion-builtin | Add to end. | yes | yes | yes | yes |
array_prepend(elem, a) | datafusion-builtin | Add to start. | yes | - | yes | yes |
array_concat(a1, a2, ...) | datafusion-builtin | Variadic concat. | yes | yes | yes | yes |
array_remove(a, elem) | datafusion-builtin | Remove first occurrence. | - | - | - | yes |
array_remove_all(a, elem) | datafusion-builtin | Remove all occurrences. | yes | - | - | yes |
array_replace(a, from, to) | datafusion-builtin | Replace first match. | yes | - | yes | yes |
array_replace_all(a, from, to) | datafusion-builtin | Replace all matches. | - | - | - | yes |
array_reverse(a) | datafusion-builtin | Reverse order. | yes | yes | yes | yes |
array_sort(a) | datafusion-builtin | Ascending sort. NULLs last. | yes | yes | yes | yes |
array_distinct(a) | datafusion-builtin | Deduplicate. Preserves first occurrence. | yes | yes | yes | yes |
array_slice(a, start, end) | datafusion-builtin | 1-based, inclusive. Negative indexes count from end. | yes | yes | yes | yes |
array_pop_front(a) / array_pop_back(a) | datafusion-builtin | Remove first / last. | yes | - | - | yes |
array_resize(a, n [, fill]) | datafusion-builtin | Truncate or pad to length n. | - | - | - | yes |
array_flatten(a) / flatten(a) | datafusion-builtin | One level of flattening. | yes | yes | yes | yes |
Array set operations
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
array_intersect(a, b) | datafusion-builtin | Common elements (set-style). | yes | yes | yes | yes |
array_union(a, b) | datafusion-builtin | Distinct combination. | yes | yes | yes | yes |
array_except(a, b) | datafusion-builtin | In a but not in b. | yes | - | yes | yes |
Array reductions
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
array_min(a) | datafusion-builtin | Minimum element. | yes | yes | yes | yes |
array_max(a) | datafusion-builtin | Maximum. | yes | yes | yes | yes |
array_sum(a) | datafusion-builtin | Sum of numeric elements. | yes | - | - | yes |
array_mean(a) | datafusion-builtin | Average. | - | - | - | - |
array_any_value(a) | datafusion-builtin | First non-NULL element. | - | - | - | - |
Array unnesting (lateral)
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
unnest(a) | datafusion-builtin | One row per element. Used in FROM. | yes | yes | yes (explode) | yes |
unnest(a) WITH ORDINALITY | datafusion-builtin | Adds 1-based offset column. | yes | - | - | - |
-- One row per (order, item) pair
SELECT order_id, item
FROM orders, UNNEST(items) AS t(item);
-- Numbered
SELECT order_id, item, idx
FROM orders, UNNEST(items) WITH ORDINALITY AS t(item, idx);
Map functions
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
map(keys_array, values_array) | datafusion-builtin | Build a map from two parallel arrays. | yes | - | yes (map_from_arrays) | yes |
map_keys(m) | datafusion-builtin | Array of keys. | yes | yes | yes | yes |
map_values(m) | datafusion-builtin | Array of values. | yes | yes | yes | yes |
map_extract(m, key) | datafusion-builtin | Lookup. NULL if missing. Also accessible via m[key]. | yes (element_at) | yes (get) | yes (element_at) | yes (element_at) |
cardinality(m) | datafusion-builtin | Number of keys. | yes | yes | yes | yes |
m['key'] | datafusion-builtin | Subscript syntax for map lookup. | yes | yes | yes | yes |
Aggregates that build maps / arrays
See Aggregate functions for array_agg, map_agg, histogram, multimap_agg, map_union. The names differ slightly across engines:
| SQE | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|
array_agg(x) | array_agg | array_agg | collect_list | array_agg / list |
map_agg(k, v) | map_agg | object_agg | map_from_arrays | map |
histogram(x) | histogram | - | - | histogram |
multimap_agg(k, v) | multimap_agg | - | - | - |
Struct / row
| Construct | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
struct(a, b, ...) | datafusion-builtin | Anonymous record. | - | yes (object_construct) | yes | yes |
named_struct('a', x, 'b', y) | datafusion-builtin | Named-field record. | yes (row(...)) | yes (object_construct) | yes | yes |
s.field | datafusion-builtin | Field access. | yes | yes | yes | yes |
(a, b, ...) (row literal) | datafusion-builtin | Anonymous tuple. | yes | - | yes | yes |
SELECT named_struct('host', host, 'port', port) AS endpoint
FROM servers;
SELECT endpoint.host, endpoint.port FROM ...;
Examples
Tag-set membership
-- Find products with both 'sale' and 'new' tags
SELECT * FROM products
WHERE array_has_all(tags, ARRAY['sale', 'new']);
-- Find products with any of the listed tags
SELECT * FROM products
WHERE array_has_any(tags, ARRAY['sale', 'clearance']);
Top-K frequencies via histogram
SELECT k, v
FROM events, UNNEST(map_keys(histogram(event_type)), map_values(histogram(event_type))) AS t(k, v)
ORDER BY v DESC
LIMIT 10;
Build a map from joined tables
SELECT
user_id,
map_agg(setting_key, setting_value) AS preferences
FROM user_settings
GROUP BY user_id;
map_agg errors on duplicate keys. For multimap-style behaviour use multimap_agg.
Lateral pattern: filter then unnest
SELECT order_id, tag
FROM orders, UNNEST(tags) AS t(tag)
WHERE order_id > 100 AND tag LIKE 'priority_%';
Lambda functions
DataFusion’s parser does not support lambda syntax (x -> x + 1). Trino, Spark, DuckDB do. The audit rows in features.md note this. Workarounds:
- Pre-compute via a CTE plus
unnest. - Use
map_filter/transformfromdatafusion-functions-nestedonce parser support lands upstream.
What is NOT registered
zip(a, b)(parallel-iterate two arrays). Useunnestagainst an indexed pair instead.reduce(a, init, lambda, finish). Aggregate within a CTE instead.- Snowflake
flattentable function (with PATH and OUTER options). UseUNNESTdirectly.
These are tracked but blocked on lambda support in DataFusion.
JSON
Two layered JSON surfaces. Both are registered in every session.
- Trino-named layer in
sqe-trino-functions:json_extract,json_extract_scalar,json_array_length,json_parse,json_format,json_object,is_json_scalar,json_array_contains,json_size,json_array_get,to_json. Maps directly to dbt-trino model expectations. - DataFusion JSON layer via
datafusion-functions-json(registered insession_context.rs:361andembedded.rs:172):json_get,json_get_str,json_get_int,json_get_float,json_get_bool,json_get_json,json_get_array,json_contains,json_as_text,json_length. Type-specific extractors that avoid an outer CAST.
JSON columns are stored as VARCHAR (Iceberg has no native JSON type yet). Both layers parse on every call, so for hot paths consider extracting once into a typed column.
Trino-named layer
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
json_parse(s) | sqe-trino-functions | Parse JSON text. Returns the same VARCHAR after validation; primarily used to fail fast on malformed input. trino_functions.rs:92 | yes | parse_json | from_json | json |
json_format(j) | sqe-trino-functions | Format / re-emit JSON. trino_functions.rs:60 | yes | - | to_json | to_json |
json_object(k1, v1, k2, v2, ...) | sqe-trino-functions | Build a JSON object from key-value pairs. trino_functions.rs:59 | yes | yes | yes | yes |
json_array_length(j) | sqe-trino-functions | Number of elements in a JSON array. NULL if not an array. trino_functions.rs:91 | yes | yes | yes | json_array_length |
json_extract(j, '$.path') | sqe-trino-functions | JSONPath extraction. Returns JSON-encoded value. trino_functions.rs:89 | yes | yes | yes | yes |
json_extract_scalar(j, '$.path') | sqe-trino-functions | Same path syntax; returns plain text scalar (or NULL). trino_functions.rs:90 | yes | - | get_json_object | json_extract_string |
is_json_scalar(j) | sqe-trino-functions (ext) | True for JSON null / boolean / number / string. False for objects and arrays. trino_functions_ext.rs:30 | yes | - | - | - |
json_array_contains(j, v) | sqe-trino-functions (ext) | True if JSON array contains the value. trino_functions_ext.rs:31 | yes | - | - | - |
json_size(j, '$.path') | sqe-trino-functions (ext) | Cardinality at the path. Object key count for objects; array length for arrays; 0 for primitives. trino_functions_ext.rs:72 | yes | - | - | - |
json_array_get(j, idx) | sqe-trino-functions (ext) | 0-based element from a JSON array. trino_functions_ext.rs:73 | yes | - | - | - |
to_json(any) | sqe-trino-functions (ext) | Serialize a SQL value (struct, array, map) as JSON. trino_functions_ext.rs:80 | - | yes | yes | yes |
DataFusion JSON layer
These return typed scalars directly, which means no outer CAST and DataFusion can push predicates through.
| Function | Origin | Notes | Returns |
|---|---|---|---|
json_get(j, key_or_index) | datafusion-functions-json | Generic typed accessor. The argument can be a string key or integer index. | Union of possible types |
json_get_str(j, key) | datafusion-functions-json | Force string return. | VARCHAR |
json_get_int(j, key) | datafusion-functions-json | Force integer. | BIGINT |
json_get_float(j, key) | datafusion-functions-json | Force float. | DOUBLE |
json_get_bool(j, key) | datafusion-functions-json | Force boolean. | BOOLEAN |
json_get_json(j, key) | datafusion-functions-json | Re-emit nested JSON. | VARCHAR (JSON) |
json_get_array(j, key) | datafusion-functions-json | Force array. | VARCHAR (JSON array) |
json_contains(j, '$.path') | datafusion-functions-json | True if path exists. | BOOLEAN |
json_as_text(j) | datafusion-functions-json | Re-emit as JSON text (round-trip). | VARCHAR |
json_length(j) | datafusion-functions-json | Number of object keys / array elements at root. | BIGINT |
Path syntax
The Trino layer uses JSONPath ($.foo.bar[0]).
The DataFusion JSON layer uses single-step keys: a string for object access or an integer for array access.
-- Trino-style: full JSONPath
SELECT json_extract_scalar(payload, '$.user.id')
FROM events;
-- DataFusion: chained single-step
SELECT json_get_str(json_get_json(payload, 'user'), 'id')
FROM events;
Both compose. Both work on the same VARCHAR column. Choose by ergonomics:
- Trino-style is one call per path; cleaner SQL.
- DataFusion JSON layer returns native types; no outer CAST.
For a deeply nested path used in a hot WHERE clause, the DataFusion layer wins on speed (no JSON re-encoding between steps).
Comparison
| Operation | SQE | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|
| Parse / validate | json_parse | json_parse | parse_json | from_json (typed) | json() cast |
| Path extract (text) | json_extract_scalar | json_extract_scalar | : syntax (payload:user.id) | get_json_object | json_extract_string |
| Path extract (JSON) | json_extract | json_extract | path (object navigation) | nested from_json | json_extract |
| Typed extract | json_get_str/int/float/bool | cast(... AS ...) | :value::TYPE | from_json schema | json_extract_* |
| Build object | json_object(k, v, ...) | json_object(k, v, ...) | object_construct | to_json(struct(...)) | json_object |
| Build array | json_array(...) (Trino style; not registered yet) | json_array(...) | array_construct | to_json(array(...)) | json_array |
| Length | json_array_length / json_length | json_array_length | array_size | json_array_length | json_array_length |
| Contains key/value | json_contains / json_array_contains | json_array_contains | array_contains | array_contains | json_contains |
| To JSON text | to_json | cast(x AS json) | to_json | to_json | to_json |
Examples
Extract a typed field for filtering
-- Slowpath: parse JSON, cast to int, compare
SELECT * FROM events
WHERE CAST(json_extract_scalar(payload, '$.user_id') AS BIGINT) = 12345;
-- Faster: typed extractor, no CAST
SELECT * FROM events
WHERE json_get_int(json_get_json(payload, 'user'), 'id') = 12345;
Project several fields at once
SELECT
json_extract_scalar(payload, '$.user.id') AS user_id,
json_extract_scalar(payload, '$.user.email') AS email,
json_extract_scalar(payload, '$.session.locale') AS locale,
CAST(json_extract_scalar(payload, '$.amount') AS DECIMAL(18, 2)) AS amount
FROM events;
Path-existence filter
SELECT * FROM events
WHERE json_contains(payload, '$.metadata.experimental_flag') = true;
Build a structured field for downstream consumers
SELECT json_object(
'id', id,
'host', url_extract_host(url),
'path', url_extract_path(url),
'received_at', cast(occurred_at AS VARCHAR)
) AS event_doc
FROM events;
Performance tips
- Decode once at the boundary: when a JSON column is queried in many downstream views, consider materialising the relevant subfields into typed columns at ingest time. Repeated parsing is the biggest JSON cost.
- Path index pruning: the DataFusion JSON layer can push
json_get_str(payload, 'user_id') = 'X'through to a manifest min/max statistic when the underlying JSON has been pre-extracted. The Trino layer is opaque. - Avoid round-trip:
json_format(json_parse(j))is a no-op semantically but burns CPU. Skip the round-trip unless you need re-canonicalisation.
Iceberg JSON column type
Iceberg V3 has no native JSON primitive yet. SQE accepts JSON in CREATE TABLE and stores it as VARCHAR underneath. Predicates and projections work as text. When Iceberg adds a JSON primitive, the storage layout will change but the SQL surface stays the same.
What is not registered
json_array(...)(Trino name for build-an-array). Usecast(make_array(...) AS VARCHAR)followed byjson_format, orto_json(make_array(...)).json_sizeat the root: covered. With a path: covered.- JSONPath wildcards (
$..foo,$[*]): not supported by DataFusion’s path parser. Useunnest(json_get_array(...))for array-level wildcarding.
Encoding, hashing, URL
Three small, related families: binary encoding (base64 / hex), cryptographic hashes, URL parsing. SQE inherits the encoding and crypto helpers from DataFusion and adds Trino-named aliases plus eight URL extractors.
Binary encoding
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
encode(input, encoding) | datafusion-builtin | Encode binary to text. encoding: 'base64' or 'hex'. | yes | yes | yes | yes |
decode(input, encoding) | datafusion-builtin | Decode text to binary. Same encoding set. | yes | yes | yes | yes |
from_base64(s) | sqe-trino-functions | Trino-named base64 decode. trino_functions.rs:78 | yes | yes (base64_decode_string) | yes (unbase64) | yes |
to_base64(b) | sqe-trino-functions | Trino-named base64 encode. trino_functions.rs:79 | yes | yes (base64_encode) | yes (base64) | yes |
from_hex(s) | sqe-trino-functions | Trino-named hex decode. trino_functions.rs:80 | yes | yes (hex_decode_string) | yes (unhex) | yes |
to_hex(n) | datafusion-builtin | Encode integer or binary as hex. NOT registered as Trino UDF: DataFusion already has it. | yes | yes | yes | yes |
SELECT to_base64(CAST('hello' AS bytea)); -- 'aGVsbG8='
SELECT from_base64('aGVsbG8='); -- bytea -> 'hello'
SELECT encode(CAST('hi' AS bytea), 'hex'); -- '6869'
SELECT decode('6869', 'hex'); -- bytea -> 'hi'
Cryptographic hashes
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
md5(s) | datafusion-builtin | 128-bit hex string. | yes | yes | yes | yes |
sha224(s) | datafusion-builtin | SHA-224 hex. | yes | - | yes | - |
sha256(s) | datafusion-builtin + sqe-policy | SHA-256 hex. SQE registers an additional UDF used by column masks. | yes | yes | yes | yes |
sha384(s) | datafusion-builtin | SHA-384 hex. | yes | - | yes | - |
sha512(s) | datafusion-builtin | SHA-512 hex. | yes | yes | yes | yes |
digest(s, algo) | datafusion-builtin | Generic. Algos: md5, sha224, sha256, sha384, sha512, blake2s, blake2b, blake3. | - | - | - | - |
checksum(b) | sqe-trino-functions (ext) | xxHash of bytes; cheaper than crypto hashes. trino_functions_ext.rs:69 | yes | - | - | - |
SELECT sha256(email) AS email_hash FROM users;
SELECT digest('hello', 'blake3'); -- 64-char blake3 hex
SELECT to_hex(checksum(payload)) FROM events;
URL parsing
All eight URL functions live in sqe-trino-functions. They wrap the url crate from crates.io for correct RFC 3986 handling.
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
url_extract_host(url) | sqe-trino-functions | Hostname only, no port. trino_functions.rs:67 | yes | parse_url(..., 'HOST') | parse_url(..., 'HOST') | - |
url_extract_path(url) | sqe-trino-functions | Path component (after host, before query). trino_functions.rs:68 | yes | parse_url(..., 'PATH') | parse_url(..., 'PATH') | - |
url_extract_port(url) | sqe-trino-functions | Port as INT. NULL when absent. trino_functions.rs:69 | yes | - | - | - |
url_extract_protocol(url) | sqe-trino-functions | Scheme (https, http, s3, etc). trino_functions.rs:70 | yes | parse_url(..., 'PROTOCOL') | parse_url(..., 'PROTOCOL') | - |
url_extract_query(url) | sqe-trino-functions | Query string (after ?, no leading ?). trino_functions.rs:71 | yes | parse_url(..., 'QUERY') | parse_url(..., 'QUERY') | - |
url_extract_parameter(url, name) | sqe-trino-functions | First value of named parameter. trino_functions.rs:72 | yes | parse_url(..., 'QUERY:name') | parse_url(..., 'QUERY', 'name') | - |
url_encode(s) | sqe-trino-functions | Percent-encode. trino_functions.rs:73 | yes | yes | yes | - |
url_decode(s) | sqe-trino-functions | Percent-decode. trino_functions.rs:74 | yes | yes | yes | - |
SELECT
url,
url_extract_protocol(url) AS proto,
url_extract_host(url) AS host,
url_extract_port(url) AS port,
url_extract_path(url) AS path,
url_extract_query(url) AS query,
url_extract_parameter(url, 'utm_source') AS utm
FROM events;
For example, on https://example.com:8443/api?utm_source=newsletter&page=2:
| Component | Value |
|---|---|
url_extract_protocol | https |
url_extract_host | example.com |
url_extract_port | 8443 |
url_extract_path | /api |
url_extract_query | utm_source=newsletter&page=2 |
url_extract_parameter(..., 'utm_source') | newsletter |
When to use which hash
| Use case | Recommendation |
|---|---|
| Equality lookups, deduplication | xxhash via checksum(). ~10x faster than crypto hashes. |
| Cryptographic integrity, signatures | sha256 or sha512. Avoid md5 for new uses. |
| Column masking via grants | sha256 (used in MASKED WITH clauses). Deterministic, no collisions in practice. |
| Hashing UTF-8 strings | All accept VARCHAR or BINARY arguments. The result type is VARCHAR (hex) for crypto hashes, BINARY for digest(). |
| Salting | Concatenate the salt: sha256(concat(salt, password)). SQE has no pbkdf2-style helpers. |
When to use which URL parser
url_extract_host, url_extract_query, url_extract_parameter use a real RFC 3986 parser, so they handle internationalised domain names (IDN), unusual ports, percent-encoded query values, and trailing fragments correctly. The regex equivalent (regexp_extract(url, 'https?://([^/]+)', 1)) breaks on these edge cases.
Use the regex form only when the input is known to be a fixed shape that the regex covers.
Why no Snowflake-style PARSE_URL
Snowflake’s parse_url(url [, permissive]) returns an OBJECT with named keys. The eight separate url_extract_* functions are equivalent in expressive power and easier to compile statically. dbt-snowflake to dbt-sqe migrations need the rewrite, but it is mechanical: parse_url(u, 'HOST') becomes url_extract_host(u), etc.
Why DataFusion’s decode is not shadowed
A Snowflake-style DECODE(expr, search1, result1, ...) would conflict with DataFusion’s binary decode(input, encoding). SQE keeps the binary one and exposes the conditional logic via CASE WHEN. See Conditional for the trade-off.
Aggregate functions
Functions used in GROUP BY queries and OVER clauses. SQE inherits ~40 aggregates from DataFusion plus 12 Trino UDAFs from sqe-trino-functions.
Standard aggregates
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
count(expr) | datafusion-builtin | Counts non-NULL rows. | count | count | count | count |
count(*) | datafusion-builtin | Counts all rows. | count | count | count | count |
count(distinct expr) | datafusion-builtin | Distinct non-NULL count. | count(distinct ...) | count(distinct ...) | count(distinct ...) | count(distinct ...) |
sum(expr) | datafusion-builtin | Sum. NULL-skipping. | sum | sum | sum | sum |
sum(distinct expr) | datafusion-builtin | Distinct sum. | sum(distinct ...) | sum(distinct ...) | sum(distinct ...) | sum(distinct ...) |
avg(expr) / mean(expr) | datafusion-builtin | Arithmetic mean. NULL-skipping. | avg | avg | avg / mean | avg / mean |
min(expr) | datafusion-builtin | Minimum. | min | min | min | min |
max(expr) | datafusion-builtin | Maximum. | max | max | max | max |
median(expr) | datafusion-builtin | Exact median. Slower than approx_median on big inputs. | - | median | median | median |
Statistical / regression
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
variance(x) / var_samp(x) | datafusion-builtin | Sample variance. | variance / var_samp | variance_samp | variance / var_samp | variance / var_samp |
var_pop(x) | datafusion-builtin | Population variance. | var_pop | variance_pop | var_pop | var_pop |
stddev(x) / stddev_samp(x) | datafusion-builtin | Sample stddev. | stddev / stddev_samp | stddev_samp | stddev / stddev_samp | stddev |
stddev_pop(x) | datafusion-builtin | Population stddev. | stddev_pop | stddev_pop | stddev_pop | stddev_pop |
corr(y, x) | datafusion-builtin | Pearson correlation. | corr | corr | corr | corr |
covar_samp(y, x) / covar_pop(y, x) | datafusion-builtin | Sample / population covariance. | covar_samp / covar_pop | covar_samp / covar_pop | covar_samp / covar_pop | covar_samp / covar_pop |
regr_slope(y, x) | datafusion-builtin | Linear regression slope. | regr_slope | regr_slope | - | regr_slope |
regr_intercept(y, x) | datafusion-builtin | y-intercept. | regr_intercept | regr_intercept | - | regr_intercept |
regr_r2(y, x) | datafusion-builtin | R-squared. | regr_r2 | regr_r2 | - | regr_r2 |
regr_count, regr_sxx, regr_syy, regr_sxy, regr_avgx, regr_avgy | datafusion-builtin | Regression sums and counts. | yes | yes | - | yes |
Distinct and approximation
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
approx_distinct(expr) | datafusion-builtin | HyperLogLog distinct count. ~1% error. | approx_distinct | approx_count_distinct | approx_count_distinct | approx_count_distinct |
approx_median(expr) | datafusion-builtin | Median estimate via t-digest. | - | - | approx_percentile(0.5) | approx_quantile(0.5) |
approx_percentile_cont(expr, p) | datafusion-builtin | Percentile estimate via t-digest. p in [0, 1]. | approx_percentile | approx_percentile | approx_percentile | approx_quantile |
approx_percentile(expr, p) | sqe-trino-functions | Trino-named alias of approx_percentile_cont. trino_functions.rs:164 | approx_percentile | - | - | - |
Boolean and bitwise
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
bool_and(x) | datafusion-builtin | True only if every row is true. | bool_and | booland_agg | bool_and / every | bool_and |
bool_or(x) | datafusion-builtin | True if any row is true. | bool_or | boolor_agg | bool_or / any | bool_or |
every(x) | sqe-trino-functions | Trino-named alias of bool_and. trino_functions.rs:170 | every | - | every | - |
bit_and(x) | datafusion-builtin | Bitwise AND of all values. | bitwise_and_agg | bitand_agg | bit_and | - |
bit_or(x) | datafusion-builtin | Bitwise OR. | bitwise_or_agg | bitor_agg | bit_or | - |
bit_xor(x) | datafusion-builtin | Bitwise XOR. | bitwise_xor_agg | bitxor_agg | bit_xor | - |
bitwise_and_agg(x) | sqe-trino-functions | Trino name for bit_and. trino_functions.rs:144 | bitwise_and_agg | - | - | - |
bitwise_or_agg(x) | sqe-trino-functions | Trino name for bit_or. trino_functions.rs:150 | bitwise_or_agg | - | - | - |
bitwise_xor_agg(x) | sqe-trino-functions | Trino name for bit_xor. trino_functions.rs:158 | bitwise_xor_agg | - | - | - |
Positional
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
first_value(expr [order by ...]) | datafusion-builtin | First row’s value. Most useful with OVER ordering. | first_value | first_value | first_value | first_value |
last_value(expr [order by ...]) | datafusion-builtin | Last row’s value. | last_value | last_value | last_value | last_value |
nth_value(expr, n [order by ...]) | datafusion-builtin | Nth row’s value. | nth_value | nth_value | nth_value | nth_value |
max_by(value, key) | sqe-trino-functions | value from the row with the max key. trino_functions.rs:177 | max_by | max_by | - | arg_max |
min_by(value, key) | sqe-trino-functions | value from the row with the min key. trino_functions.rs:178 | min_by | min_by | - | arg_min |
arbitrary(expr) | sqe-trino-functions (ext) | Any one non-NULL value. Trino-named alias of any_value. trino_functions_ext.rs:68 | arbitrary | any_value | any_value | any_value |
Collection-building
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
array_agg(expr) | datafusion-builtin | Collect into an array. NULLs included. | array_agg | array_agg | collect_list | array_agg / list |
array_agg(distinct expr) | datafusion-builtin | Distinct array. | array_agg(distinct) | array_agg(distinct) | collect_set | list_distinct |
string_agg(expr, sep) | datafusion-builtin | Concatenate with separator. SQL standard. | array_join/listagg | listagg | - | string_agg |
listagg(expr, sep) | sqe-trino-functions | Same as string_agg; Snowflake / Trino name. trino_functions.rs:138 | listagg | listagg | - | - |
histogram(expr) | sqe-trino-functions | Map of value -> count. trino_functions.rs:188 | histogram | - | - | histogram |
map_agg(key, value) | sqe-trino-functions | Build a map by aggregating key-value pairs. Last write wins. trino_functions.rs:189 | map_agg | object_agg | map_from_arrays | map |
multimap_agg(key, value) | sqe-trino-functions | Map where each value is an array (collects duplicates). trino_functions.rs:190 | multimap_agg | - | - | - |
map_union(map_col) | sqe-trino-functions | Aggregate already-built maps into one. trino_functions.rs:191 | map_union | - | - | - |
Modifiers
| Modifier | Notes |
|---|---|
agg_func(expr) FILTER (WHERE pred) | Filter rows before aggregation. Cleaner than agg_func(CASE WHEN pred THEN expr END). |
agg_func(distinct expr) | Distinct values only. |
agg_func(expr) OVER (...) | Window form. Uses PARTITION BY, ORDER BY, frame clauses. See Window functions. |
agg_func(expr) WITHIN GROUP (ORDER BY ...) | Ordered aggregate (e.g. listagg). |
Example using FILTER:
SELECT
region,
count(*) AS total_orders,
count(*) FILTER (WHERE status = 'cancelled') AS cancelled,
sum(amount) FILTER (WHERE status = 'shipped') AS shipped_revenue
FROM orders
GROUP BY region;
GROUP BY extensions
| Construct | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
GROUP BY GROUPING SETS ((a, b), (a), ()) | datafusion-builtin | Multiple grouping levels in one query. | yes | yes | yes | yes |
GROUP BY CUBE (a, b, c) | datafusion-builtin | All 2^N grouping combinations. | yes | yes | yes | yes |
GROUP BY ROLLUP (a, b, c) | datafusion-builtin | Hierarchical: (), (a), (a, b), (a, b, c). | yes | yes | yes | yes |
GROUPING(col) | datafusion-builtin | Returns 1 if col was rolled up in this row, else 0. | yes | yes | yes | yes |
SELECT
region,
product,
sum(amount) AS revenue,
GROUPING(region) AS region_rolled_up,
GROUPING(product) AS product_rolled_up
FROM orders
GROUP BY ROLLUP (region, product)
ORDER BY region, product;
Approximation vs exact: when to choose
count(distinct)exact. sub-second on millions of rows; avoid above ~1B distinct values.approx_distinctHyperLogLog. order of magnitude faster on huge inputs. ~1% relative error.medianexact. sorts the entire group; expensive on big partitions.approx_median/approx_percentile_contt-digest. sub-percent error, much cheaper memory profile.
For dashboards over multi-billion-row tables, default to approximations. For audit queries that need exact counts, default to exact.
Window functions
Window functions compute a value per row using a “window” of related rows. Unlike aggregates, they do not collapse rows; the input row count is preserved.
All window functions in SQE come from datafusion-functions-window (DataFusion’s built-in window crate). No SQE-specific window functions exist; the SQL surface matches DataFusion exactly.
Syntax
window_function(args) OVER (
[PARTITION BY col1, col2, ...]
[ORDER BY col1 [ASC|DESC] [NULLS FIRST|LAST], ...]
[frame_clause]
)
The frame clause has three forms:
ROWS BETWEEN <start> AND <end>
RANGE BETWEEN <start> AND <end>
GROUPS BETWEEN <start> AND <end>
Bounds:
UNBOUNDED PRECEDING
N PRECEDING
CURRENT ROW
N FOLLOWING
UNBOUNDED FOLLOWING
Default frame:
- With
ORDER BY:RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW. - Without
ORDER BY:ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING.
Functions
Ranking
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
row_number() | datafusion-builtin | 1-based unique rank. | row_number | row_number | row_number | row_number |
rank() | datafusion-builtin | Standard rank with gaps after ties. | rank | rank | rank | rank |
dense_rank() | datafusion-builtin | Rank with no gaps. | dense_rank | dense_rank | dense_rank | dense_rank |
percent_rank() | datafusion-builtin | (rank - 1) / (rows - 1) in [0, 1]. | percent_rank | percent_rank | percent_rank | percent_rank |
cume_dist() | datafusion-builtin | Cumulative distribution: rows <= current / total. | cume_dist | cume_dist | cume_dist | cume_dist |
ntile(n) | datafusion-builtin | Bucket rows into N equal-size groups. | ntile | ntile | ntile | ntile |
Offset
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
lag(expr [, offset [, default]]) | datafusion-builtin | Value offset rows back. Default offset 1, default value NULL. | lag | lag | lag | lag |
lead(expr [, offset [, default]]) | datafusion-builtin | Value offset rows forward. | lead | lead | lead | lead |
first_value(expr) | datafusion-builtin | First row’s value within frame. | first_value | first_value | first_value | first_value |
last_value(expr) | datafusion-builtin | Last row’s value within frame. | last_value | last_value | last_value | last_value |
nth_value(expr, n) | datafusion-builtin | Nth row’s value within frame. | nth_value | nth_value | nth_value | nth_value |
Aggregates as windows
Every aggregate function from Aggregate functions also works as a window function:
SELECT
customer_id,
order_date,
amount,
sum(amount) OVER (PARTITION BY customer_id ORDER BY order_date) AS running_total,
avg(amount) OVER (PARTITION BY customer_id) AS customer_avg
FROM orders;
Frame examples
Running total
SELECT
order_date,
amount,
sum(amount) OVER (
ORDER BY order_date
ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
) AS running_total
FROM orders;
Trailing 7-day average
SELECT
order_date,
amount,
avg(amount) OVER (
ORDER BY order_date
RANGE BETWEEN INTERVAL '7' DAY PRECEDING AND CURRENT ROW
) AS trailing_7d_avg
FROM orders;
RANGE with an INTERVAL works on date / timestamp ordering keys and respects time gaps. ROWS would just count rows regardless of time.
Top N per group via row_number
WITH ranked AS (
SELECT
category,
product,
revenue,
row_number() OVER (PARTITION BY category ORDER BY revenue DESC) AS rn
FROM products
)
SELECT * FROM ranked WHERE rn <= 5;
Rolling difference with LAG
SELECT
order_date,
amount,
amount - lag(amount, 1, 0) OVER (ORDER BY order_date) AS day_over_day
FROM orders;
The , 0 argument fills the first row (where there is no predecessor) with zero instead of NULL.
Frame variants compared
| Form | What “between -1 and +1” means |
|---|---|
ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING | Three rows by position: previous, current, next. |
RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING | Rows within [order_key - 1, order_key + 1] of the current order key value. |
GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING | Three peer groups: rows tied with the current key, plus the previous and next tied groups. |
GROUPS is rare but useful when ordering on a low-cardinality key produces many ties and you want “the previous distinct value group” semantics.
What is NOT supported (DataFusion blocked)
QUALIFYclause (filtering on window-function output without a subquery). DataFusion’s parser does not acceptQUALIFY. Workaround: wrap the SELECT and filter in an outer query, as in the “Top N per group” example above.
The audit row lives in features.md. Tracked upstream as a parser enhancement.
Performance notes
PARTITION BYenables parallelism: each partition runs on its own thread / worker. Without partitioning, the window runs single-threaded against the global ordering.ROWSframes are cheaper thanRANGEframes when the ordering key has many ties;RANGEmay need a binary search per row.- A
unbounded preceding ... unbounded followingframe on a sorted input lets DataFusion stream-compute aggregates without materialising the partition. Other frames require partition-buffering.
The EXPLAIN ANALYZE output shows partition counts and frame mode per WindowAgg node; use it when a window query is slower than expected.
Table-valued functions
A TVF returns a table you can use in a FROM clause. SQE ships two families:
- File format readers:
read_parquet,read_csv,read_json,read_delta. Implemented insqe-catalog/. - Iceberg metadata readers:
table_snapshots,table_history,table_files,table_partitions,table_manifests,table_refs. Implemented insqe-catalog/src/iceberg_metadata_tvf.rs.
DataFusion contributes the generators (generate_series, unnest) and the URL-table auto-detect path (SELECT * FROM 'file.parquet').
File format TVFs
Detailed per-function docs: File-format TVFs, read_parquet TVF. Quick reference here.
| TVF | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
read_parquet(path, ...) | sqe-catalog | Parquet on local FS / S3 / HTTPS / hf://. Inline auth args. read_parquet.rs | Hive table only | infer_schema+stage | parquet source | read_parquet |
read_csv(path, ...) | sqe-catalog | DuckDB-style aliases (sep, delim, header, nullstr, compress). Smart defaults from extension. read_csv.rs | - | infer_schema+stage | csv source | read_csv |
read_json(path, ...) | sqe-catalog | NDJSON (one document per line). read_json.rs | - | - | json source | read_json |
read_delta(path, ...) | sqe-catalog | Read-only Delta Lake reader. Time travel via version => N or timestamp => 'RFC3339'. read_delta.rs | via connector | - | native | via extension |
SELECT * FROM 'file.ext' | datafusion-builtin | Quoted-string auto-detect. Dispatches by extension to one of the readers above. Requires enable_url_table(). | - | - | - | yes |
Common arguments
All four file readers accept the same path scheme set: local, S3, HTTPS, hf://. Arguments are positional path + named keyword arguments:
SELECT * FROM read_parquet(
's3://bucket/key.parquet',
access_key => 'AKIA...',
secret_key => '...',
endpoint => 'http://localhost:9000',
region => 'us-east-1'
);
The full keyword list per reader lives in File-format TVFs. The same shape works for read_csv, read_json, read_delta.
Path schemes
| Scheme | Auth | Example |
|---|---|---|
| Local | filesystem perms | /data/sales.parquet |
s3:// | inline args, [storage] block, or AWS provider chain (V10) | s3://bucket/key.parquet |
https:// | session bearer for HF, otherwise public | https://example.com/data.csv |
hf://datasets/<org>/<name>/... | HF_TOKEN env var, optional ?revision= | hf://datasets/squad/plain_text/train.parquet |
hf://[email protected]/... | revision inline (V12.1) | hf://datasets/foo/[email protected]/train.parquet |
hf://...@~parquet/... | auto-generated parquet view (V12.1) | hf://datasets/foo/bar@~parquet/default/train/0.parquet |
Iceberg metadata TVFs
Six TVFs that expose Iceberg internal state without leaving SQL. Useful for observability, audit, planning.
| TVF | Origin | Returns | Trino | Snowflake | Spark SQL |
|---|---|---|---|---|---|
table_snapshots(ns, table) | sqe-catalog | One row per snapshot. Columns: snapshot_id, parent_snapshot_id, timestamp_ms, operation, summary. iceberg_metadata_tvf.rs:93 | t$snapshots | - | t.snapshots |
table_history(ns, table) | sqe-catalog | Linear snapshot history. Columns: made_current_at, snapshot_id, parent_id, is_current_ancestor. iceberg_metadata_tvf.rs:356 | t$history | - | t.history |
table_files(ns, table) | sqe-catalog | One row per data file in the current snapshot. Columns: content, file_path, partition, record_count, file_size_in_bytes. iceberg_metadata_tvf.rs:469 | t$files | - | t.files |
table_manifests(ns, table) | sqe-catalog | One row per manifest in the current snapshot. iceberg_metadata_tvf.rs:217 | t$manifests | - | t.manifests |
table_partitions(ns, table) | sqe-catalog | Per-partition aggregate. iceberg_metadata_tvf.rs:622 | t$partitions | - | t.partitions |
table_refs(ns, table) | sqe-catalog | One row per branch / tag. Columns: name, type, snapshot_id, max_ref_age_ms. iceberg_metadata_tvf.rs:768 | t$refs | - | t.refs |
Trino’s table$snapshots syntax is also accepted; crates/sqe-coordinator/src/query_handler.rs rewrites it to the TVF call.
-- DuckDB / Trino-style $-syntax
SELECT * FROM analytics."events$snapshots";
-- Equivalent SQE TVF call
SELECT * FROM table_snapshots('analytics', 'events');
Examples
What’s the current snapshot’s row count?
SELECT SUM(record_count) AS rows
FROM table_files('analytics', 'events');
When did each branch fork?
SELECT name, type, snapshot_id
FROM table_refs('analytics', 'events')
WHERE type = 'branch';
How big are recent snapshots in megabytes?
SELECT
snapshot_id,
summary['added-files-size'] AS added_bytes,
summary['total-files-size'] AS total_bytes
FROM table_snapshots('analytics', 'events')
ORDER BY timestamp_ms DESC
LIMIT 10;
The summary column is a MAP<VARCHAR, VARCHAR>; cast values numerically when needed.
Generators (DataFusion built-ins)
| Function | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
generate_series(start, stop) | datafusion-builtin | Integer sequence, inclusive both ends. | sequence | sequence | sequence | generate_series |
generate_series(start, stop, step) | datafusion-builtin | With step. Negative step counts down. | sequence | - | - | generate_series |
range(start, stop) / range(start, stop, step) | datafusion-builtin | Half-open: includes start, excludes stop. | - | - | - | range |
unnest(array) | datafusion-builtin | Lateral expansion: one input row -> N output rows. | unnest | flatten | explode | unnest |
Examples:
SELECT * FROM generate_series(1, 5);
-- 1, 2, 3, 4, 5
SELECT day FROM generate_series(DATE '2026-05-01', DATE '2026-05-07') AS t(day);
-- 7 dates, May 1 through May 7
SELECT id, value FROM orders, UNNEST(items) AS t(value);
-- Lateral unnest: one row per (order, item)
Quoted-string auto-detect
SELECT * FROM '<path>' works as a shortcut when the path’s extension is recognised:
| Extension | Dispatches to |
|---|---|
.parquet | read_parquet |
.csv, .tsv, .psv, .ssv (with optional .gz / .bz2 / .xz / .zst) | read_csv |
.json, .jsonl, .ndjson (with optional codec suffix) | read_json |
.avro | DataFusion’s avro reader |
The mechanism is DataFusion’s enable_url_table() SessionConfig, called at crates/sqe-cli/src/embedded.rs:158. Auto-detect works in cluster mode too.
-- All three are equivalent (assuming the file is a CSV)
SELECT * FROM read_csv('/data/sales.csv');
SELECT * FROM '/data/sales.csv';
SELECT * FROM 'hf://datasets/squad/plain_text/train.csv';
When to register vs query directly
A read_* TVF call reads on every query. Two cases where registering as a table is better:
- Repeated queries: register once via
CREATE TABLE foo AS SELECT * FROM read_parquet(...)so subsequent queries skip the URL fetch. - You need writes: TVFs are read-only. Writes need a catalog-registered Iceberg table.
For ad-hoc analytics on a one-shot file, the TVF is faster: no schema decision, no commit, no metadata.
DDL
Data Definition Language: schemas, tables, views, columns, partitions, branches, tags. Most statements parse via sqlparser-rs; SQE adds branch / tag / partition-evolution syntax that sqlparser-rs does not natively understand.
Source: sqe-sql/src/classifier.rs, sqe-sql/src/ddl.rs, sqe-sql/src/partition.rs, sqe-sql/src/partition_evolution.rs. Coordinator handlers in crates/sqe-coordinator/src/catalog_ops.rs.
Schema
| Statement | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
CREATE SCHEMA [IF NOT EXISTS] cat.ns | sqlparser-rs + sqe-coordinator | Creates a namespace in the catalog. | yes | yes | yes | yes |
CREATE SCHEMA [IF NOT EXISTS] cat.ns LOCATION 's3://...' | sqlparser-rs + sqe-coordinator | Override default location. Only on catalogs that accept location at namespace level (Polaris, S3 Tables). | yes | yes | yes | - |
DROP SCHEMA [IF EXISTS] cat.ns [CASCADE|RESTRICT] | sqlparser-rs + sqe-coordinator | CASCADE drops contained tables. | yes | yes | yes | yes |
ALTER SCHEMA cat.ns RENAME TO new_name | sqlparser-rs + sqe-coordinator | Catalog must support namespace rename. | partial | yes | yes | yes |
CREATE SCHEMA IF NOT EXISTS analytics.staging;
CREATE SCHEMA marketing LOCATION 's3://my-warehouse/marketing/';
DROP SCHEMA staging CASCADE;
Table creation
| Statement | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
CREATE TABLE t (col TYPE [DEFAULT expr], ...) | sqlparser-rs + sqe-sql + sqe-coordinator | Iceberg V3 column defaults supported. | yes | yes | yes | yes |
CREATE TABLE t (...) PARTITIONED BY (transform(col), ...) | sqe-sql/partition.rs | Partition transforms: bucket(N, col), truncate(N, col), year(col), month(col), day(col), hour(col), identity (just col). | partial | - | yes | - |
CREATE TABLE t AS SELECT ... (CTAS) | sqlparser-rs + sqe-coordinator | Inferred schema; partitioning via WITH (partitioning = ARRAY['day(ts)']). | yes | yes | yes | yes |
CREATE OR REPLACE TABLE t AS SELECT ... | sqlparser-rs + sqe-coordinator | Atomic replace. New snapshot replaces the table; old data files retained until expire_snapshots. | yes | yes | partial | yes |
CREATE TABLE [IF NOT EXISTS] t LIKE other_table | sqlparser-rs + sqe-coordinator | Copy schema only, no data. | yes | yes | yes | yes |
CREATE TABLE analytics.events (
id BIGINT,
user_id BIGINT,
event_type VARCHAR,
occurred_at TIMESTAMP(6),
payload JSON,
region VARCHAR DEFAULT 'unknown'
)
PARTITIONED BY (day(occurred_at), bucket(16, user_id));
CREATE TABLE analytics.daily_events AS
SELECT day(occurred_at) AS d, count(*) AS n
FROM analytics.events GROUP BY 1;
Schema evolution
| Statement | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
ALTER TABLE t ADD COLUMN c TYPE [DEFAULT expr] | sqlparser-rs + sqe-coordinator | New column. Existing rows get the default (V3) or NULL (V2). | yes | yes | yes | yes |
ALTER TABLE t DROP COLUMN [IF EXISTS] c | sqlparser-rs + sqe-coordinator | Logical drop. Field id retained in old data files. | yes | yes | yes | yes |
ALTER TABLE t RENAME COLUMN old TO new | sqlparser-rs + sqe-coordinator | Iceberg field id stays the same; only the name changes. | yes | yes | yes | yes |
ALTER TABLE t ALTER COLUMN c SET NOT NULL | sqlparser-rs + sqe-coordinator | Tighten nullability. Fails if existing rows have NULL. | yes | yes | yes | yes |
ALTER TABLE t ALTER COLUMN c DROP NOT NULL | sqlparser-rs + sqe-coordinator | Loosen nullability. | yes | yes | yes | yes |
ALTER TABLE t ALTER COLUMN c SET DEFAULT expr | sqlparser-rs + sqe-coordinator | Iceberg V3 column default. | partial | yes | yes | yes |
ALTER TABLE t ALTER COLUMN c TYPE new_type | sqlparser-rs + sqe-coordinator | Type promotion only (e.g. INT -> BIGINT). Lossy changes rejected. | partial | partial | partial | partial |
ALTER TABLE t RENAME TO new_t | sqlparser-rs + sqe-coordinator | Catalog rename. Different catalog support varies. | yes | yes | yes | yes |
ALTER TABLE t SET TBLPROPERTIES (...) | sqlparser-rs + sqe-coordinator | Set Iceberg properties (e.g. write.delete.mode). | yes | yes | yes | - |
COMMENT ON TABLE t IS 'description' | sqlparser-rs + sqe-coordinator | Stored in Iceberg properties. | yes | yes | yes | yes |
COMMENT ON COLUMN t.c IS 'description' | sqlparser-rs + sqe-coordinator | Stored on the column metadata. | yes | yes | yes | yes |
ALTER TABLE analytics.events ADD COLUMN device VARCHAR DEFAULT 'unknown';
ALTER TABLE analytics.events DROP COLUMN IF EXISTS deprecated_field;
ALTER TABLE analytics.events RENAME COLUMN payload TO body;
ALTER TABLE analytics.events ALTER COLUMN region SET NOT NULL;
ALTER TABLE analytics.events SET TBLPROPERTIES (
'write.delete.mode' = 'merge-on-read',
'write.parquet.bloom-filter-columns' = 'user_id,event_id'
);
Partition evolution (SQE / Iceberg-specific)
Iceberg lets you change partition spec without rewriting data. SQE parses these in sqe-sql/src/partition_evolution.rs because sqlparser-rs only knows Hive-style PARTITION (col = val).
| Statement | Notes | Trino | Snowflake | Spark SQL |
|---|---|---|---|---|
ALTER TABLE t ADD PARTITION FIELD transform(col) | Add a new partition field. Existing data stays in the old spec. | partial | - | yes |
ALTER TABLE t ADD PARTITION FIELD transform(col) AS alias | Same with explicit name for the partition column. | - | - | yes |
ALTER TABLE t DROP PARTITION FIELD transform(col) | Remove a partition field from the current spec. | partial | - | yes |
ALTER TABLE t REPLACE PARTITION FIELD old_transform(col) WITH new_transform(col) | Replace one transform with another. | - | - | yes |
-- Originally partitioned by day(ts); switch to hour() for finer granularity.
ALTER TABLE events REPLACE PARTITION FIELD day(occurred_at) WITH hour(occurred_at);
-- Add a bucketing field on top of existing daily partitions.
ALTER TABLE events ADD PARTITION FIELD bucket(64, user_id);
Branches and tags (SQE / Iceberg-specific)
Iceberg branches are named pointers to a snapshot, like git branches. Tags are immutable named pointers. SQE parses these in sqe-sql/src/ddl.rs.
| Statement | Notes |
|---|---|
ALTER TABLE t CREATE BRANCH name | New branch from current snapshot. |
ALTER TABLE t CREATE BRANCH name AS OF VERSION snapshot_id | New branch from a specific snapshot. |
ALTER TABLE t CREATE BRANCH name WITH RETENTION (max_ref_age_ms = N) | Auto-expire branch after N ms of inactivity. |
ALTER TABLE t CREATE [OR REPLACE] TAG name | New tag pointing at current snapshot. OR REPLACE is allowed because tags are not strictly immutable in iceberg-rust. |
ALTER TABLE t CREATE TAG name AS OF VERSION snapshot_id | Tag a specific snapshot. |
ALTER TABLE t DROP BRANCH [IF EXISTS] name | Remove a branch. |
ALTER TABLE t DROP TAG [IF EXISTS] name | Remove a tag. |
-- Branch a snapshot for development work
ALTER TABLE analytics.events CREATE BRANCH dev_2026_05;
-- Pin a known-good snapshot as a tag
ALTER TABLE analytics.events CREATE TAG release_2026_q2 AS OF VERSION 8472810294;
-- Query the branch
SELECT * FROM analytics.events FOR VERSION AS OF 'dev_2026_05';
Views
| Statement | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
CREATE [OR REPLACE] VIEW v AS SELECT ... | sqlparser-rs + sqe-coordinator | Standard SQL view. Iceberg views format-version 1. | yes | yes | yes | yes |
CREATE [OR REPLACE] VIEW v (col1, col2) AS SELECT ... | sqlparser-rs + sqe-coordinator | Explicit column list. | yes | yes | yes | yes |
DROP VIEW [IF EXISTS] v | sqlparser-rs + sqe-coordinator | Remove a view. | yes | yes | yes | yes |
CREATE OR REPLACE VIEW analytics.recent_events AS
SELECT * FROM analytics.events
WHERE occurred_at >= now() - INTERVAL '7' DAY;
DROP VIEW IF EXISTS analytics.recent_events;
Drop
| Statement | Origin | Notes |
|---|---|---|
DROP TABLE [IF EXISTS] t [PURGE] | sqlparser-rs + sqe-coordinator | PURGE deletes data files immediately; default keeps the metadata so system.remove_orphan_files can clean later. |
DROP VIEW [IF EXISTS] v | sqlparser-rs + sqe-coordinator | Standard. |
DROP SCHEMA [IF EXISTS] s [CASCADE|RESTRICT] | sqlparser-rs + sqe-coordinator | CASCADE drops contained tables. |
Iceberg V3 type system
These types only exist in format-version 3. Adding one to a CREATE TABLE auto-bumps the table to V3.
| Type | Notes |
|---|---|
TIMESTAMP_NS, TIMESTAMP_NS WITH TIME ZONE | Nanosecond precision timestamps. Arrow Timestamp(Nanosecond, ...). |
GEOMETRY, GEOGRAPHY | Stub types in V3; SQE accepts them in CREATE but does not yet provide spatial functions. |
Default values via DEFAULT expr | Existing rows in older snapshots inherit the default at read time. |
What CREATE / ALTER does NOT cover
CREATE INDEX. Iceberg has no equivalent. Bloom filter columns and partition fields cover the same ground; configure viaSET TBLPROPERTIESandADD PARTITION FIELD.CREATE FUNCTION/CREATE PROCEDURE. UDFs are Rust-side. SQL-defined functions and procedures are not supported.CREATE SEQUENCE. no auto-increment / sequence support today. Userow_number()over a deterministic ordering for synthetic keys.CREATE TYPE. no user-defined types. UseSTRUCT<...>orMAP<...>.
These are tracked but not on the immediate roadmap.
DML
Data Manipulation Language: reads, writes, updates, deletes, merges. SQE adds Iceberg time-travel clauses on SELECT, an Iceberg-aware MERGE INTO, and a SET WRITE_BRANCH shortcut for routing writes to a named branch.
Source: sqe-sql/src/time_travel.rs, crates/sqe-coordinator/src/{query_handler, write_handler}.rs.
SELECT
| Form | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
SELECT cols FROM t [WHERE ...] [GROUP BY ...] [HAVING ...] [ORDER BY ...] [LIMIT N] | datafusion-builtin | Standard SQL. | yes | yes | yes | yes |
WITH cte AS (...) SELECT ... | datafusion-builtin | CTE. Multiple CTEs allowed. | yes | yes | yes | yes |
WITH RECURSIVE cte AS (...) SELECT ... | datafusion-builtin | Recursive CTE. | yes | yes | no | yes |
SELECT DISTINCT cols FROM t | datafusion-builtin | Distinct rows. | yes | yes | yes | yes |
SELECT * EXCLUDE (col, ...) FROM t | datafusion-builtin | Exclude columns from *. | yes | - | - | yes |
SELECT * REPLACE (expr AS col) FROM t | datafusion-builtin | Substitute one or more columns. | yes | - | - | yes |
SELECT cols FROM t1 [INNER|LEFT|RIGHT|FULL] JOIN t2 ON ... | datafusion-builtin | All join types including SEMI / ANTI. | yes | yes | yes | yes |
SELECT cols FROM t1 USING (col1, col2) | datafusion-builtin | Equality on shared column names. | yes | yes | yes | yes |
SELECT cols FROM t1, LATERAL (SELECT ... FROM t2 WHERE ...) | datafusion-builtin | Correlated subquery in FROM. | yes | yes | partial | yes |
SELECT cols FROM t TABLESAMPLE BERNOULLI (5) | datafusion-builtin | Random sampling. | yes | yes | yes | yes |
Time travel (Iceberg-specific)
| Form | Origin | Notes |
|---|---|---|
SELECT ... FROM t FOR VERSION AS OF snapshot_id | sqe-sql/time_travel.rs | Read a specific snapshot (snapshot id, branch name, or tag name). |
SELECT ... FROM t FOR SYSTEM_TIME AS OF timestamp | datafusion-builtin (sqlparser native) | Read snapshot active at the given timestamp. |
SELECT ... FROM t FOR INCREMENTAL BETWEEN SNAPSHOT s1 AND SNAPSHOT s2 | sqe-sql/time_travel.rs | SQE-specific. Returns rows added between two snapshots; useful for CDC-style processing. |
-- By snapshot id
SELECT * FROM events FOR VERSION AS OF 8472810294;
-- By branch
SELECT * FROM events FOR VERSION AS OF 'dev_2026_05';
-- By tag
SELECT * FROM events FOR VERSION AS OF 'release_2026_q2';
-- By timestamp
SELECT * FROM events FOR SYSTEM_TIME AS OF TIMESTAMP '2026-04-01 00:00:00';
-- Incremental between two snapshots
SELECT * FROM events
FOR INCREMENTAL BETWEEN SNAPSHOT 1234 AND SNAPSHOT 5678;
INSERT
| Form | Origin | Notes |
|---|---|---|
INSERT INTO t (cols) VALUES (...), (...) | sqlparser-rs + sqe-coordinator | Multi-row literal insert. |
INSERT INTO t SELECT ... FROM s | sqlparser-rs + sqe-coordinator | Insert from query. |
INSERT INTO t (col1, col2) SELECT ... FROM s | sqlparser-rs + sqe-coordinator | Subset of columns; others get DEFAULT or NULL. |
INSERT OVERWRITE t SELECT ... FROM s | sqlparser-rs + sqe-coordinator | Replace partition or table data. Targets the partitions implied by the SELECT. |
INSERT INTO events VALUES
(1, 'click', TIMESTAMP '2026-05-08 09:00:00'),
(2, 'view', TIMESTAMP '2026-05-08 09:01:00');
INSERT INTO events (id, event_type, occurred_at)
SELECT id, kind, ts FROM staging.raw_events;
UPDATE
| Form | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
UPDATE t SET col = expr [WHERE pred] | sqlparser-rs + sqe-coordinator | CoW or MoR by table property. | yes | yes | yes | yes |
UPDATE t SET col1 = e1, col2 = e2 [WHERE pred] | sqlparser-rs + sqe-coordinator | Multi-column set. | yes | yes | yes | yes |
UPDATE t SET col = expr FROM other o WHERE t.k = o.k | sqlparser-rs + sqe-coordinator | Update from another table. | partial | yes | yes | yes |
UPDATE orders
SET status = 'shipped', shipped_at = now()
WHERE tracking_id IS NOT NULL;
DELETE
| Form | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
DELETE FROM t [WHERE pred] | sqlparser-rs + sqe-coordinator | CoW (default) or MoR (write.delete.mode = 'merge-on-read'). | yes | yes | yes | yes |
DELETE FROM t USING other o WHERE t.k = o.k | sqlparser-rs + sqe-coordinator | Delete by join. | partial | yes | yes | yes |
TRUNCATE TABLE t | sqe-sql/classifier.rs | Rewrites to DELETE FROM t. Same MoR / CoW behaviour. | yes | yes | yes | yes |
DELETE FROM events WHERE event_type = 'spam';
TRUNCATE TABLE staging.tmp;
MERGE
| Form | Origin | Notes |
|---|---|---|
MERGE INTO t USING s ON cond WHEN MATCHED THEN UPDATE SET ... WHEN NOT MATCHED THEN INSERT (...) VALUES (...) | sqlparser-rs + sqe-coordinator | CoW or MoR. Multiple WHEN MATCHED branches with extra predicates allowed. |
MERGE INTO t USING s ON cond WHEN MATCHED THEN DELETE | sqlparser-rs + sqe-coordinator | Delete matched rows. |
MERGE INTO t USING s ON cond WHEN MATCHED AND pred THEN UPDATE SET ... WHEN MATCHED THEN DELETE | sqlparser-rs + sqe-coordinator | Conditional MATCHED branches. |
MERGE INTO orders t
USING staging.order_updates s
ON t.id = s.id
WHEN MATCHED AND s.status = 'cancelled' THEN DELETE
WHEN MATCHED THEN UPDATE SET status = s.status, updated_at = s.updated_at
WHEN NOT MATCHED THEN INSERT (id, status, created_at) VALUES (s.id, s.status, s.created_at);
Copy-on-Write vs Merge-on-Read
Three table properties control write semantics:
| Property | Default | Effect |
|---|---|---|
write.delete.mode | 'copy-on-write' | Switch to 'merge-on-read' to write position / equality delete files instead of rewriting whole data files. |
write.update.mode | 'copy-on-write' | Same options. MoR UPDATE writes both deletes and inserts. |
write.merge.mode | 'copy-on-write' | Same options. |
Set per-table via ALTER TABLE:
ALTER TABLE orders SET TBLPROPERTIES (
'write.delete.mode' = 'merge-on-read',
'write.update.mode' = 'merge-on-read',
'write.merge.mode' = 'merge-on-read'
);
When to choose:
- CoW: small tables, infrequent writes, predictable read latency. Default.
- MoR: large tables, frequent small deletes, willing to trade read amplification for write speed. Compact periodically with
system.rewrite_data_files.
COPY TO
| Form | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
COPY (SELECT ...) TO 'path' (FORMAT csv|json|parquet) | datafusion-builtin | Write query result to a file. | - | yes | - | yes |
COPY t TO 'path' (FORMAT parquet) | datafusion-builtin | Write whole table. | - | yes | - | yes |
COPY (SELECT ...) TO 'path' (FORMAT parquet, PARTITION_BY 'col1, col2') | datafusion-builtin | Hive-style partitioned output. | - | yes | - | yes |
COPY (SELECT * FROM events WHERE occurred_at >= DATE '2026-05-01')
TO '/tmp/may_events.parquet'
(FORMAT parquet);
COPY events TO 's3://export/events.csv'
(FORMAT csv, COMPRESSION gzip, HEADER true);
Branch routing (SQE-specific)
Iceberg branches let you isolate writes from the production snapshot. SQE exposes branch routing as a session variable.
| Statement | Notes |
|---|---|
SET WRITE_BRANCH = 'name' | Subsequent INSERT / UPDATE / DELETE / MERGE writes go to the named branch. |
SET WRITE_BRANCH = DEFAULT | Reset to the default (main) branch. |
SET WRITE_BRANCH = NULL | Same as DEFAULT. |
ALTER TABLE events CREATE BRANCH staging;
SET WRITE_BRANCH = 'staging';
INSERT INTO events SELECT * FROM new_data;
-- ^ writes to the 'staging' branch only
SELECT count(*) FROM events FOR VERSION AS OF 'staging';
-- ^ reads from staging branch
SET WRITE_BRANCH = DEFAULT;
Session control
| Statement | Origin | Notes |
|---|---|---|
USE catalog.schema | sqe-sql/classifier.rs | Switch active catalog and schema. Subsequent unqualified table references use this scope. |
USE schema | sqe-sql/classifier.rs | Switch schema only. |
SET <variable> = <value> | sqlparser-rs (DataFusion) | DataFusion session config. See EXPLAIN ANALYZE documentation for valid keys. |
BEGIN / COMMIT / ROLLBACK | sqe-sql/classifier.rs | No-op stubs for JDBC compatibility. SQE does not implement multi-statement transactions; each commit is single-statement. |
Comparison summary
| Operation | SQE | Trino + Iceberg | Spark + Iceberg | DuckDB |
|---|---|---|---|---|
SELECT time travel | yes (FOR VERSION AS OF, branch / tag / id) | yes | yes | partial |
| Incremental SELECT | FOR INCREMENTAL BETWEEN (SQE-specific) | partial via Iceberg incremental APIs | yes | - |
INSERT INTO | yes | yes | yes | yes |
UPDATE (CoW + MoR) | yes | yes | yes | yes |
DELETE (CoW + MoR) | yes | yes | yes | yes |
MERGE INTO (CoW + MoR) | yes | yes | yes | - |
TRUNCATE TABLE | yes (rewrites to DELETE) | yes | yes | yes |
COPY TO | yes | - | - | yes |
| Branch-routed writes | SET WRITE_BRANCH (SQE) | partial | yes | - |
| Multi-statement transactions | no (no-op stubs) | no | no | yes |
CALL procedures
Iceberg maintenance operations exposed as SQL CALL statements. SQE wraps the vendored iceberg-rust action APIs (RewriteFilesAction, RemoveSnapshotAction, RewriteManifestsAction) and adds an SQE-specific bloom-filter suggestion procedure that walks recent query history.
All procedures use Iceberg’s named-argument syntax: CALL system.<proc>(name => value, ...). Unknown argument names raise a parse error so typos fail fast.
Source: sqe-sql/src/procedures.rs. Handlers in crates/sqe-coordinator/src/maintenance.rs.
Reference
| Procedure | Origin | Required args | Optional args | Notes |
|---|---|---|---|---|
system.rewrite_data_files | sqe-sql + sqe-coordinator | table => 'ns.t' | target_file_size_bytes => N, min_input_files => N, max_concurrent_file_group_rewrites => N | Bin-packs small data files into larger ones. Default target 512 MiB, min 5 files per group, max 4 concurrent groups. |
system.expire_snapshots | sqe-sql + sqe-coordinator | table => 'ns.t' | older_than => TIMESTAMP, retain_last => N | Drops old snapshots. older_than and retain_last combine: a snapshot must be older than older_than and beyond the retain_last window before it is removed. |
system.remove_orphan_files | sqe-sql + sqe-coordinator | table => 'ns.t' | older_than => TIMESTAMP | Deletes files under the table prefix not referenced by any live snapshot. Default older_than is 3 days ago, to avoid racing with in-flight writes. |
system.rewrite_manifests | sqe-sql + sqe-coordinator | table => 'ns.t' | - | Consolidates many small manifest files into fewer larger ones. Speeds up planning on large tables. |
system.suggest_bloom_filter_columns | sqe-sql + sqe-coordinator | table => 'ns.t' | history_limit => N | SQE-specific. Walks the last N finished queries (default 1000), counts equality predicates per column, returns ranked suggestions for write.parquet.bloom-filter-columns. |
Comparison to other engines
| Procedure | SQE | Trino + Iceberg | Spark + Iceberg | DuckDB |
|---|---|---|---|---|
| Compact small files | CALL system.rewrite_data_files(...) | ALTER TABLE t EXECUTE optimize | CALL t.system.rewrite_data_files(...) | - |
| Expire old snapshots | CALL system.expire_snapshots(...) | ALTER TABLE t EXECUTE expire_snapshots(...) | CALL t.system.expire_snapshots(...) | - |
| Remove orphans | CALL system.remove_orphan_files(...) | ALTER TABLE t EXECUTE remove_orphan_files(...) | CALL t.system.remove_orphan_files(...) | - |
| Rewrite manifests | CALL system.rewrite_manifests(...) | ALTER TABLE t EXECUTE optimize_manifests | CALL t.system.rewrite_manifests(...) | - |
| Suggest bloom filters | CALL system.suggest_bloom_filter_columns(...) | - | - | - |
The Spark and SQE shapes are aligned: Spark uses t.system.<proc> (table-qualified), SQE uses system.<proc>(table => 'ns.t') (named arg). Both are explicit. Trino prefers EXECUTE-as-DDL syntax which is harder to script.
Examples
Compact a partitioned fact table
CALL system.rewrite_data_files(
table => 'analytics.events',
target_file_size_bytes => 268435456, -- 256 MiB
min_input_files => 8
);
Returns one summary row:
+----------------------+----------------------+----------------------+
| files_rewritten | bytes_rewritten | snapshot_id |
+----------------------+----------------------+----------------------+
| 142 | 39283744832 | 8472810294 |
+----------------------+----------------------+----------------------+
Drop snapshots older than 30 days, keeping the last 10
CALL system.expire_snapshots(
table => 'analytics.events',
older_than => TIMESTAMP '2026-04-08 00:00:00',
retain_last => 10
);
The retain_last floor is enforced even when older_than would clear more. Useful for keeping a rollback budget while clamping storage growth.
Bloom filter suggestion before a tuning pass
CALL system.suggest_bloom_filter_columns(
table => 'analytics.events',
history_limit => 5000
);
Returns one row per column with a positive equality-predicate count, ranked descending:
+----------+-------------------+------------------+
| column | equality_pred_hits | recommendation |
+----------+-------------------+------------------+
| user_id | 4823 | strongly suggested |
| event_id | 1241 | suggested |
| device | 312 | weak |
+----------+-------------------+------------------+
Apply with:
ALTER TABLE analytics.events SET TBLPROPERTIES (
'write.parquet.bloom-filter-columns' = 'user_id,event_id'
);
The next write picks up the new property; existing files are unaffected until rewritten.
Combined maintenance run
-- Once a week, in this order:
CALL system.expire_snapshots(table => 'analytics.events',
older_than => TIMESTAMP '2026-04-08 00:00:00', retain_last => 30);
CALL system.remove_orphan_files(table => 'analytics.events',
older_than => TIMESTAMP '2026-04-08 00:00:00');
CALL system.rewrite_manifests(table => 'analytics.events');
CALL system.rewrite_data_files(table => 'analytics.events');
Order matters: expire snapshots before removing orphan files (otherwise files referenced by snapshots about to expire look orphaned), and rewrite manifests before rewriting data files (so the rewrite plan reads compact manifests).
Permissions
Procedures inherit the calling user’s grants on the target table:
system.rewrite_data_files,system.rewrite_manifestsneedMODIFY(writes new files, commits a snapshot).system.expire_snapshots,system.remove_orphan_filesneedMODIFYandDROP(alters retention, deletes files).system.suggest_bloom_filter_columnsis read-only against query history;SELECTon the table is enough.
A user without the right grant gets a clear “policy denied” error instead of a generic execution failure.
Safety notes
remove_orphan_fileswith noolder_thanuses the 3-day default, which is conservative against compaction or COPY jobs in flight. Override witholder_thanonly after confirming no concurrent writers.expire_snapshotsis destructive for time-travel queries. Once a snapshot is expired,FOR VERSION AS OF <id>for that snapshot fails. Document a retention window your team agrees on, and stick to it.rewrite_data_filesrewrites entire data files, not row groups. Two consecutive calls can churn the same files; rely on themin_input_filesfloor (default 5) to keep churn bounded.
What is not exposed
The vendored iceberg-rust crate has more transaction actions than SQE wires up. Notable omissions:
expire_snapshots_by_id(drop a specific snapshot rather than by age). easy to add if needed.rewrite_position_deletes(compact MoR delete files). not yet wrapped; on the V13 backlog.cherrypick_snapshot(apply a non-current snapshot’s changes to the head). out of scope for now; rare use case.
File an issue if you hit one of these in production.
GRANT and REVOKE
Chameleon / SBP-specific. The access-control SQL surface on this page (column masks, row filters, effective-grant inspection,
CHECK ACCESS) is an SQE security extension built for the Chameleon platform. It is not part of the core open-source Iceberg SQL surface, and the grant backend is pluggable: SQE ships a Polaris backend and a Chameleon backend. A default OSS deployment can run without it. It is documented here for completeness; treat it as an optional, platform-specific layer.
SQE-specific security extensions on top of the SQL standard GRANT / REVOKE. The base shapes are parsed by sqlparser-rs; SQE adds:
- Column masks:
GRANT SELECT ON ... TO ... MASKED WITH expr. - Row filters:
GRANT SELECT ON ... TO ... ROWS WHERE expr. - Effective-grant inspection:
SHOW EFFECTIVE GRANTS FOR USER "x"returns the resolved policy for a user across roles and inheritance. - Resource-scoped listing:
SHOW GRANTS ON ns.table. - Pre-flight check:
CHECK ACCESS SELECT ON ns.table FOR USER "x"returns boolean without executing.
These extensions are parsed in sqe-sql/src/classifier.rs (pre-parser scan) and enforced by the policy engine in sqe-policy/. The plan rewriter injects row filters above TableScan and substitutes column masks before DataFusion’s optimizer runs, so the optimizer cannot push user predicates through a mask.
The full design lives in docs/openspec.md. This page is the SQL surface reference.
Privileges
| Privilege | Applies to | Effect |
|---|---|---|
SELECT | table, view, schema, catalog | Read rows. Combines with row filters and column masks. |
INSERT | table | Append new rows. |
UPDATE | table | Modify existing rows. |
DELETE | table | Remove rows. |
MODIFY | table | Shorthand for INSERT + UPDATE + DELETE + MERGE. Required by maintenance procedures. |
DROP | table, schema | Required by DROP TABLE, DROP SCHEMA, system.expire_snapshots. |
CREATE | schema, catalog | Required to create new tables / schemas. |
ALL PRIVILEGES | any | Every privilege on the resource. |
Grantee types
| Type | Syntax | Source |
|---|---|---|
| User | TO USER "alice" | OIDC subject claim. |
| Role | TO ROLE "analyst" | Group claim from the OIDC provider, or a manually mapped role. |
| Public | TO PUBLIC | Every authenticated user. Avoid in production. |
Statements
Standard GRANT / REVOKE
GRANT SELECT ON analytics.events TO ROLE "analyst";
GRANT INSERT, UPDATE ON staging.tmp TO USER "etl";
GRANT ALL PRIVILEGES ON SCHEMA analytics TO ROLE "data_engineer";
REVOKE INSERT ON staging.tmp FROM USER "etl";
The standard form is parsed by sqlparser-rs and routed via StatementKind::Grant / StatementKind::Revoke.
Column masks (SQE extension)
GRANT SELECT (id, name, email)
ON users
TO ROLE "support"
MASKED WITH (
email = sha256(email)
);
The MASKED WITH clause is post-parse: SQE walks the AST after sqlparser succeeds and lifts the trailing extension into a PolicyStatement node. Anyone with the support role sees the masked email; the unmasked column never reaches the user’s session. Plan optimization happens after the substitution so a WHERE email = '[email protected]' predicate cannot bypass the mask.
Row filters (SQE extension)
GRANT SELECT ON orders TO ROLE "regional_eu"
ROWS WHERE region = 'EU';
The filter expression is injected as a Filter node directly above TableScan for the orders reference. DataFusion’s predicate pushdown can move user WHERE clauses through the row filter (because filters compose), but cannot eliminate it.
SHOW GRANTS
| Form | What it returns |
|---|---|
SHOW GRANTS ON ns.table | All grants on the resource. |
SHOW GRANTS ON SCHEMA ns | All grants on the schema. |
SHOW GRANTS TO USER "alice" | Direct grants to the user (does not include role-inherited). |
SHOW GRANTS TO ROLE "analyst" | Direct grants to the role. |
SHOW EFFECTIVE GRANTS FOR USER "alice" | Resolved policy: direct grants + role-inherited + masks + row filters. The view a query planner uses. |
sqe> SHOW EFFECTIVE GRANTS FOR USER "alice";
+------------------+--------+-----------+--------------+----------------------+
| resource | privilege | grantee | row_filter | column_masks |
+------------------+--------+-----------+--------------+----------------------+
| analytics.events | SELECT | role "an" | region='EU' | none |
| users | SELECT | role "su" | none | email -> sha256(...) |
+------------------+--------+-----------+--------------+----------------------+
CHECK ACCESS
A pre-flight test. Returns boolean without executing the query.
CHECK ACCESS SELECT ON analytics.events FOR USER "alice";
-- true
CHECK ACCESS DELETE ON analytics.events FOR USER "alice";
-- false
Useful in scripts that want to bail out before a long-running query if the user lacks permission, and in the test suite to verify policy logic.
Comparison
| Feature | SQE | Trino + Iceberg | Spark + Iceberg | DuckDB |
|---|---|---|---|---|
GRANT / REVOKE (SQL standard) | yes | yes (with Ranger) | yes (Ranger / Lake Formation) | no |
| Column masks | GRANT ... MASKED WITH | external (Ranger) | external (Ranger) | no |
| Row filters | GRANT ... ROWS WHERE | external (Ranger) | external (Ranger) | no |
SHOW EFFECTIVE GRANTS | yes | no | no | no |
CHECK ACCESS (pre-flight) | yes | no | no | no |
| Per-user OIDC bearer to storage | yes | no (service account) | no (service account) | no |
| Plan-level enforcement | yes (rewriter) | external middleware | external middleware | no |
The structural difference: SQE keeps policy in-engine, plan-rewritten before optimization, and tied to the per-query bearer token. Trino and Spark push the responsibility to Apache Ranger, which lives outside the engine and intercepts at the connector boundary.
Backends
PolicyStore is pluggable. Three implementations ship today:
| Backend | Use case | Where it lives |
|---|---|---|
InMemory | Single-node dev, tests. Grants stored in a hash map. | sqe-policy/src/in_memory_store.rs |
Postgres | Cluster mode default. Grants persisted in a tenant-scoped table. | sqe-policy/src/postgres_store.rs |
OPA (Open Policy Agent) | Rego-based policy. The store sends the resolved query plan + identity to OPA, OPA returns row filters / column masks as JSON. | sqe-policy/src/opa_store.rs |
Cedar | AWS Cedar-language policy. Same shape as OPA. | sqe-policy/src/cedar_store.rs |
Pick a backend in [security.policy] of the engine config:
[security.policy]
backend = "postgres" # or "opa", "cedar", "in_memory"
url = "postgres://policy_db"
OPA / Cedar add a network round trip per query but let the security team author policy in their language of choice.
Why plan rewriting, not connector hooks
A short rationale (full reasoning in docs/openspec.md):
- Optimization safety: filters added above
TableScansurvive predicate pushdown but cannot be eliminated. Connector-level hooks run after planning and can be bypassed by a clever WHERE clause. - Information leakage: a user querying
column_that_is_masked = 'secret'gets zero rows, exactly as if the column did not exist. PostgreSQL RLS uses the same model. - Auditability: the rewritten plan is logged. Reviewers see exactly what filter was applied per query, per user.
- Composability: row filters from multiple grants AND together; column masks from multiple grants are applied innermost-out. The semantics are explicit instead of implementation-defined.
Known gaps
- No
WITH GRANT OPTION. Grants are non-delegating; only an admin can grant. - No column-level INSERT (
GRANT INSERT (col1, col2) ON ...). The granularity is table-level for INSERT today. - Mask expressions are scalar only; aggregate / table-valued mask expressions are not allowed.
File an issue if any of these block your use case.
SHOW and EXPLAIN
Metadata queries (catalog / schema / table listing) and plan inspection (EXPLAIN, EXPLAIN ANALYZE, EXPLAIN FULL). Most are routed through the coordinator; EXPLAIN FULL is SQE-specific.
Source: sqe-sql/src/classifier.rs (statement routing), crates/sqe-coordinator/src/query_handler.rs (handlers).
SHOW statements
| Statement | Origin | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|---|
SHOW CATALOGS | sqe-sql/classifier.rs:154 | Lists every catalog the session can see. Honours auth: catalogs the user has no SELECT on are filtered. | yes | - | yes | yes |
SHOW SCHEMAS [IN cat] | sqlparser-rs + sqe-coordinator | List namespaces. Filters by catalog if IN supplied. | yes | yes | yes | yes |
SHOW TABLES [IN cat.ns] | sqlparser-rs + sqe-coordinator | List tables. | yes | yes | yes | yes |
SHOW VIEWS [IN cat.ns] | sqlparser-rs + sqe-coordinator | List views. | yes | yes | yes | partial |
SHOW COLUMNS FROM cat.ns.t | sqe-coordinator/query_handler.rs:1858 | Trino syntax. Rewrites to information_schema.columns query. | yes | yes | yes | yes |
SHOW CREATE TABLE cat.ns.t | sqe-sql/classifier.rs | Reconstruct the CREATE statement from current metadata. | yes | yes | yes | yes |
SHOW STATS FOR cat.ns.t | sqe-sql/classifier.rs:166 | Per-column NDV, null fraction, min, max. From Iceberg manifest stats. | yes | - | partial | yes |
DESCRIBE cat.ns.t | datafusion-builtin | Three-column projection: column_name, data_type, is_nullable. | yes | yes | yes | yes |
SHOW GRANTS ON ... | sqe-sql/classifier.rs:186 | See GRANT and REVOKE. | partial | yes | partial | - |
SHOW EFFECTIVE GRANTS FOR USER "x" | sqe-sql/classifier.rs:174 | SQE-specific. See GRANT and REVOKE. | - | - | - | - |
sqe> SHOW CATALOGS;
+---------------+
| catalog_name |
+---------------+
| default |
| analytics |
| iceberg_main |
+---------------+
sqe> SHOW TABLES IN analytics;
+--------------+--------------+--------------+
| table_catalog | table_schema | table_name |
+--------------+--------------+--------------+
| analytics | public | events |
| analytics | public | users |
| analytics | staging | tmp_dedup |
+--------------+--------------+--------------+
DESCRIBE vs SHOW COLUMNS
Both work, slightly different shapes:
DESCRIBE analytics.events;
-- column_name | data_type | is_nullable
SHOW COLUMNS FROM analytics.events;
-- column_name | data_type | is_nullable | extra
DESCRIBE is DataFusion-native (3 columns). SHOW COLUMNS is Trino syntax, rewritten by SQE to query information_schema.columns directly so external dbt models that expect 4 columns work unmodified.
SHOW STATS
Per-column statistics from manifest aggregates. Unlike DESCRIBE, this returns one row per column with summary numbers:
sqe> SHOW STATS FOR analytics.events;
+--------------+--------------+--------------+----------------+--------+--------+
| column_name | data_size | distinct | null_fraction | min | max |
+--------------+--------------+--------------+----------------+--------+--------+
| id | 96000000 | 12000000 | 0.0 | 1 | 12000000 |
| user_id | 96000000 | 8473210 | 0.0 | 1 | 9999 |
| amount | 144000000 | 9921458 | 0.001 | -50.00 | 12500.00 |
| occurred_at | 96000000 | 11973247 | 0.0 | 2024-..| 2026-...|
+--------------+--------------+--------------+----------------+--------+--------+
distinct and bounds are upper bounds from manifest stats, not exact. For exact counts use count(distinct col) or .summarize. The output drives planner cost estimates.
EXPLAIN
| Statement | Origin | Notes |
|---|---|---|
EXPLAIN SELECT ... | datafusion-builtin | Logical and physical plans, no execution. |
EXPLAIN ANALYZE SELECT ... | datafusion-builtin | Run the query; show physical plan with per-operator metrics. |
EXPLAIN FULL SELECT ... | sqe-sql/classifier.rs:159 | SQE-specific. Logical plan + physical plan + Iceberg scan plan (manifest counts, file counts, partition pruning, residual filter), no execution. |
EXPLAIN is the cheapest:
EXPLAIN SELECT user_id, count(*) FROM events
WHERE occurred_at >= DATE '2026-05-01' GROUP BY user_id;
+---------------+--------------------------------------------------------------+
| plan_type | plan |
+---------------+--------------------------------------------------------------+
| logical_plan | Projection: user_id, count(*) |
| | Aggregate: groupBy=[user_id], aggr=[count(*)] |
| | Filter: occurred_at >= Date32("2026-05-01") |
| | TableScan: events |
| physical_plan | ProjectionExec ... |
| | AggregateExec ... |
| | CoalesceBatchesExec ... |
| | FilterExec ... |
| | IcebergScanExec(events): files=12, bytes=180MB |
+---------------+--------------------------------------------------------------+
EXPLAIN ANALYZE runs the query and overlays per-operator counters:
| physical_plan | ProjectionExec, metrics=[output_rows=4823, elapsed=12ms]
| | AggregateExec, metrics=[output_rows=4823, elapsed=42ms]
| | IcebergScanExec, metrics=[files=12, files_pruned=0, bytes=180MB, elapsed=89ms]
EXPLAIN FULL shows the iceberg planning detail without executing:
| iceberg_plan | files_total=120, files_after_partition_prune=12, |
| | files_after_min_max_prune=12, residual_filter=true |
| | bytes_planned=180MB, partition_columns=[day(occurred_at)] |
Comparison
| Statement | SQE | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|
EXPLAIN | yes | yes | yes (EXPLAIN) | yes | yes |
EXPLAIN ANALYZE | yes | yes (EXPLAIN ANALYZE) | partial (query profile) | yes | yes |
EXPLAIN FULL (planning detail w/o exec) | yes (SQE-specific) | partial | - | partial | - |
SHOW STATS | yes | yes | partial (information_schema) | partial | yes |
Information schema (DataFusion-native)
Always available; standard SQL surface.
| Table | Notes |
|---|---|
information_schema.schemata | Schemas in every catalog. |
information_schema.tables | Tables in every catalog. |
information_schema.columns | Per-column metadata. |
information_schema.views | Views. |
information_schema.df_settings | DataFusion session config. |
SELECT table_schema, table_name
FROM information_schema.tables
WHERE table_catalog = 'analytics' AND table_type = 'BASE TABLE';
The dotcommands .tables, .schema, .catalogs are convenience wrappers around these.
Iceberg metadata
For Iceberg-specific metadata (snapshots, manifests, files, partitions, refs, history), see Table-valued functions. Both SQE TVF syntax and Trino t$snapshots syntax are accepted.
Operators
Built-in SQL operators. All from DataFusion’s parser; SQE adds none of its own. The list is here for completeness so users do not have to cross-reference the DataFusion docs for the basics.
Arithmetic
| Operator | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|
+ | Add. Numeric or interval. | yes | yes | yes | yes |
- | Subtract. Numeric, interval, or unary negation. | yes | yes | yes | yes |
* | Multiply. | yes | yes | yes | yes |
/ | Divide. Integer / Integer returns Double in DataFusion. | yes | yes | yes | yes |
% | Modulo. Integer or numeric. Same as mod(). | yes | yes | yes | yes |
^ | Not exponentiation in DataFusion. Use pow(x, y). | - | - | - | yes |
SELECT 10 + 5, -- 15
10 - 5, -- 5
10 * 5, -- 50
10 / 3, -- 3.333... (Double)
10 % 3, -- 1
-10 -- unary minus
;
For integer division use floor(a / b) or div(a, b) (DataFusion).
String
| Operator | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|
|| | Concatenate. NULL propagates. | yes | yes | yes | yes |
LIKE | Pattern with _ and %. | yes | yes | yes | yes |
ILIKE | Case-insensitive LIKE. | yes | yes | partial | yes |
NOT LIKE / NOT ILIKE | Negated. | yes | yes | yes | yes |
SIMILAR TO | SQL/POSIX-light regex. | yes | yes | - | yes |
~ / ~* | Regex match (case-sensitive / insensitive). | - | yes | - | yes |
!~ / !~* | Negated regex. | - | yes | - | yes |
SELECT name FROM users WHERE email ILIKE '%@example.com';
SELECT * FROM logs WHERE message ~ '^ERROR:';
Comparison
| Operator | Notes |
|---|---|
=, <>, != | Equal, not-equal. NULL propagates (returns NULL, not true / false). |
<, <=, >, >= | Ordering. |
BETWEEN x AND y | Inclusive range. NULL propagates. |
NOT BETWEEN x AND y | Negated. |
IS DISTINCT FROM | Like <> but treats NULL = NULL as false (i.e. NULLs are equal). |
IS NOT DISTINCT FROM | Like = but treats NULL = NULL as true. |
-- These differ on NULLs
SELECT a = b FROM (VALUES (1, NULL)) AS t(a, b); -- NULL
SELECT a IS NOT DISTINCT FROM b FROM (VALUES (1, NULL)) AS t(a, b); -- false
SELECT a IS NOT DISTINCT FROM b FROM (VALUES (NULL, NULL)) AS t(a, b); -- true
IS NOT DISTINCT FROM covers what Snowflake DECODE does for NULL = NULL match without needing the conditional construct.
NULL tests
| Operator | Notes |
|---|---|
IS NULL | True if NULL. |
IS NOT NULL | True if not NULL. |
IS TRUE / IS FALSE | Three-valued logic: NULL is not TRUE and is not FALSE. |
IS NOT TRUE / IS NOT FALSE | Inverse, including NULL. |
IS UNKNOWN / IS NOT UNKNOWN | Same as IS NULL / IS NOT NULL for boolean expressions. |
Logical
| Operator | Notes |
|---|---|
AND | Three-valued: NULL AND TRUE = NULL; NULL AND FALSE = FALSE. |
OR | Three-valued: NULL OR TRUE = TRUE; NULL OR FALSE = NULL. |
NOT | NULL stays NULL. |
Set membership
| Operator | Notes |
|---|---|
IN (a, b, c) | List membership. NULL in list is ignored. |
IN (subquery) | Subquery membership. |
NOT IN (...) | Negated. CARE: NOT IN with NULL in list returns NULL, not TRUE. |
EXISTS (subquery) | True if subquery returns any row. |
NOT EXISTS (subquery) | Negated. Safer than NOT IN for NULL handling. |
ANY (subquery) / SOME (subquery) | Compares to any row. x = ANY (subquery) = x IN (subquery). |
ALL (subquery) | Compares to every row. |
-- IN (NOT recommended when subquery may produce NULLs)
SELECT * FROM users WHERE id IN (SELECT user_id FROM blocked);
-- NOT EXISTS (safer)
SELECT * FROM users u WHERE NOT EXISTS (
SELECT 1 FROM blocked b WHERE b.user_id = u.id
);
Type cast
| Operator | Notes | Trino | Snowflake | Spark SQL | DuckDB |
|---|---|---|---|---|---|
CAST(expr AS type) | SQL standard cast. Errors on overflow / parse failure. | yes | yes | yes | yes |
TRY_CAST(expr AS type) | Returns NULL on failure. | yes | yes | partial | yes |
expr::type | Postgres-style shorthand for CAST. | yes | yes | - | yes |
try(CAST(...)) | try() wraps any expression; same effect for casts. | yes | - | - | - |
SELECT CAST('42' AS BIGINT); -- 42
SELECT TRY_CAST('not a number' AS BIGINT); -- NULL (no error)
SELECT '42'::BIGINT; -- 42 (Postgres style)
SELECT try(CAST(payload AS BIGINT)) FROM events;
Field access
| Operator | Notes |
|---|---|
expr.field | Struct field access. |
expr['key'] | Map subscript. |
expr[index] | Array subscript. 1-based, NULL on out-of-bounds. |
SELECT
address.city, -- struct field
settings['theme'], -- map lookup
tags[1] -- first array element
FROM users;
Quantifier shortcut
| Operator | Notes |
|---|---|
expr IN (subquery) | Equivalent to expr = ANY (subquery). |
expr NOT IN (subquery) | Equivalent to expr <> ALL (subquery). |
Operator precedence
Higher binds tighter:
::(postfix cast)[](subscript),.(field access)unary +,unary -,NOT*,/,%+,-||LIKE,ILIKE,SIMILAR TO,~,BETWEEN,IN,IS NULL,IS NOT NULL=,<>,!=,<,<=,>,>=,IS DISTINCT FROM,IS NOT DISTINCT FROMANDOR
Use parentheses when the order is not obvious; the planner does not warn on ambiguity.
What is NOT supported
@>,<@(Postgres array containment). Usearray_has_all/array_contains.->,->>(Postgres JSON arrow). Usejson_get/json_get_str(DataFusion JSON layer) orjson_extract/json_extract_scalar(Trino layer). See JSON.<<,>>(bit shift). Usepower(2, n) * xfor left shift;floor(x / power(2, n))for right shift.<=>(MySQL null-safe equals). UseIS NOT DISTINCT FROM.- Regex named captures (
(?P<name>...)). DataFusion’s regex backend (Rustregexcrate) does not support PCRE-style named captures; use numbered captures viaregexp_extract(s, p, n).
Dot-commands
Embedded-CLI shortcuts. Lines beginning with . bypass the SQL parser and run client-side; everything else is SQL. The convention matches sqlite3 and the DuckDB shell.
Dot-commands work in the embedded CLI (sqe-cli --embedded) only. Cluster mode uses the same shortcuts indirectly through dbt-sqe or a Flight SQL client; the dot-commands themselves are a REPL feature.
Source: crates/sqe-cli/src/dotcommands.rs.
Reference
| Command | Aliases | Argument | Origin | Action |
|---|---|---|---|---|
.help | .h, .? | - | sqe-cli | Print the dot-command list. |
.exit | .quit, .q | - | sqe-cli | Leave the REPL. End-of-input does the same. |
.tables [schema] | - | optional schema name | sqe-cli | Query information_schema.tables. Filter by schema if given. |
.schema <table> | .describe, .d | required table name | sqe-cli | Query information_schema.columns. Accepts 1-, 2-, or 3-part names. |
.summarize <table> | .summary | required table name | sqe-cli | Per-column count, distinct, null_count, min, max via UNION ALL. SQE’s V9 answer to DuckDB’s SUMMARIZE. |
.catalogs | .databases | - | sqe-cli | Query information_schema.schemata. |
.read <path> | - | required file path | sqe-cli | Execute a SQL script file. Errors abort. |
.timer on|off | - | required on or off | sqe-cli | Toggle per-query elapsed-time output below each result. |
.format [table|csv|tsv|json] | - | optional format | sqe-cli | Show the current format with no argument; set with one. |
Comparison to other shells
| Command | SQE | sqlite3 | DuckDB CLI | psql | Trino CLI |
|---|---|---|---|---|---|
| help | .help | .help | .help | \? | help |
| exit | .exit, .quit | .exit, .quit | .exit | \q | quit |
| list tables | .tables | .tables | .tables | \dt | SHOW TABLES |
| describe table | .schema t | .schema t | .schema t / DESCRIBE t | \d t | DESCRIBE t |
| summary stats | .summarize t | - | SUMMARIZE t | - | - |
| toggle timing | .timer on | .timer on | .timer on | \timing | - |
| run script | .read f.sql | .read f.sql | .read f.sql | \i f.sql | --file=f.sql |
Examples
Inspect a table you just created
sqe> CREATE TABLE orders AS SELECT * FROM read_parquet('s3://bucket/orders.parquet');
sqe> .schema orders
+-------------+-----------------+-------------+
| column_name | data_type | is_nullable |
+-------------+-----------------+-------------+
| id | BigInt | NO |
| customer_id | BigInt | YES |
| amount | Decimal(18, 2) | YES |
| created_at | Timestamp(Microsecond, None) | YES |
+-------------+-----------------+-------------+
.schema accepts qualified names: .schema iceberg.staging.orders works the same way against a 3-part name.
Summarize before deciding
sqe> .summarize orders
+-------------+-------+----------+------------+--------+----------+
| column | count | distinct | null_count | min | max |
+-------------+-------+----------+------------+--------+----------+
| id | 12000 | 12000 | 0 | 1 | 12000 |
| customer_id | 12000 | 8473 | 0 | 1 | 9999 |
| amount | 12000 | 9921 | 12 | -50.00 | 12500.00 |
| created_at | 12000 | 11973 | 0 | 2024-... | 2026-...|
+-------------+-------+----------+------------+--------+----------+
A count == distinct column is a candidate primary key. A high null_count rules out a NOT NULL constraint. The min / max range hints at distribution skew.
Time queries while iterating
sqe> .timer on
sqe> SELECT count(*) FROM read_parquet('hf://datasets/squad/plain_text/train.parquet');
+----------+
| count(*) |
+----------+
| 87599 |
+----------+
1 row in set (1.412s)
Pipe results to a file via .format + shell redirect
$ echo ".format csv
SELECT id, name FROM users" | sqe-cli --embedded --warehouse /data/wh > users.csv
What dot-commands do not do
- They do not run on the cluster. Use a Flight SQL client (
pyarrow,dbt-sqe) or callinformation_schemadirectly in SQL. - They do not support tab completion or up-arrow recall of dot-command syntax. Tab completion exists for SQL keywords and table names but not for
.fooarguments. - They are not pluggable. Adding a new dot-command means a code change in
crates/sqe-cli/src/dotcommands.rs.
Adding new dot-commands
The pattern is small enough to read in one sitting. Each new command needs:
- A new
DotCommandenum variant indotcommands.rs. - A match arm in
parse_dot_command(). - A line added to
help_text(). - Optional: a query builder helper if the command translates to SQL.
- A handler in the REPL loop (
crates/sqe-cli/src/repl.rs).
Two existing examples cover the spectrum: .tables (one-shot SQL builder), .summarize (multi-step: read schema, then build aggregate UNION ALL).
Quickstart
Get SQE running locally in under 5 minutes.
Prerequisites
- Rust 1.85+ (
curl --proto '=https' --tlsv1.2 -sSf https://sh.rustup.rs | sh) - A running data platform stack: Keycloak, Polaris, MinIO/S3 (see the quickstart stack in
data-platform/quickstart/full/)
1. Clone and Build
git clone https://github.com/schuberg/sqe.git
cd sqe
cargo build --release --bin sqe-server --bin sqe-cli
Or use the build script:
./scripts/build.sh release
2. Configure
Copy the example config and adjust for your environment:
cp sqe.toml.example sqe.toml
Key settings to update:
[auth]
keycloak_url = "https://your-keycloak:8443" # Your Keycloak URL
realm = "iceberg" # Your realm
client_id = "sqe-client" # OIDC client ID
[catalog]
catalog_url = "http://your-polaris:8181/api/catalog"
warehouse = "your-warehouse"
[storage]
s3_endpoint = "http://your-minio:9000"
s3_region = "us-east-1"
s3_access_key = "minioadmin" # Or set via SQE_STORAGE__S3_ACCESS_KEY
s3_secret_key = "minioadmin" # Or set via SQE_STORAGE__S3_SECRET_KEY
3. Start the Server
# Single-node coordinator (default mode)
./target/release/sqe-server --config sqe.toml
You should see:
INFO Starting sqe-server mode=Coordinator config="sqe.toml"
INFO Health endpoints on port 9091 (/healthz, /readyz)
INFO Prometheus metrics on port 9090
INFO SQE coordinator listening on 0.0.0.0:50051
4. Connect with the CLI
./target/release/sqe-cli --host localhost --port 50051
Username: alice
Password: ****
sqe-cli 0.1.0 connected to http://localhost:50051 (flight)
Type SQL queries, or \q to quit. End multi-line queries with ;
sqe> SHOW SCHEMAS;
schema_name
-------------
analytics
raw
(2 rows)
sqe> SELECT * FROM raw.orders LIMIT 5;
order_id | customer_id | amount | region
----------+-------------+--------+--------
1 | 100 | 250.00 | EU
2 | 101 | 150.00 | US
3 | 100 | 300.00 | EU
4 | 102 | 75.00 | APAC
5 | 103 | 500.00 | EU
(5 rows)
5. Run a Single Query
./target/release/sqe-cli -H localhost -p 50051 -u alice -e "SELECT COUNT(*) FROM raw.orders;"
Set SQE_PASSWORD to avoid the password prompt:
export SQE_USER=alice
export SQE_PASSWORD=secret
./target/release/sqe-cli -e "SHOW TABLES IN raw;"
Health Check
curl http://localhost:9091/healthz # → ok
curl http://localhost:9091/readyz # → 200 when ready
Pointing at a different catalog
The walkthrough above runs SQE against the local Polaris stack
over Iceberg REST. SQE supports five other catalog backends out
of the box: AWS Glue (native SDK), AWS S3 Tables (managed
Iceberg), Hive Metastore (Thrift), JDBC (Postgres / MySQL /
SQLite), and Hadoop (filesystem-only). Each uses the same
binary, just with a different [catalog.backend] block.
See Catalog backends for the full per-backend recipe with TOML examples, AWS credential setup, verification queries, and a troubleshooting checklist. Glue and S3 Tables are verified live against AWS deployments.
For the operator-friendly version of the same content (BI tool
connection, slim builds, cargo features), see
QUICKSTART.md
in the repo root.
Next Steps
- Catalog backends: per-backend TOML, credentials, verification queries
- Configuration Reference: all settings and env vars
- Docker: run in containers
- Kubernetes & Helm: production deployment
- Using the CLI: full CLI reference
Catalog backends
SQE talks to Iceberg tables through one of six catalog backends. The
choice is per-deployment in sqe.toml. Default release builds ship
every backend; slim builds drop the unused ones to save binary size.
The five non-Hadoop backends share one dispatch path through the
upstream iceberg-catalog-loader crate. Hadoop is the lone outlier
because it is filesystem-only: no metadata service to talk to,
just a warehouse path to walk.
Quick reference
| Backend | type value | Required keys | Optional keys | Cargo feature | Vendored crate |
|---|---|---|---|---|---|
| REST | rest (default) | catalog_url, warehouse (on [catalog]) | bearer / OAuth headers via runtime auth | rest (always) | iceberg-catalog-rest |
| HMS | hms | uri, warehouse | hms | iceberg-catalog-hms | |
| Glue | glue | region, warehouse | endpoint | glue | iceberg-catalog-glue |
| S3 Tables | s3tables | table_bucket_arn | endpoint_url | s3tables | iceberg-catalog-s3tables |
| JDBC | jdbc | url, warehouse | sql-postgres | iceberg-catalog-sql | |
| Hadoop | hadoop | warehouse | hadoop | (SQE-native) |
All six are smoke-tested in CI. Two of them, Glue and S3 Tables,
are verified live against production AWS deployments (account
ACCOUNT_ID, eu-example-1 and eu-example-2).
Multiple catalogs in one coordinator
SQE supports attaching several named catalogs (potentially of different backend types) to one coordinator. Each one becomes a top-level SQL identifier and cross-catalog joins work without any session-state setup.
# Legacy single-catalog block kept as a placeholder for backwards
# compatibility. When `[catalogs.*]` is populated the legacy block
# is dropped unless `query.default_catalog` names it explicitly.
[catalog]
catalog_url = ""
[catalogs.polaris]
catalog_url = "http://polaris:8181/api/catalog"
warehouse = "production"
[catalogs.polaris.backend]
type = "rest"
[catalogs.nessie]
catalog_url = "http://nessie:19120/iceberg"
warehouse = "lake"
[catalogs.nessie.backend]
type = "rest"
[catalogs.aws_glue]
catalog_url = ""
[catalogs.aws_glue.backend]
type = "glue"
region = "eu-example-1"
warehouse = "s3://my-bucket/wh"
[catalogs.aws_s3tables]
catalog_url = ""
[catalogs.aws_s3tables.backend]
type = "s3tables"
table_bucket_arn = "arn:aws:s3tables:eu-example-2:ACCOUNT_ID:bucket/my-bucket"
[catalogs.legacy_hms]
catalog_url = ""
[catalogs.legacy_hms.backend]
type = "hms"
uri = "metastore.example.com:9083"
warehouse = "s3a://my-bucket/wh"
[query]
# Optional. Picks the catalog DataFusion uses for unqualified
# names. Defaults to the first entry from `[catalogs.*]` sorted
# alphabetically (so `aws_glue` would win the example above).
default_catalog = "polaris"
3-part SQL identifiers route to the right catalog:
SELECT *
FROM polaris.sales.orders p
LEFT JOIN nessie.archive.orders n ON p.id = n.id
WHERE n.id IS NULL;
SELECT count(*) FROM aws_glue.iceberg_demo_analytics.iceberg_user_events;
SELECT * FROM aws_s3tables.testnamespace.daily_sales LIMIT 10;
Each catalog uses its own backend dispatch path (REST + bearer token, native AWS SDK, Thrift, etc.). The user’s bearer token from the session auth applies to all REST catalogs registered. Per-catalog credential scoping is a future change; today storage credentials are coordinator-wide.
The legacy single-catalog form (the [catalog] block alone, no [catalogs.*]) keeps working unchanged. Existing deployments need no migration.
Per-catalog auth and storage
Each catalog can override the global session bearer token and the global S3 credentials via optional [catalogs.<name>.auth] and [catalogs.<name>.storage] blocks. Federation across organisations becomes a config change rather than a separate deployment.
# Default Polaris uses the user's session token (V6 behaviour).
[catalogs.polaris]
catalog_url = "http://polaris:8181/api/catalog"
warehouse = "main"
[catalogs.polaris.backend]
type = "rest"
# A partner Polaris uses its own OAuth client and S3 bucket.
[catalogs.partner]
catalog_url = "https://partner.com/iceberg"
warehouse = "shared"
[catalogs.partner.backend]
type = "rest"
[catalogs.partner.auth]
type = "client_credentials"
token_endpoint = "https://partner.com/oauth/tokens"
client_id = "sqe-partner"
client_secret = "..." # use env override SQE__catalogs__partner__auth__client_secret
[catalogs.partner.storage]
s3_endpoint = "https://partner-s3.example.com"
s3_region = "us-east-1"
s3_access_key = "..."
s3_secret_key = "..."
# A public Nessie endpoint we read anonymously.
[catalogs.public_archive]
catalog_url = "https://nessie.public.example.com/iceberg"
warehouse = "public"
[catalogs.public_archive.backend]
type = "rest"
[catalogs.public_archive.auth]
type = "anonymous"
# AWS Glue lets the AWS SDK provider chain handle auth.
[catalogs.aws_glue]
[catalogs.aws_glue.backend]
type = "glue"
region = "eu-example-1"
warehouse = "s3://wh/"
[catalogs.aws_glue.auth]
type = "aws"
[catalogs.<name>.auth].type values
type | What it does | When to use |
|---|---|---|
session_bearer (default) | Pass the user’s session bearer token through unchanged | One OIDC provider fronts every Iceberg REST endpoint (the common case) |
client_credentials | Cluster-level OAuth2 client_credentials grant against the catalog’s own token endpoint | Federation with a partner Iceberg REST that has its own OAuth |
anonymous | No Authorization header | Public read-only Nessie or Polaris |
static | Pre-issued bearer token | Internal gateway with a fixed key, integration tests |
aws | AWS SDK provider chain | Glue / S3 Tables native backends, AWS REST endpoints with SigV4 |
[catalogs.<name>.storage] overrides
The block accepts the same keys as the top-level [storage] block: s3_endpoint, s3_region, s3_access_key, s3_secret_key, s3_path_style, s3_allow_http. Iceberg credential vending from REST catalogs still wins per-table over both this and the global block, so you only need to fill out per-catalog storage when the catalog does not vend credentials (Hadoop, raw Iceberg over Ceph, etc.) or when the underlying buckets live behind different S3 endpoints.
Today storage overrides apply at scan / write time. The client_credentials token is fetched once at session-build time and reused for the session lifetime; refresh-on-expiry is a future change.
REST: Polaris, Nessie, Unity OSS, AWS Glue REST, AWS S3 Tables REST
The default. Most production deployments speak Iceberg REST.
[catalog]
catalog_url = "https://polaris.example.com:18181/api/catalog"
warehouse = "production_warehouse"
[catalog.backend]
type = "rest" # default; this block can be omitted entirely
Local Polaris stack from the repo:
docker compose -f docker-compose.test.yml up -d
# Polaris listens on http://localhost:18181
AWS REST endpoints (Glue REST, S3 Tables REST) work transparently:
when the server’s /v1/config response advertises
rest.sigv4-enabled=true, SQE engages SigV4 automatically. AWS
credentials come from the standard SDK chain (env vars, profiles,
IMDS).
| Service | REST endpoint | Auth |
|---|---|---|
| Apache Polaris | https://polaris/api/catalog | OIDC bearer |
| Project Nessie 0.107+ | https://nessie/api/v1/iceberg | bearer / anonymous |
| Unity Catalog OSS | https://unity/api/2.1/unity-catalog/iceberg | bearer (Databricks) / anonymous (OSS) |
| AWS Glue Iceberg REST | https://glue.<region>.aws-endpoint/iceberg | AWS SigV4 (auto-detected) |
| AWS S3 Tables REST | https://s3tables.<region>.aws-endpoint/iceberg/v1 | AWS SigV4 (auto-detected) |
REST is the most-tested path. Every benchmark suite (TPC-H, SSB, TPC-DS, TPC-C, TPC-E, TPC-BB, ClickBench) runs against the local Polaris stack on every release build.
HMS: Hive Metastore over Thrift
For deployments still on Hive Metastore.
[catalog.backend]
type = "hms"
uri = "metastore.example.com:9083" # Thrift host:port
warehouse = "s3a://my-bucket/warehouse"
Pulls in volo-thrift and pilota (~10-15 MB).
Authentication via Kerberos / Knox is not supported directly. Deployments that need it should sit behind a sidecar that handles the SASL handshake and exposes a plain Thrift port. SQE expects the metastore to speak unauthenticated Thrift on its data plane.
The HMS path is verified by the integration test in
sqe-catalog/tests/backends_integration.rs and runs
against a docker-compose overlay during CI.
Glue: AWS Glue Data Catalog
[catalog.backend]
type = "glue"
region = "eu-example-1"
warehouse = "s3://my-bucket/warehouse"
# endpoint = "http://localhost:4566" # optional, e.g. LocalStack
Run with the right AWS credentials:
AWS_PROFILE=my-profile ./target/release/sqe-coordinator ~/sqe-config.toml
The AWS SDK reads AWS_PROFILE, AWS_ACCESS_KEY_ID, AWS_REGION,
or IMDS in that order. The region field in the config sets the
Glue API region; warehouse is the S3 path Glue uses for new
tables.
Pulls in aws-sdk-glue + aws-config (~50-80 MB).
Live verification (2026-05-05) against AWS Glue in eu-example-1
(account ACCOUNT_ID, database iceberg_demo_analytics):
sqe> SHOW SCHEMAS;
+------------------------+
| schema_name |
+------------------------+
| admin_consumer |
| admin_producer |
| default |
| iceberg-demo_catalog |
| iceberg_demo_analytics |
| saleslhdev_pub_db |
| saleslhdev_sub_db |
+------------------------+
(7 rows)
sqe> SELECT region, event_type, COUNT(*) AS n
. FROM iceberg_demo_analytics.iceberg_user_events
. GROUP BY region, event_type ORDER BY n DESC LIMIT 5;
+------------+------------+-------+
| region | event_type | n |
+------------+------------+-------+
| ap-south | login | 50524 |
| eu-example-1 | login | 50424 |
| eu-example-1 | click | 50391 |
| eu-example-1 | view | 50251 |
| us-west | click | 50155 |
+------------+------------+-------+
Aggregations, filter pushdown, and ORDER BY all work correctly across ~1.5M rows.
S3 Tables: AWS managed Iceberg
AWS’s first-class managed Iceberg service. Different from Glue (which is metadata-only): S3 Tables manages metadata and storage in one product. Tables live in a “table bucket” addressed by ARN.
[catalog.backend]
type = "s3tables"
table_bucket_arn = "arn:aws:s3tables:eu-example-2:ACCOUNT_ID:bucket/my-bucket"
# endpoint_url = "http://localhost:4566" # optional, custom endpoint
Same AWS credential story as Glue. The bucket ARN format is
arn:aws:s3tables:REGION:ACCOUNT:bucket/NAME.
Pulls in aws-sdk-s3tables. Shares the AWS SDK runtime that
glue already pulls, so the incremental binary cost on top of an
AWS-enabled build is small (~5 MB).
Live verification (2026-05-05) against
arn:aws:s3tables:eu-example-2:ACCOUNT_ID:bucket/testtablebucket:
sqe> SHOW SCHEMAS;
+---------------+
| schema_name |
+---------------+
| testnamespace |
+---------------+
sqe> SELECT product_category, COUNT(*) AS sales_count, SUM(sales_amount) AS total_sales
. FROM testnamespace.daily_sales
. GROUP BY product_category ORDER BY total_sales DESC;
+------------------+-------------+-------------+
| product_category | sales_count | total_sales |
+------------------+-------------+-------------+
| Laptop | 4 | 4500.0 |
| Monitor | 3 | 925.0 |
| Keyboard | 1 | 60.0 |
| Mouse | 1 | 25.0 |
+------------------+-------------+-------------+
Two backends in one repo, both writing to AWS through SQE’s
identical scan + aggregation path. The only thing that differs
is which CatalogBuilder the loader hands back from
load("glue") vs load("s3tables").
JDBC: Postgres / MySQL / SQLite
Iceberg’s JDBC catalog stores table metadata in a relational database. Useful when you want a single SQL endpoint without running a metadata service.
[catalog.backend]
type = "jdbc"
url = "postgresql://user:pass@host:5432/iceberg"
warehouse = "s3://my-bucket/warehouse"
The URL prefix selects the driver:
| Prefix | Driver | Notes |
|---|---|---|
sqlite:path/to/file.db | SQLite | Local file, no separate server |
postgresql://... or postgres://... | PostgreSQL | Production-grade, recommended |
mysql://... | MySQL | Tested on MySQL 8.0+ |
The catalog tables follow the Iceberg JDBC catalog schema
(iceberg_tables, iceberg_namespace_properties). SQE creates
them on first connect.
Pulls in sqlx + the requested DB driver (~5-10 MB for Postgres).
The Postgres path is verified by an integration test against a
docker-compose Postgres in sqe-catalog/tests/backends_integration.rs.
Hadoop: filesystem-only catalog
No metadata service. SQE walks warehouse for metadata.json
files and treats the prefix as the catalog. Useful for read-only
access to a warehouse another engine wrote, or for one-off
investigations on a S3 / GCS / Azure prefix without standing up
Polaris.
[catalog.backend]
type = "hadoop"
warehouse = "s3://my-bucket/warehouse"
This is SQE’s only native catalog backend. The other five all
delegate to the upstream iceberg-rust builder via the
iceberg-catalog-loader crate. Hadoop has no upstream loader
counterpart because it is not really a catalog. There is no
metadata service to talk to. Implementation lives in
sqe-catalog/src/backends/hadoop.rs.
Read-only. No commit path. Use a real catalog if you need INSERT, UPDATE, DELETE, or MERGE.
How the loader works
Every non-REST backend’s dispatch goes through one function call:
#![allow(unused)]
fn main() {
let catalog = iceberg_catalog_loader::load(catalog_type)?
.load(name.to_string(), props)
.await?;
}
catalog_type is the lowercase string ("glue", "s3tables",
etc). props is a HashMap<String, String> of the upstream
*_CATALOG_PROP_* keys. The loader’s registry is feature-gated
so a slim build only links the backends the SQE binary actually
uses.
The patch sits in
vendor/iceberg-rust/crates/catalog/loader/src/lib.rs, documented
inline at the touch site and in the vendor README under “SQE-only
patches.” It is forward-compatible with upstream: every existing
caller of the loader sees all backends present by default; nobody
loses anything.
Slim builds
Default release builds include every backend. Operators who want a smaller image can opt out:
# REST only: no AWS SDK, no Thrift, no sqlx
cargo build --release --no-default-features --features rest -p sqe-coordinator
# REST + AWS managed Iceberg
cargo build --release --no-default-features --features rest,glue,s3tables -p sqe-coordinator
# REST + Hive
cargo build --release --no-default-features --features rest,hms -p sqe-coordinator
Approximate cost on top of a rest-only build:
| Feature | Adds | Why |
|---|---|---|
hadoop | ~0 | Reuses existing object_store |
sql-postgres | 5-10 MB | sqlx + Postgres driver |
hms | 10-15 MB | volo-thrift + pilota |
glue | 50-80 MB | full AWS SDK |
s3tables | ~5 MB on top of glue | shares AWS SDK runtime |
Default release binary lands around 180-200 MB on Linux x86_64.
Verifying the connection
Once the coordinator is up, run these in order. Each one exercises a deeper layer and tells you exactly where things break if they do.
# 1. Auth + Flight handshake
SQE_PASSWORD=s3cr3t sqe-cli --port 60051 --user root -e "SELECT 1"
# 2. Catalog reachable, namespaces visible
SQE_PASSWORD=s3cr3t sqe-cli --port 60051 --user root -e "SHOW SCHEMAS"
# 3. Pick a namespace, list its tables
SQE_PASSWORD=s3cr3t sqe-cli --port 60051 --user root -e "SHOW TABLES IN <namespace>"
# 4. Read a row
SQE_PASSWORD=s3cr3t sqe-cli --port 60051 --user root \
-e "SELECT * FROM <namespace>.<table> LIMIT 1"
If step 4 works, every other Iceberg query path works too: filter pushdown, GROUP BY, JOIN, time-travel, write back.
Troubleshooting
Invalid or expired bearer token when the CLI passes
--token: the bearer was minted by something SQE’s auth chain
does not recognize. Use --user + SQE_PASSWORD instead and let
SQE mint its own token via the auth endpoint configured in
[auth].
Catalog '<X>' build failed with no further detail: check
the coordinator log. Common causes:
- AWS credentials not on the chain (no
AWS_PROFILE, no env vars, not running on EC2 / EKS). - HMS Thrift port not reachable.
- JDBC
urltypo (the prefix selects the driver). - S3 Tables ARN region mismatch (the ARN’s region must match
whatever the AWS SDK resolves; set
AWS_REGIONto be safe).
No such table but the table exists in the catalog: namespace
case sensitivity. Iceberg namespaces are usually lowercase; some
HMS deployments treat them as case-insensitive.
Slow first query every time the coordinator restarts: cold
manifest cache. Subsequent queries hit ObjectCache and run
faster. Expected.
Where to go from here
Quickstart: top-level walkthrough that covers SQE end-to-end including auth and CLI connectionIceberg Integration: REST surface, credential vending, read / write path, V3 featuresConfiguration Reference: every TOML key andSQE_*env varArchitecture: Coordinator: how the catalog plugs into session management
Storage backends
SQE separates two concerns:
- Catalog backend: where the table metadata lives. Polaris, Nessie, AWS Glue, S3 Tables, Hive Metastore, JDBC, Hadoop. See Catalog backends.
- Storage backend: where the data files live. AWS S3, GCS, ADLS Gen2, R2, MinIO, Ceph, local filesystem. This page.
Both are independent. A table whose metadata is in Polaris can have data files in any storage backend the engine knows how to talk to. The catalog hands SQE a s3://... (or gs://..., abfss://...) URL when loading a table; SQE picks the right object-store driver from the URL scheme.
Implementation lives in sqe-catalog/src/file_tvf_common.rs and sqe-catalog/src/lazy_object_store.rs.
Compatibility matrix
| Backend | URL scheme | Default build | TVF reads | Catalog reads | Writes | Notes |
|---|---|---|---|---|---|---|
| Local filesystem | /path or ./path | yes | yes | yes | yes | No setup required. |
| AWS S3 | s3://bucket/key | yes | yes | yes | yes | Provider chain (env / ~/.aws / IMDS / IRSA) when no inline creds. |
| AWS S3 (SSE / KMS) | s3://bucket/key | yes | yes | yes | yes | Server-side encryption is transparent. |
| Cloudflare R2 | s3://bucket/key (S3-compatible endpoint) | yes | yes | yes | yes | Set endpoint = https://<account>.r2.cloudflarestorage.com, region = auto. |
| MinIO | s3://bucket/key | yes | yes | yes | yes | Allow plain HTTP via s3_allow_http = true. |
| Ceph RGW | s3://bucket/key | yes | yes | yes | yes | Same as MinIO. |
| SeaweedFS | s3://bucket/key | yes | yes | yes | yes | Same as MinIO. |
| Garage | s3://bucket/key | yes | yes | yes | yes | Same as MinIO. |
| rustfs | s3://bucket/key | yes | yes | yes | yes | Same as MinIO. |
| HTTPS | https://host/path | yes | yes | partial | no | Lazy HttpStore per host (V10). Read-only. |
| HuggingFace | hf://datasets/... | yes | yes | no | no | V10 + V12.1. Auto-resolves to HTTPS. Read-only. |
| Azure ADLS Gen2 | abfss://[email protected]/path | yes | yes | yes | yes | Shared key, SAS, and Azurite emulator supported. |
| Azure (shorthand) | azure://container/path, az://container/path | yes | yes | yes | yes | Account name from [storage.azure] or azure_account => '...'. |
| Google Cloud Storage | gs://bucket/path, gcs://bucket/path | yes | yes | yes | yes | Service-account JSON path or inline; ADC fallback when neither set. |
All backends ship in the default cargo build. The object_store workspace dependency is built with aws, http, azure, and gcp features; no opt-in feature flip needed.
Local filesystem
No configuration. Pass an absolute or relative path:
SELECT * FROM '/data/orders.parquet';
SELECT * FROM read_csv('./report.csv');
SELECT * FROM read_delta('/var/lake/orders');
Catalog backends that store data on local disk (hadoop type, or any REST catalog with file:// warehouse paths) work the same way.
AWS S3
The default storage backend. Two ways to provide credentials:
1. AWS provider chain (recommended for production)
When inline access_key / secret_key are absent, SQE delegates to the AWS SDK provider chain:
AWS_ACCESS_KEY_ID+AWS_SECRET_ACCESS_KEY(+ optionalAWS_SESSION_TOKEN).~/.aws/credentialsprofile (AWS_PROFILEselects which).- EC2 IMDS instance role.
- EKS IRSA service-account role.
# config.toml
[storage]
type = "s3"
region = "eu-example-2"
# No access_key / secret_key here -> provider chain.
SELECT * FROM read_parquet('s3://bucket/key.parquet');
2. Inline credentials (per-query override)
SELECT * FROM read_parquet(
's3://bucket/path/*.parquet',
access_key => 'AKIA...',
secret_key => '...',
endpoint => 'https://s3.eu-example-2.aws-endpoint',
region => 'eu-example-2'
);
Inline values win over [storage] defaults for that query only.
Cloudflare R2
R2 speaks the S3 protocol. Two pieces are non-default:
- Endpoint points at your R2 account URL.
- Region is the literal string
auto(R2 ignores region but rejects the empty string).
SELECT * FROM read_parquet(
's3://my-r2-bucket/data.parquet',
access_key => '<R2_ACCESS_KEY_ID>',
secret_key => '<R2_SECRET_ACCESS_KEY>',
endpoint => 'https://<account-id>.r2.cloudflarestorage.com',
region => 'auto'
);
For permanent setup put the endpoint and creds in [storage]:
[storage]
type = "s3"
endpoint = "https://<account-id>.r2.cloudflarestorage.com"
region = "auto"
s3_access_key = "<R2_ACCESS_KEY_ID>"
s3_secret_key = "<R2_SECRET_ACCESS_KEY>"
MinIO, Ceph RGW, SeaweedFS, Garage, rustfs
All are S3-compatible. Two extra knobs versus AWS S3:
- Endpoint: your server URL. Typical:
http://minio:9000(Docker),https://s3.internal:9000(TLS). s3_allow_http: set totrueif the endpoint is plain HTTP. SQE refuses HTTP by default to prevent accidental cleartext credentials.
SELECT * FROM read_parquet(
's3://bucket/data.parquet',
access_key => 'minio-access-key',
secret_key => 'minio-secret-key',
endpoint => 'http://localhost:9000',
region => 'us-east-1'
);
[storage]
type = "s3"
endpoint = "http://localhost:9000"
region = "us-east-1"
s3_access_key = "minio-access-key"
s3_secret_key = "minio-secret-key"
s3_allow_http = true
HTTPS
Any https:// URL works without registration. The first request to a new scheme://host builds an HttpStore on the fly and caches it in the session’s object-store registry. Subsequent reads from the same host reuse the cached store.
SELECT * FROM 'https://example.com/data.parquet';
SELECT * FROM read_csv('https://raw.githubusercontent.com/.../titanic.csv');
SELECT * FROM read_json('https://api.example.com/events.ndjson');
Configurable via the [storage.http] block (custom headers, bearer auth):
[storage.http]
default_headers = { Authorization = "Bearer ${API_TOKEN}" }
HTTPS reads are stateless: each request fetches the byte range needed for the current Parquet operation. Range request support is required (every modern object store has it).
HuggingFace hf://
The hf:// resolver translates HuggingFace dataset, model, and spaces URLs into HTTPS calls against the Hub. Public datasets work anonymously; private datasets read HF_TOKEN from the environment.
-- Default revision (main)
SELECT * FROM read_parquet(
'hf://datasets/squad/plain_text/train-00000-of-00001.parquet'
);
-- Pinned revision (DuckDB-style, V12.1)
SELECT * FROM read_csv('hf://datasets/foo/[email protected]/data.csv');
-- Auto-generated Parquet view (V12.1)
SELECT * FROM read_parquet(
'hf://datasets/foo/bar@~parquet/default/train/0.parquet'
);
-- Equivalent ?revision query parameter
SELECT * FROM read_csv('hf://datasets/foo/bar/data.csv?revision=v1.0');
Glob expansion (**/*.parquet) on hf:// is tracked for V12.2; today the path must point to a specific file.
See File-format TVFs for the full path-form table.
Azure ADLS Gen2 / Blob
Three URL shapes are accepted:
| URL form | When to use |
|---|---|
abfss://<container>@<account>.dfs.core.windows.net/<path> | Hadoop-style; account encoded in URL. Most portable across tools. |
abfs://... | Same shape, plaintext variant. Avoid in production. |
azure://<container>/<path>, az://<container>/<path> | Shorthand. Account comes from [storage.azure] or the azure_account inline arg. |
Three auth methods:
-- 1. Shared key (storage account key)
SELECT * FROM read_parquet(
'abfss://[email protected]/path/data.parquet',
azure_access_key => '<storage-account-key>'
);
-- 2. SAS token (sub-account scope)
SELECT * FROM read_csv(
'abfss://[email protected]/2026-05-08/events.csv',
azure_sas_token => 'sv=2024-08-04&ss=b&srt=sco&sp=r...'
);
-- 3. Azurite emulator (local development)
SELECT * FROM read_parquet('azure://devstoreaccount1/test/data.parquet');
For permanent setup put credentials in [storage.azure]:
[storage]
azure_account = "myaccount"
azure_access_key = "<storage-account-key>"
# OR:
azure_sas_token = "sv=2024-08-04&..."
# Local development against Azurite:
azure_use_emulator = true
OAuth2 / managed-identity auth is not yet wired through the inline args; service-account flows go through the AWS-style env-var fallback that object_store::azure::MicrosoftAzureBuilder provides.
Google Cloud Storage
-- 1. Service-account JSON file
SELECT * FROM read_parquet(
'gs://my-bucket/path/data.parquet',
gcs_service_account_path => '/var/secrets/gcs-key.json'
);
-- 2. Inline service-account JSON
SELECT * FROM read_csv(
'gs://my-bucket/data.csv',
gcs_service_account_key => '{"type":"service_account",...}'
);
-- 3. Application Default Credentials (gcloud config / GCE metadata / GKE Workload Identity)
SELECT * FROM read_parquet('gs://my-bucket/data.parquet');
For permanent setup:
[storage]
gcs_service_account_path = "/var/secrets/gcs-key.json"
# OR inline:
gcs_service_account_key = "{\"type\":\"service_account\",...}"
When neither is set the underlying GCS driver falls back to ADC: GOOGLE_APPLICATION_CREDENTIALS env var, gcloud config, GCE metadata server, GKE Workload Identity. No SQE config needed for the workload-identity path.
The gcs:// scheme is also accepted as a synonym for gs://.
Per-query vs configured
Inline TVF arguments override [storage] defaults for one query. This matters when:
- Querying a dataset in a different region than your default storage.
- Running ad-hoc reads against a customer’s bucket without changing engine config.
- Passing through end-user credentials in a multi-tenant deployment (see Authentication Flow for the policy-controlled variant).
The Iceberg catalog backend has its own credential flow (catalog credential vending; see Iceberg Integration). Storage credentials there come from the catalog’s STS exchange, not [storage].
Why two layers
Catalogs name tables; storage holds bytes. Iceberg already separates these in the spec (every table’s location field is independent of the catalog hosting it). SQE keeps the same separation. The pay-off:
- The same dataset can be served by multiple catalogs simultaneously (Polaris in dev, Glue REST in prod, with both pointed at the same S3 prefix).
- A catalog migration (Polaris -> Nessie) does not move any data files.
- Storage feature flags (R2 vs AWS) do not touch metadata.
Implementation references
sqe-catalog/src/file_tvf_common.rs: shared inline-arg parsing forread_parquet/read_csv/read_json/read_delta. URL-scheme dispatch viais_s3_path,is_http_path,is_hf_path,is_azure_path,is_gcs_path.register_s3_store_if_needed,register_azure_store_if_needed,register_gcs_store_if_needed,register_http_store_if_needed: per-backendSessionContextregistration.extract_bucket,extract_azure_container_account,extract_gcs_bucket: URL parsers.
sqe-catalog/src/lazy_object_store.rs: V10’s lazy HTTPS object-store registry.sqe-catalog/src/iceberg_storage.rs: catalog-backed storage credential resolution (vended creds vs static[storage]).- Workspace
Cargo.tomlline 37:object_store = { ..., features = ["aws", "http", "azure", "gcp"] }. - The DuckDB-comparison audit row for backends lives in
docs/duckdb-comparision.md.
Building from Source
Prerequisites
| Tool | Version | Purpose |
|---|---|---|
| Rust | 1.85+ | Compiler |
| Cargo | (bundled) | Build system |
| protoc | 3.x+ | Protobuf compiler (for gRPC/Flight) |
| cmake | 3.x+ | Build dependency for some crates |
| pkg-config | any | Library discovery |
| OpenSSL dev | 3.x | TLS support |
macOS
brew install protobuf cmake pkg-config openssl
Ubuntu/Debian
sudo apt-get install -y protobuf-compiler cmake pkg-config libssl-dev
Build
# Debug (fast compile, slow runtime)
cargo build --bin sqe-server --bin sqe-cli
# Release (slow compile, fast runtime)
cargo build --release --bin sqe-server --bin sqe-cli
# Or use the build script
./scripts/build.sh release
Binaries are placed in target/release/ (or target/debug/):
sqe-server— the server binary (coordinator or worker)sqe-cli— the SQL CLI client
Test
# All workspace tests
cargo test --workspace
# Specific crate
cargo test -p sqe-coordinator
# Integration tests (require running quickstart stack)
cargo test --workspace -- --ignored
Docker Build
# Build the image
docker build -t sqe:latest .
# With OCI labels
docker build -t sqe:0.1.0 \
--build-arg VERSION=0.1.0 \
--build-arg BUILD_DATE=$(date -u +%Y-%m-%dT%H:%M:%SZ) \
--build-arg GIT_REVISION=$(git rev-parse HEAD) \
.
The Dockerfile uses a multi-stage build:
graph LR
subgraph "Build Stage (rust:bookworm)"
DEPS["Cache dependencies"] --> BUILD["cargo build --release"]
end
subgraph "Runtime Stage (debian:bookworm-slim)"
BIN["sqe-server + sqe-cli"] --> RUN["Non-root user (sqe:1000)"]
end
BUILD -->|COPY binaries| BIN
Workspace Structure
sqe/
├── Cargo.toml # Workspace root
├── Cargo.lock
├── Dockerfile
├── sqe.toml.example
├── crates/
│ ├── sqe-core/ # Shared types, config, errors
│ ├── sqe-auth/ # Keycloak OIDC
│ ├── sqe-catalog/ # Iceberg REST catalog client
│ ├── sqe-sql/ # SQL parser & classifier
│ ├── sqe-policy/ # Policy enforcement (pluggable)
│ ├── sqe-planner/ # Plan splitting for distributed exec
│ ├── sqe-coordinator/# Coordinator + sqe-server binary
│ ├── sqe-worker/ # Worker executor
│ ├── sqe-cli/ # SQL CLI client
│ ├── sqe-metrics/ # Prometheus + OTel + audit
│ └── sqe-trino-compat/ # Trino wire protocol adapter
├── deploy/
│ ├── helm/sqe/ # Helm chart
│ └── k8s/ # Raw K8s manifests
├── docs/
│ └── book/ # This documentation (mdBook)
├── scripts/
│ ├── build.sh
│ └── test.sh
└── tests/
└── integration_test.rs
Using the CLI
sqe-cli is the SQL client. By default it connects to a remote coordinator over Arrow Flight SQL or Trino HTTP. Pass --embedded to skip the network entirely and run an in-process engine. That mode is useful for ad-hoc analysis on local Parquet, CSV, or JSON files without standing up a cluster.
Usage
sqe-cli [OPTIONS]
Options:
-H, --host <HOST> Coordinator host [default: localhost]
-p, --port <PORT> Coordinator port [default: 50051]
--protocol <PROTOCOL> Wire protocol: flight or http [default: flight]
-u, --user <USER> Username (prompts if not set)
--token <TOKEN> Bearer token (skips password flow)
-e, --execute <SQL> Execute a single query and exit
--file <PATH> Read statements from a SQL script file
--stop-on-error Abort the script on first error (default: continue)
--embedded Run the engine in-process (no remote coordinator)
--memory-limit <SIZE> Per-process memory pool when --embedded [default: 1GB]
--warehouse <PATH> Single catalog at PATH named `iceberg`
(shorthand for --catalog iceberg=PATH)
--catalog NAME=PATH Attach a named persistent catalog (repeatable)
--memory Skip persistent catalogs entirely
-f, --format <FORMAT> Output format: table, csv, tsv, json [default: table]
--tls Use HTTPS/TLS
--insecure Accept invalid TLS certificates
-h, --help Print help
-V, --version Print version
Embedded mode
--embedded boots a single-process SessionContext with the same DataFusion tuning the cluster coordinator uses (parse_float_as_decimal, 64MB hash-join broadcast threshold, dynamic filter pushdown, Parquet filter pushdown). It registers all the same scalar functions, Trino-dialect aliases, JSON helpers, and the read_parquet(...) table-valued function. No auth, no Polaris, no network listeners.
# One-shot query against a local Parquet file
sqe-cli --embedded -e "SELECT COUNT(*) FROM read_parquet('data.parquet')"
# Trino-dialect functions work out of the box
sqe-cli --embedded -e "SELECT year(DATE '2026-05-07')"
# Run a script of statements
sqe-cli --embedded --file setup.sql
# Combine: script first, then ad-hoc query
sqe-cli --embedded --file setup.sql -e "SELECT COUNT(*) FROM staging"
# Interactive REPL (the default if no -e or --file is given)
sqe-cli --embedded
S3 access works too. Pass credentials inline to read_parquet:
SELECT *
FROM read_parquet(
's3://bucket/path/*.parquet',
access_key => 'AKIA...',
secret_key => '...',
region => 'eu-example-1'
);
File format TVFs
Alongside read_parquet(), the embedded engine ships read_csv() and read_json() for direct file access. They share the same calling convention (positional path, named keyword args) and the same S3 credential bag.
-- Local CSV (auto-detect schema, headers on by default)
SELECT count(*) FROM read_csv('/data/sales.csv');
-- Tab- or semicolon-separated, no header
SELECT * FROM read_csv('/data/raw.tsv',
delimiter => '\t',
has_header => 'false');
-- NDJSON
SELECT * FROM read_json('/data/events.jsonl');
-- S3-hosted CSV with inline credentials
SELECT * FROM read_csv('s3://bucket/sales/*.csv',
access_key => 'AKIA...',
secret_key => '...',
endpoint => 'http://minio:9000',
region => 'us-east-1');
CSV-specific named args: delimiter, has_header, quote, escape, comment, null_regex, file_extension. JSON-specific: newline_delimited, file_extension.
Auto-detect: SELECT * FROM 'file.ext'
DuckDB-style sugar for “I just want to query this file.” The engine looks at the file extension and picks the right reader:
SELECT * FROM '/data/sales.parquet';
SELECT * FROM '/data/events.jsonl';
SELECT * FROM '/data/log.csv';
Works with globs and S3 URLs too. For S3, you still need credentials configured somewhere (default in [storage], or use read_csv()/read_parquet() and pass them inline).
HTTP / HTTPS URLs
Every file-format TVF and the SELECT * FROM 'file.ext' auto-detect accept HTTP and HTTPS URLs out of the box:
-- Public CSV from any HTTP(S) host
SELECT count(*) FROM read_csv(
'https://raw.githubusercontent.com/datasciencedojo/datasets/master/titanic.csv'
);
-- Auto-detect on a quoted URL
SELECT count(*) FROM
'https://raw.githubusercontent.com/datasciencedojo/datasets/master/titanic.csv';
-- Parquet over HTTP (range-request reads, no full download)
SELECT count(*) FROM read_parquet('https://example.com/dataset.parquet');
The engine builds an HTTP object store on first request per scheme://host[:port] and caches it for the rest of the session. No configuration needed.
HuggingFace Hub: hf://
hf:// URLs resolve to public HuggingFace Hub download URLs:
-- hf://datasets/<owner>/<name>/<path>
SELECT * FROM read_csv(
'hf://datasets/datasets-examples/doc-formats-csv-1/data.csv'
);
-- Pin a revision via ?revision=
SELECT * FROM read_parquet(
'hf://datasets/squad/plain_text/train.parquet?revision=v1.0.0'
);
-- Models and Spaces work the same way
SELECT * FROM read_json('hf://models/<owner>/<name>/config.json');
The resolver expands hf://datasets/<owner>/<name>/<path> to https://huggingface.co/datasets/<owner>/<name>/resolve/<rev>/<path> and routes through the same HTTP object store as raw HTTPS URLs. Default revision is main.
Public datasets work without any auth. Private datasets are not yet supported (HF token plumbing is on the roadmap).
COPY ... TO 'file'
Export query results to disk. Format is auto-detected from the extension.
COPY (SELECT * FROM iceberg.sales.orders WHERE year = 2026)
TO '/exports/orders-2026.parquet';
COPY (SELECT customer_id, total FROM iceberg.sales.orders)
TO '/exports/orders.csv';
-- Force a format / pass options
COPY orders TO '/exports/orders.json'
(FORMAT 'json');
Persistent catalog
By default, --embedded attaches a SQLite-backed Iceberg catalog at ~/.sqe/warehouse/. Tables created here survive across sessions; SQL DDL (CREATE SCHEMA, CREATE TABLE, DROP TABLE, DROP SCHEMA) routes through the iceberg catalog without any out-of-band setup:
# Session 1: declare a schema and a table via plain SQL
sqe-cli --embedded -e "CREATE SCHEMA iceberg.staging"
sqe-cli --embedded -e \
"CREATE TABLE iceberg.staging.events (event_id BIGINT, ts TIMESTAMP, kind VARCHAR)"
# Session 2: same warehouse, table is still there
sqe-cli --embedded -e "SELECT count(*) FROM iceberg.staging.events"
CREATE TABLE ... AS SELECT ... (CTAS) is a known limitation: the embedded mode does not yet have a Parquet writer + iceberg-transaction commit pipeline, so the upstream provider rejects table providers that carry data. Use a separate CREATE TABLE (schema only) followed by INSERT INTO ... SELECT ... once the embedded INSERT path lands, or load via the cluster path for now.
The on-disk layout:
~/.sqe/warehouse/
├── sqe.db # SQLite catalog (namespaces, table pointers)
└── iceberg/ # Iceberg metadata + Parquet data files
└── staging/
└── events/
├── metadata/
└── data/
The catalog name is iceberg. Three-part identifiers (iceberg.staging.events) work; unqualified names resolve against DataFusion’s default in-memory catalog, so SELECT * FROM read_parquet(...) still works without any catalog interaction.
Override the path:
sqe-cli --embedded --warehouse /data/my-warehouse -e "..."
Skip the catalog entirely (ephemeral session, nothing written to disk):
sqe-cli --embedded --memory -e "SELECT 1"
Tables in the warehouse are valid Iceberg. If you later upgrade to a cluster deployment, point the cluster catalog at the same path and the tables come along. No migration, no re-export.
Multiple catalogs
Attach more than one warehouse with repeated --catalog NAME=PATH flags. Each becomes a top-level SQL identifier; cross-catalog joins work without any session-state setup.
sqe-cli --embedded \
--catalog prod=/data/prod \
--catalog stage=/data/stage \
-e "SELECT *
FROM prod.sales.orders p
LEFT JOIN stage.sales.orders s ON p.id = s.id
WHERE s.id IS NULL"
The catalog name shows up in information_schema.tables.table_catalog, in .catalogs, and in 3-part SQL identifiers. Names cannot contain . (it would clash with the SQL namespace separator) and cannot repeat (DataFusion’s register_catalog would silently overwrite).
--warehouse <path> remains as a shorthand for --catalog iceberg=<path>. The three flags --memory, --warehouse, and --catalog are mutually exclusive. Pick one.
Dot-commands
The REPL recognises sqlite/DuckDB-style commands that start with .. They run client-side, never reach the engine, and don’t end with ;:
sqe> .help
Dot commands:
.help show this list
.exit, .quit leave the REPL
.tables [schema] list tables (optionally filter by schema)
.schema <table> describe a table's columns
.describe <table> alias for .schema
.summarize <table> per-column count, distinct, null, min, max
.catalogs list catalogs visible to the session
.read <path> execute a SQL script file
.timer on|off toggle per-query elapsed-time output
.format [fmt] show or set output format (table|csv|tsv|json)
Examples:
sqe> .timer on
Timer: on
sqe> SELECT count(*) FROM read_parquet('events.parquet');
+----------+
| count(*) |
+----------+
| 1500000 |
+----------+
Time: 0.243s
sqe> .tables
sqe> .schema iceberg.staging.events
sqe> .summarize iceberg.staging.events
sqe> .read setup.sql
sqe> .format json
.summarize runs a per-column UNION ALL of count, null_count,
distinct_count, min, and max. It is a two-step flow: the REPL
fetches the column list from information_schema.columns, then
generates and executes the aggregate query. Min/max are cast to
VARCHAR so columns of mixed types render in one table.
SQL surface
In addition to the standard dialect, embedded mode (and the cluster coordinator) supports DuckDB-style projection sugar that DataFusion 53.1 ships natively:
-- Drop columns from the projection.
SELECT * EXCLUDE (secret, internal_id) FROM users;
-- Substitute a column with an expression while keeping order.
SELECT * REPLACE (UPPER(name) AS name, total / 100 AS total)
FROM orders;
-- Native column-level metadata.
DESCRIBE iceberg.staging.events;
The legacy \format and \q forms still work for backward compatibility.
What embedded mode does not include
- Authentication, RBAC, or column masking. Embedded mode runs as the local user. Use the cluster path when you need policy enforcement.
- Distributed execution. Embedded mode is single-process by design.
- Concurrent writers. The SQLite catalog is single-process; running two
sqe-cli --embeddedinstances against the same warehouse simultaneously will likely produce errors. The cluster path handles concurrent writes correctly.
Script files
--file reads a SQL script and executes statements in order, separated by ;. The splitter respects single-quoted strings, double-quoted identifiers, line comments (--), and block comments (/* ... */), so semicolons inside those don’t accidentally split a statement.
By default, errors print to stderr and execution continues. Pass --stop-on-error to abort on the first failure. That is the right setting for CI scripts where any failure means the schema setup is broken.
sqe-cli --embedded --file setup.sql
Interactive Mode
sqe-cli --host sqe-coordinator --port 50051 --user alice
Password: ****
sqe-cli 0.1.0 connected to http://sqe-coordinator:50051 (flight)
Type SQL queries, or \q to quit. End multi-line queries with ;
sqe> SELECT * FROM raw.orders LIMIT 3;
order_id | customer_id | amount | region
----------+-------------+--------+--------
1 | 100 | 250.00 | EU
2 | 101 | 150.00 | US
3 | 100 | 300.00 | EU
(3 rows)
sqe> \q
Multi-line Queries
Queries are executed when you type ;:
sqe> SELECT
-> region,
-> COUNT(*) AS orders,
-> SUM(amount) AS total
-> FROM raw.orders
-> GROUP BY region
-> ORDER BY total DESC;
Commands
| Command | Action |
|---|---|
\q | Quit |
quit | Quit |
exit | Quit |
Ctrl+C | Cancel current input / quit |
Ctrl+D | Quit (EOF) |
History is saved to ~/.sqe_history.
Single Query Mode
Execute one query and exit. Useful for scripts:
sqe-cli -H localhost -p 50051 -u alice -e "SELECT COUNT(*) FROM raw.orders;"
Output Formats
Table (default)
sqe-cli -e "SELECT 1 AS a, 'hello' AS b;" --format table
a | b
---+-------
1 | hello
(1 rows)
CSV
sqe-cli -e "SELECT 1 AS a, 'hello' AS b;" --format csv
a,b
1,hello
JSON (newline-delimited)
sqe-cli -e "SELECT 1 AS a, 'hello' AS b;" --format json
{"a":"1","b":"hello"}
Authentication
Username/Password
# Interactive prompt
sqe-cli --user alice
# Environment variables (no prompts)
export SQE_USER=alice
export SQE_PASSWORD=secret
sqe-cli -e "SHOW SCHEMAS;"
Bearer Token
Skip the password flow entirely with a pre-obtained token:
sqe-cli --token eyJhbGciOiJSUzI1NiIs... -e "SELECT 1;"
Connecting in Kubernetes
# Port-forward to the coordinator
kubectl port-forward svc/sqe-coordinator 50051:50051
# Then connect locally
sqe-cli --host localhost --port 50051
# Or exec directly into the pod
kubectl exec -it deploy/sqe-coordinator -- sqe-cli
Using with Trino Protocol
For compatibility with tools that speak Trino HTTP:
sqe-cli --protocol http --host localhost --port 8080 --user alice
This uses the Trino-compatible /v1/statement endpoint instead of Flight SQL.
Quickstart
Each quickstart is a self-contained directory you can run end to end. It brings up everything the use-case needs, runs a few useful queries, and captures the real output as committed evidence. These pages describe each one at a high level — what it shows and how it works — and link to the repo for the full config, compose files, queries, and captured output.
These are the user-facing source of truth for “how do I run SQE for X.” The rest of the book explains why SQE is built the way it is; the quickstarts show how to use it.
What’s possible
Catalog + authentication (local Docker stack)
| Quickstart | What it shows | Status |
|---|---|---|
| Polaris + Keycloak (client credentials) | Polaris + Keycloak; SQE mints user tokens via the OIDC password grant | validated |
| Polaris + Keycloak (user token) | Same stack; clients bring a pre-minted Keycloak token, SQE validates + passes it through | validated |
| Project Nessie | Nessie as the Iceberg REST catalog (auth-less, anonymous SQE) | validated |
| Unity Catalog OSS | Unity Catalog OSS over Iceberg REST (read-only; catalog-browse demo) | validated |
AWS managed catalogs (CDK bootstrap + teardown)
| Quickstart | What it shows | Status |
|---|---|---|
| AWS S3 Tables | AWS S3 Tables (managed Iceberg); CDK bootstrap + teardown; SQE creates the namespace | validated |
| AWS Glue | AWS Glue Data Catalog; SQE creates the DB and does a full round-trip | validated |
| AWS Glue + Lake Formation | Glue governed by Lake Formation: denied until an explicit LF grant, then succeeds | validated |
Embedded (single binary, sqe-cli)
| Quickstart | What it shows | Status |
|---|---|---|
| Query local and remote files | Read files directly with the read_* TVFs (no server, no catalog) | validated |
| Persistent local catalog (SQLite) | Local persistent Iceberg catalog backed by SQLite (no server) | validated |
| Attach multiple catalogs | Attach several persistent catalogs and JOIN across them | validated |
| Quack (DuckDB wire protocol) | SQE’s DuckDB Quack RPC endpoint, both directions | experimental |
Operations
| Quickstart | What it shows | Status |
|---|---|---|
| Observability: metrics + Grafana | Scrape SQE’s Prometheus metrics with VictoriaMetrics + Grafana | validated |
Benchmarks
| Quickstart | What it shows | Status |
|---|---|---|
| TPC-H / TPC-DS / SSB | Generate, load, and run the TPC suites against SQE with per-query timings | validated |
How a quickstart is laid out
Each directory has a README.md (the why/how), a standalone docker-compose.yml,
the annotated config, a run.sh that brings the stack up and captures output, the
demo queries.sql, and an OUTPUT.md with the real captured result.
Run any of them from a clone of the repo:
cd quickstart/<name>
cp .env.example .env
./run.sh
All quickstarts live in the repo under
quickstart/.
Polaris + Keycloak (client credentials)
slug: polaris-keycloak-client-id title: “Polaris + Keycloak (client credentials)” description: “Run SQE against Apache Polaris with Keycloak as the identity provider. SQE holds a confidential client and exchanges each user’s username + password for a bearer token via the OIDC password grant, then passes that token through to Polaris.”
Polaris + Keycloak (client credentials)
Run SQE against an Apache Polaris catalog where Keycloak issues the identities. A user connects to SQE with a username and password; SQE exchanges those for the user’s bearer token via its own confidential client (the OIDC Resource Owner Password Credentials grant), then passes the token straight through to Polaris. Polaris decides what the user can see — no service account, no shared credential.
How it works
- Keycloak acts as the identity provider. An
icebergrealm holds a confidential client (sqe-client) and three test users with different role levels. - SQE uses the
oidc_passwordauth provider: on login it posts the user’s credentials plus its own client secret to Keycloak’s token endpoint and receives the user’s bearer token. - Polaris is federated to Keycloak — it validates the token SQE forwards
(issuer, signature, audience) and maps the token’s
preferred_usernameto a Polaris principal with its own RBAC roles. - RustFS provides S3-compatible warehouse storage. A one-shot
bucket-initcontainer creates the warehouse bucket on startup. - Every query runs as the authenticated user. The user never contacts Keycloak directly.
What it demonstrates
- SQE minting a user’s token from username + password via OIDC password grant.
- Token passthrough to Polaris: Polaris enforces catalog-level RBAC per principal.
- Multi-user isolation:
adminuser(write access) andtestuser(read-only) running the same queries with different results. - Full create/write/read round-trip:
CREATE SCHEMA→CREATE TABLE→INSERT→SELECT … GROUP BY. - Role-level access control validated in two layers: the demo path and the integration test suite.
Status: validated (2026-06-06).
Run it
Full config, docker compose, queries, and captured output are in the repo:
→ quickstart/polaris-keycloak-client-id/
cd quickstart/polaris-keycloak-client-id
cp .env.example .env
./run.sh
Polaris + Keycloak (user token)
slug: polaris-keycloak-user-token title: “Polaris + Keycloak (user token)” description: “Run SQE against Apache Polaris where clients bring a pre-minted Keycloak bearer token. SQE validates the token against the realm JWKS and passes it through to Polaris. No client secret, no password grant.”
Polaris + Keycloak (user token)
The bring-your-own-token path. An upstream application or identity provider has already authenticated the user and holds their bearer token. The client sends that token to SQE; SQE validates it (signature, issuer, expiry) against the realm’s public JWKS endpoint and passes it through to Polaris. SQE never sees a password and holds no client secret.
How it works
- The Docker stack is identical to the client credentials quickstart: Keycloak, Polaris, RustFS, and SQE on one network.
- The only difference is SQE’s auth provider:
bearer_tokeninstead ofoidc_password. SQE fetches the realm’s signing keys once from the JWKS endpoint, then verifies every incoming token locally — no call to Keycloak’s token endpoint, no client secret. run.shmints a token from Keycloak’s public client (standing in for an upstream app) and queries SQE with--token.- Polaris enforces RBAC as before: the token’s
preferred_usernamemaps to a principal with its own role grants.
What it demonstrates
- SQE as a pure token validator: it verifies the JWT signature and claims, then passes the token to the catalog.
- A valid token with write access completes the full create/write/read round-trip.
- A read-only token is allowed to read but denied a write by Polaris RBAC (403).
- A malformed token is rejected by SQE’s JWKS validation before reaching the catalog.
Status: validated (2026-06-06).
Run it
Full config, docker compose, queries, and captured output are in the repo:
→ quickstart/polaris-keycloak-user-token/
cd quickstart/polaris-keycloak-user-token
cp .env.example .env
./run.sh
Project Nessie (Iceberg REST catalog)
slug: nessie title: “Project Nessie (Iceberg REST catalog)” description: “Run SQE against Project Nessie. Nessie speaks the Iceberg REST protocol, so SQE uses the same rest catalog code path it uses for Polaris. Auth-less stack: SQE runs in anonymous dev mode.”
Project Nessie (Iceberg REST catalog)
Project Nessie is a transactional, git-like
catalog for Iceberg tables. It exposes the Iceberg REST protocol — the same
surface as Polaris — so SQE talks to it through the identical rest catalog
code path. Swapping Polaris for Nessie is a one-line config change.
This quickstart is about the catalog, not auth. Nessie runs auth-less and SQE
uses its anonymous provider, so there is no identity provider to set up. For
the full auth story (real identities, RBAC, token passthrough) see the
polaris-keycloak quickstarts.
How it works
- Nessie runs with an in-memory version store and serves the Iceberg REST
protocol at its
/icebergmount point. - RustFS provides S3-compatible warehouse storage. A one-shot
bucket-initcontainer creates the warehouse bucket on startup. - SQE uses the
anonymousauth provider — every connection is accepted as a single anonymous identity. This mode logs a security warning on startup and should not be used in production. - The
polaris_urlin SQE’s catalog config simply points at Nessie’s/icebergendpoint instead of Polaris’s/api/catalog. SQE issues the same Iceberg REST handshake either way.
What it demonstrates
- SQE connecting to Nessie as an Iceberg REST catalog with no auth configuration.
- Full create/write/read round-trip:
CREATE SCHEMA→CREATE TABLE→INSERT→SELECT … GROUP BY. - The same
restcatalog code path works against both Polaris and Nessie — the catalog is swappable by config.
Status: validated (2026-06-06).
Run it
Full config, docker compose, queries, and captured output are in the repo:
cd quickstart/nessie
cp .env.example .env
./run.sh
Unity Catalog OSS (Iceberg REST, read-only)
slug: unity-oss title: “Unity Catalog OSS (Iceberg REST, read-only)” description: “Connect SQE to Unity Catalog OSS over its Iceberg REST adapter and browse the catalog. Unity OSS’s Iceberg REST is read-only at this version, so this quickstart enumerates namespaces and tables rather than running DML.”
Unity Catalog OSS (Iceberg REST, read-only)
Unity Catalog OSS exposes an
Iceberg REST adapter, so SQE connects to it through the same rest catalog code
path it uses for Polaris and Nessie. This quickstart connects SQE to Unity and
browses the catalog: it lists namespaces and tables.
Be clear-eyed about the current limitation: Unity OSS’s Iceberg REST adapter is
read-only at this version. Create, drop, and commit are not supported, and the
bundled table is not served as a loadable Iceberg table, so SELECT does not
work either. This quickstart exists to show the connection works and to document
the boundary. For full read and write against an Iceberg REST catalog, use the
Polaris or Nessie quickstarts.
How it works
- Unity Catalog OSS runs with its Iceberg REST adapter enabled. It ships a
bundled demo table (
unity.default.marksheet_uniform). - SQE connects using the same
polaris_urlconfig key, pointing at Unity’s Iceberg REST mount. Auth is theanonymousdev-mode provider (Unity OSS runs without authentication). run.shenumerates schemas and tables, then attempts aSELECTand captures the failure to document the read-only boundary.
What it demonstrates
- SQE connecting to Unity Catalog OSS over its Iceberg REST surface.
- Catalog enumeration:
SHOW SCHEMASandSHOW TABLESreturn the Unity namespace and bundled table. - The read boundary: a
SELECTon the bundled table is denied by Unity OSS (not a SQE limitation — SQE’s REST path is the same one that does full read/write against Polaris and Nessie).
Status: validated (2026-06-06).
Run it
Full config, docker compose, queries, and captured output are in the repo:
cd quickstart/unity-oss
cp .env.example .env
./run.sh
AWS S3 Tables (managed Iceberg)
slug: aws-s3-tables title: “AWS S3 Tables (managed Iceberg)” description: “Run SQE against AWS S3 Tables, AWS’s managed Iceberg service (metadata + storage in one). A CDK stack bootstraps a throwaway table bucket and tears it down; SQE creates the namespace and does a full create/write/read round-trip over the AWS SDK.”
AWS S3 Tables (managed Iceberg)
Point SQE at AWS S3 Tables, AWS’s managed Iceberg product. Unlike Glue (metadata only), S3 Tables bundles the catalog and the storage into one service: you create a table bucket, and namespaces and tables live inside it. SQE talks to it over the AWS SDK with your IAM credentials.
A small CDK stack bootstraps the throwaway table bucket and tears it down at the end, so the quickstart leaves nothing behind in your account.
How it works
- A TypeScript CDK stack creates an S3 Tables table bucket in your AWS account on deploy and removes it on destroy.
- SQE uses the
s3tablescatalog backend, configured with the table bucket ARN. The bucket ARN and your AWS region are injected at runtime byrun.sh. - AWS IAM credentials (via
AWS_PROFILEor environment variables) authenticate all catalog and storage operations — no separate identity provider. - SQE creates the namespace itself with
CREATE SCHEMA, which makes the calling principal its owner and avoids Lake Formation permission conflicts. run.shruns the full loop: CDK deploy → start SQE → run queries → capture output → delete table and namespace → CDK destroy.
What it demonstrates
- SQE connecting to AWS S3 Tables as a managed, non-REST Iceberg catalog.
- Full create/write/read round-trip:
CREATE SCHEMA→CREATE TABLE→INSERT→SELECT … GROUP BY, all against live S3 Tables. - Clean teardown: the table bucket, namespace, and table are all removed; no resources left in the account.
Status: validated (2026-06-06).
Run it
Full config, CDK stack, docker compose, queries, and captured output are in the repo:
cd quickstart/aws-s3-tables
cp .env.example .env
./run.sh
AWS Glue Data Catalog
slug: aws-glue title: “AWS Glue Data Catalog” description: “Run SQE against the AWS Glue Data Catalog with S3 as storage. A CDK stack bootstraps a throwaway S3 warehouse bucket and tears it down; SQE creates the database and does a full create/write/read round-trip over the AWS SDK.”
AWS Glue Data Catalog
Point SQE at the AWS Glue Data Catalog. Glue is the catalog (table metadata) and S3 is the storage; SQE talks to both over the AWS SDK using your IAM credentials. No Polaris, no Keycloak, no local object store.
A small CDK stack bootstraps a throwaway S3 warehouse bucket and tears it down at the end, so the quickstart leaves nothing behind in your account.
How it works
- A TypeScript CDK stack creates an S3 bucket to use as the Iceberg warehouse. The bucket is removed on CDK destroy (including any Iceberg data inside it).
- SQE uses the
gluecatalog backend, configured with your AWS region and the S3 warehouse path. Both are injected at runtime byrun.sh. - AWS IAM credentials authenticate all Glue catalog and S3 storage operations.
- SQE creates the Glue database with
CREATE SCHEMA, making the calling principal its owner. This is deliberate: in a Lake Formation-enabled account, a database created out-of-band is LF-governed with no grants, which blocksCREATE TABLE. By creating the database itself, SQE avoids that. See the glue-lake-formation quickstart for the governed variant. run.shruns the full loop: CDK deploy → start SQE → run queries → capture output → drop the Glue database → CDK destroy.
What it demonstrates
- SQE connecting to AWS Glue as a non-REST Iceberg catalog with S3 storage.
- Full create/write/read round-trip:
CREATE SCHEMA→CREATE TABLE→INSERT→SELECT … GROUP BY, all against live Glue + S3. - Clean teardown: the S3 bucket (and all Iceberg data) and the Glue database are removed; no resources left in the account.
Status: validated (2026-06-06).
Run it
Full config, CDK stack, docker compose, queries, and captured output are in the repo:
cd quickstart/aws-glue
cp .env.example .env
./run.sh
AWS Glue + Lake Formation
slug: glue-lake-formation title: “AWS Glue + Lake Formation” description: “Run SQE against a Lake-Formation-governed AWS Glue database. A CDK stack creates the database (so LF governs it); the run shows SQE denied until an explicit LF grant, then succeeding. Table/database-level LF permissions, not column/row masking.”
AWS Glue + Lake Formation
Point SQE at an AWS Glue database that Lake Formation governs. Unlike the
aws-glue quickstart, which lets SQE create the database
(making the caller its owner to sidestep LF), here the database is created by
CloudFormation. In a Lake-Formation-enabled account that means it is governed
with no grants, so SQE is denied until you grant it LF permissions
explicitly. The run shows the denial, the grant, and the same statement
succeeding.
How it works
- A TypeScript CDK stack creates both an S3 warehouse bucket and an LF-governed Glue database. Because CloudFormation creates the database, Lake Formation governs it with no permissions granted by default.
- Phase A:
run.shstarts SQE and runsqueries.sql.CREATE TABLEfails with an LFAccessDeniedException— the principal has no LF permission on the database. - Grant:
run.shcallsaws lakeformation grant-permissionsto give the principalCREATE_TABLE,ALTER,DROP, andDESCRIBEon the database. - Phase B:
run.shrestarts SQE (to flush any cached state) and runs the same queries.CREATE TABLE→INSERT→SELECTall succeed. - CDK destroy removes the database, bucket, and LF grant. Nothing is left behind.
Note the boundary: this quickstart demonstrates table- and database-level LF permission gating. SQE reads Iceberg data files from S3 directly with the caller’s IAM credentials; it does not call Lake Formation’s filtered credential-vending for column masking or row filtering. SQE’s own column/row masking is a separate policy engine (OPA/Cedar plan rewriting).
What it demonstrates
- LF enforcement in action:
CREATE TABLEdenied before the grant, succeeding after. - The deny → grant → succeed arc captured in a single
run.shrun. - Full create/write/read round-trip in Phase B:
CREATE TABLE→INSERT→SELECT … GROUP BY. - Clean teardown: no stack, database, bucket, or LF grants left in the account.
Status: validated (2026-06-07).
Run it
Full config, CDK stack, docker compose, queries, and captured output are in the repo:
→ quickstart/glue-lake-formation/
cd quickstart/glue-lake-formation
cp .env.example .env
./run.sh
Embedded: query local and remote files
slug: embedded-files title: “Embedded: query local and remote files” description: “Run SQE’s engine in-process with sqe-cli –embedded and query CSV, JSON, and Parquet files directly with read_csv / read_json / read_parquet. No server, no catalog, local files or HTTPS URLs.”
Embedded: query local and remote files
SQE’s engine runs in-process. sqe-cli --embedded --memory starts the query
engine and the file-reader table-valued functions in a single binary — no
coordinator, no workers, no network listeners, no catalog. The read_csv,
read_json, and read_parquet TVFs read files directly, whether they live on
local disk or behind an HTTPS URL.
This is the fastest way to explore data with SQL. There is no Docker stack to bring up; this quickstart just runs the CLI.
How it works
--embeddedruns the engine in-process.--memorydisables the persistent catalog — the session is ephemeral and nothing is written to disk.- Sample data files (CSV, JSON, Parquet) in the
data/directory are mounted into the container at runtime. read_csv,read_json, andread_parquetaccept a local path, anhttps://URL, or ans3://URI — the same object-store layer backs all three.run.shexercises local CSV aggregation, a cross-format JOIN (CSV + Parquet), and a remote HTTPS Parquet read, and captures the output.
What it demonstrates
- Querying CSV, JSON, and Parquet files with SQL in a single binary, no server.
- Cross-format joins:
read_csv(...)andread_parquet(...)in one query. - Remote file reads over HTTPS:
read_parquet('https://...')streams the file via range requests. - The
--memoryflag: no state survives the process (for persistent local tables, see the embedded-sqlite-catalog quickstart).
Status: validated (2026-06-06).
Run it
Full sample data, queries, and captured output are in the repo:
cd quickstart/embedded-files
cp .env.example .env
./run.sh
Embedded: persistent local catalog (SQLite)
slug: embedded-sqlite-catalog title: “Embedded: persistent local catalog (SQLite)” description: “Run SQE’s engine in-process with a persistent, SQLite-backed Iceberg catalog on local disk. CREATE TABLE survives across CLI invocations, no server or external catalog service.”
Embedded: persistent local catalog (SQLite)
sqe-cli --embedded --warehouse <dir> runs the engine in-process and attaches a
SQLite-backed Iceberg catalog rooted at <dir>. Unlike the
embedded-files quickstart (which uses --memory and keeps
nothing), here CREATE TABLE and its data persist on disk: the catalog is a
SQLite file and the table data lives next to it as Iceberg metadata and Parquet.
No server, no Polaris, no catalog service.
How it works
--embedded --warehouse <dir>names the catalogicebergand stores both the SQLite catalog metadata (sqe.db) and the Iceberg data files under<dir>.run.shruns two separatesqe-cliprocesses against the same warehouse directory: the first writes (create + insert), the second (a fresh invocation) reads it back. This proves that the data survives across process restarts.- The
./warehousedirectory persists on the host;./run.sh --cleanresets it.
What it demonstrates
- Data written in one process is readable by a separate, subsequent process — on-disk persistence via SQLite + Iceberg.
- The full local Iceberg lifecycle:
CREATE SCHEMA→CREATE TABLE→INSERTin process 1;SELECTin process 2. - The difference between
--memory(session-only) and--warehouse(persistent).
Status: validated (2026-06-06).
Run it
Full queries and captured output are in the repo:
→ quickstart/embedded-sqlite-catalog/
cd quickstart/embedded-sqlite-catalog
cp .env.example .env
./run.sh
Embedded: attach multiple catalogs
slug: attach-catalogs title: “Embedded: attach multiple catalogs” description: “Attach several persistent Iceberg catalogs in one embedded SQE session with –catalog NAME=PATH and query (JOIN) across them. No server.”
Embedded: attach multiple catalogs
sqe-cli --embedded --catalog NAME=PATH (repeatable) mounts several persistent,
SQLite-backed Iceberg catalogs in one in-process session. Each catalog shows up
under its name in 3-part SQL identifiers (name.namespace.table), and a single
query can JOIN across them. No server, no catalog service.
Useful for local analysis that spans more than one warehouse — for example, a
sales catalog and a ref (reference-data) catalog, joined in one query.
How it works
- Two seed scripts populate two independent catalogs under separate directories
(
./catalogs/salesand./catalogs/ref), each with its own SQLite metadata and Iceberg data. - A single
sqe-clisession attaches both with two--catalogflags and runs a cross-catalog JOIN. --catalog NAME=PATHis repeatable and mutually exclusive with--memoryand--warehouse../run.sh --cleanresets both catalog directories.
What it demonstrates
- Attaching two independent persistent catalogs in one embedded session.
- A cross-catalog JOIN resolving tables from each catalog by name.
- The
--catalog NAME=PATHflag as an alternative to--warehousewhen more than one named catalog is needed.
Status: validated (2026-06-06).
Run it
Full seed scripts and captured output are in the repo:
cd quickstart/attach-catalogs
cp .env.example .env
./run.sh
Quack: the DuckDB wire protocol
slug: quack title: “Quack: the DuckDB wire protocol” description: “SQE speaks DuckDB’s Quack RPC protocol both ways: as a server (a DuckDB CLI queries SQE) and as a client (SQE’s quack_query() pulls from a remote Quack endpoint). run.sh proves the forward round-trip with a local DuckDB 1.5.3; the reverse is documented and verified.”
Quack: the DuckDB wire protocol
Quack is DuckDB’s RPC protocol. A DuckDB client can ATTACH 'quack:host:port'
and query a remote engine as though it were a local database.
SQE speaks Quack both ways:
- As a server — a DuckDB client queries SQE’s catalogs over the Quack
endpoint (
quack_portin the coordinator config). - As a client — SQE’s
quack_query()table function pulls rows from a remote Quack endpoint (another SQE instance, or a DuckDB runningquack_serve).
How it works
- The stack is a queryable SQE with a Nessie catalog and RustFS warehouse storage — the same base as the Nessie quickstart — plus the Quack endpoint enabled on the coordinator.
- Setting
quack_portin the coordinator config is all that is needed to enable the endpoint. It serves aGET /identification probe and aPOST /quackRPC surface. run.shalways validates the server-side probe (GET /). If a local DuckDB 1.5.3+ is on your PATH, it also seeds an Iceberg table in SQE and has DuckDB query it over Quack via thequack_query()table function.- The reverse direction (SQE as a Quack client, pulling from a DuckDB
quack_serveinstance) is documented in the repo README and verified separately — SQE’squack_query()function is available on every session.
Note that Quack is a pre-release protocol: DuckDB plans to stabilize it
around v2.0, and the client extension ships from core_nightly. The round-trip
works today (validated with duckdb 1.5.3) but is not a stable surface yet.
What it demonstrates
- Enabling SQE’s Quack server endpoint with a single config key.
- The
GET /identification probe confirming the endpoint is live. - A DuckDB client querying an SQE Iceberg table over the Quack protocol (forward round-trip).
- SQE as a Quack client:
quack_query()pulling rows from a remote DuckDBquack_serveinstance (reverse direction, documented and verified).
Status: experimental (2026-06-07).
Run it
Full config, docker compose, queries, and captured output are in the repo:
cd quickstart/quack
cp .env.example .env
./run.sh
Quack Protocol Reference (as of DuckDB extension v1.5-variegata)
Reference notes for implementing a Quack-compatible server and client in Rust. Extracted from the duckdb/duckdb-quack source (MIT, ~356 commits, May 2026) and the DuckDB v1.5.2+ release. Cross-checked against the announcement post and the in-repo docs/usage.md.
The Quack protocol is pre-release and the DuckDB project plans to stabilise it for v2.0 in September 2026. Treat this document as a snapshot, not a stable contract.
Status of upstream documentation
The upstream documentation has two surfaces with mismatched naming:
- README + source code:
quack_serve,quack_stop,quack:URI scheme, HTTP endpointPOST /quack, content typeapplication/vnd.duckdb. docs/usage.mdand FAQ:rpc_start,rpc_stop,POST /rpc, MIME typeapplication/duckdb.
The source is authoritative. The rpc_* doc names appear to be an older or aspirational naming. We follow the source.
The FAQ states “Quack uses HTTP v2.0”. The source uses httplib (a small C++ HTTP/1.1 library) with keep_alive_max_count(128). We treat the wire as HTTP/1.1 with keep-alive, not HTTP/2.
Transport
| Field | Value |
|---|---|
| Protocol | HTTP/1.1, keep-alive enabled |
| Default port | 9494 |
| URI scheme | quack:host[:port] (HTTPS by default for non-localhost, plain HTTP for localhost) |
| Endpoint | POST /quack |
| Content-Type (request and response) | application/vnd.duckdb |
| TLS | Optional. Server generates self-signed cert via quack_generate_keys(). Production deployments expected to terminate TLS at a reverse proxy. |
| CORS | Server returns Access-Control-Allow-Origin: * on OPTIONS /quack and on every response |
There is also a root path that returns a plain-text identification string:
GET / HTTP/1.1
HTTP/1.1 200 OK
Content-Type: text/plain
This is a DuckDB Quack RPC endpoint. Use ATTACH 'quack:...' to connect here.
Useful for sniffing whether a host speaks Quack.
Wire format
Every request body and every response body is a serialised QuackMessage. The serializer is DuckDB’s BinarySerializer with SerializationCompatibility::FromIndex(7). This is the same code path DuckDB uses for its Write-Ahead Log files.
Each message on the wire is:
[ serialized MessageHeader (BinarySerializer Begin/End block) ]
[ serialized message body (BinarySerializer Begin/End block) ]
BinarySerializer uses field-tagged encoding. Every field has a numeric ID, a type, and a value. Optional fields can be omitted. The schema (with stable field IDs) is captured in src/include/quack_message.json in the upstream repo and reproduced below for stability.
Message header
| Field ID | Name | Type | Notes |
|---|---|---|---|
| 1 | type | MessageType (enum) | See message types below |
| 2 | connection_id | string | Server-assigned, returned in CONNECTION_RESPONSE |
| 3 | client_query_id | optional_idx (u64) | Monotonic per-client query ID for log correlation |
MessageType is an enum encoded as idx_t:
INVALID = 0
CONNECTION_REQUEST = 1
CONNECTION_RESPONSE = 2
PREPARE_REQUEST = 3
PREPARE_RESPONSE = 4
FETCH_REQUEST = 5
FETCH_RESPONSE = 6
APPEND_REQUEST = 7
SUCCESS_RESPONSE = 8
DISCONNECT_MESSAGE = 9
ERROR_RESPONSE = 10
The exact wire ordering of enum tags depends on DuckDB’s EnumUtil; do not hard-code numeric values. Always go through the named enum.
Message bodies
ConnectionRequest
Initial handshake. Sent once per connection.
| Field | Type | Notes |
|---|---|---|
1 auth_string | string | Bearer token. Server’s auth function decides validity |
2 client_duckdb_version | string | e.g. "v1.5.2" |
3 client_platform | string | e.g. "osx_arm64" |
4 min_supported_quack_version | idx_t | client min |
5 max_supported_quack_version | idx_t | client max |
ConnectionResponse
| Field | Type | Notes |
|---|---|---|
1 server_duckdb_version | string | |
2 server_platform | string | |
3 quack_version | idx_t | Currently 1 |
Header carries the server-assigned connection_id; clients echo it in subsequent requests.
PrepareRequest
| Field | Type | Notes |
|---|---|---|
1 sql_query | string | Raw SQL |
PrepareResponse
| Field | Type | Notes |
|---|---|---|
1 result_types | vector<LogicalType> | Per-column DuckDB type |
2 result_names | vector<string> | Column names |
3 needs_more_fetch | bool | If true, client must follow up with FETCH_REQUEST using result_uuid |
4 results | vector<DataChunkWrapper> | Optional first batch of rows |
5 result_uuid | hugeint_t | Server-side handle for follow-up fetches |
The server may inline the entire result if it fits; otherwise it returns a result_uuid and the client pulls more via FETCH_REQUEST.
FetchRequest
| Field | Type | Notes |
|---|---|---|
1 uuid | hugeint_t | Result handle from PrepareResponse |
FetchResponse
| Field | Type | Notes |
|---|---|---|
1 results | vector<DataChunkWrapper> | Batched chunks |
2 batch_index | optional_idx | Sequence number for ordering |
AppendRequest
Bulk insert from client to server.
| Field | Type | Notes |
|---|---|---|
1 schema_name | string | Target schema |
2 table_name | string | Target table |
3 append_chunk | DataChunkWrapper | Row data |
SuccessResponse
Empty body. Used to acknowledge DisconnectMessage, AppendRequest, etc.
DisconnectMessage
Empty body. Client signals end of session. Server responds with SuccessResponse and closes the connection.
ErrorResponse
| Field | Type | Notes |
|---|---|---|
1 message | string | Raw error message |
DataChunk wire format
Results travel as DataChunkWrapper, which serialises one DuckDB DataChunk (vectorised columnar batch). The wrapper has a single field:
| Field ID | Name | Type |
|---|---|---|
| 300 | chunk | DataChunk |
A DataChunk is DuckDB’s native columnar batch type. Its serialisation includes:
- Number of columns
- Per-column
LogicalType(recursive for nested types) - Per-column
Vectordata (validity bitmap + data buffer + optional dictionary/auxiliary buffers)
This is not Arrow IPC. DuckDB has its own columnar layout. The two formats are not interchangeable without conversion.
For SQE to read these, we either:
- Link
libduckdband let DuckDB’s C++ code deserialise into a DataChunk, then convert to Arrow inside our process; or - Reimplement DuckDB’s
BinarySerializerandDataChunk::Serializesemantics in Rust.
Option 1 ties us to a specific DuckDB version but gets correctness for free. Option 2 is purer Rust but the maintenance cost tracks DuckDB releases. Decision recorded in openspec/changes/duckdb-quack-protocol-support/design.md (Open Questions section).
Authentication
The server’s quack_authentication_function (default quack_check_token) is a SQL scalar function with signature (sid VARCHAR, token VARCHAR) -> BOOLEAN. The default implementation compares the token against quack_default_token.
Users can plug their own auth by registering a scalar function with that signature and pointing the setting at it.
The token travels in ConnectionRequestMessage.auth_string. There is no separate Auth frame. Once ConnectionResponse returns, the connection is authenticated for the lifetime of that connection.
Per-query authorisation: quack_authorization_function is (sid VARCHAR, query VARCHAR) -> BOOLEAN. Default allows everything. Called server-side before executing each PrepareRequest.
Pushdown semantics
The server supports the following pushdowns when a client ATTACHes and then scans a remote table:
- Projection pushdown: only requested columns are returned
- Filter pushdown: constant comparisons (
=,<,>,<=,>=,<>),IS NULL,IS NOT NULL,IN (...), andAND/ORcombinations
Filters are evaluated server-side. Other predicates (function calls, joins) execute on the client.
For SQE-as-server: the SQL the client sends is already the filtered/projected SQL. We do not need to extract pushdowns from a separate field. The SQL string carries everything.
Logging
The extension registers two log types:
quacklog: structured per-message (message_type,connection_id,client_query_id,query,duration_ms,error)HTTPlog: per-request URL + status
For SQE compatibility, we should emit equivalent structured logs from the server crate.
Compatibility matrix
Server quack_version | Client min..max | Behaviour |
|---|---|---|
| 1 | min<=1<=max | OK |
| 1 | min>1 | Server returns ErrorResponse |
| Future N | client max < N | Server should downgrade if possible; otherwise reject |
Current quack_version = 1. The protocol is expected to bump versions before v2.0 stabilisation.
Things SQE will need to handle differently from DuckDB
- Iceberg-backed catalogs: DuckDB Quack assumes its own catalog. Our
Attachreturns SQE’s Iceberg catalog tree. DuckDB clients see Iceberg namespaces as schemas. - OIDC tokens vs static tokens: the auth function receives an opaque string. We treat it as an OIDC bearer and validate via
sqe-auth. Bare static tokens are still accepted ifsqe-authis configured for them. - Result format: SQE’s existing query engine produces Arrow
RecordBatch. We must convert eachRecordBatchto a DuckDBDataChunkbefore serialising. This conversion is non-trivial but tractable (both are columnar, both have validity bitmaps). - Policy enforcement: server-side SQL goes through
sqe-policySQL-text rewriter (seeopenspec/changes/duckdb-quack-protocol-support/design.md) before reaching the planner.
References
- Upstream repo: https://github.com/duckdb/duckdb-quack (MIT)
- Announcement: https://duckdb.org/2026/05/12/quack-remote-protocol
- DuckDB docs (overview): https://duckdb.org/docs/current/quack/overview
- FAQ: https://duckdb.org/quack/faq
- Local reference clone:
/tmp/duckdb-quack-src/(during research; delete after Phase 1)
Quack RPC datatype matrix
How DuckDB, Arrow/DataFusion, SQE’s LogicalTypeId, and Iceberg primitive types line up for the Quack RPC path. Status reflects what works through a real duckdb 1.5.3 CLI session (SELECT ... FROM quack_query('quack:localhost:9494', ...)) against sqe-server on a feature branch and later.
Scalar types
| DuckDB | Arrow / DataFusion | LogicalTypeId | Iceberg | Quack | Notes |
|---|---|---|---|---|---|
BOOLEAN | Boolean | Boolean | boolean | ✅ | nulls round-trip |
TINYINT | Int8 | TinyInt | (none) | ✅ | |
SMALLINT | Int16 | SmallInt | (none) | ✅ | |
INTEGER | Int32 | Integer | int | ✅ | |
BIGINT | Int64 | BigInt | long | ✅ | |
UTINYINT / USMALLINT / UINTEGER / UBIGINT | UInt8 / UInt16 / UInt32 / UInt64 | UTinyInt etc. | (none) | ⚠️ | wire encoding works; DataFusion SQL planner rejects unsigned literals in SELECT (upstream limitation, not ours) |
HUGEINT / UHUGEINT | (no native Arrow) | HugeInt / UHugeInt | decimal(38, 0) | ⚠️ | wire encoding works; DataFusion SQL planner rejects HUGEINT |
FLOAT | Float32 | Float | float | ✅ | |
DOUBLE | Float64 | Double | double | ✅ | |
DECIMAL(p, s) | Decimal128 | Decimal + ExtraTypeInfo::Decimal { precision, scale } | decimal(p, s) | ✅ | physical width tier-narrowed to i16/i32/i64/i128 per DuckDB; Decimal256 not supported; negative scale rejected |
VARCHAR | Utf8 / LargeUtf8 / Utf8View | Varchar | string | ✅ | DataFusion 53 emits Utf8View by default |
BLOB | Binary / LargeBinary / BinaryView | Blob | binary | ✅ | nulls round-trip |
DATE | Date32 | Date | date | ✅ | both sides use days-since-1970-01-01 |
DATE from Date64 | Date64 | Date | date | ✅ | narrowed to i32 days |
TIMESTAMP_S / _MS / _US (default TIMESTAMP) / _NS | Timestamp(Second/Millisecond/Microsecond/Nanosecond, None) | TimestampSec / TimestampMs / Timestamp / TimestampNs | timestamp | ✅ | timezone discarded; see follow-ups |
TIMESTAMP WITH TIME ZONE | Timestamp(*, Some(tz)) | TimestampTz | timestamptz | ❌ | timezone stripped today |
TIME | Time32(Second/Millisecond) / Time64(Microsecond) | Time | time | ✅ | Time32 variants widen ×1_000_000 / ×1_000 to i64 microseconds-of-day |
TIME_NS | Time64(Nanosecond) | TimeNs | (none, project as time) | ✅ | i64 nanoseconds-of-day passthrough |
UUID | FixedSizeBinary(16) | Uuid | uuid | ✅ | 16-byte raw passthrough; other widths rejected |
INTERVAL | Interval(YearMonth/DayTime/MonthDayNano) | Interval | (none) | ✅ | widens into DuckDB’s 16-byte interval_t { months, days, micros }; ns floored to micros |
BIT | (no native Arrow) | Bit | (none) | ❌ |
Nested types
| DuckDB | Arrow | LogicalTypeId | Iceberg | Quack |
|---|---|---|---|---|
LIST<T> | List / LargeList | List + ExtraTypeInfo::List { child } | list<T> | ✅ recursive child type; child element vector reused under field 106 |
STRUCT(...) | Struct | Struct + ExtraTypeInfo::Struct { fields } | struct<...> | ✅ pairs of (name, LogicalType) via child_list_t (pair fields 0/1) |
MAP<K, V> | Map | Map + ExtraTypeInfo::List { child: STRUCT(key, value) } | map<K, V> | ✅ DuckDB stores MAP as LIST<STRUCT<key,value>>; we reuse the LIST vector layout and stamp the parent type id as Map |
ARRAY<T, N> (fixed) | FixedSizeList | Array + ExtraTypeInfo::Array { child, size } | (none) | ✅ fields 103 (array_size) + 104 (child vector with size*count elements); size=0 honors WritePropertyWithDefault elision |
UNION | Union (dense or sparse) | Union + ExtraTypeInfo::Struct { fields } (tag-prefixed) | (none) | ⚠️ codec verified by unit test: DuckDB’s LogicalType::UNION factory builds a StructTypeInfo with a UTINYINT “tag” prepended to members, so we reuse the STRUCT wire layout and stamp the parent id as Union. DataFusion doesn’t emit UnionArray in practice; Arrow bridge mapping is deferred. |
ENUM | Dictionary(Int8/16/32/64 or UInt8/16/32/64, Utf8/LargeUtf8) | Enum + ExtraTypeInfo::Enum { values } | (none, project as string) | ⚠️ wire codec verified by unit tests (hand-written EnumTypeInfo: fields 200=values_count u64, 201=string list; per-row indices narrow to u8/u16/u32 by dict-size tier). DataFusion’s SQL planner rejects the ENUM(...) type literal and doesn’t dictionary-encode repeated strings, so end-to-end SQL exercising it via DataFusion isn’t currently possible — the path is ready for non-DataFusion engines or future DataFusion support |
Parameterised types
DuckDB’s LogicalType carries optional ExtraTypeInfo on the wire (field 101 of the LogicalType object). Wave 2a added the framework plus the DECIMAL variant; the remaining variants still surface as WireError::UnsupportedExtraTypeInfo:
DECIMAL(p, s)— ✅ encoded viaExtraTypeInfo::Decimal { precision, scale }. Storage tier follows DuckDB: precision 1-4 -> i16, 5-9 -> i32, 10-18 -> i64, 19-38 -> i128.LIST<T>— ✅ encoded viaExtraTypeInfo::List { child }. Recursive child type, child element vector under field 106.STRUCT(...)— ✅ encoded viaExtraTypeInfo::Struct { fields }. Pair entries with field 0 (name) + field 1 (LogicalType).MAP<K, V>— ✅ reusesExtraTypeInfo::List { child: STRUCT(key, value) }per DuckDB’s internalLogicalType::MAPfactory; no separateMapTypeInfo.ARRAY<T, N>— ✅ encoded viaExtraTypeInfo::Array { child, size }. Field 200 (child_type, WriteProperty) + field 201 (size, WritePropertyWithDefault default 0).ENUM— ✅ encoded viaExtraTypeInfo::Enum { values }. Custom serializer: field 200 (values_count u64,WriteProperty) + field 201 (WriteList<string>). Per-row index width follows DuckDB’sEnumTypeInfo::DictType: <=256 entries -> u8, <=65536 -> u16, otherwise u32.UNION— ✅ codec routes through STRUCT’s wire layout. DuckDB modelsUNION(members)as a StructTypeInfo with a UTINYINT tag prepended to the members, so no new ExtraTypeInfo variant is needed. Arrow bridge mapping is deferred because DataFusion never emitsUnionArray.- User-defined types — not implemented.
The full parameterised-type family is wired in the codec, in both directions. Forward (Arrow -> DataChunk) handles every type DataFusion emits; reverse (DataChunk -> Arrow) handles every type a remote DuckDB returns through the quack_query() TVF — LIST, STRUCT, MAP, ARRAY, ENUM (as Dictionary(UIntX, Utf8)), and arbitrarily deep compositions like STRUCT(tags VARCHAR[], counts MAP(VARCHAR, INT)). The remaining gaps are upstream (DataFusion’s planner rejecting certain SQL syntax) or low-traffic enough to defer (Arrow bridge for UNION’s UnionArray, which DataFusion does not emit in practice).
ExtraTypeInfo wire layout (verified against DuckDB v1.5.3 generated serializer):
- Base field 100 (u8):
ExtraTypeInfoTypediscriminant —WriteProperty, always written. - Base field 101 (string):
alias—WritePropertyWithDefault, omitted when “”. - Base field 102: deleted; readers tolerate but writers never emit.
- Base field 103 (
unique_ptr<ExtensionTypeInfo>):WritePropertyWithDefault, omitted when null. Unsupported in the codec. - Subclass fields per variant. For
DECIMAL: field 200 (width, u8,WritePropertyWithDefaultdefault 0) and field 201 (scale, u8,WritePropertyWithDefaultdefault 0). Scale 0 is the common case and omits field 201 entirely.
How to reproduce the matrix
# 1. Start the test stack and bootstrap once.
docker compose -f docker-compose.test.yml up -d
./scripts/bootstrap-test.sh
# 2. Start sqe-server with a Quack listener + BearerPassthrough auth.
cargo build --release --bin sqe-server
target/release/sqe-server --config tests/sqe-quack-test.toml &
# 3. Grab a Polaris bearer.
TOKEN=$(curl -s -X POST http://localhost:18181/api/catalog/v1/oauth/tokens \
-d "grant_type=client_credentials&client_id=root&client_secret=s3cr3t&scope=PRINCIPAL_ROLE:ALL" \
| python3 -c "import sys,json; print(json.load(sys.stdin)['access_token'])")
# 4. Query through real DuckDB CLI (1.5.2+):
duckdb -c "
INSTALL quack FROM core_nightly; LOAD quack;
CREATE SECRET (TYPE quack, TOKEN '${TOKEN}');
SELECT * FROM quack_query('quack:localhost:9494',
'SELECT 42 AS id, ''alice'' AS name, DATE ''2026-05-25'' AS joined');
"
tests/sqe-quack-test.toml is a copy of tests/sqe-test.toml with coordinator.quack_port = 9494 and an [[auth.providers]] type = "bearer_passthrough" entry.
Status
Every row marked ✅ has been verified end-to-end with a real duckdb 1.5.3 CLI session. The verification command for each row is SELECT <literal>::<type> ... against quack_query, and the assertion is that DuckDB renders the value back without error.
Client mode (Option B)
In addition to serving Quack RPC, SQE can act as a Quack client and pull rows from a remote DuckDB or another sqe-server:
-
sqe-quack-clientcrate exposes a synchronousQuackClientfor programmatic use. -
QuackTableProvideradapts a Quack query result to a DataFusionTableProvider(eager fetch, in-memory). -
quack_query(uri, [token,] sql)is registered as a TVF on every coordinator session, so any SQL client can pull remote tables inline:-- 2-arg form (no auth) SELECT * FROM quack_query('quack:remote-duckdb:9495', 'SELECT * FROM colors'); -- 3-arg form (bearer / static token) SELECT * FROM quack_query('quack:remote-duckdb:9495', 'remote-secret', 'SELECT * FROM colors');
This is symmetric to DuckDB’s own quack_query built-in. Composing the two lets a single DuckDB CLI session route queries through sqe-server, which itself fetches from a remote DuckDB — useful for federated reads or for treating DuckDB as an execution backend for specific workloads.
Federation: Iceberg + Quack in one query
Because both quack_query() and Iceberg tables surface as DataFusion TableProviders on the same session, a single SQL statement can freely mix the two:
-- JOIN an Iceberg table with rows pulled from a remote DuckDB
SELECT p.id, p.name AS person, r.color
FROM "default".quack_demo p -- Iceberg / Polaris
JOIN quack_query(
'quack:remote-duckdb:9495',
'remote-secret',
'SELECT id, name AS color FROM colors'
) r -- remote DuckDB via Quack
ON p.id = r.id;
Live-verified shapes:
- INNER JOIN (Iceberg ↔ Quack) on matching keys.
- COUNT/SUM aggregation across the join.
- UNION ALL of Iceberg rows with Quack rows (same projected schema).
- CROSS JOIN with DECIMAL preserved end-to-end.
- Filter/projection from either side.
DataFusion plans each query end-to-end; the Iceberg scan reads Parquet from object storage and the Quack TVF round-trips Arrow batches over HTTP/Quack — both feed into the same execution plan.
Observability: metrics + Grafana
slug: observability title: “Observability: metrics + Grafana” description: “Scrape SQE’s Prometheus metrics with VictoriaMetrics and visualise them in Grafana. A minimal queryable SQE (Nessie catalog) generates real metrics: queries, cache, sessions, scan pruning, coordinator memory.”
Observability: metrics + Grafana
SQE exposes Prometheus metrics on a configurable prometheus_port. This
quickstart scrapes them with VictoriaMetrics and renders them in Grafana,
so you can watch query rate, cache hit/miss, active sessions, scan pruning, and
coordinator memory while running queries.
A minimal queryable SQE (Nessie catalog + RustFS, anonymous auth) generates real metrics; the focus here is the monitoring pipeline.
How it works
- SQE serves Prometheus text metrics at
/metricson itsprometheus_port. The coordinator binds the metrics port on all interfaces so VictoriaMetrics can reach it in-network. - VictoriaMetrics scrapes the SQE metrics endpoint every 5 seconds using a
scrape config that names the target
sqe-coordinator. - Grafana is provisioned with VictoriaMetrics as the default datasource and
an SQE Overview dashboard. Open it at
http://localhost:13000(admin / admin) after the stack starts. run.shstarts the stack, runsqueries.sqla few times to generate metrics, waits for a scrape, then queries the VictoriaMetrics API to assert SQE’s counters are present.
A selection of the metrics SQE exposes: sqe_rows_returned_total,
sqe_active_sessions, sqe_cache_hits_total / sqe_cache_misses_total,
sqe_files_pruned_minmax_total, sqe_coordinator_memory_used_bytes,
sqe_s3_requests_total / sqe_s3_bytes_read_total.
What it demonstrates
- The full Prometheus → VictoriaMetrics → Grafana scrape pipeline against a live SQE instance.
- SQE’s
prometheus_portconfig and the metrics it exposes out of the box. - The provisioned SQE Overview Grafana dashboard.
- Asserted via the VictoriaMetrics query API: the
up{job="sqe-coordinator"}target is 1 and SQE counters are populated.
Status: validated (2026-06-07).
Run it
Full config, docker compose, dashboard, and captured output are in the repo:
cd quickstart/observability
cp .env.example .env
./run.sh
Benchmarks: TPC-H / TPC-DS / SSB
slug: benchmark title: “Benchmarks: TPC-H / TPC-DS / SSB” description: “Generate a TPC dataset, load it into SQE as Iceberg tables, and run the suite’s queries with per-query timings. Uses sqe-bench against a Nessie catalog over RustFS, all in Docker.”
Benchmarks: TPC-H / TPC-DS / SSB
Generate a benchmark dataset, load it into SQE as Iceberg tables, and run the
suite’s queries with per-query timings. Everything runs in Docker: a Nessie
catalog over RustFS holds the tables, and sqe-bench drives generate, load, and
test. The default is TPC-H at scale factor 0.01, which completes in seconds.
How it works
The run has three phases:
- Generate —
sqe-bench generatewrites the suite’s tables as Parquet files to a shared Docker volume. - Load —
sqe-bench loadissues oneCREATE TABLE … AS SELECT * FROM read_parquet(…)per table. The coordinator reads the Parquet from the shared volume; tables land in the Nessie catalog. - Test —
sqe-bench testruns every.sqlin the suite over Flight SQL and reports pass / fail / error plus a per-query timing table.
sqe-bench is a separate image built from Dockerfile.bench. Both images build
from this repo on first run if absent.
What it demonstrates
- All three
sqe-benchphases — generate, load, and test — running end to end. - TPC-H (default, SF0.01): all 22 queries pass with per-query timings.
- Configurable suite (
BENCH=ssb,BENCH=tpcds) and scale factor (SCALE=0.1,SCALE=1). sqe-bench testoutput: a pass/fail/diff/skip/error summary and aBENCH_SUMMARY:line for machine parsing.
Status: validated (2026-06-07).
Run it
Full config, docker compose, suite SQL, and captured output are in the repo:
cd quickstart/benchmark
cp .env.example .env
./run.sh
Configuration
SQE is configured via a TOML file with environment variable overrides. Environment variables take precedence over the config file.
Config File
Default path: sqe.toml in the current directory. Override with:
sqe-server --config /etc/sqe/sqe.toml
# or
SQE_CONFIG=/etc/sqe/sqe.toml sqe-server
Full Reference
[coordinator]
flight_sql_port = 50051 # Flight SQL gRPC port
trino_http_port = 8080 # Trino-compat HTTP port (0 to disable)
mode = "hybrid" # "coordinator", "worker", "hybrid", "local", "distributed"
worker_urls = [] # Worker Flight URLs for distributed mode
debug = false # When true, error messages include internal details (dev only)
[coordinator.tls]
cert_file = "" # PEM certificate — TLS enabled when both cert + key are set
key_file = "" # PEM private key
ca_file = "" # Optional PEM CA for mTLS client certificate verification
[worker]
coordinator_url = "http://coordinator:50051"
flight_port = 50052 # Worker Flight port
heartbeat_interval_secs = 5 # Health check interval
memory_limit = "8GB" # Worker memory limit (supports B/KB/MB/GB/TB)
spill_to_disk = true # Allow spilling large sorts/joins to disk
spill_dir = "/tmp/sqe-spill" # Temp directory for spilling
[auth]
keycloak_url = "" # Keycloak base URL (OIDC password grant mode)
realm = "" # Keycloak realm name
token_endpoint = "" # Generic OAuth2 token endpoint (client_credentials mode)
client_id = "sqe-client" # OIDC client ID (required)
client_secret = "" # Set via SQE_AUTH__CLIENT_SECRET env var
token_refresh_buffer_secs = 60 # Refresh tokens this many seconds before expiry
ssl_verification = true # Set false for dev (self-signed certs)
[catalog]
catalog_url = "http://polaris:8181/api/catalog" # REST catalog endpoint
warehouse = "iceberg" # warehouse identifier the catalog expects
metadata_cache_ttl_secs = 30 # Table metadata cache TTL
default_table_format_version = 2 # Iceberg table format version (2 or 3)
# `catalog_url` accepts any Iceberg REST endpoint. SQE has been
# verified live against Apache Polaris, Project Nessie 0.107+,
# Unity Catalog OSS, AWS Glue Iceberg REST, and AWS S3 Tables REST.
# For AWS REST endpoints the vendored REST client signs requests
# with SigV4 when the server advertises `rest.sigv4-enabled=true`
# in its /v1/config defaults.
# When `[catalog.backend]` is omitted, SQE defaults to `type = "rest"`
# and uses `catalog_url` + `warehouse` above. To target a non-REST
# catalog (HMS, AWS Glue native, AWS S3 Tables native, JDBC, Hadoop),
# set the backend block explicitly. See `docs/book/src/getting-started/
# catalogs.md` for the full per-backend reference.
# [catalog.backend]
# type = "hms"
# uri = "metastore.example.com:9083"
# warehouse = "s3a://my-bucket/warehouse"
# [catalog.backend]
# type = "glue"
# region = "eu-example-1"
# warehouse = "s3://my-bucket/warehouse"
# # endpoint = "http://localhost:4566" # optional, e.g. LocalStack
# [catalog.backend]
# type = "s3tables"
# table_bucket_arn = "arn:aws:s3tables:eu-example-2:ACCOUNT_ID:bucket/my-bucket"
# # endpoint_url = "http://localhost:4566"
# [catalog.backend]
# type = "jdbc"
# url = "postgresql://user:pass@host:5432/iceberg"
# warehouse = "s3://my-bucket/warehouse"
# [catalog.backend]
# type = "hadoop"
# warehouse = "s3://my-bucket/warehouse"
# Non-REST backends dispatch through the upstream
# `iceberg-catalog-loader` crate. End-to-end SQL through HMS, Glue,
# S3 Tables, and JDBC works on main today. Hadoop has its own
# dispatch in `sqe-catalog/src/backends/hadoop.rs`.
[storage]
s3_endpoint = "http://s3:9000"
s3_region = "us-east-1"
s3_access_key = "" # Set via SQE_STORAGE__S3_ACCESS_KEY
s3_secret_key = "" # Set via SQE_STORAGE__S3_SECRET_KEY
s3_path_style = true # true for MinIO/Ceph, false for AWS S3
[policy]
engine = "passthrough" # "passthrough" (only option currently; "opa", "cedar" planned)
[session]
idle_timeout_secs = 900 # 15 min — sessions idle longer are expired
absolute_timeout_secs = 28800 # 8 hours — hard session lifetime cap
[query]
timeout_secs = 300 # 5 min — max execution time per query
[query.role_overrides] # Per-role timeout overrides (seconds)
# admin = 3600 # Admins get 1 hour
# analyst = 600 # Analysts get 10 minutes
[rate_limit]
enabled = false # Enable per-user and global rate limiting
per_user_queries_per_minute = 60
global_queries_per_minute = 1000
[metrics]
prometheus_port = 9090 # Prometheus /metrics endpoint
otlp_endpoint = "" # OTLP gRPC endpoint (empty = disabled)
audit_log_path = "" # Audit JSONL file (empty = disabled)
Environment Variable Overrides
Every config field can be overridden via environment variable. Convention: SQE_<SECTION>__<FIELD> (double underscore separating section from field).
| Env Var | Config Field | Type |
|---|---|---|
| Coordinator | ||
SQE_COORDINATOR__FLIGHT_SQL_PORT | coordinator.flight_sql_port | u16 |
SQE_COORDINATOR__TRINO_HTTP_PORT | coordinator.trino_http_port | u16 |
SQE_COORDINATOR__MODE | coordinator.mode | string |
SQE_COORDINATOR__DEBUG | coordinator.debug | bool |
| TLS | ||
SQE_TLS__CERT_FILE | coordinator.tls.cert_file | string |
SQE_TLS__KEY_FILE | coordinator.tls.key_file | string |
SQE_TLS__CA_FILE | coordinator.tls.ca_file | string |
| Worker | ||
SQE_WORKER__COORDINATOR_URL | worker.coordinator_url | string |
SQE_WORKER__FLIGHT_PORT | worker.flight_port | u16 |
SQE_WORKER__HEARTBEAT_INTERVAL_SECS | worker.heartbeat_interval_secs | u64 |
SQE_WORKER__MEMORY_LIMIT | worker.memory_limit | string |
SQE_WORKER__SPILL_TO_DISK | worker.spill_to_disk | bool |
SQE_WORKER__SPILL_DIR | worker.spill_dir | string |
| Auth | ||
SQE_AUTH__KEYCLOAK_URL | auth.keycloak_url | string |
SQE_AUTH__REALM | auth.realm | string |
SQE_AUTH__TOKEN_ENDPOINT | auth.token_endpoint | string |
SQE_AUTH__CLIENT_ID | auth.client_id | string |
SQE_AUTH__CLIENT_SECRET | auth.client_secret | string |
SQE_AUTH__TOKEN_REFRESH_BUFFER_SECS | auth.token_refresh_buffer_secs | u64 |
SQE_AUTH__SSL_VERIFICATION | auth.ssl_verification | bool |
| Catalog | ||
SQE_CATALOG__CATALOG_URL | catalog.catalog_url | string |
SQE_CATALOG__POLARIS_URL | catalog.catalog_url (legacy alias) | string |
SQE_CATALOG__WAREHOUSE | catalog.warehouse | string |
SQE_CATALOG__METADATA_CACHE_TTL_SECS | catalog.metadata_cache_ttl_secs | u64 |
SQE_CATALOG__DEFAULT_TABLE_FORMAT_VERSION | catalog.default_table_format_version | u8 |
| Storage | ||
SQE_STORAGE__S3_ENDPOINT | storage.s3_endpoint | string |
SQE_STORAGE__S3_REGION | storage.s3_region | string |
SQE_STORAGE__S3_ACCESS_KEY | storage.s3_access_key | string |
SQE_STORAGE__S3_SECRET_KEY | storage.s3_secret_key | string |
SQE_STORAGE__S3_PATH_STYLE | storage.s3_path_style | bool |
| Policy | ||
SQE_POLICY__ENGINE | policy.engine | string |
| Session | ||
SQE_SESSION__IDLE_TIMEOUT_SECS | session.idle_timeout_secs | u64 |
SQE_SESSION__ABSOLUTE_TIMEOUT_SECS | session.absolute_timeout_secs | u64 |
| Query | ||
SQE_QUERY__TIMEOUT_SECS | query.timeout_secs | u64 |
| Rate Limit | ||
SQE_RATE_LIMIT__ENABLED | rate_limit.enabled | bool |
SQE_RATE_LIMIT__PER_USER_QUERIES_PER_MINUTE | rate_limit.per_user_queries_per_minute | u32 |
SQE_RATE_LIMIT__GLOBAL_QUERIES_PER_MINUTE | rate_limit.global_queries_per_minute | u32 |
| Metrics | ||
SQE_METRICS__PROMETHEUS_PORT | metrics.prometheus_port | u16 |
SQE_METRICS__OTLP_ENDPOINT | metrics.otlp_endpoint | string |
SQE_METRICS__AUDIT_LOG_PATH | metrics.audit_log_path | string |
Boolean values accept: true/false, 1/0, yes/no.
TLS
SQE supports optional TLS encryption for the Flight SQL gRPC listener.
Server-side TLS: Set cert_file and key_file to enable. When both are set, the server listens on TLS; when omitted, plaintext.
mTLS (mutual TLS): Set ca_file to a PEM CA bundle. Clients must present a certificate signed by this CA.
[coordinator.tls]
cert_file = "/etc/sqe/server.crt"
key_file = "/etc/sqe/server.key"
ca_file = "/etc/sqe/ca.crt" # Optional: enables mTLS
Validation rules:
- If either
cert_fileorkey_fileis set, both must be set - All referenced files must exist when TLS is enabled
ca_fileis optional – when set, it must also exist
Authentication Modes
SQE supports two OAuth2 flows, selected by which config fields are populated:
OIDC Password Grant (Keycloak)
For environments with Keycloak (or any OIDC provider supporting ROPC). The coordinator exchanges the user’s username/password for tokens:
[auth]
keycloak_url = "https://keycloak.example.com"
realm = "iceberg"
client_id = "sqe-client"
OAuth2 Client Credentials
For service-to-service auth or providers without ROPC support. The coordinator obtains tokens using a client ID and secret. Set token_endpoint directly:
[auth]
token_endpoint = "http://polaris:8181/api/catalog/v1/oauth/tokens"
client_id = "root"
client_secret = "s3cr3t"
At least one of keycloak_url or token_endpoint must be configured. If both are set, keycloak_url takes priority (OIDC mode).
Validation
SQE validates config at startup and fails fast on errors:
auth.client_idmust not be emptycatalog.catalog_urlmust not be empty- At least one of
auth.keycloak_urlorauth.token_endpointmust be set coordinator.flight_sql_portmust differ fromcoordinator.trino_http_portcoordinator.flight_sql_portmust differ frommetrics.prometheus_port- TLS: if either cert or key is set, both must be set; referenced files must exist
Priority Order
CLI flags (--mode, --config) > Environment variables > Config file > Defaults
Sensitive Values
Never put secrets in the TOML config file. Use environment variables or Kubernetes Secrets:
# Environment
export SQE_AUTH__CLIENT_SECRET="my-secret"
export SQE_STORAGE__S3_ACCESS_KEY="minioadmin"
export SQE_STORAGE__S3_SECRET_KEY="minioadmin"
# Kubernetes Secret (via Helm)
helm install sqe deploy/helm/sqe/ \
--set secrets.SQE_AUTH__CLIENT_SECRET=xxx \
--set secrets.SQE_STORAGE__S3_SECRET_KEY=xxx
Docker
SQE ships as a single Docker image containing two binaries: sqe-server (the engine) and sqe-cli (the client).
Image Layout
graph TB
subgraph "sqe:latest (debian:bookworm-slim)"
BIN1["/usr/local/bin/sqe-server"]
BIN2["/usr/local/bin/sqe-cli"]
USER["User: sqe (UID 1000)"]
EP["ENTRYPOINT: sqe-server"]
end
- Base:
debian:bookworm-slim(~80MB) — provides glibc, OpenSSL, CA certificates - User: Non-root
sqe(UID 1000) - Entrypoint:
sqe-server— the mode is selected via--modeflag orSQE_MODEenv var
Build
docker build -t sqe:latest .
# With metadata labels
docker build -t sqe:0.1.0 \
--build-arg VERSION=0.1.0 \
--build-arg BUILD_DATE=$(date -u +%Y-%m-%dT%H:%M:%SZ) \
--build-arg GIT_REVISION=$(git rev-parse HEAD) \
.
Run Coordinator
docker run -d \
--name sqe-coordinator \
-p 50051:50051 \
-p 8080:8080 \
-p 9090:9090 \
-p 9091:9091 \
-v $(pwd)/sqe.toml:/etc/sqe/sqe.toml:ro \
-e SQE_AUTH__CLIENT_SECRET=my-secret \
-e SQE_STORAGE__S3_ACCESS_KEY=minioadmin \
-e SQE_STORAGE__S3_SECRET_KEY=minioadmin \
sqe:latest --config /etc/sqe/sqe.toml
The default mode is coordinator, so no --mode flag needed.
Run Worker
docker run -d \
--name sqe-worker-1 \
-p 50052:50052 \
-v $(pwd)/sqe.toml:/etc/sqe/sqe.toml:ro \
sqe:latest --mode worker --config /etc/sqe/sqe.toml
Use the CLI
# Interactive SQL against running coordinator
docker exec -it sqe-coordinator sqe-cli
# One-shot query
docker exec sqe-coordinator sqe-cli -e "SELECT COUNT(*) FROM raw.orders;"
# With explicit connection
docker run --rm -it --network host \
sqe:latest sqe-cli --host localhost --port 50051 --user alice
Note: when using docker exec, sqe-cli connects to localhost:50051 by default — which is the coordinator running in the same container.
Docker Compose
services:
coordinator:
image: sqe:latest
command: ["--config", "/etc/sqe/sqe.toml"]
ports:
- "50051:50051"
- "8080:8080"
- "9090:9090"
- "9091:9091"
volumes:
- ./sqe.toml:/etc/sqe/sqe.toml:ro
environment:
SQE_AUTH__CLIENT_SECRET: ${SQE_AUTH_SECRET}
SQE_STORAGE__S3_ACCESS_KEY: ${S3_ACCESS_KEY}
SQE_STORAGE__S3_SECRET_KEY: ${S3_SECRET_KEY}
healthcheck:
test: ["CMD", "curl", "-f", "http://localhost:9091/healthz"]
interval: 10s
timeout: 5s
retries: 3
worker:
image: sqe:latest
command: ["--mode", "worker", "--config", "/etc/sqe/sqe.toml"]
deploy:
replicas: 2
volumes:
- ./sqe.toml:/etc/sqe/sqe.toml:ro
depends_on:
coordinator:
condition: service_healthy
Why One Image?
| Concern | Answer |
|---|---|
| Version skew | Coordinator, workers, and CLI are always the same build |
| CI/CD | One image to build, scan, and promote |
| K8s simplicity | Same image: field, different --mode arg |
| Debugging | kubectl exec into any pod → sqe-cli is right there |
| Size overhead | Minimal — both roles share 95% of their code |
Kubernetes & Helm
SQE includes a Helm chart for production Kubernetes deployment.
Architecture on K8s
graph TB
subgraph "Kubernetes Cluster"
subgraph "Coordinator Deployment"
C1["sqe-server<br/>--mode coordinator"]
end
subgraph "Worker Deployment (optional)"
W1["sqe-server<br/>--mode worker"]
W2["sqe-server<br/>--mode worker"]
end
SVC["Service: sqe-coordinator<br/>ClusterIP"]
CM["ConfigMap: sqe-config<br/>(sqe.toml)"]
SEC["Secret: sqe-secrets<br/>(credentials)"]
SM["ServiceMonitor<br/>(optional)"]
SVC --> C1
CM --> C1
CM --> W1
CM --> W2
SEC --> C1
SEC --> W1
SEC --> W2
C1 --> W1
C1 --> W2
end
CLIENT["Clients"] --> SVC
PROM["Prometheus"] --> SM
Install with Helm
Single-Node (small environments)
helm install sqe deploy/helm/sqe/ \
--set config.auth.keycloak_url=https://keycloak.example.com \
--set config.catalog.catalog_url=http://polaris:8181/api/catalog \
--set secrets.SQE_AUTH__CLIENT_SECRET=my-secret \
--set secrets.SQE_STORAGE__S3_ACCESS_KEY=minioadmin \
--set secrets.SQE_STORAGE__S3_SECRET_KEY=minioadmin
Workers are disabled by default — the coordinator runs queries locally.
Distributed (production)
helm install sqe deploy/helm/sqe/ \
--set worker.enabled=true \
--set worker.replicas=4 \
--set coordinator.resources.limits.memory=4Gi \
--set worker.resources.limits.memory=16Gi \
--set worker.resources.limits.cpu=8 \
--set config.auth.keycloak_url=https://keycloak.example.com \
--set config.catalog.catalog_url=http://polaris:8181/api/catalog \
--set existingSecret=sqe-credentials
Using an Existing Secret
Create the secret separately (e.g., via sealed-secrets or external-secrets-operator):
apiVersion: v1
kind: Secret
metadata:
name: sqe-credentials
type: Opaque
stringData:
SQE_AUTH__CLIENT_SECRET: "my-secret"
SQE_STORAGE__S3_ACCESS_KEY: "AKIA..."
SQE_STORAGE__S3_SECRET_KEY: "wJalrXUtnFEMI/K7MDENG..."
Then reference it:
helm install sqe deploy/helm/sqe/ --set existingSecret=sqe-credentials
Values Reference
Image
image:
repository: sqe
tag: latest # Defaults to Chart.appVersion
pullPolicy: IfNotPresent
imagePullSecrets: []
Coordinator
coordinator:
replicas: 1
resources:
requests: { memory: "512Mi", cpu: "500m" }
limits: { memory: "2Gi", cpu: "2" }
nodeSelector: {}
tolerations: []
affinity: {}
podAnnotations: {}
Workers
worker:
enabled: false # Enable for distributed execution
replicas: 2
resources:
requests: { memory: "1Gi", cpu: "1" }
limits: { memory: "8Gi", cpu: "4" }
nodeSelector: {}
tolerations: []
affinity: {}
podAnnotations: {}
Service
service:
type: ClusterIP
flightSqlPort: 50051
trinoHttpPort: 8080
metricsPort: 9090
Health Probes
healthPort: 9091
livenessProbe:
initialDelaySeconds: 5
periodSeconds: 10
readinessProbe:
initialDelaySeconds: 5
periodSeconds: 5
Monitoring
serviceMonitor:
enabled: false
interval: 30s
labels: {} # e.g., { release: prometheus }
Operations
Scaling Workers
kubectl scale deployment sqe-worker --replicas=8
# or
helm upgrade sqe deploy/helm/sqe/ --set worker.replicas=8
Rolling Update
Config changes trigger automatic rolling restarts (via checksum annotation on the ConfigMap):
helm upgrade sqe deploy/helm/sqe/ --set config.catalog.metadata_cache_ttl_secs=60
Interactive SQL
kubectl exec -it deploy/sqe-coordinator -- sqe-cli
Port Forwarding
# Flight SQL
kubectl port-forward svc/sqe-coordinator 50051:50051
# Trino HTTP (for dashboards)
kubectl port-forward svc/sqe-coordinator 8080:8080
# Metrics
kubectl port-forward svc/sqe-coordinator 9090:9090
Logs
kubectl logs deploy/sqe-coordinator -f
kubectl logs deploy/sqe-worker -f
Logs are structured JSON — pipe to jq for readability:
kubectl logs deploy/sqe-coordinator | jq .
OpenLineage
SQE emits OpenLineage 2-0-2 events for queries that mutate data: INSERT, CTAS, MERGE, UPDATE, DELETE, plus DDL on tables. SELECT events are off by default and can be enabled per environment.
Use these events to drive a lineage UI (Marquez, DataHub) or to feed a metadata catalog. Off by default. Zero hot-path overhead when disabled.
What gets emitted
| Statement | Emits? | Inputs | Outputs |
|---|---|---|---|
| SELECT | only if emit_selects = true | yes | none |
| INSERT, CTAS, MERGE | always | yes | yes |
| UPDATE, DELETE | always | yes | yes |
| CREATE TABLE, ALTER, DROP | always | none | yes (with schema facet) |
| OPTIMIZE, VACUUM, REWRITE_MANIFESTS | never | n/a | n/a |
Each query produces two events: a START at submit time and a COMPLETE on success or FAIL on error.
Configuration
[metrics.openlineage]
enabled = true
job_namespace = "sqe-prod" # per-env
# at least one sink required
file_path = "/var/log/sqe/lineage.jsonl"
http_endpoint = "https://marquez.example.com/api/v1/lineage"
# HTTP transport
auth_mode = "bearer" # "none" | "bearer" | "user_token"
api_key = "..." # required when auth_mode = "bearer"
http_timeout_ms = 5000
http_retry_attempts = 1
# disk spool fallback (recommended when HTTP is configured)
spool_path = "/var/spool/sqe-ol"
spool_max_bytes = 104857600
replay_interval_secs = 30
# back-pressure (rarely needs tuning)
channel_capacity = 10000
emit_selects = false
Every TOML key has an SQE_METRICS__OPENLINEAGE__* env override.
Sink choice
Pick one or both:
- File sink: append-only JSONL at
file_path. Use for SIEM ingestion, debug, or local development. - HTTP sink: POST to an OL collector. Use for Marquez/DataHub. Add a
spool_pathso a collector outage does not lose events.
When both are set, every event goes to both. Failures on one sink do not block the other.
Marquez quickstart
docker run -p 5000:5000 -p 5001:5001 marquezproject/marquez
In sqe.toml:
[metrics.openlineage]
enabled = true
http_endpoint = "http://localhost:5000/api/v1/lineage"
auth_mode = "none"
Restart SQE. Run a query. Browse Marquez at http://localhost:3000.
DataHub quickstart
DataHub’s OL receiver expects a bearer token:
[metrics.openlineage]
enabled = true
http_endpoint = "https://datahub.example.com/openapi/openlineage/api/v1/lineage"
auth_mode = "bearer"
api_key = "${DATAHUB_TOKEN}"
spool_path = "/var/spool/sqe-ol"
Troubleshooting
No events appear in Marquez or DataHub. Check enabled = true and that at least one sink is configured. Validation is enforced at startup; the server refuses to start with a misconfigured block.
Spool directory grows. The collector is unreachable. SQE buffers up to spool_max_bytes then drops newest events. Inspect /var/spool/sqe-ol/spool.jsonl*. Restore the collector and the replay loop drains within replay_interval_secs.
sqe_lineage_dropped_events_total counter increments. The bounded mpsc channel was full. Increase channel_capacity or investigate why the emitter is slow (collector too slow? HTTP timeouts?).
sqe_lineage_sink_errors_total{sink="http"} increments. Same: collector unreachable. With spool_path configured, events still reach disk.
v1 limitations
- No mTLS to the collector. Bearer auth only.
auth_mode = "user_token"(forwarding the user’s OIDC bearer per event) is wired but currently falls back to the staticapi_keyin v1; full per-event token forwarding is a follow-up.- Maintenance procedures (OPTIMIZE, VACUUM, REWRITE_MANIFESTS) are never emitted.
- MERGE column-level lineage is emitted at the dataset level; per-branch annotations (MERGE_INSERT vs MERGE_UPDATE) are deferred until DataFusion exposes a Merge LogicalPlan node.
- DDL paths (CREATE TABLE, DROP, etc.) emit events with
plan = None. The dataset target is captured but column lineage is empty for DDL. - Embedded CLI (
sqe-cliad-hoc mode) does not emit lineage. Production server (sqe-server) is the only emit path.
Runtime catalog management
SQE supports DuckDB-style ATTACH and DETACH for mounting Iceberg
catalogs from SQL at runtime. Credentials live in a session-local
secret store managed with CREATE SECRET / DROP SECRET. The same
six backends documented in Catalog backends
work here, plus a SQLite backend for local prototyping.
Use this when:
- An analyst wants to point at a partner’s catalog for a single session without redeploying.
- The dev loop needs a quick local catalog without editing TOML.
- Operators want to provision shared bearer tokens or AWS profiles centrally and have queries reference them by name.
ATTACH is process-local. Catalogs attached via SQL are wiped on
coordinator restart. Static TOML catalogs (the [catalog] and
[catalogs.*] blocks) are the right shape for “this is part of the
deployment.” ATTACH is the right shape for “this is part of this
session.”
Syntax
ATTACH '<location>' AS <name> (TYPE <kind>, <key> = <value>, ...);
DETACH <name>;
CREATE SECRET <name> (TYPE <kind>, <key> = <value>, ...);
DROP SECRET <name>;
SHOW SECRETS;
<location> is the connection target (URL, ARN, file path) and its
meaning depends on TYPE. Option keys are case-insensitive. String
values are single-quoted. The one exception is SECRET <name> which
takes a bare identifier so it can be looked up in the secret store.
Catalog kinds
TYPE value | <location> shape | Required options | Optional options |
|---|---|---|---|
iceberg_rest | URL of the Iceberg REST endpoint | WAREHOUSE | SECRET (bearer) |
glue | empty string (region drives discovery) | WAREHOUSE | SECRET (aws), REGION |
s3tables | empty string | TABLE_BUCKET_ARN | SECRET (aws), ENDPOINT_URL |
hms | Thrift URI (thrift://host:9083) | WAREHOUSE | |
jdbc | JDBC connection string | WAREHOUSE | SECRET (basic) |
sqlite | local directory path | ||
hadoop | warehouse path on object store or local FS |
Secret kinds
TYPE value | Required keys | Used by |
|---|---|---|
bearer | TOKEN | iceberg_rest |
basic | USERNAME, PASSWORD | jdbc |
aws | any of ACCESS_KEY_ID, SECRET_ACCESS_KEY, SESSION_TOKEN, REGION, PROFILE | glue, s3tables |
A bearer secret stores one token. A basic secret stores a username and password. An AWS secret can hold any combination of credential fields; missing fields fall through to the standard AWS credential chain (env vars, profile, IMDS).
Example: REST catalog with bearer token
CREATE SECRET partner_tok (TYPE bearer, TOKEN 'eyJhbGciOiJSUzI1...');
ATTACH 'http://catalog.example.com:9090/api/catalog' AS partner
(TYPE iceberg_rest, WAREHOUSE 'analytics', SECRET partner_tok);
SELECT * FROM partner.sales.orders LIMIT 10;
DETACH partner;
DROP SECRET partner_tok;
The token never appears in plan history or query logs after the
CREATE SECRET statement; subsequent statements reference it by
name. The token bytes are zeroized when the secret is dropped or
the coordinator exits cleanly.
Example: AWS Glue with explicit credentials
CREATE SECRET aws_dev (TYPE aws,
ACCESS_KEY_ID = 'AKIA...',
SECRET_ACCESS_KEY = 'wJalrXUt...',
REGION = 'eu-example-1');
ATTACH '' AS glue_dev
(TYPE glue, WAREHOUSE 's3://my-warehouse/', SECRET aws_dev);
SELECT * FROM glue_dev.public.events LIMIT 5;
Example: AWS Glue using the standard credential chain
Skip SECRET and the AWS SDK uses its default chain (env vars,
shared profile, IMDS, container credentials).
ATTACH '' AS glue_prod
(TYPE glue, WAREHOUSE 's3://prod-warehouse/');
This is the same chain aws-sdk-glue uses everywhere else. EKS
service accounts, EC2 instance roles, and ~/.aws/credentials
profiles all work without an explicit CREATE SECRET.
Example: SQLite for local prototyping
ATTACH '/tmp/sqe-dev' AS local (TYPE sqlite);
CREATE SCHEMA local.tutorial;
CREATE TABLE local.tutorial.events (id BIGINT, ts TIMESTAMP);
INSERT INTO local.tutorial.events VALUES (1, NOW());
The location is a directory. SQE creates <dir>/catalog.db
(SQLite-backed Iceberg catalog) and a <dir>/warehouse/ subdirectory
for table data. Useful for dbt model development without a Polaris
deployment.
SHOW CATALOGS and SHOW SECRETS
SHOW CATALOGS includes every TOML-configured catalog plus the two
coordinator-registered system catalogs (system, datafusion) plus
every name added via ATTACH. The list updates immediately after
each ATTACH or DETACH.
SHOW SECRETS;
returns a two-column result (name, type). Secret values are not
exposed; the table is for inventory only.
Authorization
Out of the box, ATTACH and CREATE SECRET are open to any authenticated
session. Lock them down through the same OPA / Cedar policy backend
that gates GRANT and REVOKE: write a rule that denies
statement_kind == "attach" for non-admin roles. The plan rewriter
sees the statement before it reaches the registry, so a denied ATTACH
errors at policy enforcement time, not at catalog build time.
Lifecycle and persistence
The runtime catalog registry is process-local and in-memory. A restart wipes every attached catalog and every created secret. There is no on-disk persistence in v1.
This is intentional. Persistent ATTACH (where catalogs survive a restart) is a feature operators ask for but most do not want once they think it through. A catalog attached at 9 AM on Monday is in the system at 3 AM on Sunday because someone forgot to DETACH it. The credentials behind it have rotated. Queries against it return 401. The on-call engineer wakes up to a query failure for a catalog they did not know existed. Static TOML is the right place for “part of the deployment.” ATTACH is the right place for “part of this session.”
Troubleshooting
catalog '<name>' is already attached; DETACH it first. A catalog
with that name is in the registry. Issue DETACH <name> first or
choose a different name. The check is case-sensitive.
catalog '<name>' is not attached. DETACH was issued for an
unknown name. Check SHOW CATALOGS for the spelling.
secret '<name>' is referenced by attached catalogs: <list>.
DROP SECRET while one or more attached catalogs reference it. DETACH
the listed catalogs first, then retry the drop. The error names every
referencing catalog so you do not have to chase them one at a time.
secret '<name>' not found. ATTACH ... SECRET nonexistent was
issued without a matching CREATE SECRET. Names are case-sensitive.
Failed to list namespaces: ... during ATTACH. The catalog was
built but the initial list_namespaces call against the backend
failed. Check that <location> and the credentials are correct, and
that the network path between the coordinator and the catalog is
reachable. The error message includes the upstream HTTP status or
SDK error.
Bearer token is in the request but the catalog returns 401. Check that the token is valid against the catalog’s expected issuer. Bearer tokens stored as secrets are forwarded as-is; SQE does not reissue or refresh them.
Dynamic Polaris catalog discovery
ATTACH and [catalogs.*] both name catalogs explicitly. For
dynamically-provisioned Polaris warehouses (IaC, per-tenant, random-suffixed),
enable lazy discovery instead:
[query]
catalog_discovery = "polaris-auto" # default is "static"
With polaris-auto, a query referencing a 3-part identifier whose catalog is
not statically declared triggers a one-time probe against Polaris for a
warehouse of that name, using the caller’s own bearer token. If Polaris
resolves it (and the caller is authorized), SQE registers it into the session
exactly like a static catalog – same policy enforcement, dynamic-filter
pushdown, and credential passthrough – and the query proceeds. No sqe.toml
change, no restart:
-- main_warehouse_9d679d was created in Polaris at runtime, never declared in TOML
SELECT count(*) FROM "main_warehouse_9d679d".analytics.orders;
Properties:
- Authorization is unchanged. The probe uses the caller’s bearer; Polaris
rejects warehouses they are not authorized for. A denied or nonexistent
warehouse returns the same
unknown catalogerror – existence is not leaked. - Per-session scoping. The discovered catalog is registered into the caller’s session, not shared process-wide, so vended credentials and visibility stay per-user. A second reference in the same session reuses it without re-probing.
- Drop-out within the session TTL. A renamed or dropped warehouse stops resolving on the next session refresh.
static(the default) is unchanged – an undeclared catalog errors with no Polaris probe.- REST/Polaris only. Glue, S3 Tables, and HMS still require static
declaration or
ATTACH.SHOW CATALOGSlists statically-configured and already-discovered catalogs, not warehouses never yet referenced.
v1 limitations
- No on-disk persistence. ATTACH does not survive a restart.
- No encryption-at-rest for secrets. The store holds plaintext bytes
in memory;
Dropzeroizes on clean shutdown but does not protect against process dumps or memory snapshots taken while running. - No mTLS to attached REST catalogs. Bearer tokens only.
- No KERBEROS for HMS. The HMS path uses the upstream Thrift client’s default authentication.
- Authorization is enforced through the policy backend (OPA / Cedar). There is no built-in role check for ATTACH or CREATE SECRET out of the box; operators wire it themselves through policy rules.
- The embedded CLI (
sqe-cliad-hoc mode) supports the same syntax as the cluster server. Embedded ATTACH targets the same in-memory registry but does not share state across CLI invocations.
Web UI
SQE serves a read-only web dashboard on the coordinator’s health port
(metrics_port + 1, default 9091). It shows the queries the engine is running
and has run, per-query timing and fragments, the cluster nodes, and live engine
metrics. The data comes from the coordinator’s in-memory query tracker and
worker registry. The page adds no new instrumentation and never touches the
query path.

Access
-
Open
http://<coordinator-host>:<metrics_port + 1>/. With the defaultmetrics.prometheus_port = 9090, that ishttp://localhost:9091/. -
The same port also serves
/healthz,/readyz, and/api/v1/status. -
The UI is on by default. Turn it off with:
[metrics] web_ui = falseWhen off,
/healthz,/readyz, and/api/v1/statusstill respond; the dashboard and the/api/v1/queries*endpoints return 404.
Security
There is no login. The dashboard is network-gated, the same posture as the
Prometheus /metrics endpoint: anyone who can reach the port sees every user’s
query text and the cluster state. Keep the health port on an internal network.
The UI is strictly read-only. It cannot submit queries, cancel them, or change
configuration. The query-detail endpoint deliberately omits session id, client
IP, and roles so the unauthenticated surface stays small.
Tabs
- Overview carries the node identity and capabilities (enabled protocols and ports, catalog backend and URL, storage, memory limit), live resource gauges (memory pool used, concurrency against the configured cap), and the engine metrics (queries by state, rows out, average latency) as stat cards. Each card has a one-hour sparkline, and a query-activity histogram sits below them.
- Queries lists recent queries with id, user, state, SQL, elapsed time, rows, and bytes scanned. Click a row for the detail: the queue, planning, and execution timing, the rows/bytes/spill/peak-memory totals, and the per-fragment breakdown showing which worker ran each fragment.
- Cluster shows the worker nodes with health and in-flight load. In single-node mode the coordinator lists itself as one node doing both roles.
Every chart is hoverable. Pointing at a bar or a sparkline point shows the time and value.
JSON API
The page is a thin client over a small JSON API on the same port. The endpoints are stable and safe to scrape directly:
| Endpoint | Returns |
|---|---|
GET /api/v1/overview | node, capabilities, resources, metrics |
GET /api/v1/queries?state=<running|finished|failed|all>&limit=<n> | recent queries, newest first |
GET /api/v1/queries/{id} | one query plus its fragments (404 if unknown) |
GET /api/v1/workers | worker nodes with health and in-flight load |
GET /api/v1/metrics/history | time-bucketed series for the charts |
GET /api/v1/status | Ballista/DataFusion-style cluster status |
How it is built
- One HTML page with vanilla JavaScript, embedded in the binary with
include_str!. No Node toolchain, no bundler, no external assets, no web fonts or logos. The visual language follows the Schuberg Philis palette and layout with system fonts. - The metrics history is an in-memory ring buffer. The coordinator samples query
counts, rows, latency, active queries, and memory-pool usage every five seconds
and keeps a rolling one-hour window.
GET /api/v1/metrics/historyaggregates the samples into one-minute buckets, so the charts advance a bar each minute and the current bar refreshes every sample.
For a longer history, scrape /metrics into Prometheus and chart it in Grafana.
The web UI is the at-a-glance view that ships in the binary.
Rust Crate Structure
SQE is organized as a Cargo workspace with 11 crates. Each crate has a focused responsibility.
Dependency Graph
graph TB
SERVER["sqe-server binary<br/>(in sqe-coordinator)"]
CLI["sqe-cli"]
SERVER --> COORD["sqe-coordinator"]
SERVER --> WORKER["sqe-worker"]
COORD --> AUTH["sqe-auth"]
COORD --> CAT["sqe-catalog"]
COORD --> SQL["sqe-sql"]
COORD --> POLICY["sqe-policy"]
COORD --> PLANNER["sqe-planner"]
COORD --> METRICS["sqe-metrics"]
COORD --> TRINO["sqe-trino-compat"]
COORD --> CORE["sqe-core"]
WORKER --> PLANNER
WORKER --> METRICS
WORKER --> CORE
AUTH --> CORE
CAT --> CORE
SQL --> CORE
POLICY --> CORE
PLANNER --> CORE
METRICS --> CORE
TRINO --> CORE
CLI --> CORE
style SERVER fill:#6f9,stroke:#333
style CLI fill:#6f9,stroke:#333
Crate Reference
sqe-core
Shared types used across all crates.
| Module | Contents |
|---|---|
config.rs | SqeConfig and all sub-configs, TOML loading, env var overrides |
error.rs | SqeError enum (Auth, Catalog, Execution, Config, NotImplemented, Internal) |
session.rs | Session struct (id, user, tokens, expiry), SessionUser (username, roles) |
lib.rs | VERSION constant |
sqe-auth
Keycloak OIDC authentication.
| Module | Contents |
|---|---|
authenticator.rs | Authenticator — ROPC grant, token refresh, background refresh task |
keycloak.rs | KeycloakClient — HTTP calls to Keycloak token endpoint, role extraction |
token_cache.rs | TokenCache — DashMap of session → cached tokens, expiry tracking |
sqe-catalog
Iceberg REST catalog client (wraps iceberg-rust).
| Module | Contents |
|---|---|
rest_catalog.rs | SessionCatalog — per-user catalog with bearer token, namespace/table ops |
catalog_provider.rs | SqeCatalogProvider — DataFusion CatalogProvider bridge |
schema_provider.rs | SqeSchemaProvider — DataFusion SchemaProvider for Iceberg namespaces |
table_provider.rs | Iceberg → DataFusion TableProvider |
credential_vending.rs | Extract S3 credentials from Polaris table load response |
iceberg_scan.rs | Iceberg scan configuration |
info_schema.rs | Virtual information_schema (tables, schemata, columns) |
sqe-sql
SQL parsing and statement classification.
| Module | Contents |
|---|---|
classifier.rs | parse_and_classify(sql) → StatementKind, routes all SQL statement types |
sqe-policy
Policy enforcement framework (pluggable backend).
| Module | Contents |
|---|---|
lib.rs | PolicyEnforcer trait, PassthroughEnforcer (default no-op) |
sqe-planner
Distributed query planning.
| Module | Contents |
|---|---|
scan_task.rs | ScanTask — serializable message from coordinator to worker |
splitter.rs | split_files() — divide data files across workers |
sqe-coordinator
The coordinator: SQL routing, session management, write handling.
| Module | Contents |
|---|---|
flight_sql.rs | SqeFlightSqlService — Arrow Flight SQL server (735 lines) |
query_handler.rs | QueryHandler — central query routing and execution (596 lines) |
session_manager.rs | SessionManager — session lifecycle, token refresh integration |
worker_registry.rs | WorkerRegistry — worker discovery, health checking |
catalog_ops.rs | DDL operations (DROP TABLE, CREATE/DROP SCHEMA) |
write_handler.rs | CTAS and INSERT INTO handling |
writer.rs | Parquet file writing to S3 |
distributed_scan.rs | Distributed scan coordination |
mode.rs | Mode enum, resolve_mode() for sqe-server |
bin/sqe_server.rs | Unified server binary entry point |
sqe-worker
Stateless scan executor.
| Module | Contents |
|---|---|
executor.rs | execute_scan(ScanTask) — read Parquet from S3, return Arrow batches |
flight_service.rs | WorkerFlightService — Flight server for receiving scan tasks |
sqe-cli
Interactive SQL client.
| Module | Contents |
|---|---|
main.rs | CLI argument parsing (clap), REPL loop, auth flow |
client.rs | SqlClient trait, QueryResult type |
flight.rs | FlightClient — Flight SQL client with handshake and token auth |
http.rs | HttpClient — Trino HTTP protocol client |
display.rs | Output formatting: ASCII table, CSV, JSON |
sqe-metrics
Observability stack.
| Module | Contents |
|---|---|
lib.rs | MetricsRegistry — Prometheus counters, histograms, gauges |
server.rs | Prometheus HTTP /metrics endpoint (axum) |
audit.rs | AuditLogger — JSONL audit log writer |
otel.rs | OpenTelemetry init (traces, metrics, logs via OTLP/gRPC) |
sqe-trino-compat
Trino wire protocol compatibility layer.
| Module | Contents |
|---|---|
server.rs | Trino HTTP server (/v1/statement endpoint) |
protocol.rs | RecordBatch → Trino JSON response serialization |
types.rs | Arrow → Trino type mapping |
Key External Dependencies
| Crate | Version | Purpose |
|---|---|---|
datafusion | 51 | Query engine (SQL planning, optimization, execution) |
arrow / arrow-flight | 57 | Columnar data format and Flight SQL protocol |
iceberg / iceberg-catalog-rest | 0.8 | Iceberg table format and REST catalog |
tonic | 0.14 | gRPC framework (Flight SQL server + client) |
axum | 0.8 | HTTP framework (health, metrics, Trino compat) |
tokio | 1 | Async runtime |
sqlparser | 0.59 | SQL parsing |
moka | 0.12 | Async TTL cache (metadata caching) |
clap | 4 | CLI argument parsing |
tracing | 0.1 | Structured logging |
opentelemetry | 0.31 | Distributed tracing and metrics |
Testing
Test Structure
crates/
├── sqe-core/src/ # 40 unit tests (config validation, error types, session, memory limit parsing)
├── sqe-auth/src/ # 17 unit tests (authenticator, OIDC, session)
├── sqe-coordinator/
│ ├── src/
│ │ ├── mode.rs # 10 unit tests (mode selection)
│ │ ├── worker_registry.rs # 5 unit tests (health checking)
│ │ ├── write_handler.rs # 2 unit tests (schema conversion)
│ │ ├── catalog_ops.rs # 5 unit tests (table ref parsing)
│ │ └── distributed_scan.rs # 3 unit tests
│ └── tests/
│ ├── integration_test.rs # 45 integration tests (end-to-end)
│ └── sql_compat_test.rs # 5 SQL compatibility tests
├── sqe-catalog/src/
│ ├── credential_vending.rs # 5 unit tests
│ └── info_schema.rs # 4 unit tests
├── sqe-sql/src/
│ └── classifier.rs # 29 unit tests (statement classification)
├── sqe-planner/src/
│ ├── scan_task.rs # 2 unit tests (serialization)
│ └── splitter.rs # 5 unit tests (file splitting)
├── sqe-policy/src/ # 8 unit tests (policy enforcer, passthrough)
├── sqe-metrics/src/
│ ├── lib.rs # 4 unit tests (metrics registry)
│ ├── server.rs # 1 unit test (metrics endpoint)
│ ├── audit.rs # 3 unit tests (audit logging)
│ └── otel.rs # 1 unit test
├── sqe-trino-compat/src/ # 12 unit tests (type mapping, serialization)
└── sqe-worker/src/
└── executor.rs # 3 unit tests (S3 URL parsing)
Running Tests
# All workspace tests (fast -- unit tests only)
cargo test --workspace
# Specific crate
cargo test -p sqe-sql
cargo test -p sqe-coordinator
# Specific test
cargo test -p sqe-coordinator -- mode
# Integration tests (require test stack)
cargo test --workspace -- --ignored
# With output
cargo test --workspace -- --nocapture
Unit Tests
Unit tests run without external dependencies. They test:
- Config validation – environment variable parsing, default values, memory limit parsing
- Error types – error construction, display formatting, conversion
- Session management – session creation, token fingerprint, expiry
- Authentication – OIDC flow, token validation, client credentials
- Policy enforcement – passthrough policy, enforcer trait behavior
- SQL classification – every statement type routes correctly
- Mode selection – config/env var priority, case insensitivity, error cases
- Worker health – state transitions, failure thresholds, recovery
- Schema conversion – Arrow to Iceberg type mapping
- Serialization – ScanTask JSON roundtrip
- File splitting – even/uneven distribution across workers
- Metrics – counter increment, histogram observation
- Audit – JSONL serialization, file writing, no-op mode
Integration Tests
Integration tests live in crates/sqe-coordinator/tests/integration_test.rs and require a running test stack (Polaris, S3-compatible storage). They are marked #[ignore] and run with --ignored.
Test inventory (45 tests)
| Category | Tests | What they validate |
|---|---|---|
| Core queries | test_simple_select, test_where_conditions, test_order_limit_offset, test_case_expression, test_math_expressions, test_string_functions | Basic SELECT, filtering, ordering, CASE/WHEN, arithmetic, string functions |
| Joins | test_inner_join, test_left_join, test_right_join, test_full_outer_join, test_cross_join, test_self_join, test_three_way_join, test_join_with_aggregation | All join types including multi-table and join+GROUP BY |
| Aggregation | test_aggregation_basic, test_having_clause, test_window_functions, test_window_running_total | GROUP BY, HAVING, OVER(), running totals |
| Subqueries | test_subquery_where, test_in_subquery, test_exists_subquery, test_scalar_subquery_select | Correlated and uncorrelated subqueries, IN, EXISTS |
| CTEs | test_cte_join, test_multiple_ctes | WITH clauses, multi-CTE queries |
| Set operations | test_union_all | UNION ALL across tables |
| DDL/DML | test_ctas_roundtrip, test_insert_into, test_drop_table, test_drop_table_if_exists_no_error, test_create_and_drop_view, test_view_with_aggregation | CREATE TABLE AS, INSERT INTO, DROP TABLE, views |
| EXPLAIN | test_explain_plan, test_explain_analyze, test_explain_full, test_explain_policy_aware | Plan output, execution stats, policy annotation |
| Metadata | test_information_schema_tables, test_information_schema_schemata | information_schema virtual tables |
| Auth | test_authentication, test_token_fingerprint, test_keycloak_auth_with_test_users, test_keycloak_token_refresh, test_different_user_catalog_visibility | Token flow, session fingerprinting, per-user catalog isolation |
| Distributed | test_distributed_select, test_local_fallback_without_workers | Coordinator-to-worker scan, graceful fallback to local mode |
| Trino compat | test_trino_http_query | Query via Trino HTTP protocol adapter |
Running Integration Tests
The lightweight test stack is the primary way to run integration tests:
# Start the lightweight test stack (Polaris in-memory + RustFS)
docker compose -f docker-compose.test.yml up -d
# Bootstrap (idempotent: creates buckets, warehouse, namespaces)
./scripts/bootstrap-test.sh
# Run all integration tests
./scripts/integration-test.sh
# Run a single test by name
./scripts/integration-test.sh test_simple_select
The test config is at tests/sqe-test.toml and uses token_endpoint (client_credentials mode) against Polaris’s built-in OAuth.
SQL Compatibility Tests
SQL compatibility tests live in crates/sqe-coordinator/tests/sql_compat_test.rs. These 5 tests validate SQL semantic correctness beyond what the integration tests cover – they focus on edge cases in SQL behavior that must match ANSI SQL or Trino semantics to avoid surprises for users migrating queries.
The SQL compat tests use the same test stack and configuration as the integration tests. They are also marked #[ignore] and run as part of ./scripts/integration-test.sh.
Test Configuration
# tests/sqe-test.toml
[coordinator]
flight_sql_port = 50051
trino_http_port = 8080
[auth]
token_endpoint = "http://localhost:8181/api/catalog/v1/oauth/tokens"
client_id = "root"
client_secret = "s3cr3t"
[catalog]
catalog_url = "http://localhost:8181/api/catalog"
warehouse = "test_warehouse"
[storage]
s3_endpoint = "http://localhost:9000"
s3_access_key = "s3admin"
s3_secret_key = "s3admin"
s3_region = "us-east-1"
s3_path_style = true
Benchmark Testing
Beyond unit and integration tests, SQE ships with sqe-bench — a benchmark CLI that validates SQL correctness and measures performance across industry-standard query suites.
Benchmark tests differ from integration tests in scope and purpose:
| Integration tests | Benchmark tests | |
|---|---|---|
| Data | Synthetic fixtures (small) | TPC/SSB scale factor data (GB scale) |
| Queries | Targeted feature tests | Full benchmark query suites (22–99 queries) |
| Validation | Pass/fail assertions | PASS / DIFF / FAIL / SKIP / ERROR with timing |
| Purpose | Regression detection | SQL correctness + performance tracking |
Quick benchmark run
# Generate TPC-H data at scale factor 1
cargo run -p sqe-bench -- generate tpch --scale 1 --output ./data
# Load into SQE (requires running stack)
cargo run -p sqe-bench -- load tpch --scale 1 --data ./data \
--host localhost --port 60051 --username root --password ""
# Run all 22 TPC-H queries
cargo run -p sqe-bench -- test tpch --scale 1 \
--host localhost --port 60051 --username root --password ""
# Or use the script wrapper
./scripts/benchmark-test.sh tpch
Supported benchmarks
| Benchmark | Queries | Notes |
|---|---|---|
tpch | 22 | Standard first check for any SQL engine |
tpcds | 99 | Complex SQL: correlated subqueries, window functions, GROUPING SETS |
ssb | 13 | Fast smoke test — denormalized star schema |
tpcc | 8 | OLTP reads; write queries skip until DELETE/MERGE land |
tpce | 11 | Brokerage OLTP reads |
tpcbb | 10 | SQL-only subset over TPC-DS data |
Benchmark test in CI
TPC-H at SF1 runs as a post-merge smoke test. The full suite (TPC-H + TPC-DS + SSB) runs nightly. JSON reports are written to benchmarks/results/ and archived as CI artifacts for regression tracking.
# CI smoke test (TPC-H SF1 only, fails on any ERROR or FAIL)
./scripts/benchmark-test.sh tpch
# Nightly full suite
./scripts/benchmark-test.sh tpch
./scripts/benchmark-test.sh tpcds
./scripts/benchmark-test.sh ssb
For full documentation of benchmark commands, scale factors, result formats, and how to add new benchmarks, see Benchmark Suite.
Roadmap
SQE is developed in phases, each building on the previous.
Phase Overview
gantt
title SQE Implementation Phases
dateFormat YYYY-MM
axisFormat %b %Y
section Core
Phase 1 - Single Node :done, p1, 2026-01, 2026-02
Phase 2 - Write Path :done, p2, 2026-02, 2026-03
Phase 3 - Row-Level Writes :done, p3, 2026-03, 2026-04
section Scale
Phase 3b - Benchmarks :done, p3b, 2026-03, 2026-04
Phase 4 - Pluggable Auth :done, p4, 2026-03, 2026-04
Phase 4b - Streaming/Distributed :done, p4b, 2026-04, 2026-04
section Shipped
Phase 2c - dbt Compatibility :done, p2c, 2026-03, 2026-04
Phase 5 - Pluggable Catalogs :done, p5, 2026-04, 2026-05
Phase 7 - Iceberg V3 :done, p7, 2026-04, 2026-05
Phase 9 - OpenLineage :done, p9, 2026-05, 2026-05
section Next
Phase 6 - Security Policies :p6, 2026-05, 2026-07
Phase 8 - Trino Decommission :p8, 2026-07, 2026-09
Phase 1 — Single-Node Engine (Done)
The foundation: a working SQL engine that queries Iceberg tables through Polaris with Keycloak auth.
- DataFusion query execution
- Keycloak OIDC authentication (ROPC grant)
- Per-session catalog with bearer token passthrough
- Arrow Flight SQL server
- CLI client (
sqe-cli) SELECT,SHOW CATALOGS/SCHEMAS/TABLES,EXPLAIN- Prometheus metrics + structured JSON logging
Phase 2 — Write Path & Views (Done)
SQL write operations and catalog DDL.
CREATE TABLE AS SELECTCREATE OR REPLACE TABLEINSERT INTO SELECTCREATE VIEW/DROP VIEWCREATE SCHEMA/DROP SCHEMADROP TABLE/DROP TABLE IF EXISTS- Parquet writer (to S3 via Iceberg)
- Audit logging (JSONL)
- OpenTelemetry export (OTLP/gRPC)
- Trino-compatible HTTP endpoint
Phase 2c — dbt Compatibility (Done)
Native dbt support via dbt-sqe adapter over ADBC Flight SQL.
information_schemavirtual providers (tables, schemata, columns)dbt-sqePython adapter (connection manager, materializations)ALTER TABLE RENAME- dbt
table,view, and append-onlyincrementalmaterializations incrementalwithmergestrategy (CoW + MoR)- Adapter lives at
adapters/dbt-sqe/dbt/
Phase 3 — Row-Level Writes (Done)
DELETE FROM, UPDATE, and MERGE INTO are implemented via Copy-on-Write using the iceberg-rust fork vendored at vendor/iceberg-rust/ (DF 53.1 + Arrow 58 rebase of risingwavelabs/iceberg-rust), which provides rewrite_files() transaction support.
Strategy: Copy-on-Write
graph TB
subgraph "Copy-on-Write (Implemented)"
READ["Read affected<br/>data files"] --> FILTER["Apply WHERE filter"]
FILTER --> REWRITE["Rewrite without<br/>deleted/modified rows"]
REWRITE --> COMMIT["Commit via<br/>rewrite_files()"]
end
subgraph "Merge-on-Read (Future)"
DELFILE["Write position<br/>delete files"] --> COMMITDEL["Commit via<br/>RowDeltaAction"]
COMMITDEL --> COMPACT["Background<br/>compaction"]
end
style READ fill:#6f9
style FILTER fill:#6f9
style REWRITE fill:#6f9
style COMMIT fill:#6f9
CoW rewrites affected data files entirely. MoR has shipped: set TBLPROPERTIES ('write.delete.mode' = 'merge-on-read') to opt in. SQE writes a position-delete file (no PK) or an equality-delete file (with PK) and commits via FastAppendAction / RowDeltaAction. CoW remains the default for backward compatibility.
Delivered
DELETE FROM table WHERE condition— removes matching rows; supports cross-table subqueries; DELETE without WHERE = truncateUPDATE table SET col = expr WHERE condition— modifies matching rows; supports CASE WHEN transformations and cross-table subqueriesMERGE INTO target USING source ON condition WHEN MATCHED/NOT MATCHED ...— full outer join approach with WHEN MATCHED/NOT MATCHED clauses- All operations atomic via Iceberg snapshot isolation
- dbt
incrementalwithmergestrategy - Integration tests against Polaris + MinIO
- TPC-C write queries (17/17 pass), TPC-E write queries enabled
Iceberg Dependency
Uses the iceberg-rust fork vendored at vendor/iceberg-rust/ for rewrite_files(). When upstream apache/iceberg-rust ships OverwriteAction (tracked in Epic #2186), the dependency can be migrated back to the official crate.
SQE Changes
| File | Change |
|---|---|
Cargo.toml | Vendored iceberg-rust fork (DF 53 + Arrow 58 rebase) at vendor/iceberg-rust/ |
crates/sqe-coordinator/src/delete_handler.rs | DELETE FROM execution via CoW |
crates/sqe-coordinator/src/update_handler.rs | UPDATE execution via CoW |
crates/sqe-coordinator/src/merge_handler.rs | MERGE INTO execution via CoW |
crates/sqe-coordinator/src/query_handler.rs | Routes Merge/Delete/Update to handlers |
crates/sqe-coordinator/src/write_handler.rs | Shared CoW rewrite logic |
Phase 7 — Iceberg V3 (Done)
Iceberg V3 table format support landed end-to-end. The vendored fork at vendor/iceberg-rust/ is rebased onto DataFusion 53.1 + Arrow 58 and carries V3 spec coverage. SQE Iceberg matrix score: 167/189 = 88.4% (per docs/iceberg-matrix.md).
V3 Features Shipped
| Feature | Status |
|---|---|
Default values (ALTER TABLE ADD COLUMN ... DEFAULT) | Done |
Schema evolution (ALTER TABLE ADD/DROP COLUMN) | Done |
Nanosecond timestamps (TIMESTAMP_NS, TIMESTAMPTZ_NS) | Done |
| Partition evolution | Done |
| Equality deletes + position deletes (MoR) | Done |
V3 Features Still Blocked Upstream
| Feature | Blocker |
|---|---|
| Variant type | iceberg-rust #2188 not merged |
| Geometry type | DataFusion UDT #12644 |
| Vector / Embedding type | Iceberg V3 vector spec not finalised |
| Multi-arg partition transforms | Iceberg Java spec alignment in progress |
| Row lineage | Deferred upstream |
Other Hardening
- Metadata cache invalidation on DDL
- Large result-set streaming (Flight SQL
do_getback-pressure) - Error messages tuned for catalog and auth failures
- Partition pruning across all predicate types
Phase 4b/4c — Distributed Execution (Done)
Scale-out query execution with stateless workers. Implemented via streaming execution in two phases.
graph TB
subgraph Coordinator
PLAN["Query Planner"] --> STAGE["Stage Decomposition"]
STAGE --> SCHED["Scheduler"]
end
subgraph Workers
SCHED -->|ScanTask| W1["Worker 1"]
SCHED -->|ScanTask| W2["Worker 2"]
SCHED -->|ScanTask| W3["Worker 3"]
end
W1 <-->|DoExchange shuffle| W2
W2 <-->|DoExchange shuffle| W3
W1 -->|Arrow stream| MERGE["Multi-endpoint merge"]
W2 -->|Arrow stream| MERGE
W3 -->|Arrow stream| MERGE
MERGE --> CLIENT["Client"]
Delivered
- Phase A (spill-to-disk): FairSpillPool with watermarks, late materialization, file/page pruning, TopK, S3 I/O pipeline (coalescing, footer cache, prefetch), SortMergeJoin fallback
- Phase B (distributed): DoExchange shuffle, distributed sort (range-partition with sampling), two-phase aggregation, distributed joins (broadcast, shuffle hash, pre-sorted merge, predicate transfer), multi-endpoint Flight SQL, stage decomposition
- Adaptive sort stripping — memory-aware sort mode selection
- Metrics — spill, shuffle, late-mat, pruning, time-to-first-row, S3 I/O, auth, write path
Benchmark Results (SF1, distributed 2-worker)
| Suite | Pass Rate | Time | Speedup vs single |
|---|---|---|---|
| TPC-H | 22/22 | 13.5s | 2.1x |
| TPC-DS | 98/99 | 36.1s | 2.8x |
| SSB | 13/13 | 5.3s | 2.7x |
| TPC-C | 17/17 | 8.6s | 2.6x |
Phase 5 — Pluggable Catalogs (Done)
CatalogBackend trait replaced the hard-coded Polaris REST catalog. Seven catalog backends ship today:
| Backend | Status |
|---|---|
| Iceberg REST (Polaris, Lakeformation) | Done — default |
| AWS Glue | Done |
| Nessie | Done |
| Hive Metastore (Thrift) | Done |
| AWS S3 Tables | Done |
| Snowflake Horizon (REST-compatible) | Done |
| JDBC | Done |
Multi-cloud storage via object_store: S3 (+ endpoint override for R2, Ceph, Garage, MinIO), Azure ADLS Gen2/Blob, GCS, local filesystem. Engine-side session-manager wiring for Delta + remaining edge cases is the only deferred item; tracked in nextsteps.md.
Phase 6 — Security Policies (Planned)
Fine-grained access control via LogicalPlan rewriting.
PolicyEnforcerimplementations (OPA via Rego, Cedar)GRANT/REVOKEwithROWS WHEREandMASKED WITHSHOW GRANTS/SHOW EFFECTIVE POLICY- Column restriction (invisible columns)
- Policy caching with TTL (moka)
- No-information-leakage model (PostgreSQL RLS style)
Phase 9 — OpenLineage Emitter (Done)
Coordinator-side OpenLineage 2-0-2 emitter with column-level lineage. Off by default; zero hot-path overhead when disabled.
sqe-lineagecrate: event types, observer, emitter task, file/HTTP/spool sinks, multi-catalog dataset extractor, column-lineage trace rules across 11 LogicalPlan node typesOpenLineageConfiginsqe-corewith TOML + env-var overrides + startup validation- Hooks in
QueryHandler::execute_statement(START / COMPLETE / FAIL) - Documented at
docs/book/src/operations/openlineage.md
Deferrals (documented): mTLS, per-event user-OIDC bearer forwarding, MERGE per-branch column annotations, embedded CLI emit, DDL hint extraction.
Phase 10 — Performance & Reliability Testing (Planned)
Validate SQE is production-ready through systematic benchmarking and reliability testing.
Performance Benchmarks
graph LR
subgraph "Benchmark Suite"
TPCH["TPC-H<br/>22 queries<br/>SF 10/100/1000"]
TPCDS["TPC-DS<br/>99 queries<br/>SF 10/100"]
CUSTOM["Custom workloads<br/>Iceberg-specific<br/>partition pruning"]
end
subgraph "Comparison Targets"
TRINO["Trino<br/>(current prod)"]
SQE["SQE"]
SPARK["Spark SQL<br/>(reference)"]
end
TPCH --> TRINO
TPCH --> SQE
TPCH --> SPARK
TPCDS --> TRINO
TPCDS --> SQE
| Benchmark | Scale Factors | Purpose |
|---|---|---|
| TPC-H | SF10, SF100, SF1000 | Standard analytical workload, join-heavy |
| TPC-DS | SF10, SF100 | Complex analytics, subqueries, window functions |
| Iceberg-specific | Varies | Partition pruning, metadata operations, time travel |
| Write path | 1M, 10M, 100M rows | CTAS, INSERT, MERGE throughput |
| Concurrent users | 10, 50, 100 sessions | Connection handling, session isolation |
Key Metrics
- Query latency — P50, P95, P99 per query
- Throughput — queries/second under load
- Memory usage — peak RSS per query complexity
- Startup time — cold start to first query
- Scan speed — GB/s from S3 (single-node vs distributed)
Performance Targets
| Metric | Target | Rationale |
|---|---|---|
| TPC-H SF100 geometric mean | Within 2x of Trino | Parity goal for migration |
| Cold start to ready | < 2 seconds | K8s autoscaling responsiveness |
| Peak memory (SF100 query) | < 4GB coordinator | Fit in standard K8s pod limits |
| Concurrent session overhead | < 10MB per session | Support 100+ sessions |
Reliability Testing
| Test | Method | What it validates |
|---|---|---|
| Chaos: kill worker mid-query | kubectl delete pod during scan | Coordinator retries/fails gracefully |
| Chaos: kill coordinator | SIGKILL during query | In-flight queries fail cleanly, no data corruption |
| Chaos: Polaris unavailable | Block network to Polaris | Graceful error, no hang, cached metadata still works |
| Chaos: Keycloak unavailable | Block network to Keycloak | Existing sessions continue, new auth fails cleanly |
| Chaos: S3 latency spike | tc netem delay on S3 | Query timeout, not hang |
| Memory pressure | Large query + small memory limit | Spill-to-disk or clean OOM, no silent corruption |
| Token expiry during query | Set very short token TTL | Refresh mid-query, or clean auth error |
| Concurrent DDL + DML | CTAS while DROP TABLE on same table | Iceberg conflict detection, clean error |
| Long-running soak test | 24h mixed workload | No memory leaks, no connection leaks, stable latency |
Profiling & Optimization
- CPU profiling —
perf+ flamegraphs on hot queries - Memory profiling —
jemallocstats, allocation tracking - I/O profiling — S3 request counts, Parquet read amplification
- Query plan analysis — DataFusion
EXPLAIN ANALYZEfor bottleneck identification
Deliverables
- Automated benchmark harness (run TPC-H/DS, collect results, compare)
- Performance regression CI (catch slowdowns before merge)
- Published benchmark results: SQE vs Trino on identical data
- Reliability test playbook with pass/fail criteria
- Memory/CPU profiling report with optimization recommendations
- Soak test (24h) passing without degradation
Phase 8 — Trino Decommission (Future)
Complete migration from Trino DCAF fork.
- Full Trino wire protocol compatibility for remaining tools
- Dashboard migration playbook (Superset, Grafana, etc.)
- JDBC driver migration guide (Trino JDBC → Flight SQL JDBC)
- Performance parity validation (benchmark comparison)
- Runbook for operators
- Trino fork sunset and decommission