From 0d376479dd7e240ac24e1ee648da1d54e4a2722f Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 15:38:03 -0800 Subject: [PATCH 01/20] docs: Add spec for Parquet dump/restore Define the directory layout, metadata.json schema, and per-entity Parquet file format for dumping and restoring subgraph data. --- docs/specs/dump-restore.md | 269 +++++++++++++++++++++++++++++++++++++ 1 file changed, 269 insertions(+) create mode 100644 docs/specs/dump-restore.md diff --git a/docs/specs/dump-restore.md b/docs/specs/dump-restore.md new file mode 100644 index 00000000000..6d7e117b78d --- /dev/null +++ b/docs/specs/dump-restore.md @@ -0,0 +1,269 @@ +# Spec: Parquet Dump/Restore for Subgraph Data + +## Problem + +Subgraph entity data lives exclusively in PostgreSQL. There's no way to export a subgraph's data for backup, migration between environments, or sharing. We need a file-based dump/restore mechanism. + +## Goals + +1. **Dump** a subgraph's entity data to parquet files +2. **Restore** a subgraph from parquet files +3. **Incremental append** -- add newly arrived data to an existing dump without rewriting it +4. Long-term: make dumping an **ongoing process** (not just a one-off CLI operation) + +## Non-goals (for now) + +- S3/GCS output +- Schema evolution / migration between different schema versions + +## Data Model Recap + +Each subgraph deployment has: +- A PostgreSQL schema (e.g., `sgd123`) +- One table per entity type, with columns: + - `vid` (bigserial) -- row version ID, primary key + - `block_range` (int4range) for mutable entities OR `block$` (int) for immutable + - `causality_region` (int, optional) -- for offchain data sources + - Data columns matching GraphQL fields (text, int, bigint, numeric, bytea, bool, timestamptz, arrays, enums) +- `data_sources$` table -- dynamic data sources created at runtime (defined in `dynds/private.rs`, separate from Layout): + - `vid` (int, identity PK), `block_range` (int4range), `causality_region` (int) + - `manifest_idx` (int), `parent` (int, self-ref FK), `id` (bytea) + - `param` (bytea, nullable), `context` (jsonb, nullable), `done_at` (int, nullable) + - Note: `parent` and `id` exist in the DDL but are currently unused by `insert()`, `load()`, and `copy_to()`. The dump should include them for completeness (they may contain data in older deployments). +- `poi2$` table -- Proof of Indexing data (a **mutable** entity table in the Layout, i.e. it uses `block_range` not `block$`; has `digest` (bytea), `id` (text), optionally `block_time$` (int8); `has_causality_region: false`). Conditionally created when `catalog.use_poi` is true. Excluded from some Layout operations like `find_changes()`. +- Metadata in `subgraphs.subgraph_manifest`, `subgraphs.deployment`, and `subgraphs.head` tables + +## Dump Format + +### Directory layout + +``` +/ + metadata.json -- deployment metadata + per-table state + schema.graphql -- raw GraphQL schema text + subgraph.yaml -- raw subgraph manifest YAML (optional) + / + chunk_000000.parquet -- rows ordered by vid + chunk_000001.parquet -- incremental append + ... + data_sources$/ + chunk_000000.parquet -- dynamic data sources +``` + +One parquet file per entity type (each has a different columnar schema). The `data_sources$` table is dumped alongside entity tables. The `poi2$` table, when present, is a regular entity table in the `Layout` (conditionally created when `catalog.use_poi` is true) and appears as any other entity type directory. Incremental dumps produce new chunk files rather than rewriting existing ones. + +The GraphQL schema and subgraph manifest YAML are stored as separate files rather than embedded in `metadata.json`. This matches the existing dump code in `dump.rs` and keeps the files human-readable and diffable. + +### Parquet schema per entity type + +System columns (always present): +- `vid` -> Int64 +- Immutable entities: `block$` -> Int32 +- Mutable entities: `block_range_start` -> Int32, `block_range_end` -> Int32 (nullable; null = unbounded/current) +- `causality_region` -> Int32 (only if table has it) + +Data columns mapped from `ColumnType` (all 10 variants defined in `relational.rs:1342`): + +| ColumnType | Arrow DataType | Notes | +|--------------------|---------------------------|-------------------------------| +| Boolean | Boolean | | +| Int | Int32 | | +| Int8 | Int64 | | +| Bytes | Binary | Raw bytes | +| BigInt | Utf8 | Arbitrary precision as string | +| BigDecimal | Utf8 | Arbitrary precision as string | +| Timestamp | TimestampMicrosecond(None) | Matches Value::Timestamp | +| String | Utf8 | | +| Enum(EnumType) | Utf8 | String value of enum variant | +| TSVector(FulltextConfig) | **Skip** | Generated; rebuild on restore | + +**List/array columns:** `List(T)` is not a `ColumnType` variant. Whether a column is an array is determined by `Column.is_list()` (delegates to the GraphQL `field_type`). A `[String]` field has `column_type: ColumnType::String` with a list-typed `field_type`. For Arrow mapping, check `column.is_list()` and wrap the base Arrow type in `List`. In `OidValue`, arrays have separate variants (`StringArray`, `BytesArray`, `BoolArray`, `Ints`, `Int8Array`, `BigDecimalArray`, `TimestampArray`). + +Nullability follows the GraphQL schema (non-null fields -> non-nullable Arrow columns). + +### metadata.json + +Contains everything needed to reconstruct the deployment's table structure, plus diagnostic information (health, indexes) captured at dump time. The GraphQL schema and manifest YAML are stored in separate files (`schema.graphql`, `subgraph.yaml`), not embedded here. + +The struct backing this file is `Metadata` (evolved from the existing `Control` struct in `dump.rs`). + +```json +{ + "version": 1, + "deployment": "Qm...", + "network": "mainnet", + + "manifest": { + "spec_version": "1.0.0", + "description": "Optional subgraph description", + "repository": "https://github.com/...", + "features": ["..."], + "entities_with_causality_region": ["EntityType1"], + "history_blocks": 2147483647 + }, + + "earliest_block_number": 12345, + "start_block": { "number": 12345, "hash": "0xabc..." }, + "head_block": { "number": 99999, "hash": "0xdef..." }, + "entity_count": 150000, + + "graft_base": null, + "graft_block": null, + "debug_fork": null, + + "health": { + "failed": false, + "health": "healthy", + "fatal_error": null, + "non_fatal_errors": [] + }, + + "indexes": { + "token": [ + "CREATE INDEX CONCURRENTLY IF NOT EXISTS attr_0_0_id ON sgd.token USING btree (id)" + ] + }, + + "tables": { + "Token": { + "immutable": true, + "has_causality_region": false, + "chunks": [ + { "file": "Token/chunk_000000.parquet", "min_vid": 0, "max_vid": 50000, "row_count": 50000 } + ], + "max_vid": 50000 + }, + "data_sources$": { + "immutable": false, + "has_causality_region": true, + "chunks": [ + { "file": "data_sources$/chunk_000000.parquet", "min_vid": 0, "max_vid": 100, "row_count": 100 } + ], + "max_vid": 100 + } + } +} +``` + +**Field sources:** + +| Field | Source | Code path | +|-------|--------|-----------| +| `manifest.*` | `subgraphs.subgraph_manifest` | `SubgraphManifestEntity` via `deployment_entity()` in `detail.rs` | +| `start_block` | `subgraphs.subgraph_manifest` | `start_block_number`, `start_block_hash` columns; available via `StoredSubgraphManifest` in `detail.rs:542-543`, assembled into `SubgraphDeploymentEntity.start_block` | +| `earliest_block_number` | `subgraphs.deployment` | `SubgraphDeploymentEntity.earliest_block_number` | +| `graft_base`, `graft_block` | `subgraphs.deployment` | `SubgraphDeploymentEntity.graft_base`, `.graft_block` | +| `debug_fork` | `subgraphs.deployment` | `SubgraphDeploymentEntity.debug_fork` | +| `head_block` | `subgraphs.head` | `SubgraphDeploymentEntity.latest_block` | +| `entity_count` | `subgraphs.head` | `DeploymentDetail.entity_count` (i64 in DB, usize in Rust) | +| `health.*` | `subgraphs.deployment` + `subgraph_error` | `SubgraphDeploymentEntity.{failed, health, fatal_error, non_fatal_errors}` | +| `indexes` | `pg_indexes` catalog | `IndexList::load()` → `CreateIndex::to_sql()` (existing code in `dump.rs:163-179`) | +| `network` | `deployment_schemas` | `Site.network` | +| `tables.*` | `Layout.tables` | `Table.{immutable, has_causality_region}` | + +**Notes:** +- `use_bytea_prefix` is not stored in the dump. It is hardcoded to `true` in `create_deployment` (deployment.rs:1302) and will always be set to `true` on restore. +- `health` and `indexes` are point-in-time diagnostic snapshots. They are not used during restore (a restored deployment starts healthy; indexes are auto-created by `Layout::create_relational_schema()`). They are included for inspection and debugging. +- `indexes` are serialized as SQL strings using `CreateIndex::with_nsp("sgd")` + `to_sql(true, true)`, producing `CREATE INDEX CONCURRENTLY IF NOT EXISTS` statements with a normalized `sgd` namespace. +- The `manifest` fields mirror the existing `Manifest` struct in `dump.rs` (derived from `SubgraphManifestEntity`). The `schema` and `raw_yaml` fields of `SubgraphManifestEntity` are written to separate files instead. +- The `poi2$` table, when present, is a regular mutable entity table in `Layout.tables` and appears in the `tables` map like any other entity. It does not need special handling. + +The raw GraphQL schema (in `schema.graphql`) is sufficient to reconstruct the full relational layout via `InputSchema::parse(spec_version, schema, deployment_hash)` → `Layout::new()`. The `InputSchema::parse()` call requires `manifest.spec_version` for version-specific parsing logic. + +## Dump Process + +**Existing code:** There is already a metadata-only dump in `store/postgres/src/relational/dump.rs` (`Layout::dump()`) that writes `control.json`, `schema.graphql`, and `subgraph.yaml`. It is called via `DeploymentStore::dump()` (deployment_store.rs:901) which loads `Layout` + `IndexList` and passes both to `Layout::dump()`. The connection is `AsyncPgConnection` via `pool.get_permitted()`. The new parquet dump extends this to include entity data. + +The existing `Control` struct is renamed to `Metadata` and extended with the fields described above. The existing `Manifest`, `BlockPtr`, `Health`, and `Error` structs in `dump.rs` are reused and extended. + +1. Resolve the deployment (by name, hash, or sgdN) +2. Read deployment metadata from `subgraph_manifest` + `deployment` + `head` tables (via `deployment_entity()` in `detail.rs`) +3. Write `schema.graphql` and `subgraph.yaml` (existing behavior) +4. For each entity type table in `Layout.tables` (sorted by name for determinism; includes `poi2$` when present): + a. Query rows in vid order, batched (adaptive sizing like `VidBatcher`) + b. Convert PG rows directly to Arrow `RecordBatch` (no JSON intermediate) + c. Write batches to parquet file + d. Record chunk info (file path, min_vid, max_vid, row_count) +5. Dump `data_sources$` table (fixed schema, same batch approach; include all DDL columns: `vid`, `block_range`, `causality_region`, `manifest_idx`, `parent`, `id`, `param`, `context`, `done_at`). Note: `parent` and `id` are in the DDL (`private.rs:68-69`) but not in the `DataSourcesTable` struct — dumping them requires raw SQL or extending the struct. +6. Write `metadata.json` atomically (write to tmp file, rename) + +### Incremental append + +- Read existing `metadata.json` to get `max_vid` per entity type +- Query rows with `vid > max_vid` +- Write as new chunk files (`chunk_000001.parquet`, etc.) +- Update metadata atomically + +## Restore Process + +1. Read `metadata.json` and `schema.graphql` +2. Parse schema via `InputSchema::parse(manifest.spec_version, schema_text, deployment_hash)` +3. Create a `Site` entry in `deployment_schemas` (needed for the deployment to be discoverable) +4. Create deployment via `create_deployment(conn, site, DeploymentCreate { .. })` -- this populates three tables: + - `subgraphs.head` (block pointers, entity count -- initially null/0) + - `subgraphs.deployment` (deployment hash, earliest_block, graft info, health) + - `subgraphs.subgraph_manifest` (schema from file, features/spec_version/etc. from metadata) +5. Create tables via `Layout::create_relational_schema()` -- this generates DDL from the parsed schema and creates all entity tables with default indexes +6. Restore `data_sources$` table via DDL from `DataSourcesTable::new().as_ddl()` + batch-insert +7. For each entity type (including `poi2$` if present), read all parquet chunks in order, batch-insert into PG +8. Reset vid sequences to `max_vid + 1` for all entity tables and data_sources$ +9. Update `subgraphs.head` with `head_block.number`, `head_block.hash`, and `entity_count` from dump metadata + +## PG Read Strategy: OidValue-based Dynamic Columns + +Use the existing `dsl::Table::select_cols()` + `DynamicRow` pattern (see `store/postgres/src/relational/dsl.rs` and `store/postgres/src/relational/value.rs`). This already solves dynamic-schema typed extraction through the connection pool: + +1. `select_cols()` builds a typed SELECT using `DynamicSelectClause` for any set of columns +2. Results are `DynamicRow` where `OidValue` dispatches on PG OID at runtime +3. `OidValue` captures all needed types: String, Bytes, Bool, Int, Int8, BigDecimal, Timestamp, plus array variants +4. Convert `OidValue` -> Arrow `ArrayBuilder` (analogous to existing `OidValue` -> `Entity` in `FromOidRow`) + +No JSON, no separate connection. The existing `DeploymentStore::dump()` already uses `AsyncPgConnection` via `pool.get_permitted()`. + +**Block range handling:** Add `OidValue::Int4Range(Bound, Bound)` variant (OID 3904). Diesel already has `FromSql, Pg>` for `(Bound, Bound)` which parses the binary format. ~15 lines of code in `value.rs` + fix the `BLOCK_RANGE_COL` placeholder in `dsl.rs:46-49` (currently `ColumnType::Bytes`, with comment "we can't deserialize in4range"). This resolves the existing TODO at dsl.rs line 294. + +**Key existing code:** +- `dsl::Table::select_cols()` (`store/postgres/src/relational/dsl.rs:305`) +- `OidValue` enum and `FromSql` impl (`store/postgres/src/relational/value.rs:33`) +- `FromOidRow` trait for result deserialization (`value.rs:206`) +- `selected_columns()` for building column list with system columns (`dsl.rs:246`) + +### Vid continuity on restore + +Preserve original vid values. Needed for incremental consistency and simpler to implement. Reset vid sequence to `max_vid + 1` after restore. + +## Where Code Lives + +- `store/postgres/src/relational/dump.rs` -- **Existing** metadata-only dump. Contains `Manifest`, `BlockPtr`, `Error`, `Health`, `Control` structs. `Control` will be renamed to `Metadata` and extended. The existing helper structs (`Manifest`, `BlockPtr`, `Health`, `Error`) are reused. Called via `DeploymentStore::dump()` → `Layout::dump()`. +- `store/postgres/src/parquet/` -- New module for parquet read/write/schema mapping +- `node/src/manager/commands/dump.rs` -- **Existing** CLI command skeleton (resolves deployment, calls `SubgraphStore::dump()`) +- `node/src/manager/commands/restore.rs` -- New CLI command for restore +- Expose via `command_support` in `store/postgres/src/lib.rs` + +## Dependencies to Add + +- `parquet = "=57.3.0"` (same version as existing `arrow`) to workspace and `store/postgres` +- `arrow` workspace dep to `store/postgres` + +## Existing Code to Reuse + +- `Layout`, `Table`, `Column`, `ColumnType` (`store/postgres/src/relational.rs`) -- schema introspection; `poi2$` is in Layout as a mutable entity table (conditionally, when `catalog.use_poi` is true) +- `Column.is_list()` (`relational.rs:1574`) -- determines if a column is an array type (delegates to GraphQL `field_type.is_list()`) +- `DataSourcesTable` (`store/postgres/src/dynds/private.rs`) -- `data_sources$` DDL via `as_ddl()` method; note that `parent` and `id` columns are in the DDL but not in the struct's typed fields +- `VidBatcher` (`store/postgres/src/vid_batcher.rs`) -- adaptive batch iteration using PG histogram statistics +- `copy.rs` pattern -- progress reporting, batch operation lifecycle; already handles both entity tables and `data_sources$` copying +- `InputSchema::parse(spec_version, raw, id)` (`graph/src/schema/input/mod.rs:965`) -- schema reconstruction from text +- `DeploymentSearch` (`node/src/manager/deployment.rs`) -- CLI deployment resolution (supports name, Qm hash, sgdN namespace) +- `create_deployment` (`store/postgres/src/deployment.rs:1224`) -- populates `head`, `deployment`, and `subgraph_manifest` tables +- `DeploymentCreate` + `SubgraphManifestEntity` (`graph/src/data/subgraph/schema.rs:103`) -- structs needed by `create_deployment` +- `deployment_entity()` (`store/postgres/src/detail.rs`) -- reads deployment metadata into `SubgraphDeploymentEntity`; note that `start_block_*` is in the DB table (`StoredSubgraphManifest`) but only partially exposed through `SubgraphDeploymentEntity` +- `IndexList::load()` + `CreateIndex::to_sql()` (`store/postgres/src/relational/index.rs`) -- loads and serializes indexes +- Existing `dump.rs` (`store/postgres/src/relational/dump.rs`) -- metadata serialization types: `Manifest`, `BlockPtr`, `Health`, `Error`, `Control` (to be renamed `Metadata`) + +## Implementation Order + +1. Schema mapping + metadata types (foundation, unit-testable) +2. Parquet writer (dump from PG) + graphman `dump` command +3. Incremental append support +4. Parquet reader (restore to PG) + graphman `restore` command +5. Ongoing dump integration (run as part of graph-node, not just CLI) From a64cb2b3644c1de015b82e7407384911e70a362c Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Mon, 23 Feb 2026 16:17:00 -0800 Subject: [PATCH 02/20] docs: Document the deployment dump format --- docs/implementation/README.md | 1 + docs/implementation/dump.md | 238 ++++++++++++++++++++++++++++++++++ 2 files changed, 239 insertions(+) create mode 100644 docs/implementation/dump.md diff --git a/docs/implementation/README.md b/docs/implementation/README.md index 31d4eb694a6..d54a39babbe 100644 --- a/docs/implementation/README.md +++ b/docs/implementation/README.md @@ -10,3 +10,4 @@ the code should go into comments. * [SQL Query Generation](./sql-query-generation.md) * [Adding support for a new chain](./add-chain.md) * [Pruning](./pruning.md) +* [Dump Format](./dump.md) diff --git a/docs/implementation/dump.md b/docs/implementation/dump.md new file mode 100644 index 00000000000..12af2ba04d7 --- /dev/null +++ b/docs/implementation/dump.md @@ -0,0 +1,238 @@ +## Dump Format + +The `graphman dump` command exports all entity data and metadata for a +single subgraph deployment into a self-contained directory of Parquet files +and JSON metadata. The resulting dump can be used to restore the deployment +into a different `graph-node` instance via `graphman restore`. Dumps are +consistent snapshots of the deployment's state at a specific point in time. + +### Directory layout + +A dump directory has the following structure: + +``` +/ + metadata.json -- deployment metadata + per-table state + schema.graphql -- raw GraphQL schema text + subgraph.yaml -- raw subgraph manifest YAML (optional) + / + chunk_000000.parquet -- rows ordered by vid + chunk_000001.parquet -- incremental append (future chunks) + ... + data_sources$/ + chunk_000000.parquet -- dynamic data sources +``` + +Each entity type defined in the GraphQL schema gets its own subdirectory, +named after the entity type exactly as it appears in the schema (e.g. +`Token/`, `Pool/`). The Proof of Indexing appears as a regular entity +directory name `Poi$`. The special `data_sources$` directory holds dynamic +data sources created at runtime. + +Within each directory, data is stored in numbered chunk files +(`chunk_000000.parquet`, `chunk_000001.parquet`, ...). A fresh dump +produces a single `chunk_000000.parquet` per table. Incremental dumps +append new chunks rather than rewriting existing ones. + +The GraphQL schema and subgraph manifest are stored as separate plain-text +files `schema.graphql` and `subgraph.yaml`. + +### metadata.json + +The top-level `metadata.json` contains everything needed to reconstruct the +deployment's table structure, plus diagnostic information captured at dump +time. Its structure is: + +```json +{ + "version": 1, + "deployment": "Qm...", + "network": "mainnet", + + "manifest": { + "spec_version": "1.0.0", + "description": "Optional subgraph description", + "repository": "https://github.com/...", + "features": ["..."], + "entities_with_causality_region": ["EntityType1"], + "history_blocks": 2147483647 + }, + + "earliest_block_number": 12345, + "start_block": { "number": 12345, "hash": "0xabc..." }, + "head_block": { "number": 99999, "hash": "0xdef..." }, + "entity_count": 150000, + + "graft_base": null, + "graft_block": null, + "debug_fork": null, + + "health": { + "failed": false, + "health": "healthy", + "fatal_error": null, + "non_fatal_errors": [] + }, + + "indexes": { + "token": [ + "CREATE INDEX CONCURRENTLY IF NOT EXISTS attr_0_0_id ON sgd.token USING btree (id)" + ] + }, + + "tables": { + "Token": { + "immutable": true, + "has_causality_region": false, + "chunks": [ + { + "file": "Token/chunk_000000.parquet", + "min_vid": 0, + "max_vid": 50000, + "row_count": 50000 + } + ], + "max_vid": 50000 + }, + "data_sources$": { + "immutable": false, + "has_causality_region": true, + "chunks": [ + { + "file": "data_sources$/chunk_000000.parquet", + "min_vid": 0, + "max_vid": 100, + "row_count": 100 + } + ], + "max_vid": 100 + } + } +} +``` + +**Field descriptions:** + +| Field | Description | +| ----------------------------------------- | ----------------------------------------------------------------------------------------------------------------------------------- | +| `version` | Format version. Must be `1`. | +| `deployment` | Deployment hash (`Qm...`). | +| `network` | The blockchain network (e.g. `mainnet`, `goerli`). | +| `manifest` | Manifest metadata extracted from `subgraphs.subgraph_manifest`. | +| `manifest.spec_version` | Subgraph API version. Required to parse `schema.graphql`. | +| `manifest.entities_with_causality_region` | Entity types that have a `causality_region` column. | +| `manifest.history_blocks` | How many blocks of entity version history are retained. | +| `earliest_block_number` | Earliest block for which data exists (accounts for pruning). | +| `start_block` | The block where indexing started. Null if not set. | +| `head_block` | The latest indexed block at dump time. | +| `entity_count` | Total entity count across all tables. | +| `graft_base` | Deployment hash of the graft base, if any. | +| `graft_block` | Block pointer of the graft point, if any. | +| `debug_fork` | Debug fork deployment hash, if any. | +| `health` | Point-in-time health snapshot. Not used during restore. | +| `indexes` | Point-in-time index definitions as SQL. Not used during restore (indexes are auto-created by `Layout::create_relational_schema()`). | +| `tables` | Per-table metadata keyed by entity type name (or `data_sources$`). | + +Each entry in `tables` contains: + +| Field | Description | +| ---------------------- | ------------------------------------------------------------------------------ | +| `immutable` | Whether the entity type is immutable (uses `block$` instead of `block_range`). | +| `has_causality_region` | Whether rows have a `causality_region` column. | +| `chunks` | Ordered list of Parquet chunk files for this table. | +| `chunks[].file` | Relative path from the dump directory. | +| `chunks[].min_vid` | Minimum `vid` value in this chunk. | +| `chunks[].max_vid` | Maximum `vid` value in this chunk. | +| `chunks[].row_count` | Number of rows in this chunk. | +| `max_vid` | Maximum `vid` across all chunks. `-1` if the table is empty. | + +### Parquet schema: entity tables + +Each entity table's Parquet file uses an Arrow schema derived from the +entity's GraphQL definition. Columns are ordered as follows: + +1. **System columns** (always present, in this order): + - `vid` (Int64, non-nullable) -- row version ID + - Block tracking (one of): + - Immutable entities: `block$` (Int32, non-nullable) + - Mutable entities: `block_range_start` (Int32, non-nullable), + `block_range_end` (Int32, nullable -- null means unbounded/current) + - `causality_region` (Int32, non-nullable) -- only if the entity has one + +2. **Data columns** in GraphQL declaration order, skipping fulltext + (`TSVector`) columns which are generated and rebuilt on restore. + +The PostgreSQL `int4range` type used for `block_range` is decomposed into +two scalar columns (`block_range_start`, `block_range_end`) in the Parquet +representation. This avoids the need for a custom range type in Arrow. + +#### Type mapping + +GraphQL/PostgreSQL column types map to Arrow data types as follows: + +| ColumnType | Arrow DataType | Notes | +| --------------- | ------------------------------ | -------------------------------------------------------------- | +| `Boolean` | `Boolean` | | +| `Int` | `Int32` | | +| `Int8` | `Int64` | | +| `Bytes` | `Binary` | Raw bytes, no hex encoding | +| `BigInt` | `Utf8` | Stored as decimal string for arbitrary precision | +| `BigDecimal` | `Utf8` | Stored as decimal string for arbitrary precision | +| `Timestamp` | `Timestamp(Microsecond, None)` | Microseconds since epoch, no timezone | +| `String` | `Utf8` | | +| `Enum(...)` | `Utf8` | Enum variant as string (cast from PG enum to text during dump) | +| `TSVector(...)` | _skipped_ | Fulltext index columns are generated; rebuilt on restore | + +**Array columns:** A GraphQL list field (e.g. `tags: [String!]!`) is +stored as `List` where `T` is the base Arrow type from the table +above. Whether a column is a list is determined by the GraphQL field type, +not by `ColumnType`. For example, `[String!]!` becomes `List` and +`[Int!]` becomes `List`. + +**Nullability** follows the GraphQL schema: non-null fields produce +non-nullable Arrow columns; optional fields produce nullable columns. List +elements within list columns are always marked nullable in the Arrow schema. + +### Parquet schema: data_sources$ + +The `data_sources$` table has a fixed schema independent of the GraphQL +definition: + +| Column | Arrow DataType | Nullable | Description | +| ------------------- | -------------- | -------- | -------------------------------------------------- | +| `vid` | `Int64` | no | Row version ID | +| `block_range_start` | `Int32` | no | Lower bound of `block_range` | +| `block_range_end` | `Int32` | yes | Upper bound (null = unbounded) | +| `causality_region` | `Int32` | no | Causality region | +| `manifest_idx` | `Int32` | no | Index into the manifest's data source list | +| `parent` | `Int32` | yes | Self-referencing parent data source | +| `id` | `Binary` | yes | Data source identifier | +| `param` | `Binary` | yes | Data source parameter | +| `context` | `Utf8` | yes | JSON context | +| `done_at` | `Int32` | yes | Block number where the data source was marked done | + +### Compression + +All Parquet files use ZSTD compression (default level). + +### Row ordering + +Within each Parquet chunk file, rows are ordered by `vid` (ascending). +This matches the primary key ordering in PostgreSQL and enables efficient +sequential reads during restore. + +### Incremental dumps + +An incremental dump reads the existing `metadata.json`, determines the +`max_vid` for each table, and queries only rows with `vid > max_vid`. New +rows are written to new chunk files (e.g. `chunk_000001.parquet`) and the +metadata is updated atomically (write to a temp file, then rename). + +### Atomicity + +The `metadata.json` file is always written atomically: the dump writes to +`metadata.json.tmp` first, then renames it to `metadata.json`. This +ensures that a reader never sees a partially-written metadata file. If the +dump process crashes mid-write, the previous `metadata.json` remains +intact. The Parquet chunk files are written before `metadata.json` is +updated, so chunk files referenced by `metadata.json` are always complete. From c6ff4e1e82895ded0ab280da9bf5f4e8060c920f Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 15:39:26 -0800 Subject: [PATCH 03/20] store: Add OidValue::Int4Range and fix block_range binding Add `OidValue::Int4Range(Bound, Bound)` variant to properly deserialize PostgreSQL int4range columns (OID 3904). Update `select_cols` in dsl.rs to bind block_range as `Range` instead of using the Bytes placeholder, resolving the existing TODO. --- store/postgres/src/relational/dsl.rs | 11 +++++++---- store/postgres/src/relational/value.rs | 19 ++++++++++++++++++- 2 files changed, 25 insertions(+), 5 deletions(-) diff --git a/store/postgres/src/relational/dsl.rs b/store/postgres/src/relational/dsl.rs index 8620fe7c1fa..5ba41a8d497 100644 --- a/store/postgres/src/relational/dsl.rs +++ b/store/postgres/src/relational/dsl.rs @@ -16,7 +16,7 @@ use diesel::query_dsl::methods::SelectDsl; use diesel::query_source::QuerySource; use diesel::sql_types::{ - Array, BigInt, Binary, Bool, Integer, Nullable, Numeric, SingleValue, Text, Timestamptz, + Array, BigInt, Binary, Bool, Integer, Nullable, Numeric, Range, SingleValue, Text, Timestamptz, Untyped, }; use diesel::{AppearsOnTable, Expression, QueryDsl, QueryResult, SelectableExpression}; @@ -43,8 +43,8 @@ lazy_static! { pub static ref TYPENAME_COL: RelColumn = RelColumn::pseudo_column(TYPENAME, ColumnType::String); pub static ref VID_COL: RelColumn = RelColumn::pseudo_column("vid", ColumnType::Int8); pub static ref BLOCK_COL: RelColumn = RelColumn::pseudo_column(BLOCK_COLUMN, ColumnType::Int8); - // The column type is a placeholder, we can't deserialize in4range; but - // we also never try to use it when we get data from the database + // The column type is a placeholder; block_range is deserialized as + // int4range via special handling in select_cols pub static ref BLOCK_RANGE_COL: RelColumn = RelColumn::pseudo_column(BLOCK_RANGE_COLUMN, ColumnType::Bytes); pub static ref PARENT_STRING_COL: RelColumn = RelColumn::pseudo_column(PARENT_ID, ColumnType::String); @@ -291,7 +291,6 @@ impl<'a> Table<'a> { if self.meta.immutable { cols.push(&*BLOCK_COL); } else { - // TODO: We can't deserialize in4range cols.push(&*BLOCK_RANGE_COL); } } @@ -348,6 +347,10 @@ impl<'a> Table<'a> { ))); continue; } + if column.name == BLOCK_RANGE_COL.name { + selection.add_field(self.bind::>(&column.name).unwrap()); + continue; + } match column.column_type { ColumnType::Boolean => add_field::(&mut selection, self, column), ColumnType::BigDecimal => add_field::(&mut selection, self, column), diff --git a/store/postgres/src/relational/value.rs b/store/postgres/src/relational/value.rs index a59dcfe511c..6e4764b14c0 100644 --- a/store/postgres/src/relational/value.rs +++ b/store/postgres/src/relational/value.rs @@ -1,8 +1,9 @@ //! Helpers to use diesel dynamic schema to retrieve values from Postgres use std::num::NonZeroU32; +use std::ops::Bound; -use diesel::sql_types::{Array, BigInt, Binary, Bool, Integer, Numeric, Text, Timestamptz}; +use diesel::sql_types::{Array, BigInt, Binary, Bool, Integer, Numeric, Range, Text, Timestamptz}; use diesel::{deserialize::FromSql, pg::Pg}; use diesel_dynamic_schema::dynamic_value::{Any, DynamicRow}; @@ -45,6 +46,7 @@ pub enum OidValue { BigDecimalArray(Vec), Timestamp(Timestamp), TimestampArray(Vec), + Int4Range(Bound, Bound), Null, } @@ -66,6 +68,7 @@ impl FromSql for OidValue { const NUMERIC_ARY_OID: NonZeroU32 = NonZeroU32::new(1231).unwrap(); const TIMESTAMPTZ_OID: NonZeroU32 = NonZeroU32::new(1184).unwrap(); const TIMESTAMPTZ_ARY_OID: NonZeroU32 = NonZeroU32::new(1185).unwrap(); + const INT4RANGE_OID: NonZeroU32 = NonZeroU32::new(3904).unwrap(); match value.get_oid() { VARCHAR_OID | TEXT_OID => { @@ -102,6 +105,10 @@ impl FromSql for OidValue { as FromSql, Pg>>::from_sql(value) .map(OidValue::TimestampArray) } + INT4RANGE_OID => { + <(Bound, Bound) as FromSql, Pg>>::from_sql(value) + .map(|(lo, hi)| OidValue::Int4Range(lo, hi)) + } e => Err(format!("Unknown type: {e}").into()), } } @@ -143,6 +150,11 @@ impl FromOidValue for r::Value { O::BigDecimalArray(b) => as_list(b, |b| Self::String(b.to_string())), O::Timestamp(t) => Self::Timestamp(t), O::TimestampArray(t) => as_list(t, Self::Timestamp), + O::Int4Range(..) => { + return Err(StoreError::InternalError( + "int4range can not be converted to r::Value".to_string(), + )) + } O::Null => Self::Null, }; Ok(value) @@ -195,6 +207,11 @@ impl FromOidValue for graph::prelude::Value { }, O::Timestamp(t) => Self::Timestamp(t), O::TimestampArray(t) => as_list(t, Self::Timestamp), + O::Int4Range(..) => { + return Err(StoreError::InternalError( + "int4range can not be converted to Value".to_string(), + )) + } O::Null => Self::Null, }; Ok(value) From ad0845fd4fa40db1dd23eb2be5ca71272734a863 Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 15:39:50 -0800 Subject: [PATCH 04/20] store: Make infrastructure columns visible in Table::column() Add block$ (immutable), block_range (mutable), and causality_region to the column lookup so callers that need these columns by name can find them. Also add CAUSALITY_REGION_COL pseudo-column static and tighten with_nsp visibility to pub(crate). --- store/postgres/src/relational.rs | 2 +- store/postgres/src/relational/dsl.rs | 16 +++++++++++++++- store/postgres/src/relational/index.rs | 2 +- 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/store/postgres/src/relational.rs b/store/postgres/src/relational.rs index ecd88298fac..82487ba5127 100644 --- a/store/postgres/src/relational.rs +++ b/store/postgres/src/relational.rs @@ -76,7 +76,7 @@ use graph::prelude::{ QueryExecutionError, StoreError, ValueType, }; -use crate::block_range::{BoundSide, BLOCK_COLUMN, BLOCK_RANGE_COLUMN}; +use crate::block_range::{BoundSide, BLOCK_COLUMN, BLOCK_RANGE_COLUMN, CAUSALITY_REGION_COLUMN}; pub use crate::catalog::Catalog; use crate::ForeignServer; use crate::{catalog, deployment, AsyncPgConnection}; diff --git a/store/postgres/src/relational/dsl.rs b/store/postgres/src/relational/dsl.rs index 5ba41a8d497..0dacfa2e69b 100644 --- a/store/postgres/src/relational/dsl.rs +++ b/store/postgres/src/relational/dsl.rs @@ -32,7 +32,7 @@ use crate::relational_queries::PARENT_ID; use super::value::FromOidRow; use super::Column as RelColumn; use super::SqlName; -use super::{BLOCK_COLUMN, BLOCK_RANGE_COLUMN}; +use super::{BLOCK_COLUMN, BLOCK_RANGE_COLUMN, CAUSALITY_REGION_COLUMN}; const TYPENAME: &str = "__typename"; @@ -50,6 +50,8 @@ lazy_static! { pub static ref PARENT_STRING_COL: RelColumn = RelColumn::pseudo_column(PARENT_ID, ColumnType::String); pub static ref PARENT_BYTES_COL: RelColumn = RelColumn::pseudo_column(PARENT_ID, ColumnType::Bytes); pub static ref PARENT_INT_COL: RelColumn = RelColumn::pseudo_column(PARENT_ID, ColumnType::Int8); + pub static ref CAUSALITY_REGION_COL: RelColumn = + RelColumn::pseudo_column(CAUSALITY_REGION_COLUMN, ColumnType::Int); pub static ref META_COLS: [&'static RelColumn; 2] = [&*TYPENAME_COL, &*VID_COL]; } @@ -184,10 +186,22 @@ impl<'a> Table<'a> { /// is useful if just the name of the column qualified with the table /// name/alias is needed pub fn column(&self, name: &str) -> Option> { + let block_col: &RelColumn = if self.meta.immutable { + &BLOCK_COL + } else { + &BLOCK_RANGE_COL + }; + let cr_col: Option<&RelColumn> = if self.meta.has_causality_region { + Some(&*CAUSALITY_REGION_COL) + } else { + None + }; self.meta .columns .iter() .chain(*META_COLS) + .chain(std::iter::once(block_col)) + .chain(cr_col) .find(|c| &c.name == name) .map(|c| Column::new(*self, c)) } diff --git a/store/postgres/src/relational/index.rs b/store/postgres/src/relational/index.rs index 483fab23619..5cfb8f024e8 100644 --- a/store/postgres/src/relational/index.rs +++ b/store/postgres/src/relational/index.rs @@ -441,7 +441,7 @@ impl CreateIndex { } } - pub fn with_nsp(&self, nsp2: String) -> Result { + pub(crate) fn with_nsp(&self, nsp2: String) -> Result { let s = self.clone(); match s { CreateIndex::Unknown { defn: _ } => Err(anyhow!("Failed to parse the index")), From 6ebc0e881838c434f5b856aadf51e5b5c4c9afd6 Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 15:40:18 -0800 Subject: [PATCH 05/20] store: Add parquet module with Arrow schema mapping Create the foundation for Parquet-based dump/restore: a schema mapping module that converts relational Table definitions to Arrow Schema objects. - arrow_schema() maps Table -> Arrow Schema with system columns (vid, block tracking, causality_region) followed by data columns - data_sources_arrow_schema() provides fixed schema for data_sources$ table - All ColumnType variants mapped (TSVector skipped, Enum -> Utf8) - List columns wrapped in Arrow List type based on Column.is_list() --- Cargo.lock | 119 ++++++++- Cargo.toml | 1 + store/postgres/Cargo.toml | 2 + store/postgres/src/lib.rs | 1 + store/postgres/src/parquet/mod.rs | 6 + store/postgres/src/parquet/schema.rs | 378 +++++++++++++++++++++++++++ 6 files changed, 499 insertions(+), 8 deletions(-) create mode 100644 store/postgres/src/parquet/mod.rs create mode 100644 store/postgres/src/parquet/schema.rs diff --git a/Cargo.lock b/Cargo.lock index 13fe8973cd7..1ca4f40cbd4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -22,10 +22,10 @@ dependencies = [ ] [[package]] -name = "adler" -version = "1.0.2" +name = "adler2" +version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" +checksum = "320119579fcad9c21884f5c4861d16174d0e06250625266f50fe6898340abefa" [[package]] name = "ahash" @@ -3355,12 +3355,13 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.30" +version = "1.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f54427cfd1c7829e2a139fcefea601bf088ebca651d2bf53ebc600eac295dae" +checksum = "843fba2746e448b37e26a819579957415c8cef339bf08564fe8b7ddbd959573c" dependencies = [ "crc32fast", "miniz_oxide", + "zlib-rs", ] [[package]] @@ -3997,6 +3998,7 @@ version = "0.41.2" dependencies = [ "Inflector", "anyhow", + "arrow", "async-trait", "blake3 1.8.2", "chrono", @@ -4018,6 +4020,7 @@ dependencies = [ "lazy_static", "lru_time_cache", "openssl", + "parquet", "postgres", "postgres-openssl", "pretty_assertions", @@ -4747,6 +4750,12 @@ dependencies = [ "unicode-width", ] +[[package]] +name = "integer-encoding" +version = "3.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" + [[package]] name = "interprocess" version = "2.2.3" @@ -5164,6 +5173,15 @@ version = "0.11.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9106e1d747ffd48e6be5bb2d97fa706ed25b144fbee4d5c02eae110cd8d6badd" +[[package]] +name = "lz4_flex" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ab6473172471198271ff72e9379150e9dfd70d8e533e0752a27e515b48dd375e" +dependencies = [ + "twox-hash", +] + [[package]] name = "mach2" version = "0.4.2" @@ -5260,11 +5278,12 @@ dependencies = [ [[package]] name = "miniz_oxide" -version = "0.7.4" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b8a240ddb74feaf34a79a7add65a741f3167852fba007066dcac1ca548d89c08" +checksum = "1fa76a2c86f704bdb222d66965fb3d63269ce38518b83cb0575fca855ebb6316" dependencies = [ - "adler", + "adler2", + "simd-adler32", ] [[package]] @@ -5612,6 +5631,15 @@ version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "04744f49eae99ab78e0d5c0b603ab218f515ea8cfe5a456d7629ad883a3b6e7d" +[[package]] +name = "ordered-float" +version = "2.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "68f19d67e5a2795c94e73e0bb1cc1a7edeb2e28efd39e2e1c9b7a40c1108b11c" +dependencies = [ + "num-traits", +] + [[package]] name = "parity-scale-codec" version = "3.6.12" @@ -5673,6 +5701,40 @@ dependencies = [ "windows-link 0.2.0", ] +[[package]] +name = "parquet" +version = "57.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6ee96b29972a257b855ff2341b37e61af5f12d6af1158b6dcdb5b31ea07bb3cb" +dependencies = [ + "ahash", + "arrow-array", + "arrow-buffer", + "arrow-cast", + "arrow-data", + "arrow-ipc", + "arrow-schema", + "arrow-select", + "base64 0.22.1", + "brotli", + "bytes", + "chrono", + "flate2", + "half", + "hashbrown 0.16.1", + "lz4_flex", + "num-bigint 0.4.6", + "num-integer", + "num-traits", + "paste", + "seq-macro", + "simdutf8", + "snap", + "thrift", + "twox-hash", + "zstd", +] + [[package]] name = "paste" version = "1.0.15" @@ -6999,6 +7061,12 @@ version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cd0b0ec5f1c1ca621c432a25813d8d60c88abe6d3e08a3eb9cf37d97a0fe3d73" +[[package]] +name = "seq-macro" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1bc711410fbe7399f390ca1c3b60ad0f53f80e95c5eb935e52268a0e2cd49acc" + [[package]] name = "serde" version = "1.0.226" @@ -7255,6 +7323,12 @@ dependencies = [ "rand_core 0.6.4", ] +[[package]] +name = "simd-adler32" +version = "0.3.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e320a6c5ad31d271ad523dcf3ad13e2767ad8b1cb8f047f75a8aeaf8da139da2" + [[package]] name = "simdutf8" version = "0.1.5" @@ -7366,6 +7440,12 @@ dependencies = [ "serde", ] +[[package]] +name = "snap" +version = "1.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" + [[package]] name = "socket2" version = "0.5.7" @@ -7777,6 +7857,17 @@ dependencies = [ "num_cpus", ] +[[package]] +name = "thrift" +version = "0.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7e54bc85fc7faa8bc175c4bab5b92ba8d9a3ce893d0e9f42cc455c8ab16a9e09" +dependencies = [ + "byteorder", + "integer-encoding", + "ordered-float", +] + [[package]] name = "time" version = "0.3.47" @@ -8357,6 +8448,12 @@ dependencies = [ "utf-8", ] +[[package]] +name = "twox-hash" +version = "2.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ea3136b675547379c4bd395ca6b938e5ad3c3d20fad76e7fe85f9e0d011419c" + [[package]] name = "typenum" version = "1.17.0" @@ -9680,6 +9777,12 @@ dependencies = [ "syn 2.0.117", ] +[[package]] +name = "zlib-rs" +version = "0.6.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c745c48e1007337ed136dc99df34128b9faa6ed542d80a1c673cf55a6d7236c8" + [[package]] name = "zstd" version = "0.13.3" diff --git a/Cargo.toml b/Cargo.toml index 65aad95bc71..92fd17d2462 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -116,6 +116,7 @@ url = "2.5.8" ahash = "0.8.11" arrow = { version = "=57.3.0" } arrow-flight = { version = "=57.3.0", features = ["flight-sql-experimental"] } +parquet = { version = "=57.3.0" } futures = "0.3.31" half = "2.7.1" indoc = "2.0.7" diff --git a/store/postgres/Cargo.toml b/store/postgres/Cargo.toml index 3b215868ccc..aa7aaa31fbf 100644 --- a/store/postgres/Cargo.toml +++ b/store/postgres/Cargo.toml @@ -38,6 +38,8 @@ pretty_assertions = "1.4.1" sqlparser = { workspace = true } thiserror = { workspace = true } serde_yaml.workspace = true +arrow = { workspace = true } +parquet = { workspace = true } [dev-dependencies] clap.workspace = true diff --git a/store/postgres/src/lib.rs b/store/postgres/src/lib.rs index 0de50af4d60..84011378898 100644 --- a/store/postgres/src/lib.rs +++ b/store/postgres/src/lib.rs @@ -24,6 +24,7 @@ mod fork; mod functions; mod jobs; mod notification_listener; +mod parquet; mod pool; mod primary; pub mod query_store; diff --git a/store/postgres/src/parquet/mod.rs b/store/postgres/src/parquet/mod.rs new file mode 100644 index 00000000000..3d67aace96d --- /dev/null +++ b/store/postgres/src/parquet/mod.rs @@ -0,0 +1,6 @@ +#[allow(dead_code)] +pub(crate) mod convert; +#[allow(dead_code)] +pub(crate) mod schema; +#[allow(dead_code)] +pub(crate) mod writer; diff --git a/store/postgres/src/parquet/schema.rs b/store/postgres/src/parquet/schema.rs new file mode 100644 index 00000000000..234c6c2b203 --- /dev/null +++ b/store/postgres/src/parquet/schema.rs @@ -0,0 +1,378 @@ +use std::sync::Arc; + +use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; + +use crate::relational::{ColumnType, Table}; + +/// Map a relational `Table` to an Arrow `Schema`. +/// +/// System columns are emitted first (`vid`, then block tracking, then +/// `causality_region`), followed by data columns in declaration order. +/// Fulltext (TSVector) columns are skipped — they are generated and +/// rebuilt on restore. +pub fn arrow_schema(table: &Table) -> Schema { + let mut fields = Vec::new(); + + // vid — always present, non-nullable + fields.push(Field::new("vid", DataType::Int64, false)); + + // Block tracking + if table.immutable { + fields.push(Field::new("block$", DataType::Int32, false)); + } else { + fields.push(Field::new("block_range_start", DataType::Int32, false)); + fields.push(Field::new("block_range_end", DataType::Int32, true)); + } + + // Causality region + if table.has_causality_region { + fields.push(Field::new("causality_region", DataType::Int32, false)); + } + + // Data columns + for col in &table.columns { + if col.is_fulltext() { + continue; + } + let base_type = column_type_to_arrow(&col.column_type); + let dt = if col.is_list() { + DataType::List(Arc::new(Field::new("item", base_type, true))) + } else { + base_type + }; + fields.push(Field::new(col.name.as_str(), dt, col.is_nullable())); + } + + Schema::new(fields) +} + +/// Fixed Arrow schema for the `data_sources$` table. +pub fn data_sources_arrow_schema() -> Schema { + Schema::new(vec![ + Field::new("vid", DataType::Int64, false), + Field::new("block_range_start", DataType::Int32, false), + Field::new("block_range_end", DataType::Int32, true), + Field::new("causality_region", DataType::Int32, false), + Field::new("manifest_idx", DataType::Int32, false), + Field::new("parent", DataType::Int32, true), + Field::new("id", DataType::Binary, true), + Field::new("param", DataType::Binary, true), + Field::new("context", DataType::Utf8, true), + Field::new("done_at", DataType::Int32, true), + ]) +} + +fn column_type_to_arrow(ct: &ColumnType) -> DataType { + match ct { + ColumnType::Boolean => DataType::Boolean, + ColumnType::Int => DataType::Int32, + ColumnType::Int8 => DataType::Int64, + ColumnType::Bytes => DataType::Binary, + ColumnType::BigInt | ColumnType::BigDecimal => DataType::Utf8, + ColumnType::Timestamp => DataType::Timestamp(TimeUnit::Microsecond, None), + ColumnType::String | ColumnType::Enum(_) => DataType::Utf8, + ColumnType::TSVector(_) => { + unreachable!("TSVector columns should be skipped before calling column_type_to_arrow") + } + } +} + +#[cfg(test)] +mod tests { + use std::collections::BTreeSet; + use std::sync::Arc; + + use arrow::datatypes::{DataType, TimeUnit}; + use graph::prelude::DeploymentHash; + use graph::schema::InputSchema; + + use crate::layout_for_tests::{make_dummy_site, Catalog, Layout, Namespace}; + + use super::*; + + fn test_layout(gql: &str) -> Layout { + let subgraph = DeploymentHash::new("subgraph").unwrap(); + let schema = InputSchema::parse_latest(gql, subgraph.clone()).expect("Test schema invalid"); + let namespace = Namespace::new("sgd0815".to_owned()).unwrap(); + let site = Arc::new(make_dummy_site(subgraph, namespace, "anet".to_string())); + let catalog = + Catalog::for_tests(site.clone(), BTreeSet::new()).expect("Can not create catalog"); + Layout::new(site, &schema, catalog).expect("Failed to construct Layout") + } + + fn test_layout_with_causality(gql: &str, entity_name: &str) -> Layout { + let subgraph = DeploymentHash::new("subgraph").unwrap(); + let schema = InputSchema::parse_latest(gql, subgraph.clone()).expect("Test schema invalid"); + let namespace = Namespace::new("sgd0815".to_owned()).unwrap(); + let site = Arc::new(make_dummy_site(subgraph, namespace, "anet".to_string())); + let entity_type = schema.entity_type(entity_name).unwrap(); + let ents = BTreeSet::from_iter(vec![entity_type]); + let catalog = Catalog::for_tests(site.clone(), ents).expect("Can not create catalog"); + Layout::new(site, &schema, catalog).expect("Failed to construct Layout") + } + + #[test] + fn immutable_table_schema() { + let layout = test_layout( + "type Token @entity(immutable: true) { id: ID!, name: String!, decimals: Int! }", + ); + let table = layout.table("token").unwrap(); + let schema = arrow_schema(table); + + let names: Vec<&str> = schema.fields().iter().map(|f| f.name().as_str()).collect(); + assert_eq!(names, &["vid", "block$", "id", "name", "decimals"]); + + assert_eq!( + schema.field_with_name("vid").unwrap().data_type(), + &DataType::Int64 + ); + assert!(!schema.field_with_name("vid").unwrap().is_nullable()); + + assert_eq!( + schema.field_with_name("block$").unwrap().data_type(), + &DataType::Int32 + ); + assert!(!schema.field_with_name("block$").unwrap().is_nullable()); + + assert_eq!( + schema.field_with_name("id").unwrap().data_type(), + &DataType::Utf8 + ); + assert!(!schema.field_with_name("id").unwrap().is_nullable()); + + assert_eq!( + schema.field_with_name("name").unwrap().data_type(), + &DataType::Utf8 + ); + assert_eq!( + schema.field_with_name("decimals").unwrap().data_type(), + &DataType::Int32 + ); + } + + #[test] + fn mutable_table_with_causality_region() { + let layout = test_layout_with_causality( + "type Transfer @entity { id: ID!, amount: BigInt! }", + "Transfer", + ); + let table = layout.table("transfer").unwrap(); + let schema = arrow_schema(table); + + let names: Vec<&str> = schema.fields().iter().map(|f| f.name().as_str()).collect(); + assert_eq!( + names, + &[ + "vid", + "block_range_start", + "block_range_end", + "causality_region", + "id", + "amount" + ] + ); + + assert_eq!( + schema + .field_with_name("block_range_start") + .unwrap() + .data_type(), + &DataType::Int32 + ); + assert!(!schema + .field_with_name("block_range_start") + .unwrap() + .is_nullable()); + + assert_eq!( + schema + .field_with_name("block_range_end") + .unwrap() + .data_type(), + &DataType::Int32 + ); + assert!(schema + .field_with_name("block_range_end") + .unwrap() + .is_nullable()); + + assert_eq!( + schema + .field_with_name("causality_region") + .unwrap() + .data_type(), + &DataType::Int32 + ); + + assert_eq!( + schema.field_with_name("amount").unwrap().data_type(), + &DataType::Utf8 + ); + } + + #[test] + fn mutable_table_without_causality_region() { + let layout = test_layout("type Pair @entity { id: ID!, price: BigDecimal! }"); + let table = layout.table("pair").unwrap(); + let schema = arrow_schema(table); + + let names: Vec<&str> = schema.fields().iter().map(|f| f.name().as_str()).collect(); + assert_eq!( + names, + &["vid", "block_range_start", "block_range_end", "id", "price"] + ); + + // No causality_region field + assert!(schema.field_with_name("causality_region").is_err()); + } + + #[test] + fn nullable_and_list_columns() { + let layout = test_layout( + "type Pool @entity(immutable: true) { id: ID!, tags: [String!]!, description: String }", + ); + let table = layout.table("pool").unwrap(); + let schema = arrow_schema(table); + + // tags is a non-nullable list of strings + let tags = schema.field_with_name("tags").unwrap(); + assert!(!tags.is_nullable()); + match tags.data_type() { + DataType::List(inner) => { + assert_eq!(inner.data_type(), &DataType::Utf8); + } + other => panic!("expected List, got {:?}", other), + } + + // description is nullable + let desc = schema.field_with_name("description").unwrap(); + assert!(desc.is_nullable()); + assert_eq!(desc.data_type(), &DataType::Utf8); + } + + #[test] + fn all_scalar_column_types() { + let layout = test_layout( + "type Everything @entity(immutable: true) { \ + id: ID!, \ + flag: Boolean!, \ + small: Int!, \ + big: Int8!, \ + amount: BigInt!, \ + price: BigDecimal!, \ + data: Bytes!, \ + ts: Timestamp!, \ + label: String! \ + }", + ); + let table = layout.table("everything").unwrap(); + let schema = arrow_schema(table); + + assert_eq!( + schema.field_with_name("flag").unwrap().data_type(), + &DataType::Boolean + ); + assert_eq!( + schema.field_with_name("small").unwrap().data_type(), + &DataType::Int32 + ); + assert_eq!( + schema.field_with_name("big").unwrap().data_type(), + &DataType::Int64 + ); + assert_eq!( + schema.field_with_name("amount").unwrap().data_type(), + &DataType::Utf8 + ); + assert_eq!( + schema.field_with_name("price").unwrap().data_type(), + &DataType::Utf8 + ); + assert_eq!( + schema.field_with_name("data").unwrap().data_type(), + &DataType::Binary + ); + assert_eq!( + schema.field_with_name("ts").unwrap().data_type(), + &DataType::Timestamp(TimeUnit::Microsecond, None) + ); + assert_eq!( + schema.field_with_name("label").unwrap().data_type(), + &DataType::Utf8 + ); + } + + #[test] + fn data_sources_schema() { + let schema = data_sources_arrow_schema(); + let names: Vec<&str> = schema.fields().iter().map(|f| f.name().as_str()).collect(); + assert_eq!( + names, + &[ + "vid", + "block_range_start", + "block_range_end", + "causality_region", + "manifest_idx", + "parent", + "id", + "param", + "context", + "done_at" + ] + ); + + // Check nullability + assert!(!schema.field_with_name("vid").unwrap().is_nullable()); + assert!(!schema + .field_with_name("block_range_start") + .unwrap() + .is_nullable()); + assert!(schema + .field_with_name("block_range_end") + .unwrap() + .is_nullable()); + assert!(!schema + .field_with_name("causality_region") + .unwrap() + .is_nullable()); + assert!(!schema + .field_with_name("manifest_idx") + .unwrap() + .is_nullable()); + assert!(schema.field_with_name("parent").unwrap().is_nullable()); + assert!(schema.field_with_name("id").unwrap().is_nullable()); + assert!(schema.field_with_name("param").unwrap().is_nullable()); + assert!(schema.field_with_name("context").unwrap().is_nullable()); + assert!(schema.field_with_name("done_at").unwrap().is_nullable()); + + // Check types + assert_eq!( + schema.field_with_name("vid").unwrap().data_type(), + &DataType::Int64 + ); + assert_eq!( + schema.field_with_name("context").unwrap().data_type(), + &DataType::Utf8 + ); + assert_eq!( + schema.field_with_name("param").unwrap().data_type(), + &DataType::Binary + ); + } + + #[test] + fn enum_columns_map_to_utf8() { + let layout = test_layout( + "enum Status { Active, Inactive } \ + type Item @entity(immutable: true) { id: ID!, status: Status! }", + ); + let table = layout.table("item").unwrap(); + let schema = arrow_schema(table); + + assert_eq!( + schema.field_with_name("status").unwrap().data_type(), + &DataType::Utf8 + ); + } +} From cf6951aedd09d0f6c8d7ed1496f386099f4258aa Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 15:40:28 -0800 Subject: [PATCH 06/20] store: Add OidValue to Arrow RecordBatch conversion Implement rows_to_record_batch() which converts Vec> into Arrow RecordBatch objects. Uses a type-erased ColumnBuilder enum that dispatches on Arrow DataType to create the appropriate array builder. Supports all OidValue scalar variants (Bool, Int, Int8, Bytes, String, BigDecimal, Timestamp) and all array variants (BoolArray, Ints, Int8Array, BytesArray, StringArray, BigDecimalArray, TimestampArray). Block range columns arrive as separate Int32 values from the dump query, keeping the converter as a clean 1:1 mapping. --- store/postgres/src/parquet/convert.rs | 490 ++++++++++++++++++++++++++ 1 file changed, 490 insertions(+) create mode 100644 store/postgres/src/parquet/convert.rs diff --git a/store/postgres/src/parquet/convert.rs b/store/postgres/src/parquet/convert.rs new file mode 100644 index 00000000000..49fc7bf53b4 --- /dev/null +++ b/store/postgres/src/parquet/convert.rs @@ -0,0 +1,490 @@ +use std::sync::Arc; + +use arrow::array::{ + ArrayRef, BinaryBuilder, BooleanBuilder, Int32Builder, Int64Builder, ListBuilder, RecordBatch, + StringBuilder, TimestampMicrosecondBuilder, +}; +use arrow::datatypes::{DataType, Schema}; + +use graph::components::store::StoreError; + +use crate::relational::value::{OidRow, OidValue}; + +/// Convert a batch of `DynamicRow` rows into an Arrow `RecordBatch`. +/// +/// The `schema` defines the expected column names, types, and nullability. +/// Each row must have exactly `schema.fields().len()` values, positionally +/// matching the schema fields. +/// +/// The dump query selects `lower(block_range)` and `upper(block_range)` as +/// separate Int32 columns, so the converter never sees `OidValue::Int4Range`. +pub fn rows_to_record_batch(schema: &Schema, rows: &[OidRow]) -> Result { + let num_fields = schema.fields().len(); + let num_rows = rows.len(); + + // Create builders for each column + let mut builders: Vec = schema + .fields() + .iter() + .map(|field| ColumnBuilder::new(field.data_type(), num_rows)) + .collect::>()?; + + // Append each row's values to the builders + for (row_idx, row) in rows.iter().enumerate() { + if row.len() != num_fields { + return Err(StoreError::InternalError(format!( + "row {row_idx} has {} values but schema has {num_fields} fields", + row.len() + ))); + } + for (col_idx, value) in row.into_iter().enumerate() { + builders[col_idx].append(value, row_idx)?; + } + } + + // Finish builders into arrays + let arrays: Vec = builders.into_iter().map(|b| b.finish()).collect(); + + RecordBatch::try_new(Arc::new(schema.clone()), arrays) + .map_err(|e| StoreError::InternalError(format!("failed to build RecordBatch: {e}"))) +} + +/// A type-erased column builder that wraps the appropriate Arrow array builder. +enum ColumnBuilder { + Boolean(BooleanBuilder), + Int32(Int32Builder), + Int64(Int64Builder), + Binary(BinaryBuilder), + Utf8(StringBuilder), + TimestampMicro(TimestampMicrosecondBuilder), + BooleanList(ListBuilder), + Int32List(ListBuilder), + Int64List(ListBuilder), + BinaryList(ListBuilder), + Utf8List(ListBuilder), + TimestampMicroList(ListBuilder), +} + +impl ColumnBuilder { + fn new(data_type: &DataType, capacity: usize) -> Result { + match data_type { + DataType::Boolean => Ok(Self::Boolean(BooleanBuilder::with_capacity(capacity))), + DataType::Int32 => Ok(Self::Int32(Int32Builder::with_capacity(capacity))), + DataType::Int64 => Ok(Self::Int64(Int64Builder::with_capacity(capacity))), + DataType::Binary => Ok(Self::Binary(BinaryBuilder::with_capacity(capacity, 0))), + DataType::Utf8 => Ok(Self::Utf8(StringBuilder::with_capacity(capacity, 0))), + DataType::Timestamp(arrow::datatypes::TimeUnit::Microsecond, None) => Ok( + Self::TimestampMicro(TimestampMicrosecondBuilder::with_capacity(capacity)), + ), + DataType::List(inner) => match inner.data_type() { + DataType::Boolean => Ok(Self::BooleanList(ListBuilder::with_capacity( + BooleanBuilder::new(), + capacity, + ))), + DataType::Int32 => Ok(Self::Int32List(ListBuilder::with_capacity( + Int32Builder::new(), + capacity, + ))), + DataType::Int64 => Ok(Self::Int64List(ListBuilder::with_capacity( + Int64Builder::new(), + capacity, + ))), + DataType::Binary => Ok(Self::BinaryList(ListBuilder::with_capacity( + BinaryBuilder::new(), + capacity, + ))), + DataType::Utf8 => Ok(Self::Utf8List(ListBuilder::with_capacity( + StringBuilder::new(), + capacity, + ))), + DataType::Timestamp(arrow::datatypes::TimeUnit::Microsecond, None) => { + Ok(Self::TimestampMicroList(ListBuilder::with_capacity( + TimestampMicrosecondBuilder::new(), + capacity, + ))) + } + other => Err(StoreError::InternalError(format!( + "unsupported list element type: {other:?}" + ))), + }, + other => Err(StoreError::InternalError(format!( + "unsupported Arrow data type for column builder: {other:?}" + ))), + } + } + + fn append(&mut self, value: &OidValue, row_idx: usize) -> Result<(), StoreError> { + match (self, value) { + // Null for any builder type + (Self::Boolean(b), OidValue::Null) => b.append_null(), + (Self::Int32(b), OidValue::Null) => b.append_null(), + (Self::Int64(b), OidValue::Null) => b.append_null(), + (Self::Binary(b), OidValue::Null) => b.append_null(), + (Self::Utf8(b), OidValue::Null) => b.append_null(), + (Self::TimestampMicro(b), OidValue::Null) => b.append_null(), + (Self::BooleanList(b), OidValue::Null) => b.append_null(), + (Self::Int32List(b), OidValue::Null) => b.append_null(), + (Self::Int64List(b), OidValue::Null) => b.append_null(), + (Self::BinaryList(b), OidValue::Null) => b.append_null(), + (Self::Utf8List(b), OidValue::Null) => b.append_null(), + (Self::TimestampMicroList(b), OidValue::Null) => b.append_null(), + + // Scalar values + (Self::Boolean(b), OidValue::Bool(v)) => b.append_value(*v), + (Self::Int32(b), OidValue::Int(v)) => b.append_value(*v), + (Self::Int64(b), OidValue::Int8(v)) => b.append_value(*v), + (Self::Int64(b), OidValue::Int(v)) => b.append_value(*v as i64), + (Self::Binary(b), OidValue::Bytes(v)) => b.append_value(v.as_ref()), + (Self::Utf8(b), OidValue::String(v)) => b.append_value(v), + (Self::Utf8(b), OidValue::BigDecimal(v)) => b.append_value(v.to_string()), + (Self::TimestampMicro(b), OidValue::Timestamp(v)) => { + b.append_value(v.as_microseconds_since_epoch()) + } + + // Array values + (Self::BooleanList(b), OidValue::BoolArray(vals)) => { + for v in vals { + b.values().append_value(*v); + } + b.append(true); + } + (Self::Int32List(b), OidValue::Ints(vals)) => { + for v in vals { + b.values().append_value(*v); + } + b.append(true); + } + (Self::Int64List(b), OidValue::Int8Array(vals)) => { + for v in vals { + b.values().append_value(*v); + } + b.append(true); + } + (Self::BinaryList(b), OidValue::BytesArray(vals)) => { + for v in vals { + b.values().append_value(v.as_ref()); + } + b.append(true); + } + (Self::Utf8List(b), OidValue::StringArray(vals)) => { + for v in vals { + b.values().append_value(v); + } + b.append(true); + } + (Self::Utf8List(b), OidValue::BigDecimalArray(vals)) => { + for v in vals { + b.values().append_value(v.to_string()); + } + b.append(true); + } + (Self::TimestampMicroList(b), OidValue::TimestampArray(vals)) => { + for v in vals { + b.values().append_value(v.as_microseconds_since_epoch()); + } + b.append(true); + } + + (builder, value) => { + return Err(StoreError::InternalError(format!( + "row {row_idx}: type mismatch - cannot append {value:?} to {builder_type}", + builder_type = builder.type_name() + ))); + } + } + Ok(()) + } + + fn finish(self) -> ArrayRef { + match self { + Self::Boolean(mut b) => Arc::new(b.finish()), + Self::Int32(mut b) => Arc::new(b.finish()), + Self::Int64(mut b) => Arc::new(b.finish()), + Self::Binary(mut b) => Arc::new(b.finish()), + Self::Utf8(mut b) => Arc::new(b.finish()), + Self::TimestampMicro(mut b) => Arc::new(b.finish()), + Self::BooleanList(mut b) => Arc::new(b.finish()), + Self::Int32List(mut b) => Arc::new(b.finish()), + Self::Int64List(mut b) => Arc::new(b.finish()), + Self::BinaryList(mut b) => Arc::new(b.finish()), + Self::Utf8List(mut b) => Arc::new(b.finish()), + Self::TimestampMicroList(mut b) => Arc::new(b.finish()), + } + } + + fn type_name(&self) -> &'static str { + match self { + Self::Boolean(_) => "Boolean", + Self::Int32(_) => "Int32", + Self::Int64(_) => "Int64", + Self::Binary(_) => "Binary", + Self::Utf8(_) => "Utf8", + Self::TimestampMicro(_) => "TimestampMicrosecond", + Self::BooleanList(_) => "List", + Self::Int32List(_) => "List", + Self::Int64List(_) => "List", + Self::BinaryList(_) => "List", + Self::Utf8List(_) => "List", + Self::TimestampMicroList(_) => "List", + } + } +} + +#[cfg(test)] +mod tests { + use arrow::array::{ + Array, BinaryArray, BooleanArray, Int32Array, Int64Array, ListArray, StringArray, + TimestampMicrosecondArray, + }; + use arrow::datatypes::{Field, Schema, TimeUnit}; + use graph::data::store::scalar::{BigDecimal, Bytes, Timestamp}; + use std::str::FromStr; + use std::sync::Arc; + + use super::*; + + fn make_row(values: Vec) -> OidRow { + values.into_iter().collect() + } + + #[test] + fn scalar_columns() { + let schema = Schema::new(vec![ + Field::new("vid", DataType::Int64, false), + Field::new("block$", DataType::Int32, false), + Field::new("id", DataType::Utf8, false), + Field::new("flag", DataType::Boolean, false), + Field::new("data", DataType::Binary, false), + Field::new("amount", DataType::Utf8, false), + Field::new( + "ts", + DataType::Timestamp(TimeUnit::Microsecond, None), + false, + ), + ]); + + let rows = vec![ + make_row(vec![ + OidValue::Int8(1), + OidValue::Int(100), + OidValue::String("token-1".to_string()), + OidValue::Bool(true), + OidValue::Bytes(Bytes::from(vec![0xab, 0xcd])), + OidValue::BigDecimal(BigDecimal::from_str("12345678901234567890").unwrap()), + OidValue::Timestamp( + Timestamp::from_microseconds_since_epoch(1_000_000_000_000).unwrap(), + ), + ]), + make_row(vec![ + OidValue::Int8(2), + OidValue::Int(101), + OidValue::String("token-2".to_string()), + OidValue::Bool(false), + OidValue::Bytes(Bytes::from(vec![0xff])), + OidValue::BigDecimal(BigDecimal::from_str("999").unwrap()), + OidValue::Timestamp( + Timestamp::from_microseconds_since_epoch(2_000_000_000_000).unwrap(), + ), + ]), + ]; + + let batch = rows_to_record_batch(&schema, &rows).unwrap(); + assert_eq!(batch.num_rows(), 2); + assert_eq!(batch.num_columns(), 7); + + // vid + let vid = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(vid.value(0), 1); + assert_eq!(vid.value(1), 2); + + // block$ + let block = batch + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(block.value(0), 100); + assert_eq!(block.value(1), 101); + + // id + let id = batch + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id.value(0), "token-1"); + assert_eq!(id.value(1), "token-2"); + + // flag + let flag = batch + .column(3) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(flag.value(0)); + assert!(!flag.value(1)); + + // data + let data = batch + .column(4) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(data.value(0), &[0xab, 0xcd]); + assert_eq!(data.value(1), &[0xff]); + + // amount (BigDecimal → Utf8) + let amount = batch + .column(5) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(amount.value(0), "12345678901234567890"); + assert_eq!(amount.value(1), "999"); + + // ts + let ts = batch + .column(6) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(ts.value(0), 1_000_000_000_000); + assert_eq!(ts.value(1), 2_000_000_000_000); + } + + #[test] + fn nullable_columns() { + let schema = Schema::new(vec![ + Field::new("vid", DataType::Int64, false), + Field::new("name", DataType::Utf8, true), + Field::new("count", DataType::Int32, true), + ]); + + let rows = vec![ + make_row(vec![ + OidValue::Int8(1), + OidValue::String("hello".to_string()), + OidValue::Int(42), + ]), + make_row(vec![OidValue::Int8(2), OidValue::Null, OidValue::Null]), + ]; + + let batch = rows_to_record_batch(&schema, &rows).unwrap(); + assert_eq!(batch.num_rows(), 2); + + let name = batch + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(!name.is_null(0)); + assert_eq!(name.value(0), "hello"); + assert!(name.is_null(1)); + + let count = batch + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(count.value(0), 42); + assert!(count.is_null(1)); + } + + #[test] + fn list_columns() { + let schema = Schema::new(vec![ + Field::new("vid", DataType::Int64, false), + Field::new( + "tags", + DataType::List(Arc::new(Field::new("item", DataType::Utf8, true))), + false, + ), + Field::new( + "scores", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + ), + ]); + + let rows = vec![ + make_row(vec![ + OidValue::Int8(1), + OidValue::StringArray(vec!["foo".to_string(), "bar".to_string()]), + OidValue::Ints(vec![10, 20, 30]), + ]), + make_row(vec![ + OidValue::Int8(2), + OidValue::StringArray(vec![]), + OidValue::Null, + ]), + ]; + + let batch = rows_to_record_batch(&schema, &rows).unwrap(); + assert_eq!(batch.num_rows(), 2); + + // tags + let tags = batch + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + let tags_0 = tags.value(0); + let tags_0_str = tags_0.as_any().downcast_ref::().unwrap(); + assert_eq!(tags_0_str.len(), 2); + assert_eq!(tags_0_str.value(0), "foo"); + assert_eq!(tags_0_str.value(1), "bar"); + + let tags_1 = tags.value(1); + let tags_1_str = tags_1.as_any().downcast_ref::().unwrap(); + assert_eq!(tags_1_str.len(), 0); + + // scores + let scores = batch + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + let scores_0 = scores.value(0); + let scores_0_int = scores_0.as_any().downcast_ref::().unwrap(); + assert_eq!(scores_0_int.len(), 3); + assert_eq!(scores_0_int.value(0), 10); + assert_eq!(scores_0_int.value(2), 30); + assert!(scores.is_null(1)); + } + + #[test] + fn empty_batch() { + let schema = Schema::new(vec![ + Field::new("vid", DataType::Int64, false), + Field::new("name", DataType::Utf8, false), + ]); + + let batch = rows_to_record_batch(&schema, &[]).unwrap(); + assert_eq!(batch.num_rows(), 0); + assert_eq!(batch.num_columns(), 2); + } + + #[test] + fn column_count_mismatch_errors() { + let schema = Schema::new(vec![ + Field::new("vid", DataType::Int64, false), + Field::new("name", DataType::Utf8, false), + ]); + + let rows = vec![make_row(vec![OidValue::Int8(1)])]; + let err = rows_to_record_batch(&schema, &rows).unwrap_err(); + assert!(err.to_string().contains("1 values but schema has 2 fields")); + } + + #[test] + fn type_mismatch_errors() { + let schema = Schema::new(vec![Field::new("vid", DataType::Int64, false)]); + let rows = vec![make_row(vec![OidValue::String("wrong".to_string())])]; + let err = rows_to_record_batch(&schema, &rows).unwrap_err(); + assert!(err.to_string().contains("type mismatch")); + } +} From 16147918cc903755d5b575c893c6948c9d9a1225 Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 15:40:36 -0800 Subject: [PATCH 07/20] store: Add Parquet chunk file writer Add ParquetChunkWriter that wraps Arrow's ArrowWriter to stream RecordBatches into a ZSTD-compressed Parquet file while tracking row count and vid range. Returns ChunkInfo metadata on finish for inclusion in metadata.json. --- store/postgres/src/parquet/writer.rs | 354 +++++++++++++++++++++++++++ 1 file changed, 354 insertions(+) create mode 100644 store/postgres/src/parquet/writer.rs diff --git a/store/postgres/src/parquet/writer.rs b/store/postgres/src/parquet/writer.rs new file mode 100644 index 00000000000..9ab6df53c20 --- /dev/null +++ b/store/postgres/src/parquet/writer.rs @@ -0,0 +1,354 @@ +use std::fs; +use std::path::PathBuf; +use std::sync::Arc; + +use arrow::array::RecordBatch; +use arrow::datatypes::Schema; +use graph::components::store::StoreError; +use parquet::arrow::ArrowWriter; +use parquet::basic::{Compression, ZstdLevel}; +use parquet::file::properties::WriterProperties; + +use serde::{Deserialize, Serialize}; + +/// Per-chunk metadata recorded in `metadata.json`. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub struct ChunkInfo { + pub file: String, + pub min_vid: i64, + pub max_vid: i64, + pub row_count: usize, +} + +/// Writes `RecordBatch`es into a single Parquet file using ZSTD compression. +/// +/// Tracks row count and vid range across all written batches. Call +/// `finish()` to flush and close the file, returning a `ChunkInfo` +/// summary. +pub struct ParquetChunkWriter { + writer: ArrowWriter, + /// Relative path from the dump directory (e.g. `"Token/chunk_000000.parquet"`). + relative_path: String, + row_count: usize, + min_vid: i64, + max_vid: i64, +} + +impl ParquetChunkWriter { + /// Create a new writer for a Parquet chunk file. + /// + /// `path` is the absolute file path. `relative_path` is stored in the + /// resulting `ChunkInfo` (e.g. `"Token/chunk_000000.parquet"`). + pub fn new(path: PathBuf, relative_path: String, schema: &Schema) -> Result { + let props = WriterProperties::builder() + .set_compression(Compression::ZSTD(ZstdLevel::default())) + .build(); + + let file = fs::File::create(&path).map_err(|e| { + StoreError::InternalError(format!( + "failed to create parquet file {}: {e}", + path.display() + )) + })?; + + let writer = + ArrowWriter::try_new(file, Arc::new(schema.clone()), Some(props)).map_err(|e| { + StoreError::InternalError(format!( + "failed to create ArrowWriter for {}: {e}", + path.display() + )) + })?; + + Ok(Self { + writer, + relative_path, + row_count: 0, + min_vid: i64::MAX, + max_vid: i64::MIN, + }) + } + + /// Write a `RecordBatch` and update tracking stats. + /// + /// `batch_min_vid` and `batch_max_vid` are the vid bounds of this + /// batch (typically the first and last vid values). + pub fn write_batch( + &mut self, + batch: &RecordBatch, + batch_min_vid: i64, + batch_max_vid: i64, + ) -> Result<(), StoreError> { + if batch.num_rows() == 0 { + return Ok(()); + } + self.writer + .write(batch) + .map_err(|e| StoreError::InternalError(format!("failed to write RecordBatch: {e}")))?; + self.row_count += batch.num_rows(); + self.min_vid = self.min_vid.min(batch_min_vid); + self.max_vid = self.max_vid.max(batch_max_vid); + Ok(()) + } + + /// Flush and close the Parquet file, returning chunk metadata. + pub fn finish(self) -> Result { + self.writer.close().map_err(|e| { + StoreError::InternalError(format!("failed to close parquet writer: {e}")) + })?; + Ok(ChunkInfo { + file: self.relative_path, + min_vid: self.min_vid, + max_vid: self.max_vid, + row_count: self.row_count, + }) + } +} + +#[cfg(test)] +mod tests { + use arrow::array::{ + Array, BinaryArray, BooleanArray, Int32Array, Int64Array, StringArray, + TimestampMicrosecondArray, + }; + use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; + use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; + use std::sync::Arc; + + use super::*; + + /// Create a unique temp file path for a test. The caller is responsible + /// for cleanup via the returned `TempFile` guard. + struct TempFile(PathBuf); + + impl TempFile { + fn new(name: &str) -> Self { + let path = std::env::temp_dir().join(format!( + "graph_node_test_{}_{name}.parquet", + std::process::id() + )); + Self(path) + } + + fn path(&self) -> &std::path::Path { + &self.0 + } + } + + impl Drop for TempFile { + fn drop(&mut self) { + let _ = std::fs::remove_file(&self.0); + } + } + + fn test_schema() -> Schema { + Schema::new(vec![ + Field::new("vid", DataType::Int64, false), + Field::new("block$", DataType::Int32, false), + Field::new("id", DataType::Utf8, false), + Field::new("flag", DataType::Boolean, true), + Field::new("data", DataType::Binary, true), + Field::new("ts", DataType::Timestamp(TimeUnit::Microsecond, None), true), + ]) + } + + fn test_batch(schema: &Schema) -> RecordBatch { + RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Int64Array::from(vec![1, 2, 3])), + Arc::new(Int32Array::from(vec![100, 101, 102])), + Arc::new(StringArray::from(vec!["a", "b", "c"])), + Arc::new(BooleanArray::from(vec![Some(true), None, Some(false)])), + Arc::new(BinaryArray::from_opt_vec(vec![ + Some(b"\xab\xcd"), + None, + Some(b"\xff"), + ])), + Arc::new(TimestampMicrosecondArray::from(vec![ + Some(1_000_000), + Some(2_000_000), + None, + ])), + ], + ) + .unwrap() + } + + /// Read all record batches from a parquet file. + fn read_parquet(path: &std::path::Path) -> Vec { + let file = std::fs::File::open(path).unwrap(); + let reader = ParquetRecordBatchReaderBuilder::try_new(file) + .unwrap() + .build() + .unwrap(); + reader.map(|r| r.unwrap()).collect() + } + + #[test] + fn write_and_read_back() { + let schema = test_schema(); + let batch = test_batch(&schema); + let tmp = TempFile::new("write_read"); + + let mut writer = ParquetChunkWriter::new( + tmp.path().to_path_buf(), + "Test/chunk_000000.parquet".into(), + &schema, + ) + .unwrap(); + writer.write_batch(&batch, 1, 3).unwrap(); + let chunk_info = writer.finish().unwrap(); + + assert_eq!(chunk_info.file, "Test/chunk_000000.parquet"); + assert_eq!(chunk_info.min_vid, 1); + assert_eq!(chunk_info.max_vid, 3); + assert_eq!(chunk_info.row_count, 3); + + // Read back and verify + let batches = read_parquet(tmp.path()); + assert_eq!(batches.len(), 1); + let read_batch = &batches[0]; + + assert_eq!(read_batch.num_rows(), 3); + assert_eq!(read_batch.num_columns(), 6); + + let vid = read_batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(vid.values(), &[1, 2, 3]); + + let block = read_batch + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(block.values(), &[100, 101, 102]); + + let id = read_batch + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id.value(0), "a"); + assert_eq!(id.value(2), "c"); + + let flag = read_batch + .column(3) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(flag.value(0)); + assert!(flag.is_null(1)); + assert!(!flag.value(2)); + + let data = read_batch + .column(4) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(data.value(0), b"\xab\xcd"); + assert!(data.is_null(1)); + + let ts = read_batch + .column(5) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(ts.value(0), 1_000_000); + assert!(ts.is_null(2)); + } + + #[test] + fn multiple_batches_accumulate_stats() { + let schema = test_schema(); + + let batch1 = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Int64Array::from(vec![10, 20])), + Arc::new(Int32Array::from(vec![1, 1])), + Arc::new(StringArray::from(vec!["x", "y"])), + Arc::new(BooleanArray::from(vec![true, false])), + Arc::new(BinaryArray::from_vec(vec![b"a", b"b"])), + Arc::new(TimestampMicrosecondArray::from(vec![100, 200])), + ], + ) + .unwrap(); + + let batch2 = RecordBatch::try_new( + Arc::new(schema.clone()), + vec![ + Arc::new(Int64Array::from(vec![30])), + Arc::new(Int32Array::from(vec![2])), + Arc::new(StringArray::from(vec!["z"])), + Arc::new(BooleanArray::from(vec![true])), + Arc::new(BinaryArray::from_vec(vec![b"c"])), + Arc::new(TimestampMicrosecondArray::from(vec![300])), + ], + ) + .unwrap(); + + let tmp = TempFile::new("multi_batch"); + + let mut writer = ParquetChunkWriter::new( + tmp.path().to_path_buf(), + "Foo/chunk_000000.parquet".into(), + &schema, + ) + .unwrap(); + writer.write_batch(&batch1, 10, 20).unwrap(); + writer.write_batch(&batch2, 30, 30).unwrap(); + let chunk_info = writer.finish().unwrap(); + + assert_eq!(chunk_info.min_vid, 10); + assert_eq!(chunk_info.max_vid, 30); + assert_eq!(chunk_info.row_count, 3); + + // Verify all 3 rows readable + let batches = read_parquet(tmp.path()); + let total_rows: usize = batches.iter().map(|b| b.num_rows()).sum(); + assert_eq!(total_rows, 3); + } + + #[test] + fn empty_batch_is_noop() { + let schema = test_schema(); + let empty = RecordBatch::new_empty(Arc::new(schema.clone())); + let tmp = TempFile::new("empty_batch"); + + let mut writer = ParquetChunkWriter::new( + tmp.path().to_path_buf(), + "X/chunk_000000.parquet".into(), + &schema, + ) + .unwrap(); + writer.write_batch(&empty, 0, 0).unwrap(); + let chunk_info = writer.finish().unwrap(); + + assert_eq!(chunk_info.row_count, 0); + // min_vid/max_vid stay at initial sentinel values when nothing was written + assert_eq!(chunk_info.min_vid, i64::MAX); + assert_eq!(chunk_info.max_vid, i64::MIN); + } + + #[test] + fn chunk_info_serialization() { + let info = ChunkInfo { + file: "Token/chunk_000000.parquet".into(), + min_vid: 0, + max_vid: 50000, + row_count: 50000, + }; + let json = serde_json::to_string_pretty(&info).unwrap(); + assert!(json.contains("Token/chunk_000000.parquet")); + assert!(json.contains("50000")); + + let deserialized: ChunkInfo = serde_json::from_str(&json).unwrap(); + assert_eq!(deserialized.file, info.file); + assert_eq!(deserialized.min_vid, info.min_vid); + assert_eq!(deserialized.max_vid, info.max_vid); + assert_eq!(deserialized.row_count, info.row_count); + } +} From c5eebfe807af710f0aefb43d44a3d9b5fde255a8 Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 15:40:54 -0800 Subject: [PATCH 08/20] store: Implement Parquet dump for subgraph data Wire dump queries and Parquet writing into Layout::dump(). This is the core of the dump feature, covering both entity tables and the special data_sources$ table. For entity tables: build a DynamicSelectClause that selects vid, block columns (split into lower/upper for mutable tables), causality_region, and data columns. Use VidBatcher for adaptive batching, convert rows to Arrow RecordBatch, and write via ParquetChunkWriter. For data_sources$: use a concrete QueryableByName struct with raw SQL (fixed schema, no DynamicSelectClause needed). Check table existence via catalog::table_exists before attempting dump. Metadata includes version, network, block pointers, entity count, graft info, health, indexes, and per-table chunk tracking. Written atomically via tmp+rename so its presence signals a complete dump. Add entity_count() helper in detail.rs. Wire dump() through DeploymentStore and SubgraphStore. --- store/postgres/src/deployment_store.rs | 13 + store/postgres/src/detail.rs | 12 + store/postgres/src/parquet/mod.rs | 3 - store/postgres/src/relational.rs | 1 + store/postgres/src/relational/dump.rs | 677 +++++++++++++++++++++++++ store/postgres/src/subgraph_store.rs | 11 + 6 files changed, 714 insertions(+), 3 deletions(-) create mode 100644 store/postgres/src/relational/dump.rs diff --git a/store/postgres/src/deployment_store.rs b/store/postgres/src/deployment_store.rs index a9fcc833e99..43a1dfb574f 100644 --- a/store/postgres/src/deployment_store.rs +++ b/store/postgres/src/deployment_store.rs @@ -30,6 +30,7 @@ use std::collections::{BTreeMap, HashMap}; use std::convert::Into; use std::ops::Bound; use std::ops::{Deref, Range}; +use std::path::PathBuf; use std::str::FromStr; use std::sync::{atomic::AtomicUsize, Arc, Mutex}; use std::time::{Duration, Instant}; @@ -896,6 +897,18 @@ impl DeploymentStore { Ok(relational::prune::Viewer::new(self.pool.clone(), layout)) } + + pub(crate) async fn dump(&self, site: Arc, dir: PathBuf) -> Result<(), StoreError> { + let mut conn = self.pool.get_permitted().await?; + let layout = self.layout(&mut conn, site.cheap_clone()).await?; + let entity_count = crate::detail::entity_count(&mut conn, &site).await?; + // Loading the IndexList should happen inside dump, but the + // interface does not allow it; should be changed + let index_list = IndexList::load(&mut conn, site.cheap_clone(), self.clone()).await?; + layout + .dump(&mut conn, index_list, dir, &site.network, entity_count) + .await + } } /// Methods that back the trait `WritableStore`, but have small variations in their signatures diff --git a/store/postgres/src/detail.rs b/store/postgres/src/detail.rs index 3be35320d8d..9e1827fcddb 100644 --- a/store/postgres/src/detail.rs +++ b/store/postgres/src/detail.rs @@ -657,6 +657,18 @@ pub async fn deployment_entity( StoredDeploymentEntity(detail, manifest).into_subgraph_deployment_entity(schema) } +/// Load the entity count for a deployment from `subgraphs.head`. +pub async fn entity_count(conn: &mut AsyncPgConnection, site: &Site) -> Result { + use subgraph_head as h; + + let count: i64 = h::table + .find(site.id) + .select(h::entity_count) + .first(conn) + .await?; + Ok(count as usize) +} + #[derive(Queryable, Identifiable, Insertable)] #[diesel(table_name = graph_node_versions)] pub struct GraphNodeVersion { diff --git a/store/postgres/src/parquet/mod.rs b/store/postgres/src/parquet/mod.rs index 3d67aace96d..8aedff85d6f 100644 --- a/store/postgres/src/parquet/mod.rs +++ b/store/postgres/src/parquet/mod.rs @@ -1,6 +1,3 @@ -#[allow(dead_code)] pub(crate) mod convert; -#[allow(dead_code)] pub(crate) mod schema; -#[allow(dead_code)] pub(crate) mod writer; diff --git a/store/postgres/src/relational.rs b/store/postgres/src/relational.rs index 82487ba5127..ba7d0d46157 100644 --- a/store/postgres/src/relational.rs +++ b/store/postgres/src/relational.rs @@ -15,6 +15,7 @@ mod ddl_tests; mod query_tests; pub(crate) mod dsl; +pub(crate) mod dump; pub(crate) mod index; pub(crate) mod prune; pub(crate) mod rollup; diff --git a/store/postgres/src/relational/dump.rs b/store/postgres/src/relational/dump.rs new file mode 100644 index 00000000000..f01c11b86d3 --- /dev/null +++ b/store/postgres/src/relational/dump.rs @@ -0,0 +1,677 @@ +use std::collections::BTreeMap; +use std::io::Write; +use std::path::{Path, PathBuf}; +use std::{collections::HashMap, fs}; + +use diesel::dsl::sql; +use diesel::sql_types::{ + Array, BigInt, Binary, Bool, Integer, Nullable, Numeric, Text, Timestamptz, Untyped, +}; +use diesel::QueryDsl; +use diesel_async::RunQueryDsl; +use diesel_dynamic_schema::DynamicSelectClause; + +use graph::data::subgraph::schema::{SubgraphError, SubgraphHealth, SubgraphManifestEntity}; +use graph::prelude::{DeploymentHash, StoreError, SubgraphDeploymentEntity}; +use serde::{Deserialize, Serialize}; + +use crate::catalog; +use crate::detail::deployment_entity; +use crate::parquet::convert::rows_to_record_batch; +use crate::parquet::schema::{arrow_schema, data_sources_arrow_schema}; +use crate::parquet::writer::{ChunkInfo, ParquetChunkWriter}; +use crate::relational::dsl; +use crate::relational::index::IndexList; +use crate::relational::value::OidRow; +use crate::relational::{ColumnType, SqlName, Table as RelTable}; +use crate::vid_batcher::{VidBatcher, VidRange}; +use crate::AsyncPgConnection; + +use super::Layout; + +#[derive(Serialize, Deserialize)] +struct Manifest { + spec_version: String, + description: Option, + repository: Option, + features: Vec, + entities_with_causality_region: Vec, + history_blocks: i32, +} + +impl Manifest { + fn new(data: SubgraphManifestEntity) -> (Self, String, Option) { + let SubgraphManifestEntity { + spec_version, + description, + repository, + features, + schema, + raw_yaml, + entities_with_causality_region, + history_blocks, + } = data; + let this = Self { + spec_version, + description, + repository, + features, + entities_with_causality_region: entities_with_causality_region + .into_iter() + .map(|v| v.to_string()) + .collect(), + history_blocks, + }; + (this, schema, raw_yaml) + } +} + +#[derive(Serialize, Deserialize)] +struct BlockPtr { + number: i32, + hash: String, +} + +impl BlockPtr { + fn new(block_ptr: graph::prelude::BlockPtr) -> Self { + let graph::prelude::BlockPtr { number, hash } = block_ptr; + let hash = hash.hash_hex(); + Self { number, hash } + } +} + +#[derive(Serialize, Deserialize)] +struct Error { + message: String, + block_ptr: Option, + handler: Option, + deterministic: bool, +} + +impl Error { + fn new(error: SubgraphError) -> Self { + let SubgraphError { + subgraph_id: _, + message, + block_ptr, + handler, + deterministic, + } = error; + let block_ptr = block_ptr.map(BlockPtr::new); + Self { + message, + block_ptr, + handler, + deterministic, + } + } +} + +#[derive(Serialize, Deserialize)] +struct Health { + failed: bool, + health: String, + fatal_error: Option, + non_fatal_errors: Vec, +} + +impl Health { + fn new( + failed: bool, + health: SubgraphHealth, + fatal_error: Option, + non_fatal_errors: Vec, + ) -> Self { + let health = health.as_str().to_string(); + let fatal_error = fatal_error.map(Error::new); + let non_fatal_errors = non_fatal_errors.into_iter().map(Error::new).collect(); + Self { + failed, + health, + fatal_error, + non_fatal_errors, + } + } +} + +/// Per-table metadata recorded in `metadata.json`. +#[derive(Debug, Clone, Serialize, Deserialize)] +pub(crate) struct TableInfo { + pub immutable: bool, + pub has_causality_region: bool, + pub chunks: Vec, + pub max_vid: i64, +} + +/// Top-level metadata for a subgraph dump (written as `metadata.json`). +/// +/// The `tables` field is initially empty and populated during the dump +/// as entity tables are written to Parquet files. +#[derive(Serialize, Deserialize)] +pub(crate) struct Metadata { + version: u32, + deployment: DeploymentHash, + network: String, + manifest: Manifest, + earliest_block_number: i32, + start_block: Option, + head_block: Option, + entity_count: usize, + graft_base: Option, + graft_block: Option, + debug_fork: Option, + health: Health, + indexes: HashMap>, + pub tables: BTreeMap, +} + +impl Metadata { + fn new( + deployment: DeploymentHash, + network: String, + entity_count: usize, + data: SubgraphDeploymentEntity, + index_list: IndexList, + ) -> (Self, String, Option) { + let indexes = Self::indexes(index_list); + let SubgraphDeploymentEntity { + manifest, + failed, + health, + synced_at: _, + fatal_error, + non_fatal_errors, + earliest_block_number, + start_block, + latest_block, + graft_base, + graft_block, + debug_fork, + reorg_count: _, + current_reorg_depth: _, + max_reorg_depth: _, + } = data; + let (manifest, schema, yaml) = Manifest::new(manifest); + let health = Health::new(failed, health, fatal_error, non_fatal_errors); + let start_block = start_block.map(BlockPtr::new); + let head_block = latest_block.map(BlockPtr::new); + let graft_block = graft_block.map(BlockPtr::new); + let this = Self { + version: 1, + deployment, + network, + manifest, + earliest_block_number, + start_block, + head_block, + entity_count, + graft_base, + graft_block, + debug_fork, + health, + indexes, + tables: BTreeMap::new(), + }; + (this, schema, yaml) + } + + fn indexes(index_list: IndexList) -> HashMap> { + let mut res = HashMap::new(); + for (name, indexes) in index_list.indexes { + let mut indexes2 = Vec::new(); + for index in indexes { + let Ok(index) = index.with_nsp("sgd".to_string()) else { + continue; + }; + let Ok(index) = index.to_sql(true, true) else { + continue; + }; + indexes2.push(index); + } + res.insert(name, indexes2); + } + res + } +} + +/// Build a `DynamicSelectClause` for dumping an entity table. The +/// selected columns match the Arrow schema from `parquet::schema::arrow_schema`: +/// `vid`, block columns (split for mutable), `causality_region`, then data columns. +fn dump_select<'a>( + table: &'a dsl::Table<'a>, + meta: &'a RelTable, +) -> diesel::query_builder::BoxedSelectStatement< + 'a, + Untyped, + diesel::query_builder::FromClause>, + diesel::pg::Pg, +> { + type SelectClause<'b> = DynamicSelectClause<'b, diesel::pg::Pg, dsl::Table<'b>>; + + fn add_data_field<'b>( + select: &mut SelectClause<'b>, + table: &'b dsl::Table<'b>, + column: &'b crate::relational::Column, + ) { + fn add_typed<'c, ST: diesel::sql_types::SingleValue + Send>( + select: &mut SelectClause<'c>, + table: &'c dsl::Table<'c>, + name: &str, + is_list: bool, + is_nullable: bool, + ) { + match (is_list, is_nullable) { + (true, true) => { + select.add_field(table.column(name).unwrap().bind::>>()) + } + (true, false) => select.add_field(table.column(name).unwrap().bind::>()), + (false, true) => { + select.add_field(table.column(name).unwrap().bind::>()) + } + (false, false) => select.add_field(table.column(name).unwrap().bind::()), + } + } + + let name = column.name.as_str(); + let is_list = column.is_list(); + let is_nullable = column.is_nullable(); + + match &column.column_type { + ColumnType::Boolean => add_typed::(select, table, name, is_list, is_nullable), + ColumnType::BigDecimal | ColumnType::BigInt => { + add_typed::(select, table, name, is_list, is_nullable) + } + ColumnType::Bytes => add_typed::(select, table, name, is_list, is_nullable), + ColumnType::Int => add_typed::(select, table, name, is_list, is_nullable), + ColumnType::Int8 => add_typed::(select, table, name, is_list, is_nullable), + ColumnType::Timestamp => { + add_typed::(select, table, name, is_list, is_nullable) + } + ColumnType::String => add_typed::(select, table, name, is_list, is_nullable), + ColumnType::Enum(_) => { + // Cast enum to text for dump + let alias = table.meta.name.as_str(); + let cast = if is_list { "text[]" } else { "text" }; + let expr = format!("c.\"{}\"::{}", name, cast); + match (is_list, is_nullable) { + (true, true) => select.add_field(sql::>>(&expr)), + (true, false) => select.add_field(sql::>(&expr)), + (false, true) => select.add_field(sql::>(&expr)), + (false, false) => select.add_field(sql::(&expr)), + } + let _ = alias; + } + ColumnType::TSVector(_) => { /* skip fulltext columns */ } + } + } + + let mut selection = DynamicSelectClause::new(); + + // vid + selection.add_field(table.column("vid").unwrap().bind::()); + + // Block columns + if meta.immutable { + selection.add_field(table.column("block$").unwrap().bind::()); + } else { + selection.add_field(sql::("lower(c.block_range)")); + selection.add_field(sql::>("upper(c.block_range)")); + } + + // Causality region + if meta.has_causality_region { + selection.add_field(table.column("causality_region").unwrap().bind::()); + } + + // Data columns + for col in &meta.columns { + if col.is_fulltext() { + continue; + } + add_data_field(&mut selection, table, col); + } + + diesel::QueryDsl::select(*table, selection).into_boxed() +} + +/// Query the vid range for a table. Returns `(min_vid, max_vid)`. +async fn vid_range(conn: &mut AsyncPgConnection, table: &RelTable) -> Result { + let query = format!( + "select coalesce(min(vid), 0)::int8 as min_vid, coalesce(max(vid), -1)::int8 as max_vid from {}", + table.qualified_name + ); + diesel::sql_query(&query) + .get_result::(conn) + .await + .map_err(StoreError::from) +} + +/// Dump a single entity table to a Parquet file. +async fn dump_entity_table( + conn: &mut AsyncPgConnection, + table: &RelTable, + dir: &Path, +) -> Result { + let arrow_schema = arrow_schema(table); + let table_dir_name = table.object.as_str(); + let table_dir = dir.join(table_dir_name); + fs::create_dir_all(&table_dir) + .map_err(|e| StoreError::InternalError(format!("failed to create dir: {e}")))?; + + let range = vid_range(conn, table).await?; + + if range.is_empty() { + return Ok(TableInfo { + immutable: table.immutable, + has_causality_region: table.has_causality_region, + chunks: vec![], + max_vid: -1, + }); + } + + let relative_path = format!("{}/chunk_000000.parquet", table_dir_name); + let abs_path = table_dir.join("chunk_000000.parquet"); + let mut writer = ParquetChunkWriter::new(abs_path, relative_path, &arrow_schema)?; + + let dsl_table = dsl::Table::new(table); + let mut batcher = VidBatcher::load(conn, &table.nsp, table, range).await?; + + while !batcher.finished() { + batcher + .step(async |start, end| { + let query = dump_select(&dsl_table, table) + .filter(sql::("vid >= ").bind::(start)) + .filter(sql::("vid <= ").bind::(end)) + .order(sql::("vid")); + + let rows: Vec = query.load(conn).await?; + if rows.is_empty() { + return Ok(()); + } + + let batch = rows_to_record_batch(&arrow_schema, &rows)?; + // Extract vid bounds from first/last row + let batch_min_vid = start; + let batch_max_vid = end; + writer.write_batch(&batch, batch_min_vid, batch_max_vid)?; + Ok(()) + }) + .await?; + } + + let chunk_info = writer.finish()?; + + let max_vid = if chunk_info.row_count > 0 { + chunk_info.max_vid + } else { + -1 + }; + + let chunks = if chunk_info.row_count > 0 { + vec![chunk_info] + } else { + vec![] + }; + + Ok(TableInfo { + immutable: table.immutable, + has_causality_region: table.has_causality_region, + chunks, + max_vid, + }) +} + +/// Row type for the `data_sources$` table dump query. +#[derive(diesel::QueryableByName)] +struct DataSourceRow { + #[diesel(sql_type = BigInt)] + vid: i64, + #[diesel(sql_type = Integer)] + block_range_start: i32, + #[diesel(sql_type = Nullable)] + block_range_end: Option, + #[diesel(sql_type = Integer)] + causality_region: i32, + #[diesel(sql_type = Integer)] + manifest_idx: i32, + #[diesel(sql_type = Nullable)] + parent: Option, + #[diesel(sql_type = Nullable)] + id: Option>, + #[diesel(sql_type = Nullable)] + param: Option>, + #[diesel(sql_type = Nullable)] + context: Option, + #[diesel(sql_type = Nullable)] + done_at: Option, +} + +/// Convert `DataSourceRow`s to an Arrow `RecordBatch` using the fixed +/// `data_sources$` schema. +fn data_source_rows_to_record_batch( + schema: &arrow::datatypes::Schema, + rows: &[DataSourceRow], +) -> Result { + use arrow::array::{BinaryBuilder, Int32Builder, Int64Builder, RecordBatch, StringBuilder}; + use std::sync::Arc; + + let n = rows.len(); + let mut vid_b = Int64Builder::with_capacity(n); + let mut brs_b = Int32Builder::with_capacity(n); + let mut bre_b = Int32Builder::with_capacity(n); + let mut cr_b = Int32Builder::with_capacity(n); + let mut mi_b = Int32Builder::with_capacity(n); + let mut parent_b = Int32Builder::with_capacity(n); + let mut id_b = BinaryBuilder::with_capacity(n, 0); + let mut param_b = BinaryBuilder::with_capacity(n, 0); + let mut ctx_b = StringBuilder::with_capacity(n, 0); + let mut da_b = Int32Builder::with_capacity(n); + + for row in rows { + vid_b.append_value(row.vid); + brs_b.append_value(row.block_range_start); + match row.block_range_end { + Some(v) => bre_b.append_value(v), + None => bre_b.append_null(), + } + cr_b.append_value(row.causality_region); + mi_b.append_value(row.manifest_idx); + match row.parent { + Some(v) => parent_b.append_value(v), + None => parent_b.append_null(), + } + match &row.id { + Some(v) => id_b.append_value(v), + None => id_b.append_null(), + } + match &row.param { + Some(v) => param_b.append_value(v), + None => param_b.append_null(), + } + match &row.context { + Some(v) => ctx_b.append_value(v), + None => ctx_b.append_null(), + } + match row.done_at { + Some(v) => da_b.append_value(v), + None => da_b.append_null(), + } + } + + let arrays: Vec = vec![ + Arc::new(vid_b.finish()), + Arc::new(brs_b.finish()), + Arc::new(bre_b.finish()), + Arc::new(cr_b.finish()), + Arc::new(mi_b.finish()), + Arc::new(parent_b.finish()), + Arc::new(id_b.finish()), + Arc::new(param_b.finish()), + Arc::new(ctx_b.finish()), + Arc::new(da_b.finish()), + ]; + + RecordBatch::try_new(Arc::new(schema.clone()), arrays) + .map_err(|e| StoreError::InternalError(format!("failed to build data_sources batch: {e}"))) +} + +const DATA_SOURCES_TABLE: &str = "data_sources$"; +const DATA_SOURCES_BATCH_SIZE: i64 = 10_000; + +/// Dump the `data_sources$` table to a Parquet file, if it exists. +async fn dump_data_sources( + conn: &mut AsyncPgConnection, + namespace: &str, + dir: &Path, +) -> Result, StoreError> { + let table_name = SqlName::verbatim(DATA_SOURCES_TABLE.to_string()); + if !catalog::table_exists(conn, namespace, &table_name).await? { + return Ok(None); + } + + let qualified = format!("\"{}\".\"{}\"", namespace, DATA_SOURCES_TABLE); + + // Get vid range + let range_query = format!( + "select coalesce(min(vid), 0)::int8 as min_vid, coalesce(max(vid), -1)::int8 as max_vid from {}", + qualified, + ); + let range: VidRange = diesel::sql_query(&range_query) + .get_result(conn) + .await + .map_err(StoreError::from)?; + + if range.is_empty() { + return Ok(Some(TableInfo { + immutable: false, + has_causality_region: true, + chunks: vec![], + max_vid: -1, + })); + } + + let schema = data_sources_arrow_schema(); + let ds_dir = dir.join(DATA_SOURCES_TABLE); + fs::create_dir_all(&ds_dir) + .map_err(|e| StoreError::InternalError(format!("failed to create dir: {e}")))?; + + let relative_path = format!("{}/chunk_000000.parquet", DATA_SOURCES_TABLE); + let abs_path = ds_dir.join("chunk_000000.parquet"); + let mut writer = ParquetChunkWriter::new(abs_path, relative_path, &schema)?; + + let mut start = range.min; + while start <= range.max { + let end = (start + DATA_SOURCES_BATCH_SIZE - 1).min(range.max); + let query = format!( + "select vid::int8, \ + lower(block_range) as block_range_start, \ + upper(block_range) as block_range_end, \ + causality_region, \ + manifest_idx, \ + parent, \ + id, \ + param, \ + context::text, \ + done_at \ + from {} \ + where vid >= $1 and vid <= $2 \ + order by vid", + qualified, + ); + let rows: Vec = diesel::sql_query(&query) + .bind::(start) + .bind::(end) + .load(conn) + .await + .map_err(StoreError::from)?; + + if !rows.is_empty() { + let batch = data_source_rows_to_record_batch(&schema, &rows)?; + writer.write_batch(&batch, start, end)?; + } + + start = end + 1; + } + + let chunk_info = writer.finish()?; + + let max_vid = if chunk_info.row_count > 0 { + chunk_info.max_vid + } else { + -1 + }; + + let chunks = if chunk_info.row_count > 0 { + vec![chunk_info] + } else { + vec![] + }; + + Ok(Some(TableInfo { + immutable: false, + has_causality_region: true, + chunks, + max_vid, + })) +} + +impl Layout { + pub(crate) async fn dump( + &self, + conn: &mut AsyncPgConnection, + index_list: IndexList, + dir: PathBuf, + network: &str, + entity_count: usize, + ) -> Result<(), StoreError> { + fn write_file(name: PathBuf, contents: &str) -> Result<(), StoreError> { + let mut file = fs::File::create(name).map_err(|e| StoreError::Unknown(e.into()))?; + file.write_all(contents.as_bytes()) + .map_err(|e| StoreError::Unknown(e.into()))?; + Ok(()) + } + + let deployment = deployment_entity(conn, &self.site, &self.input_schema).await?; + let (mut metadata, schema, yaml) = Metadata::new( + self.site.deployment.clone(), + network.to_string(), + entity_count, + deployment, + index_list, + ); + + write_file(dir.join("schema.graphql"), &schema)?; + + if let Some(yaml) = yaml { + write_file(dir.join("subgraph.yaml"), &yaml)?; + } + + // Dump entity tables sorted by name for determinism + let mut tables: Vec<_> = self.tables.values().collect(); + tables.sort_by_key(|t| t.name.as_str().to_string()); + + for table in tables { + let table_info = dump_entity_table(conn, table, &dir).await?; + metadata + .tables + .insert(table.object.as_str().to_string(), table_info); + } + + // Dump data_sources$ if it exists + let namespace = self.site.namespace.as_str(); + if let Some(ds_info) = dump_data_sources(conn, namespace, &dir).await? { + metadata + .tables + .insert(DATA_SOURCES_TABLE.to_string(), ds_info); + } + + // Write metadata.json atomically via tmp file + rename + let tmp_path = dir.join("metadata.json.tmp"); + write_file(tmp_path.clone(), &serde_json::to_string_pretty(&metadata)?)?; + fs::rename(&tmp_path, dir.join("metadata.json")).map_err(|e| { + StoreError::InternalError(format!("failed to rename metadata.json.tmp: {e}")) + })?; + + Ok(()) + } +} diff --git a/store/postgres/src/subgraph_store.rs b/store/postgres/src/subgraph_store.rs index e5b9ee0a529..c1f8b6df5fb 100644 --- a/store/postgres/src/subgraph_store.rs +++ b/store/postgres/src/subgraph_store.rs @@ -1435,6 +1435,17 @@ impl Inner { let src_store = self.for_site(&site)?; src_store.load_indexes(site).await } + + pub async fn dump( + &self, + loc: &DeploymentLocator, + directory: std::path::PathBuf, + ) -> Result<(), StoreError> { + let site = self.find_site(loc.id.into()).await?; + let store = self.for_site(&site)?; + + store.dump(site, directory).await + } } const STATE_ENS_NOT_CHECKED: u8 = 0; From 0ddb12b7181e56923421938dcfe1da6cd7b7ad9a Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 15:41:01 -0800 Subject: [PATCH 09/20] node: Add graphman dump command Wire the dump subcommand into graphman CLI. Takes a deployment search argument and an output directory, resolves the deployment, and delegates to SubgraphStore::dump(). --- node/src/bin/manager.rs | 25 +++++++++++++++++++++++++ node/src/manager/commands/dump.rs | 30 ++++++++++++++++++++++++++++++ node/src/manager/commands/mod.rs | 1 + 3 files changed, 56 insertions(+) create mode 100644 node/src/manager/commands/dump.rs diff --git a/node/src/bin/manager.rs b/node/src/bin/manager.rs index 792df8853c9..5ac90b73300 100644 --- a/node/src/bin/manager.rs +++ b/node/src/bin/manager.rs @@ -327,6 +327,21 @@ pub enum Command { #[clap(long, short, default_value = "http://localhost:8020")] url: String, }, + + /// Dump a subgraph deployment into a directory + /// + /// EXPERIMENTAL - NOT FOR PRODUCTION USE + /// + /// This will create a dump of the subgraph deployment in the specified + /// directory. The dump includes the subgraph manifest, the mapping, and + /// the data in the database as parquet files. The dump can be used to + /// restore the subgraph deployment later with the `restore` command. + Dump { + /// The deployment (see `help info`) + deployment: DeploymentSearch, + /// The name of the directory to dump to + directory: String, + }, } impl Command { @@ -1732,6 +1747,16 @@ async fn main() -> anyhow::Result<()> { commands::deploy::run(subgraph_store, deployment, name, url).await } + + Dump { + deployment, + directory, + } => { + let (store, primary_pool) = ctx.store_and_primary().await; + let subgraph_store = store.subgraph_store(); + + commands::dump::run(subgraph_store, primary_pool, deployment, directory).await + } } } diff --git a/node/src/manager/commands/dump.rs b/node/src/manager/commands/dump.rs new file mode 100644 index 00000000000..c17cb0da5c8 --- /dev/null +++ b/node/src/manager/commands/dump.rs @@ -0,0 +1,30 @@ +use std::fs; +use std::sync::Arc; + +use graph::{bail, prelude::anyhow::Result}; + +use graph_store_postgres::{ConnectionPool, SubgraphStore}; + +use crate::manager::deployment::DeploymentSearch; + +pub async fn run( + subgraph_store: Arc, + primary_pool: ConnectionPool, + deployment: DeploymentSearch, + directory: String, +) -> Result<()> { + let directory = fs::canonicalize(&directory)?; + let stat = fs::metadata(&directory)?; + + if !stat.is_dir() { + bail!( + "The path `{}` is not a directory", + directory.to_string_lossy() + ); + } + + let loc = deployment.locate_unique(&primary_pool).await?; + + subgraph_store.dump(&loc, directory).await?; + Ok(()) +} diff --git a/node/src/manager/commands/mod.rs b/node/src/manager/commands/mod.rs index 42e45605ebd..e2f413c1f1c 100644 --- a/node/src/manager/commands/mod.rs +++ b/node/src/manager/commands/mod.rs @@ -7,6 +7,7 @@ pub mod create; pub mod database; pub mod deploy; pub mod deployment; +pub mod dump; pub mod index; pub mod listen; pub mod provider_checks; From 63de396a78f9b5715cfe499f21cac4e5ad3350ac Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 16:30:12 -0800 Subject: [PATCH 10/20] store: Add Parquet reader for restore Add `parquet/reader.rs` with `read_batches()` to read Parquet files back into Arrow RecordBatches. This is the foundation for the restore pipeline (step 1 of the restore plan). --- store/postgres/src/parquet/mod.rs | 1 + store/postgres/src/parquet/reader.rs | 310 +++++++++++++++++++++++++++ 2 files changed, 311 insertions(+) create mode 100644 store/postgres/src/parquet/reader.rs diff --git a/store/postgres/src/parquet/mod.rs b/store/postgres/src/parquet/mod.rs index 8aedff85d6f..d436709199f 100644 --- a/store/postgres/src/parquet/mod.rs +++ b/store/postgres/src/parquet/mod.rs @@ -1,3 +1,4 @@ pub(crate) mod convert; +pub(crate) mod reader; pub(crate) mod schema; pub(crate) mod writer; diff --git a/store/postgres/src/parquet/reader.rs b/store/postgres/src/parquet/reader.rs new file mode 100644 index 00000000000..7fbdcc6e32c --- /dev/null +++ b/store/postgres/src/parquet/reader.rs @@ -0,0 +1,310 @@ +use std::fs; +use std::path::Path; + +use arrow::array::RecordBatch; +use graph::components::store::StoreError; +use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; + +/// Read all record batches from a Parquet file. +/// +/// Opens the file, reads all row groups, and returns them as a vector +/// of `RecordBatch`es. The batches retain the schema embedded in the +/// Parquet file. +#[allow(dead_code)] +pub fn read_batches(path: &Path) -> Result, StoreError> { + let file = fs::File::open(path).map_err(|e| { + StoreError::InternalError(format!( + "failed to open parquet file {}: {e}", + path.display() + )) + })?; + + let reader = ParquetRecordBatchReaderBuilder::try_new(file) + .map_err(|e| { + StoreError::InternalError(format!( + "failed to create parquet reader for {}: {e}", + path.display() + )) + })? + .build() + .map_err(|e| { + StoreError::InternalError(format!( + "failed to build parquet reader for {}: {e}", + path.display() + )) + })?; + + reader + .into_iter() + .map(|batch| { + batch.map_err(|e| { + StoreError::InternalError(format!( + "failed to read record batch from {}: {e}", + path.display() + )) + }) + }) + .collect() +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use arrow::array::{ + Array, BinaryArray, BooleanArray, Int32Array, Int64Array, ListArray, StringArray, + TimestampMicrosecondArray, + }; + use arrow::datatypes::{DataType, Field, Schema, TimeUnit}; + use graph::data::store::scalar::{BigDecimal, Bytes, Timestamp}; + use std::str::FromStr; + + use super::*; + use crate::parquet::convert::rows_to_record_batch; + use crate::parquet::writer::ParquetChunkWriter; + use crate::relational::value::{OidRow, OidValue}; + + fn make_row(values: Vec) -> OidRow { + values.into_iter().collect() + } + + struct TempFile(std::path::PathBuf); + + impl TempFile { + fn new(name: &str) -> Self { + let path = std::env::temp_dir().join(format!( + "graph_node_reader_test_{}_{name}.parquet", + std::process::id() + )); + Self(path) + } + + fn path(&self) -> &Path { + &self.0 + } + } + + impl Drop for TempFile { + fn drop(&mut self) { + let _ = std::fs::remove_file(&self.0); + } + } + + fn write_and_read(name: &str, schema: &Schema, rows: &[OidRow]) -> Vec { + let tmp = TempFile::new(name); + let batch = rows_to_record_batch(schema, rows).unwrap(); + + let min_vid = if rows.is_empty() { 0 } else { 1 }; + let max_vid = rows.len() as i64; + + let mut writer = ParquetChunkWriter::new( + tmp.path().to_path_buf(), + "test/chunk_000000.parquet".into(), + schema, + ) + .unwrap(); + writer.write_batch(&batch, min_vid, max_vid).unwrap(); + writer.finish().unwrap(); + + read_batches(tmp.path()).unwrap() + } + + #[test] + fn roundtrip_scalar_columns() { + let schema = Schema::new(vec![ + Field::new("vid", DataType::Int64, false), + Field::new("block$", DataType::Int32, false), + Field::new("id", DataType::Utf8, false), + Field::new("flag", DataType::Boolean, true), + Field::new("data", DataType::Binary, true), + Field::new("amount", DataType::Utf8, false), + Field::new("ts", DataType::Timestamp(TimeUnit::Microsecond, None), true), + ]); + + let rows = vec![ + make_row(vec![ + OidValue::Int8(1), + OidValue::Int(100), + OidValue::String("token-1".into()), + OidValue::Bool(true), + OidValue::Bytes(Bytes::from(vec![0xab, 0xcd])), + OidValue::BigDecimal(BigDecimal::from_str("12345678901234567890").unwrap()), + OidValue::Timestamp(Timestamp::from_microseconds_since_epoch(1_000_000).unwrap()), + ]), + make_row(vec![ + OidValue::Int8(2), + OidValue::Int(101), + OidValue::String("token-2".into()), + OidValue::Null, + OidValue::Null, + OidValue::BigDecimal(BigDecimal::from_str("999").unwrap()), + OidValue::Null, + ]), + ]; + + let batches = write_and_read("scalar", &schema, &rows); + assert_eq!(batches.len(), 1); + let batch = &batches[0]; + assert_eq!(batch.num_rows(), 2); + assert_eq!(batch.num_columns(), 7); + assert_eq!(batch.schema().fields().len(), schema.fields().len()); + + // Verify schema matches + for (got, expected) in batch.schema().fields().iter().zip(schema.fields().iter()) { + assert_eq!(got.name(), expected.name()); + assert_eq!(got.data_type(), expected.data_type()); + } + + // Spot-check values + let vid = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(vid.value(0), 1); + assert_eq!(vid.value(1), 2); + + let id = batch + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id.value(0), "token-1"); + assert_eq!(id.value(1), "token-2"); + + let flag = batch + .column(3) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(flag.value(0)); + assert!(flag.is_null(1)); + + let data = batch + .column(4) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(data.value(0), &[0xab, 0xcd]); + assert!(data.is_null(1)); + + let ts = batch + .column(6) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(ts.value(0), 1_000_000); + assert!(ts.is_null(1)); + } + + #[test] + fn roundtrip_list_columns() { + let schema = Schema::new(vec![ + Field::new("vid", DataType::Int64, false), + Field::new( + "tags", + DataType::List(Arc::new(Field::new("item", DataType::Utf8, true))), + false, + ), + Field::new( + "scores", + DataType::List(Arc::new(Field::new("item", DataType::Int32, true))), + true, + ), + ]); + + let rows = vec![ + make_row(vec![ + OidValue::Int8(1), + OidValue::StringArray(vec!["foo".into(), "bar".into()]), + OidValue::Ints(vec![10, 20]), + ]), + make_row(vec![ + OidValue::Int8(2), + OidValue::StringArray(vec![]), + OidValue::Null, + ]), + ]; + + let batches = write_and_read("list", &schema, &rows); + assert_eq!(batches.len(), 1); + let batch = &batches[0]; + assert_eq!(batch.num_rows(), 2); + + let tags = batch + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + let tags_0 = tags.value(0); + let tags_0_str = tags_0.as_any().downcast_ref::().unwrap(); + assert_eq!(tags_0_str.len(), 2); + assert_eq!(tags_0_str.value(0), "foo"); + assert_eq!(tags_0_str.value(1), "bar"); + + let tags_1 = tags.value(1); + let tags_1_str = tags_1.as_any().downcast_ref::().unwrap(); + assert_eq!(tags_1_str.len(), 0); + + let scores = batch + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(!scores.is_null(0)); + assert!(scores.is_null(1)); + } + + #[test] + fn roundtrip_mutable_block_range() { + let schema = Schema::new(vec![ + Field::new("vid", DataType::Int64, false), + Field::new("block_range_start", DataType::Int32, false), + Field::new("block_range_end", DataType::Int32, true), + Field::new("id", DataType::Utf8, false), + ]); + + let rows = vec![ + make_row(vec![ + OidValue::Int8(1), + OidValue::Int(100), + OidValue::Int(200), + OidValue::String("a".into()), + ]), + make_row(vec![ + OidValue::Int8(2), + OidValue::Int(150), + OidValue::Null, // unbounded (current) + OidValue::String("b".into()), + ]), + ]; + + let batches = write_and_read("mutable", &schema, &rows); + let batch = &batches[0]; + assert_eq!(batch.num_rows(), 2); + + let start = batch + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(start.value(0), 100); + assert_eq!(start.value(1), 150); + + let end = batch + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(end.value(0), 200); + assert!(end.is_null(1)); + } + + #[test] + fn nonexistent_file_returns_error() { + let result = read_batches(Path::new("/tmp/nonexistent_graph_node_test.parquet")); + assert!(result.is_err()); + let err = result.unwrap_err().to_string(); + assert!(err.contains("failed to open parquet file")); + } +} From da9be36737a976e2ff0094c50c6a2f520b80390b Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 16:36:49 -0800 Subject: [PATCH 11/20] store: Add Arrow-to-Value conversion for restore Add `RestoreRow`, `DataSourceRestoreRow` structs and conversion functions `record_batch_to_restore_rows()` and `record_batch_to_data_source_rows()` that convert Arrow RecordBatches back into typed data suitable for database insertion. This is step 2 of the restore plan. --- store/postgres/src/parquet/convert.rs | 655 +++++++++++++++++++++++++- 1 file changed, 653 insertions(+), 2 deletions(-) diff --git a/store/postgres/src/parquet/convert.rs b/store/postgres/src/parquet/convert.rs index 49fc7bf53b4..2d229834eae 100644 --- a/store/postgres/src/parquet/convert.rs +++ b/store/postgres/src/parquet/convert.rs @@ -1,14 +1,21 @@ +use std::str::FromStr; use std::sync::Arc; use arrow::array::{ - ArrayRef, BinaryBuilder, BooleanBuilder, Int32Builder, Int64Builder, ListBuilder, RecordBatch, - StringBuilder, TimestampMicrosecondBuilder, + Array, ArrayRef, BinaryArray, BinaryBuilder, BooleanArray, BooleanBuilder, Int32Array, + Int32Builder, Int64Array, Int64Builder, ListArray, ListBuilder, RecordBatch, StringArray, + StringBuilder, TimestampMicrosecondArray, TimestampMicrosecondBuilder, }; use arrow::datatypes::{DataType, Schema}; use graph::components::store::StoreError; +use graph::data::store::scalar; +use graph::data::store::Value; +use graph::data::value::Word; +use graph::prelude::BlockNumber; use crate::relational::value::{OidRow, OidValue}; +use crate::relational::{ColumnType, Table}; /// Convert a batch of `DynamicRow` rows into an Arrow `RecordBatch`. /// @@ -230,18 +237,341 @@ impl ColumnBuilder { } } +/// A row extracted from a Parquet file, ready for insertion into an entity table. +#[allow(dead_code)] +pub struct RestoreRow { + pub vid: i64, + pub block: BlockNumber, + /// `None` for immutable entities, `Some` for mutable (block_range_end). + pub block_range_end: Option>, + pub causality_region: Option, + pub values: Vec<(Word, Value)>, +} + +/// Convert an Arrow `RecordBatch` back into `RestoreRow`s using the entity +/// table's schema to guide type interpretation. +/// +/// The batch must follow the column layout produced by `schema::arrow_schema`: +/// `vid`, block tracking columns, optional `causality_region`, then data columns. +#[allow(dead_code)] +pub fn record_batch_to_restore_rows( + batch: &RecordBatch, + table: &Table, +) -> Result, StoreError> { + let num_rows = batch.num_rows(); + let mut rows = Vec::with_capacity(num_rows); + + // System column indices follow the layout in schema::arrow_schema + let vid_array = downcast_i64(batch.column(0), "vid")?; + + let mut col_idx = 1; + + // Block tracking + let block_array = downcast_i32(batch.column(col_idx), "block")?; + col_idx += 1; + + let block_end_array = if !table.immutable { + let arr = downcast_i32(batch.column(col_idx), "block_range_end")?; + col_idx += 1; + Some(arr) + } else { + None + }; + + // Causality region + let cr_array = if table.has_causality_region { + let arr = downcast_i32(batch.column(col_idx), "causality_region")?; + col_idx += 1; + Some(arr) + } else { + None + }; + + // Data columns: iterate table columns, skip fulltext + let data_columns: Vec<_> = table.columns.iter().filter(|c| !c.is_fulltext()).collect(); + + // Pre-resolve Arrow column references for data columns + let arrow_data_cols: Vec<_> = data_columns + .iter() + .enumerate() + .map(|(i, _)| batch.column(col_idx + i)) + .collect(); + + for row in 0..num_rows { + let vid = vid_array.value(row); + let block = block_array.value(row); + + let block_range_end = block_end_array.as_ref().map(|arr| { + if arr.is_null(row) { + None + } else { + Some(arr.value(row)) + } + }); + + let causality_region = cr_array.as_ref().map(|arr| arr.value(row)); + + let mut values = Vec::with_capacity(data_columns.len()); + for (i, col) in data_columns.iter().enumerate() { + let arrow_col = arrow_data_cols[i].as_ref(); + let value = if arrow_col.is_null(row) { + Value::Null + } else if col.is_list() { + extract_list_value(arrow_col, row, &col.column_type)? + } else { + extract_scalar_value(arrow_col, row, &col.column_type)? + }; + values.push((Word::from(col.name.as_str()), value)); + } + + rows.push(RestoreRow { + vid, + block, + block_range_end, + causality_region, + values, + }); + } + + Ok(rows) +} + +/// A row extracted from a `data_sources$` Parquet file. +#[allow(dead_code)] +pub struct DataSourceRestoreRow { + pub vid: i64, + pub block_range_start: BlockNumber, + pub block_range_end: Option, + pub causality_region: i32, + pub manifest_idx: i32, + pub parent: Option, + pub id: Option>, + pub param: Option>, + pub context: Option, + pub done_at: Option, +} + +/// Convert an Arrow `RecordBatch` from a `data_sources$` Parquet file +/// into `DataSourceRestoreRow`s. +/// +/// The batch must follow the fixed column layout from +/// `schema::data_sources_arrow_schema`. +#[allow(dead_code)] +pub fn record_batch_to_data_source_rows( + batch: &RecordBatch, +) -> Result, StoreError> { + let num_rows = batch.num_rows(); + let mut rows = Vec::with_capacity(num_rows); + + let vid = downcast_i64(batch.column(0), "vid")?; + let block_start = downcast_i32(batch.column(1), "block_range_start")?; + let block_end = downcast_i32(batch.column(2), "block_range_end")?; + let cr = downcast_i32(batch.column(3), "causality_region")?; + let manifest_idx = downcast_i32(batch.column(4), "manifest_idx")?; + let parent = downcast_i32(batch.column(5), "parent")?; + let id_arr = downcast_binary(batch.column(6), "id")?; + let param_arr = downcast_binary(batch.column(7), "param")?; + let context_arr = downcast_utf8(batch.column(8), "context")?; + let done_at = downcast_i32(batch.column(9), "done_at")?; + + for row in 0..num_rows { + rows.push(DataSourceRestoreRow { + vid: vid.value(row), + block_range_start: block_start.value(row), + block_range_end: if block_end.is_null(row) { + None + } else { + Some(block_end.value(row)) + }, + causality_region: cr.value(row), + manifest_idx: manifest_idx.value(row), + parent: if parent.is_null(row) { + None + } else { + Some(parent.value(row)) + }, + id: if id_arr.is_null(row) { + None + } else { + Some(id_arr.value(row).to_vec()) + }, + param: if param_arr.is_null(row) { + None + } else { + Some(param_arr.value(row).to_vec()) + }, + context: if context_arr.is_null(row) { + None + } else { + Some(context_arr.value(row).to_string()) + }, + done_at: if done_at.is_null(row) { + None + } else { + Some(done_at.value(row)) + }, + }); + } + + Ok(rows) +} + +// -- Downcasting helpers -- + +fn downcast_i64<'a>(array: &'a ArrayRef, name: &str) -> Result<&'a Int64Array, StoreError> { + array.as_any().downcast_ref::().ok_or_else(|| { + StoreError::InternalError(format!("expected Int64Array for column '{name}'")) + }) +} + +fn downcast_i32<'a>(array: &'a ArrayRef, name: &str) -> Result<&'a Int32Array, StoreError> { + array.as_any().downcast_ref::().ok_or_else(|| { + StoreError::InternalError(format!("expected Int32Array for column '{name}'")) + }) +} + +fn downcast_binary<'a>(array: &'a ArrayRef, name: &str) -> Result<&'a BinaryArray, StoreError> { + array.as_any().downcast_ref::().ok_or_else(|| { + StoreError::InternalError(format!("expected BinaryArray for column '{name}'")) + }) +} + +fn downcast_utf8<'a>(array: &'a ArrayRef, name: &str) -> Result<&'a StringArray, StoreError> { + array.as_any().downcast_ref::().ok_or_else(|| { + StoreError::InternalError(format!("expected StringArray for column '{name}'")) + }) +} + +/// Extract a scalar `Value` from an Arrow array at the given row. +/// +/// The `column_type` determines how to interpret the Arrow data (e.g. +/// a Utf8 array might be String, BigInt, or BigDecimal). +fn extract_scalar_value( + array: &dyn Array, + row: usize, + column_type: &ColumnType, +) -> Result { + match column_type { + ColumnType::Boolean => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| err("BooleanArray", "Boolean"))?; + Ok(Value::Bool(arr.value(row))) + } + ColumnType::Int => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| err("Int32Array", "Int"))?; + Ok(Value::Int(arr.value(row))) + } + ColumnType::Int8 => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| err("Int64Array", "Int8"))?; + Ok(Value::Int8(arr.value(row))) + } + ColumnType::Bytes => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| err("BinaryArray", "Bytes"))?; + Ok(Value::Bytes(scalar::Bytes::from(arr.value(row)))) + } + ColumnType::BigInt => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| err("StringArray", "BigInt"))?; + let s = arr.value(row); + let big = scalar::BigInt::from_str(s) + .map_err(|e| StoreError::InternalError(format!("invalid BigInt '{s}': {e}")))?; + Ok(Value::BigInt(big)) + } + ColumnType::BigDecimal => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| err("StringArray", "BigDecimal"))?; + let s = arr.value(row); + let bd = scalar::BigDecimal::from_str(s) + .map_err(|e| StoreError::InternalError(format!("invalid BigDecimal '{s}': {e}")))?; + Ok(Value::BigDecimal(bd)) + } + ColumnType::Timestamp => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| err("TimestampMicrosecondArray", "Timestamp"))?; + let micros = arr.value(row); + let ts = scalar::Timestamp::from_microseconds_since_epoch(micros).map_err(|e| { + StoreError::InternalError(format!("invalid Timestamp {micros}: {e}")) + })?; + Ok(Value::Timestamp(ts)) + } + ColumnType::String | ColumnType::Enum(_) => { + let arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| err("StringArray", "String/Enum"))?; + Ok(Value::String(arr.value(row).to_string())) + } + ColumnType::TSVector(_) => Err(StoreError::InternalError( + "TSVector columns should not appear in parquet data".into(), + )), + } +} + +/// Extract a list `Value` from a `ListArray` at the given row. +fn extract_list_value( + array: &dyn Array, + row: usize, + column_type: &ColumnType, +) -> Result { + let list_arr = array + .as_any() + .downcast_ref::() + .ok_or_else(|| StoreError::InternalError("expected ListArray for list column".into()))?; + + let inner = list_arr.value(row); + let len = inner.len(); + let mut items = Vec::with_capacity(len); + + for i in 0..len { + if inner.is_null(i) { + items.push(Value::Null); + } else { + items.push(extract_scalar_value(inner.as_ref(), i, column_type)?); + } + } + + Ok(Value::List(items)) +} + +fn err(expected: &str, column_type: &str) -> StoreError { + StoreError::InternalError(format!("expected {expected} for {column_type} column")) +} + #[cfg(test)] mod tests { + use std::collections::BTreeSet; + use arrow::array::{ Array, BinaryArray, BooleanArray, Int32Array, Int64Array, ListArray, StringArray, TimestampMicrosecondArray, }; use arrow::datatypes::{Field, Schema, TimeUnit}; use graph::data::store::scalar::{BigDecimal, Bytes, Timestamp}; + use graph::prelude::DeploymentHash; + use graph::schema::InputSchema; use std::str::FromStr; use std::sync::Arc; use super::*; + use crate::layout_for_tests::{make_dummy_site, Catalog, Layout, Namespace}; + use crate::parquet::schema::arrow_schema; fn make_row(values: Vec) -> OidRow { values.into_iter().collect() @@ -487,4 +817,325 @@ mod tests { let err = rows_to_record_batch(&schema, &rows).unwrap_err(); assert!(err.to_string().contains("type mismatch")); } + + // -- Restore direction tests -- + + fn test_layout(gql: &str) -> Layout { + let subgraph = DeploymentHash::new("subgraph").unwrap(); + let schema = InputSchema::parse_latest(gql, subgraph.clone()).expect("Test schema invalid"); + let namespace = Namespace::new("sgd0815".to_owned()).unwrap(); + let site = Arc::new(make_dummy_site(subgraph, namespace, "anet".to_string())); + let catalog = + Catalog::for_tests(site.clone(), BTreeSet::new()).expect("Can not create catalog"); + Layout::new(site, &schema, catalog).expect("Failed to construct Layout") + } + + fn test_layout_with_causality(gql: &str, entity_name: &str) -> Layout { + let subgraph = DeploymentHash::new("subgraph").unwrap(); + let schema = InputSchema::parse_latest(gql, subgraph.clone()).expect("Test schema invalid"); + let namespace = Namespace::new("sgd0815".to_owned()).unwrap(); + let site = Arc::new(make_dummy_site(subgraph, namespace, "anet".to_string())); + let entity_type = schema.entity_type(entity_name).unwrap(); + let ents = BTreeSet::from_iter(vec![entity_type]); + let catalog = Catalog::for_tests(site.clone(), ents).expect("Can not create catalog"); + Layout::new(site, &schema, catalog).expect("Failed to construct Layout") + } + + #[test] + fn restore_immutable_entity() { + let layout = test_layout( + "type Token @entity(immutable: true) { id: ID!, name: String!, decimals: Int! }", + ); + let table = layout.table("token").unwrap(); + let schema = arrow_schema(table); + + // Build a RecordBatch via the dump direction + let rows = vec![ + make_row(vec![ + OidValue::Int8(1), + OidValue::Int(100), + OidValue::String("token-1".into()), + OidValue::String("Token One".into()), + OidValue::Int(18), + ]), + make_row(vec![ + OidValue::Int8(2), + OidValue::Int(101), + OidValue::String("token-2".into()), + OidValue::String("Token Two".into()), + OidValue::Int(6), + ]), + ]; + + let batch = rows_to_record_batch(&schema, &rows).unwrap(); + let restore_rows = record_batch_to_restore_rows(&batch, table).unwrap(); + + assert_eq!(restore_rows.len(), 2); + + // Row 0 + assert_eq!(restore_rows[0].vid, 1); + assert_eq!(restore_rows[0].block, 100); + assert!(restore_rows[0].block_range_end.is_none()); // immutable + assert!(restore_rows[0].causality_region.is_none()); + + assert_eq!(restore_rows[0].values.len(), 3); // id, name, decimals + assert_eq!(restore_rows[0].values[0].0.as_str(), "id"); + assert_eq!(restore_rows[0].values[0].1, Value::String("token-1".into())); + assert_eq!(restore_rows[0].values[1].0.as_str(), "name"); + assert_eq!( + restore_rows[0].values[1].1, + Value::String("Token One".into()) + ); + assert_eq!(restore_rows[0].values[2].0.as_str(), "decimals"); + assert_eq!(restore_rows[0].values[2].1, Value::Int(18)); + + // Row 1 + assert_eq!(restore_rows[1].vid, 2); + assert_eq!(restore_rows[1].block, 101); + assert_eq!(restore_rows[1].values[2].1, Value::Int(6)); + } + + #[test] + fn restore_mutable_entity_with_causality() { + let layout = test_layout_with_causality( + "type Transfer @entity { id: ID!, amount: BigInt! }", + "Transfer", + ); + let table = layout.table("transfer").unwrap(); + let schema = arrow_schema(table); + + let rows = vec![ + make_row(vec![ + OidValue::Int8(10), + OidValue::Int(500), + OidValue::Int(600), + OidValue::Int(0), + OidValue::String("tx-1".into()), + OidValue::String("12345678901234567890".into()), + ]), + make_row(vec![ + OidValue::Int8(11), + OidValue::Int(550), + OidValue::Null, // unbounded (current) + OidValue::Int(1), + OidValue::String("tx-2".into()), + OidValue::String("99999".into()), + ]), + ]; + + let batch = rows_to_record_batch(&schema, &rows).unwrap(); + let restore_rows = record_batch_to_restore_rows(&batch, table).unwrap(); + + assert_eq!(restore_rows.len(), 2); + + // Row 0: closed block range + assert_eq!(restore_rows[0].vid, 10); + assert_eq!(restore_rows[0].block, 500); + assert_eq!(restore_rows[0].block_range_end, Some(Some(600))); + assert_eq!(restore_rows[0].causality_region, Some(0)); + assert_eq!(restore_rows[0].values[0].0.as_str(), "id"); + assert_eq!(restore_rows[0].values[0].1, Value::String("tx-1".into())); + assert_eq!(restore_rows[0].values[1].0.as_str(), "amount"); + assert_eq!( + restore_rows[0].values[1].1, + Value::BigInt(scalar::BigInt::from_str("12345678901234567890").unwrap()) + ); + + // Row 1: open block range (current) + assert_eq!(restore_rows[1].vid, 11); + assert_eq!(restore_rows[1].block, 550); + assert_eq!(restore_rows[1].block_range_end, Some(None)); // unbounded + assert_eq!(restore_rows[1].causality_region, Some(1)); + } + + #[test] + fn restore_nullable_and_list_columns() { + let layout = test_layout( + "type Pool @entity(immutable: true) { id: ID!, tags: [String!]!, description: String }", + ); + let table = layout.table("pool").unwrap(); + let schema = arrow_schema(table); + + let rows = vec![ + make_row(vec![ + OidValue::Int8(1), + OidValue::Int(100), + OidValue::String("pool-1".into()), + OidValue::StringArray(vec!["defi".into(), "amm".into()]), + OidValue::String("A pool".into()), + ]), + make_row(vec![ + OidValue::Int8(2), + OidValue::Int(101), + OidValue::String("pool-2".into()), + OidValue::StringArray(vec![]), + OidValue::Null, + ]), + ]; + + let batch = rows_to_record_batch(&schema, &rows).unwrap(); + let restore_rows = record_batch_to_restore_rows(&batch, table).unwrap(); + + assert_eq!(restore_rows.len(), 2); + + // Row 0: tags is a list of strings + let tags_val = &restore_rows[0].values[1].1; + match tags_val { + Value::List(items) => { + assert_eq!(items.len(), 2); + assert_eq!(items[0], Value::String("defi".into())); + assert_eq!(items[1], Value::String("amm".into())); + } + other => panic!("expected List, got {:?}", other), + } + assert_eq!(restore_rows[0].values[2].1, Value::String("A pool".into())); + + // Row 1: empty list and null description + let tags_val = &restore_rows[1].values[1].1; + match tags_val { + Value::List(items) => assert_eq!(items.len(), 0), + other => panic!("expected List, got {:?}", other), + } + assert_eq!(restore_rows[1].values[2].1, Value::Null); + } + + #[test] + fn restore_all_scalar_types() { + let layout = test_layout( + "type Everything @entity(immutable: true) { \ + id: ID!, \ + flag: Boolean!, \ + small: Int!, \ + big: Int8!, \ + amount: BigInt!, \ + price: BigDecimal!, \ + data: Bytes!, \ + ts: Timestamp!, \ + label: String! \ + }", + ); + let table = layout.table("everything").unwrap(); + let schema = arrow_schema(table); + + let rows = vec![make_row(vec![ + OidValue::Int8(1), + OidValue::Int(100), + OidValue::String("thing-1".into()), + OidValue::Bool(true), + OidValue::Int(42), + OidValue::Int8(9_999_999), + OidValue::String("12345678901234567890".into()), + OidValue::BigDecimal(BigDecimal::from_str("3.14159").unwrap()), + OidValue::Bytes(Bytes::from(vec![0xab, 0xcd])), + OidValue::Timestamp(Timestamp::from_microseconds_since_epoch(1_000_000).unwrap()), + OidValue::String("hello".into()), + ])]; + + let batch = rows_to_record_batch(&schema, &rows).unwrap(); + let restore_rows = record_batch_to_restore_rows(&batch, table).unwrap(); + + assert_eq!(restore_rows.len(), 1); + let vals = &restore_rows[0].values; + + // id + assert_eq!(vals[0].1, Value::String("thing-1".into())); + // flag + assert_eq!(vals[1].1, Value::Bool(true)); + // small + assert_eq!(vals[2].1, Value::Int(42)); + // big + assert_eq!(vals[3].1, Value::Int8(9_999_999)); + // amount (BigInt) + assert_eq!( + vals[4].1, + Value::BigInt(scalar::BigInt::from_str("12345678901234567890").unwrap()) + ); + // price (BigDecimal) + assert_eq!( + vals[5].1, + Value::BigDecimal(BigDecimal::from_str("3.14159").unwrap()) + ); + // data (Bytes) + assert_eq!(vals[6].1, Value::Bytes(Bytes::from(vec![0xab, 0xcd]))); + // ts (Timestamp) + assert_eq!( + vals[7].1, + Value::Timestamp(Timestamp::from_microseconds_since_epoch(1_000_000).unwrap()) + ); + // label + assert_eq!(vals[8].1, Value::String("hello".into())); + } + + #[test] + fn restore_data_source_rows() { + use crate::parquet::schema::data_sources_arrow_schema; + + let schema = data_sources_arrow_schema(); + let rows = vec![ + make_row(vec![ + OidValue::Int8(1), + OidValue::Int(100), + OidValue::Int(200), + OidValue::Int(1), + OidValue::Int(0), + OidValue::Null, + OidValue::Null, + OidValue::Bytes(Bytes::from(vec![0x01, 0x02])), + OidValue::String(r#"{"key":"value"}"#.into()), + OidValue::Null, + ]), + make_row(vec![ + OidValue::Int8(2), + OidValue::Int(150), + OidValue::Null, // unbounded + OidValue::Int(2), + OidValue::Int(1), + OidValue::Int(5), + OidValue::Bytes(Bytes::from(vec![0xaa])), + OidValue::Null, + OidValue::Null, + OidValue::Int(300), + ]), + ]; + + let batch = rows_to_record_batch(&schema, &rows).unwrap(); + let ds_rows = record_batch_to_data_source_rows(&batch).unwrap(); + + assert_eq!(ds_rows.len(), 2); + + // Row 0 + assert_eq!(ds_rows[0].vid, 1); + assert_eq!(ds_rows[0].block_range_start, 100); + assert_eq!(ds_rows[0].block_range_end, Some(200)); + assert_eq!(ds_rows[0].causality_region, 1); + assert_eq!(ds_rows[0].manifest_idx, 0); + assert!(ds_rows[0].parent.is_none()); + assert!(ds_rows[0].id.is_none()); + assert_eq!(ds_rows[0].param.as_deref(), Some(&[0x01, 0x02][..])); + assert_eq!(ds_rows[0].context.as_deref(), Some(r#"{"key":"value"}"#)); + assert!(ds_rows[0].done_at.is_none()); + + // Row 1 + assert_eq!(ds_rows[1].vid, 2); + assert_eq!(ds_rows[1].block_range_start, 150); + assert!(ds_rows[1].block_range_end.is_none()); + assert_eq!(ds_rows[1].causality_region, 2); + assert_eq!(ds_rows[1].manifest_idx, 1); + assert_eq!(ds_rows[1].parent, Some(5)); + assert_eq!(ds_rows[1].id.as_deref(), Some(&[0xaa][..])); + assert!(ds_rows[1].param.is_none()); + assert!(ds_rows[1].context.is_none()); + assert_eq!(ds_rows[1].done_at, Some(300)); + } + + #[test] + fn restore_empty_batch() { + let layout = test_layout("type Token @entity(immutable: true) { id: ID!, name: String! }"); + let table = layout.table("token").unwrap(); + let schema = arrow_schema(table); + + let batch = rows_to_record_batch(&schema, &[]).unwrap(); + let restore_rows = record_batch_to_restore_rows(&batch, table).unwrap(); + assert!(restore_rows.is_empty()); + } } From 2d6d4c8f5962e598f1dc46e730b55fe6b9fc81c0 Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 16:53:52 -0800 Subject: [PATCH 12/20] store: Add InsertQuery::for_restore() for Parquet restore Add a new constructor to InsertQuery that accepts RestoreRow data from Parquet files, bypassing the WriteChunk/EntityWrite pipeline. This is the insertion foundation for the restore path. Also add From impl for CausalityRegion to allow constructing values from deserialized Parquet data. --- graph/src/data_source/causality_region.rs | 6 ++ store/postgres/src/relational_queries.rs | 93 +++++++++++++++++++++++ 2 files changed, 99 insertions(+) diff --git a/graph/src/data_source/causality_region.rs b/graph/src/data_source/causality_region.rs index 489247c1b9b..5755657f157 100644 --- a/graph/src/data_source/causality_region.rs +++ b/graph/src/data_source/causality_region.rs @@ -45,6 +45,12 @@ impl ToSql for CausalityRegion { } } +impl From for CausalityRegion { + fn from(value: i32) -> Self { + CausalityRegion(value) + } +} + impl CausalityRegion { /// The causality region of all onchain data sources. pub const ONCHAIN: CausalityRegion = CausalityRegion(0); diff --git a/store/postgres/src/relational_queries.rs b/store/postgres/src/relational_queries.rs index 56ceea2e27e..11e1eed3080 100644 --- a/store/postgres/src/relational_queries.rs +++ b/store/postgres/src/relational_queries.rs @@ -41,6 +41,7 @@ use std::str::FromStr; use std::string::ToString; use crate::block_range::{BoundSide, EntityBlockRange}; +use crate::parquet::convert::RestoreRow; use crate::relational::dsl::AtBlock; use crate::relational::{ dsl, rollup::Rollup, Column, ColumnType, Layout, SqlName, Table, BYTE_ARRAY_PREFIX_SIZE, @@ -2348,6 +2349,74 @@ impl<'a> InsertRow<'a> { vid, }) } + + /// Build an `InsertRow` from a `RestoreRow` (Parquet restore path). + /// + /// Unlike `new()`, this looks up values by SQL column name rather than + /// entity field name, since `RestoreRow.values` is keyed by SQL name. + /// Fulltext columns are regenerated from their source fields. + #[allow(dead_code)] + fn from_restore( + columns: &[&'a Column], + row: &'a RestoreRow, + table: &'a Table, + ) -> Result { + let mut values = Vec::with_capacity(columns.len()); + for column in columns { + let iv = if let Some(fields) = column.fulltext_fields.as_ref() { + // Fulltext columns: `fields` contains GraphQL field names, + // but `RestoreRow.values` is keyed by SQL column names. + // Resolve via the table's columns. + let fulltext_field_values: Vec<_> = fields + .iter() + .filter_map(|field_name| { + table + .columns + .iter() + .find(|c| c.field.as_str() == field_name.as_str()) + .and_then(|src_col| { + row.values + .iter() + .find(|(w, _)| w.as_str() == src_col.name.as_str()) + .map(|(_, v)| v) + }) + }) + .map(|value| match value { + Value::String(s) => Ok(s), + _ => Err(internal_error!( + "fulltext fields must be strings but got {:?}", + value + )), + }) + .collect::>()?; + if let ColumnType::TSVector(config) = &column.column_type { + InsertValue::Fulltext(fulltext_field_values, config) + } else { + return Err(StoreError::FulltextColumnMissingConfig); + } + } else { + let value = row + .values + .iter() + .find(|(w, _)| w.as_str() == column.name.as_str()) + .map(|(_, v)| v) + .unwrap_or(&NULL); + let qv = QueryValue::new(value, &column.column_type)?; + InsertValue::Value(qv) + }; + values.push(iv); + } + let end = row.block_range_end.flatten(); + let br_value = BlockRangeValue::new(table, row.block, end); + let causality_region = CausalityRegion::from(row.causality_region.unwrap_or(0)); + let vid = row.vid; + Ok(Self { + values, + br_value, + causality_region, + vid, + }) + } } #[derive(Debug)] @@ -2386,6 +2455,30 @@ impl<'a> InsertQuery<'a> { }) } + /// Build an `InsertQuery` from restore rows (Parquet restore path). + /// + /// All data columns from the dump are present, so `unique_columns` + /// includes every column in the table. Fulltext columns are + /// regenerated from their source fields. + #[allow(dead_code)] + pub fn for_restore( + table: &'a Table, + rows: &'a [RestoreRow], + ) -> Result, StoreError> { + let unique_columns: Vec<&Column> = table.columns.iter().collect(); + + let rows: Vec<_> = rows + .iter() + .map(|row| InsertRow::from_restore(&unique_columns, row, table)) + .collect::>()?; + + Ok(InsertQuery { + table, + rows, + unique_columns, + }) + } + /// Build the column name list using the subset of all keys among present entities. fn unique_columns(table: &'a Table, rows: &'a WriteChunk<'a>) -> Vec<&'a Column> { table From d7a29a9c6f8a30a2f8025f1cdbede70a73c409dc Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 17:00:07 -0800 Subject: [PATCH 13/20] store: Add Metadata::from_file() for dump deserialization Add a method to read and validate metadata.json from a dump directory. Make fields of Metadata, Manifest, BlockPtr, Health, and Error structs pub(crate) so they are accessible from the restore module. --- store/postgres/src/relational/dump.rs | 84 ++++++++++++++++----------- 1 file changed, 51 insertions(+), 33 deletions(-) diff --git a/store/postgres/src/relational/dump.rs b/store/postgres/src/relational/dump.rs index f01c11b86d3..c568f2d0011 100644 --- a/store/postgres/src/relational/dump.rs +++ b/store/postgres/src/relational/dump.rs @@ -30,13 +30,13 @@ use crate::AsyncPgConnection; use super::Layout; #[derive(Serialize, Deserialize)] -struct Manifest { - spec_version: String, - description: Option, - repository: Option, - features: Vec, - entities_with_causality_region: Vec, - history_blocks: i32, +pub(crate) struct Manifest { + pub spec_version: String, + pub description: Option, + pub repository: Option, + pub features: Vec, + pub entities_with_causality_region: Vec, + pub history_blocks: i32, } impl Manifest { @@ -67,9 +67,9 @@ impl Manifest { } #[derive(Serialize, Deserialize)] -struct BlockPtr { - number: i32, - hash: String, +pub(crate) struct BlockPtr { + pub number: i32, + pub hash: String, } impl BlockPtr { @@ -81,11 +81,11 @@ impl BlockPtr { } #[derive(Serialize, Deserialize)] -struct Error { - message: String, - block_ptr: Option, - handler: Option, - deterministic: bool, +pub(crate) struct Error { + pub message: String, + pub block_ptr: Option, + pub handler: Option, + pub deterministic: bool, } impl Error { @@ -108,11 +108,11 @@ impl Error { } #[derive(Serialize, Deserialize)] -struct Health { - failed: bool, - health: String, - fatal_error: Option, - non_fatal_errors: Vec, +pub(crate) struct Health { + pub failed: bool, + pub health: String, + pub fatal_error: Option, + pub non_fatal_errors: Vec, } impl Health { @@ -149,23 +149,41 @@ pub(crate) struct TableInfo { /// as entity tables are written to Parquet files. #[derive(Serialize, Deserialize)] pub(crate) struct Metadata { - version: u32, - deployment: DeploymentHash, - network: String, - manifest: Manifest, - earliest_block_number: i32, - start_block: Option, - head_block: Option, - entity_count: usize, - graft_base: Option, - graft_block: Option, - debug_fork: Option, - health: Health, - indexes: HashMap>, + pub version: u32, + pub deployment: DeploymentHash, + pub network: String, + pub manifest: Manifest, + pub earliest_block_number: i32, + pub start_block: Option, + pub head_block: Option, + pub entity_count: usize, + pub graft_base: Option, + pub graft_block: Option, + pub debug_fork: Option, + pub health: Health, + pub indexes: HashMap>, pub tables: BTreeMap, } impl Metadata { + /// Read and validate a dump's `metadata.json`. + #[allow(dead_code)] + pub fn from_file(path: &Path) -> Result { + let content = fs::read_to_string(path).map_err(|e| { + StoreError::InternalError(format!("failed to read {}: {e}", path.display())) + })?; + let metadata: Self = serde_json::from_str(&content).map_err(|e| { + StoreError::InternalError(format!("failed to parse {}: {e}", path.display())) + })?; + if metadata.version != 1 { + return Err(StoreError::InternalError(format!( + "unsupported dump version {} (expected 1)", + metadata.version + ))); + } + Ok(metadata) + } + fn new( deployment: DeploymentHash, network: String, From 7e7452fd38de4b745f41b8afbdcbcabae02fd7c9 Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 17:13:26 -0800 Subject: [PATCH 14/20] store: Add core restore module for schema & deployment creation --- store/postgres/src/relational.rs | 1 + store/postgres/src/relational/restore.rs | 142 +++++++++++++++++++++++ 2 files changed, 143 insertions(+) create mode 100644 store/postgres/src/relational/restore.rs diff --git a/store/postgres/src/relational.rs b/store/postgres/src/relational.rs index ba7d0d46157..efdfe6a5d8d 100644 --- a/store/postgres/src/relational.rs +++ b/store/postgres/src/relational.rs @@ -18,6 +18,7 @@ pub(crate) mod dsl; pub(crate) mod dump; pub(crate) mod index; pub(crate) mod prune; +pub(crate) mod restore; pub(crate) mod rollup; pub(crate) mod value; diff --git a/store/postgres/src/relational/restore.rs b/store/postgres/src/relational/restore.rs new file mode 100644 index 00000000000..4a0cd50ac90 --- /dev/null +++ b/store/postgres/src/relational/restore.rs @@ -0,0 +1,142 @@ +//! Restore a subgraph deployment from a dump directory. +//! +//! The dump directory must contain: +//! - `metadata.json` — deployment metadata and per-table state +//! - `schema.graphql` — raw GraphQL schema text +//! - Per-entity Parquet files in subdirectories + +use std::collections::BTreeSet; +use std::fs; +use std::path::Path; +use std::sync::Arc; + +use diesel_async::SimpleAsyncConnection; + +use graph::blockchain::BlockHash; +use graph::data::subgraph::schema::{DeploymentCreate, SubgraphManifestEntity}; +use graph::prelude::{BlockPtr as GraphBlockPtr, StoreError}; +use graph::schema::{EntityType, InputSchema}; +use graph::semver::Version; + +use crate::deployment::create_deployment; +use crate::dynds::DataSourcesTable; +use crate::primary::Site; +use crate::relational::dump::Metadata; +use crate::relational::Layout; +use crate::AsyncPgConnection; + +/// Convert a dump `BlockPtr` (hex hash string) to a graph `BlockPtr`. +fn to_graph_block_ptr(bp: &super::dump::BlockPtr) -> Result { + let hash = BlockHash::try_from(bp.hash.as_str()) + .map_err(|e| StoreError::InternalError(format!("invalid block hash '{}': {e}", bp.hash)))?; + Ok(GraphBlockPtr { + number: bp.number, + hash, + }) +} + +/// Restore a subgraph deployment's schema and metadata from a dump +/// directory. +/// +/// This creates the deployment metadata rows (`subgraphs.head`, +/// `subgraphs.deployment`, `subgraphs.subgraph_manifest`), entity +/// tables, and optionally the `data_sources$` table. +/// +/// The caller is responsible for: +/// - Reading `metadata.json` via `Metadata::from_file()` +/// - Site allocation and conflict resolution (force-drop) +/// - Obtaining the right shard connection +/// +/// Entity data import and finalization are handled separately. +#[allow(dead_code)] +pub async fn create_schema( + conn: &mut AsyncPgConnection, + site: Arc, + metadata: &Metadata, + dir: &Path, +) -> Result { + // 1. Read schema.graphql + let schema_path = dir.join("schema.graphql"); + let schema_text = fs::read_to_string(&schema_path).map_err(|e| { + StoreError::InternalError(format!("failed to read {}: {e}", schema_path.display())) + })?; + + // 2. Read subgraph.yaml (optional) + let yaml_path = dir.join("subgraph.yaml"); + let raw_yaml = fs::read_to_string(&yaml_path).ok(); + + // 3. Parse schema + let spec_version = Version::parse(&metadata.manifest.spec_version).map_err(|e| { + StoreError::InternalError(format!( + "invalid spec_version '{}': {e}", + metadata.manifest.spec_version + )) + })?; + let input_schema = InputSchema::parse(&spec_version, &schema_text, site.deployment.clone())?; + + // 4. Resolve entities_with_causality_region from names + let entities_with_causality_region: BTreeSet = metadata + .manifest + .entities_with_causality_region + .iter() + .map(|name| input_schema.entity_type(name)) + .collect::>() + .map_err(StoreError::from)?; + + // 5. Build SubgraphManifestEntity for create_deployment + let manifest_entity = SubgraphManifestEntity { + spec_version: metadata.manifest.spec_version.clone(), + description: metadata.manifest.description.clone(), + repository: metadata.manifest.repository.clone(), + features: metadata.manifest.features.clone(), + schema: schema_text, + raw_yaml, + entities_with_causality_region: entities_with_causality_region.iter().cloned().collect(), + history_blocks: metadata.manifest.history_blocks, + }; + + let start_block = metadata + .start_block + .as_ref() + .map(to_graph_block_ptr) + .transpose()?; + let graft_block = metadata + .graft_block + .as_ref() + .map(to_graph_block_ptr) + .transpose()?; + + let create = DeploymentCreate { + manifest: manifest_entity, + start_block, + graft_base: metadata.graft_base.clone(), + graft_block, + debug_fork: metadata.debug_fork.clone(), + history_blocks_override: None, + }; + + // 6. Create deployment metadata rows + create_deployment(conn, &site, create, false, false).await?; + + // 7. Create database schema and entity tables + let query = format!("create schema {}", &site.namespace); + conn.batch_execute(&query).await?; + + let layout = Layout::create_relational_schema( + conn, + site.clone(), + &input_schema, + entities_with_causality_region, + None, + ) + .await?; + + // 8. Create data_sources$ table if present in dump + if metadata.tables.contains_key("data_sources$") { + let ds_table = DataSourcesTable::new(site.namespace.clone()); + let ddl = ds_table.as_ddl(); + conn.batch_execute(&ddl).await?; + } + + Ok(layout) +} From 748cfb1f8654b5df8f0aaae7685b92d069e772bc Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 17:19:35 -0800 Subject: [PATCH 15/20] store: Add resumable entity data import for restore Add import_data() to the restore module that reads Parquet chunks and inserts entity data into PostgreSQL tables. Supports resumability by checking max(vid) in each table and skipping already-imported rows. Entity tables use InsertQuery::for_restore() for efficient batch inserts. The data_sources$ table uses raw SQL with bind parameters since it has a fixed schema outside the Layout. --- store/postgres/src/relational/restore.rs | 227 ++++++++++++++++++++++- 1 file changed, 221 insertions(+), 6 deletions(-) diff --git a/store/postgres/src/relational/restore.rs b/store/postgres/src/relational/restore.rs index 4a0cd50ac90..e2e4d2131db 100644 --- a/store/postgres/src/relational/restore.rs +++ b/store/postgres/src/relational/restore.rs @@ -10,21 +10,29 @@ use std::fs; use std::path::Path; use std::sync::Arc; -use diesel_async::SimpleAsyncConnection; - +use diesel::sql_types::{BigInt, Binary, Integer, Nullable, Text}; +use diesel_async::{RunQueryDsl, SimpleAsyncConnection}; use graph::blockchain::BlockHash; use graph::data::subgraph::schema::{DeploymentCreate, SubgraphManifestEntity}; -use graph::prelude::{BlockPtr as GraphBlockPtr, StoreError}; +use graph::prelude::{info, BlockPtr as GraphBlockPtr, Logger, StoreError}; use graph::schema::{EntityType, InputSchema}; use graph::semver::Version; use crate::deployment::create_deployment; use crate::dynds::DataSourcesTable; +use crate::parquet::convert::{ + record_batch_to_data_source_rows, record_batch_to_restore_rows, DataSourceRestoreRow, +}; +use crate::parquet::reader::read_batches; use crate::primary::Site; -use crate::relational::dump::Metadata; -use crate::relational::Layout; +use crate::relational::dump::{Metadata, TableInfo}; +use crate::relational::{Layout, Table}; +use crate::relational_queries::InsertQuery; +use crate::vid_batcher::VidRange; use crate::AsyncPgConnection; +const DATA_SOURCES_TABLE: &str = "data_sources$"; + /// Convert a dump `BlockPtr` (hex hash string) to a graph `BlockPtr`. fn to_graph_block_ptr(bp: &super::dump::BlockPtr) -> Result { let hash = BlockHash::try_from(bp.hash.as_str()) @@ -35,6 +43,167 @@ fn to_graph_block_ptr(bp: &super::dump::BlockPtr) -> Result Result { + let query = format!( + "select coalesce(min(vid), 0)::int8 as min_vid, \ + coalesce(max(vid), -1)::int8 as max_vid \ + from {}", + qualified_name + ); + let range: VidRange = diesel::sql_query(&query) + .get_result(conn) + .await + .map_err(StoreError::from)?; + Ok(range.max) +} + +/// Import a single entity table from Parquet chunks. +/// +/// Supports resumability: checks the current max(vid) in the DB table +/// and skips already-imported rows. +async fn import_entity_table( + conn: &mut AsyncPgConnection, + table: &Table, + table_info: &TableInfo, + dir: &Path, + logger: &Logger, +) -> Result { + if table_info.chunks.is_empty() || table_info.max_vid < 0 { + return Ok(0); + } + + let max_vid_db = current_max_vid(conn, table.qualified_name.as_str()).await?; + if max_vid_db >= table_info.max_vid { + info!( + logger, + "Table {} already fully restored, skipping", + table.object.as_str() + ); + return Ok(0); + } + + let chunk_size = InsertQuery::chunk_size(table); + let mut total_inserted = 0usize; + + for chunk_info in &table_info.chunks { + // Skip chunks that are fully imported + if chunk_info.max_vid <= max_vid_db { + continue; + } + + let chunk_path = dir.join(&chunk_info.file); + let batches = read_batches(&chunk_path)?; + + for batch in &batches { + let mut rows = record_batch_to_restore_rows(batch, table)?; + + // Filter out already-imported rows (for boundary chunks on resume) + if max_vid_db >= 0 { + rows.retain(|row| row.vid > max_vid_db); + } + + if rows.is_empty() { + continue; + } + + // Split into InsertQuery-sized chunks and execute + for chunk in rows.chunks(chunk_size) { + InsertQuery::for_restore(table, chunk)? + .execute(conn) + .await?; + total_inserted += chunk.len(); + } + } + } + + info!( + logger, + "Restored {} rows into {}", + total_inserted, + table.object.as_str() + ); + Ok(total_inserted) +} + +/// Insert a single data_sources$ row via raw SQL. +async fn insert_data_source_row( + conn: &mut AsyncPgConnection, + qualified_table: &str, + row: &DataSourceRestoreRow, +) -> Result<(), StoreError> { + let query = format!( + "INSERT INTO {} (vid, block_range, causality_region, manifest_idx, \ + parent, id, param, context, done_at) \ + VALUES ($1, int4range($2, $3), $4, $5, $6, $7, $8, $9::jsonb, $10)", + qualified_table, + ); + diesel::sql_query(&query) + .bind::(row.vid) + .bind::(row.block_range_start) + .bind::, _>(row.block_range_end) + .bind::(row.causality_region) + .bind::(row.manifest_idx) + .bind::, _>(row.parent) + .bind::, _>(row.id.as_deref()) + .bind::, _>(row.param.as_deref()) + .bind::, _>(row.context.as_deref()) + .bind::, _>(row.done_at) + .execute(conn) + .await + .map_err(StoreError::from)?; + Ok(()) +} + +/// Import the `data_sources$` table from Parquet chunks. +async fn import_data_sources( + conn: &mut AsyncPgConnection, + namespace: &str, + table_info: &TableInfo, + dir: &Path, + logger: &Logger, +) -> Result { + if table_info.chunks.is_empty() || table_info.max_vid < 0 { + return Ok(0); + } + + let qualified = format!("\"{}\".\"{DATA_SOURCES_TABLE}\"", namespace); + let max_vid_db = current_max_vid(conn, &qualified).await?; + if max_vid_db >= table_info.max_vid { + info!(logger, "data_sources$ already fully restored, skipping"); + return Ok(0); + } + + let mut total_inserted = 0usize; + + for chunk_info in &table_info.chunks { + if chunk_info.max_vid <= max_vid_db { + continue; + } + + let chunk_path = dir.join(&chunk_info.file); + let batches = read_batches(&chunk_path)?; + + for batch in &batches { + let rows = record_batch_to_data_source_rows(batch)?; + + for row in &rows { + if max_vid_db >= 0 && row.vid <= max_vid_db { + continue; + } + insert_data_source_row(conn, &qualified, row).await?; + total_inserted += 1; + } + } + } + + info!(logger, "Restored {} data_sources$ rows", total_inserted); + Ok(total_inserted) +} + /// Restore a subgraph deployment's schema and metadata from a dump /// directory. /// @@ -132,7 +301,7 @@ pub async fn create_schema( .await?; // 8. Create data_sources$ table if present in dump - if metadata.tables.contains_key("data_sources$") { + if metadata.tables.contains_key(DATA_SOURCES_TABLE) { let ds_table = DataSourcesTable::new(site.namespace.clone()); let ddl = ds_table.as_ddl(); conn.batch_execute(&ddl).await?; @@ -140,3 +309,49 @@ pub async fn create_schema( Ok(layout) } + +/// Import entity data and data_sources$ from Parquet files into the +/// database tables created by `create_schema`. +/// +/// This is resumable: if interrupted, it can be called again and will +/// skip already-imported rows by checking the current max(vid) in each +/// table. +#[allow(dead_code)] +pub async fn import_data( + conn: &mut AsyncPgConnection, + layout: &Layout, + metadata: &Metadata, + dir: &Path, + logger: &Logger, +) -> Result<(), StoreError> { + // Import entity tables (sorted by name for determinism) + let mut table_names: Vec<_> = metadata + .tables + .keys() + .filter(|name| name.as_str() != DATA_SOURCES_TABLE) + .collect(); + table_names.sort(); + + for table_name in table_names { + let table_info = &metadata.tables[table_name]; + let table = layout + .tables + .values() + .find(|t| t.object.as_str() == table_name) + .ok_or_else(|| { + StoreError::InternalError(format!( + "table '{}' from dump not found in layout", + table_name, + )) + })?; + import_entity_table(conn, table, table_info, dir, logger).await?; + } + + // Import data_sources$ if present + if let Some(ds_info) = metadata.tables.get(DATA_SOURCES_TABLE) { + let namespace = layout.site.namespace.as_str(); + import_data_sources(conn, namespace, ds_info, dir, logger).await?; + } + + Ok(()) +} From e0fcf7ca58ba08871273c0aea02eb191dacc0b54 Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 17:29:02 -0800 Subject: [PATCH 16/20] store: Add restore finalization with vid sequence reset and head block update --- store/postgres/src/relational/restore.rs | 113 ++++++++++++++++++++++- 1 file changed, 112 insertions(+), 1 deletion(-) diff --git a/store/postgres/src/relational/restore.rs b/store/postgres/src/relational/restore.rs index e2e4d2131db..b98f540b7ac 100644 --- a/store/postgres/src/relational/restore.rs +++ b/store/postgres/src/relational/restore.rs @@ -10,6 +10,8 @@ use std::fs; use std::path::Path; use std::sync::Arc; +use diesel::dsl::update; +use diesel::prelude::{ExpressionMethods, QueryDsl}; use diesel::sql_types::{BigInt, Binary, Integer, Nullable, Text}; use diesel_async::{RunQueryDsl, SimpleAsyncConnection}; use graph::blockchain::BlockHash; @@ -18,6 +20,7 @@ use graph::prelude::{info, BlockPtr as GraphBlockPtr, Logger, StoreError}; use graph::schema::{EntityType, InputSchema}; use graph::semver::Version; +use crate::catalog; use crate::deployment::create_deployment; use crate::dynds::DataSourcesTable; use crate::parquet::convert::{ @@ -26,7 +29,7 @@ use crate::parquet::convert::{ use crate::parquet::reader::read_batches; use crate::primary::Site; use crate::relational::dump::{Metadata, TableInfo}; -use crate::relational::{Layout, Table}; +use crate::relational::{Layout, Table, VID_COLUMN}; use crate::relational_queries::InsertQuery; use crate::vid_batcher::VidRange; use crate::AsyncPgConnection; @@ -355,3 +358,111 @@ pub async fn import_data( Ok(()) } + +/// Finalize a restored deployment by resetting vid sequences and setting +/// the head block pointer. +/// +/// This must be called after `import_data` has completed successfully. +/// Setting the head block is the very last operation — it marks the +/// deployment as "ready". +#[allow(dead_code)] +pub async fn finalize( + conn: &mut AsyncPgConnection, + layout: &Layout, + metadata: &Metadata, + logger: &Logger, +) -> Result<(), StoreError> { + let nsp = layout.site.namespace.as_str(); + + // 1. Reset vid sequences for entity tables that use bigserial. + // Tables where has_vid_seq() is true use plain bigint (no sequence). + let mut table_names: Vec<_> = metadata + .tables + .keys() + .filter(|name| name.as_str() != DATA_SOURCES_TABLE) + .collect(); + table_names.sort(); + + for table_name in table_names { + let table_info = &metadata.tables[table_name]; + if table_info.max_vid < 0 { + continue; + } + + let table = layout + .tables + .values() + .find(|t| t.object.as_str() == table_name) + .ok_or_else(|| { + StoreError::InternalError(format!( + "table '{}' from dump not found in layout", + table_name, + )) + })?; + + if table.object.has_vid_seq() { + continue; + } + + let vid_seq = catalog::seq_name(&table.name, VID_COLUMN); + let query = format!( + "SELECT setval('\"{nsp}\".\"{vid_seq}\"', {})", + table_info.max_vid + ); + conn.batch_execute(&query).await.map_err(|e| { + StoreError::InternalError(format!("reset vid seq for {table_name}: {e}")) + })?; + } + + // 2. Reset data_sources$ vid sequence if present + if let Some(ds_info) = metadata.tables.get(DATA_SOURCES_TABLE) { + if ds_info.max_vid >= 0 { + let qualified = format!("\"{nsp}\".\"{DATA_SOURCES_TABLE}\""); + let query = format!( + "SELECT setval(pg_get_serial_sequence('{qualified}', 'vid'), {})", + ds_info.max_vid + ); + conn.batch_execute(&query).await.map_err(|e| { + StoreError::InternalError(format!("reset data_sources$ vid seq: {e}")) + })?; + } + } + + // 3. Update earliest_block_number (may differ from start_block after + // pruning) and set the head block pointer. Setting the head block + // is the very last step: it makes the deployment "ready". + { + use crate::deployment::deployment as d; + use crate::deployment::head as h; + + update(d::table.filter(d::id.eq(layout.site.id))) + .set(d::earliest_block_number.eq(metadata.earliest_block_number)) + .execute(conn) + .await + .map_err(StoreError::from)?; + + if let Some(head) = &metadata.head_block { + let head_ptr = to_graph_block_ptr(head)?; + update(h::table.filter(h::id.eq(layout.site.id))) + .set(( + h::block_number.eq(head_ptr.number), + h::block_hash.eq(head_ptr.hash_slice()), + h::entity_count.eq(metadata.entity_count as i64), + )) + .execute(conn) + .await + .map_err(StoreError::from)?; + + info!( + logger, + "Finalized restore: head block #{}, entity count {}", + head_ptr.number, + metadata.entity_count + ); + } else { + info!(logger, "Finalized restore (no head block in dump)"); + } + } + + Ok(()) +} From c36de68882b4be21ba8af05658348014fd81c76c Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Sat, 21 Feb 2026 22:27:28 -0800 Subject: [PATCH 17/20] store: Add shard-aware restore planning with RestoreMode and RestoreAction --- graph/src/components/store/err.rs | 8 +++- store/postgres/src/lib.rs | 2 +- store/postgres/src/primary.rs | 74 ++++++++++++++++++++++++++++++- 3 files changed, 81 insertions(+), 3 deletions(-) diff --git a/graph/src/components/store/err.rs b/graph/src/components/store/err.rs index cbf500884df..995f0c67648 100644 --- a/graph/src/components/store/err.rs +++ b/graph/src/components/store/err.rs @@ -74,6 +74,10 @@ pub enum StoreError { StatementTimeout, #[error("database constraint violated: {0}")] ConstraintViolation(String), + /// The input to some operation, usually user input, makes it impossible + /// to complete the operation. This must be a deterministic error + #[error("{0}")] + Input(String), } // Convenience to report an internal error @@ -132,6 +136,7 @@ impl Clone for StoreError { } Self::StatementTimeout => Self::StatementTimeout, Self::ConstraintViolation(arg0) => Self::ConstraintViolation(arg0.clone()), + Self::Input(arg0) => Self::Input(arg0.clone()), } } } @@ -187,7 +192,8 @@ impl StoreError { | UnknownAttribute(_, _) | InvalidIdentifier(_) | UnsupportedFilter(_, _) - | ConstraintViolation(_) => true, + | ConstraintViolation(_) + | Input(_) => true, // non-deterministic errors Unknown(_) diff --git a/store/postgres/src/lib.rs b/store/postgres/src/lib.rs index 84011378898..44b53a7130d 100644 --- a/store/postgres/src/lib.rs +++ b/store/postgres/src/lib.rs @@ -65,7 +65,7 @@ pub use self::notification_listener::NotificationSender; pub use self::pool::{ AsyncPgConnection, ConnectionPool, ForeignServer, PoolCoordinator, PoolRole, ScopedFutureExt, }; -pub use self::primary::{db_version, UnusedDeployment}; +pub use self::primary::{db_version, RestoreMode, UnusedDeployment}; pub use self::store::Store; pub use self::store_events::SubscriptionManager; pub use self::subgraph_store::{unused, DeploymentPlacer, Shard, SubgraphStore, PRIMARY_SHARD}; diff --git a/store/postgres/src/primary.rs b/store/postgres/src/primary.rs index d7f506ff024..dde3764561d 100644 --- a/store/postgres/src/primary.rs +++ b/store/postgres/src/primary.rs @@ -776,6 +776,29 @@ mod queries { } } +/// How to handle conflicts when restoring a deployment. +/// Constructed from CLI flags --replace, --add, --force. +pub enum RestoreMode { + /// No conflict flags given — error if deployment exists anywhere + Default, + /// --replace: drop and recreate in the target shard + Replace, + /// --add: create a copy in a shard that doesn't have the deployment + Add, + /// --force: restore no matter what (replace if in shard, add if not) + Force, +} + +/// The action `plan_restore` decided on based on `RestoreMode` and current +/// state. +pub enum RestoreAction { + /// Create a new site (active=true if fresh, active=false if copy + /// exists elsewhere) + Create { active: bool }, + /// Drop existing site in target shard, then recreate + Replace { existing: Site }, +} + /// A wrapper for a database connection that provides access to functionality /// that works only on the primary database pub struct Connection { @@ -1406,7 +1429,7 @@ impl Connection { shard, namespace, network, - active: true, + active, schema_version, _creation_disallowed: (), }) @@ -1546,6 +1569,55 @@ impl Connection { .await } + /// Determine what action to take when restoring `subgraph` into `shard` + /// based on the `mode` and the current state of the deployment. + pub async fn plan_restore( + &mut self, + shard: &Shard, + subgraph: &DeploymentHash, + mode: &RestoreMode, + ) -> Result { + let conn = &mut self.conn; + let in_shard = queries::find_site_in_shard(conn, subgraph, shard).await?; + let active = queries::find_active_site(conn, subgraph).await?; + + match (in_shard, active, mode) { + // Deployment exists in target shard + (Some(existing), _, RestoreMode::Replace | RestoreMode::Force) => { + Ok(RestoreAction::Replace { existing }) + } + (Some(_), _, RestoreMode::Default | RestoreMode::Add) => { + Err(StoreError::Input(format!( + "deployment {} already exists in shard {}; use --replace or --force", + subgraph, + shard.as_str() + ))) + } + // Deployment does not exist in target shard but exists elsewhere + (None, Some(ref active_site), RestoreMode::Add | RestoreMode::Force) => { + let _ = active_site; + Ok(RestoreAction::Create { active: false }) + } + (None, Some(active_site), RestoreMode::Default) => Err(StoreError::Input(format!( + "deployment {} already exists in shard {}; use --add --shard {} or --force", + subgraph, + active_site.shard.as_str(), + shard.as_str() + ))), + (None, Some(_), RestoreMode::Replace) => Err(StoreError::Input(format!( + "deployment {} is not in shard {}; nothing to replace", + subgraph, + shard.as_str() + ))), + // Deployment does not exist anywhere + (None, None, RestoreMode::Replace) => Err(StoreError::Input(format!( + "deployment {} does not exist; nothing to replace", + subgraph + ))), + (None, None, _) => Ok(RestoreAction::Create { active: true }), + } + } + pub async fn locate_site( &mut self, locator: DeploymentLocator, From f6fb2a1a5c9b30618b814ad3986831bdfe85fb2a Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Mon, 23 Feb 2026 15:10:29 -0800 Subject: [PATCH 18/20] store: Add restore orchestration in SubgraphStore and DeploymentStore Wire up the restore pipeline through SubgraphStore::restore() and DeploymentStore::restore(). Uses plan_restore() to determine whether to create or replace the deployment site, validates the target shard exists, resolves the subgraph name for deployment rule matching, and assigns the restored deployment to a node. Changes: - DeploymentStore::restore() coordinates schema creation, data import, and finalization - Inner::restore() handles conflict resolution, site allocation, and node assignment via deployment rules - Expose create_site() and find_active_site() on primary::Connection - Make create_site() accept an `active` parameter --- store/postgres/src/deployment_store.rs | 17 ++- store/postgres/src/parquet/convert.rs | 4 - store/postgres/src/parquet/reader.rs | 1 - store/postgres/src/primary.rs | 9 +- store/postgres/src/relational/dump.rs | 1 - store/postgres/src/relational/restore.rs | 3 - store/postgres/src/relational_queries.rs | 2 - store/postgres/src/subgraph_store.rs | 145 ++++++++++++++++++++++- 8 files changed, 166 insertions(+), 16 deletions(-) diff --git a/store/postgres/src/deployment_store.rs b/store/postgres/src/deployment_store.rs index 43a1dfb574f..2238aab8ae5 100644 --- a/store/postgres/src/deployment_store.rs +++ b/store/postgres/src/deployment_store.rs @@ -30,7 +30,7 @@ use std::collections::{BTreeMap, HashMap}; use std::convert::Into; use std::ops::Bound; use std::ops::{Deref, Range}; -use std::path::PathBuf; +use std::path::{Path, PathBuf}; use std::str::FromStr; use std::sync::{atomic::AtomicUsize, Arc, Mutex}; use std::time::{Duration, Instant}; @@ -909,6 +909,21 @@ impl DeploymentStore { .dump(&mut conn, index_list, dir, &site.network, entity_count) .await } + + pub(crate) async fn restore( + &self, + site: Arc, + dir: &Path, + metadata: &crate::relational::dump::Metadata, + ) -> Result<(), StoreError> { + let mut conn = self.pool.get_permitted().await?; + let layout = + crate::relational::restore::create_schema(&mut conn, site, metadata, dir).await?; + crate::relational::restore::import_data(&mut conn, &layout, metadata, dir, &self.logger) + .await?; + crate::relational::restore::finalize(&mut conn, &layout, metadata, &self.logger).await?; + Ok(()) + } } /// Methods that back the trait `WritableStore`, but have small variations in their signatures diff --git a/store/postgres/src/parquet/convert.rs b/store/postgres/src/parquet/convert.rs index 2d229834eae..441d74a4fcf 100644 --- a/store/postgres/src/parquet/convert.rs +++ b/store/postgres/src/parquet/convert.rs @@ -238,7 +238,6 @@ impl ColumnBuilder { } /// A row extracted from a Parquet file, ready for insertion into an entity table. -#[allow(dead_code)] pub struct RestoreRow { pub vid: i64, pub block: BlockNumber, @@ -253,7 +252,6 @@ pub struct RestoreRow { /// /// The batch must follow the column layout produced by `schema::arrow_schema`: /// `vid`, block tracking columns, optional `causality_region`, then data columns. -#[allow(dead_code)] pub fn record_batch_to_restore_rows( batch: &RecordBatch, table: &Table, @@ -337,7 +335,6 @@ pub fn record_batch_to_restore_rows( } /// A row extracted from a `data_sources$` Parquet file. -#[allow(dead_code)] pub struct DataSourceRestoreRow { pub vid: i64, pub block_range_start: BlockNumber, @@ -356,7 +353,6 @@ pub struct DataSourceRestoreRow { /// /// The batch must follow the fixed column layout from /// `schema::data_sources_arrow_schema`. -#[allow(dead_code)] pub fn record_batch_to_data_source_rows( batch: &RecordBatch, ) -> Result, StoreError> { diff --git a/store/postgres/src/parquet/reader.rs b/store/postgres/src/parquet/reader.rs index 7fbdcc6e32c..f299a93fe82 100644 --- a/store/postgres/src/parquet/reader.rs +++ b/store/postgres/src/parquet/reader.rs @@ -10,7 +10,6 @@ use parquet::arrow::arrow_reader::ParquetRecordBatchReaderBuilder; /// Opens the file, reads all row groups, and returns them as a vector /// of `RecordBatch`es. The batches retain the schema embedded in the /// Parquet file. -#[allow(dead_code)] pub fn read_batches(path: &Path) -> Result, StoreError> { let file = fs::File::open(path).map_err(|e| { StoreError::InternalError(format!( diff --git a/store/postgres/src/primary.rs b/store/postgres/src/primary.rs index dde3764561d..4fab04424fb 100644 --- a/store/postgres/src/primary.rs +++ b/store/postgres/src/primary.rs @@ -1391,7 +1391,7 @@ impl Connection { /// function only performs the basic operations for creation, and the /// caller must check that other conditions (like whether there already /// is an active site for the deployment) are met - async fn create_site( + pub(crate) async fn create_site( &mut self, shard: Shard, deployment: DeploymentHash, @@ -1467,6 +1467,13 @@ impl Connection { .map(|site| (site, site_was_created)) } + pub async fn find_active_site( + &mut self, + subgraph: &DeploymentHash, + ) -> Result, StoreError> { + queries::find_active_site(&mut self.conn, subgraph).await + } + pub async fn assigned_node(&mut self, site: &Site) -> Result, StoreError> { queries::assigned_node(&mut self.conn, site).await } diff --git a/store/postgres/src/relational/dump.rs b/store/postgres/src/relational/dump.rs index c568f2d0011..c10de5a982f 100644 --- a/store/postgres/src/relational/dump.rs +++ b/store/postgres/src/relational/dump.rs @@ -167,7 +167,6 @@ pub(crate) struct Metadata { impl Metadata { /// Read and validate a dump's `metadata.json`. - #[allow(dead_code)] pub fn from_file(path: &Path) -> Result { let content = fs::read_to_string(path).map_err(|e| { StoreError::InternalError(format!("failed to read {}: {e}", path.display())) diff --git a/store/postgres/src/relational/restore.rs b/store/postgres/src/relational/restore.rs index b98f540b7ac..3cf010b8f9e 100644 --- a/store/postgres/src/relational/restore.rs +++ b/store/postgres/src/relational/restore.rs @@ -220,7 +220,6 @@ async fn import_data_sources( /// - Obtaining the right shard connection /// /// Entity data import and finalization are handled separately. -#[allow(dead_code)] pub async fn create_schema( conn: &mut AsyncPgConnection, site: Arc, @@ -319,7 +318,6 @@ pub async fn create_schema( /// This is resumable: if interrupted, it can be called again and will /// skip already-imported rows by checking the current max(vid) in each /// table. -#[allow(dead_code)] pub async fn import_data( conn: &mut AsyncPgConnection, layout: &Layout, @@ -365,7 +363,6 @@ pub async fn import_data( /// This must be called after `import_data` has completed successfully. /// Setting the head block is the very last operation — it marks the /// deployment as "ready". -#[allow(dead_code)] pub async fn finalize( conn: &mut AsyncPgConnection, layout: &Layout, diff --git a/store/postgres/src/relational_queries.rs b/store/postgres/src/relational_queries.rs index 11e1eed3080..4931e02d81e 100644 --- a/store/postgres/src/relational_queries.rs +++ b/store/postgres/src/relational_queries.rs @@ -2355,7 +2355,6 @@ impl<'a> InsertRow<'a> { /// Unlike `new()`, this looks up values by SQL column name rather than /// entity field name, since `RestoreRow.values` is keyed by SQL name. /// Fulltext columns are regenerated from their source fields. - #[allow(dead_code)] fn from_restore( columns: &[&'a Column], row: &'a RestoreRow, @@ -2460,7 +2459,6 @@ impl<'a> InsertQuery<'a> { /// All data columns from the dump are present, so `unique_columns` /// includes every column in the table. Fulltext columns are /// regenerated from their source fields. - #[allow(dead_code)] pub fn for_restore( table: &'a Table, rows: &'a [RestoreRow], diff --git a/store/postgres/src/subgraph_store.rs b/store/postgres/src/subgraph_store.rs index c1f8b6df5fb..40ab177bc2b 100644 --- a/store/postgres/src/subgraph_store.rs +++ b/store/postgres/src/subgraph_store.rs @@ -18,8 +18,8 @@ use graph::{ components::{ server::index_node::VersionInfo, store::{ - self, BlockPtrForNumber, BlockStore, DeploymentLocator, EnsLookup as EnsLookupTrait, - PruneReporter, PruneRequest, SubgraphFork, + self, BlockPtrForNumber, BlockStore, DeploymentLocator, DeploymentSchemaVersion, + EnsLookup as EnsLookupTrait, PruneReporter, PruneRequest, SubgraphFork, }, }, data::{ @@ -42,7 +42,9 @@ use graph::{derive::CheapClone, futures03::future::join_all, prelude::alloy::pri use crate::{ catalog::Catalog, deployment::{OnSync, SubgraphHealth}, - primary::{self, DeploymentId, Mirror as PrimaryMirror, Primary, Site}, + primary::{ + self, DeploymentId, Mirror as PrimaryMirror, Primary, RestoreAction, RestoreMode, Site, + }, relational::{ self, index::{IndexList, Method}, @@ -735,6 +737,35 @@ impl Inner { } } + /// Determine the target node for a deployment using the configured + /// deployment rules, ignoring the shard selection. Returns an error + /// if no rule matches. + async fn node_for_deployment( + &self, + name: &SubgraphName, + network: &str, + ) -> Result { + let placement = self + .placer + .place(name.as_str(), network) + .map_err(|msg| internal_error!("illegal indexer name in deployment rule: {}", msg))?; + + match placement { + Some((_, nodes)) if !nodes.is_empty() => { + if nodes.len() == 1 { + Ok(nodes.into_iter().next().unwrap()) + } else { + let mut pconn = self.primary_conn().await?; + // unwrap: nodes is not empty + Ok(pconn.least_assigned_node(&nodes).await?.unwrap()) + } + } + _ => Err(StoreError::InternalError( + "no deployment rule matches this deployment".into(), + )), + } + } + pub async fn copy_deployment( &self, src: &DeploymentLocator, @@ -1446,6 +1477,114 @@ impl Inner { store.dump(site, directory).await } + + pub async fn restore( + &self, + dir: &std::path::Path, + shard: Shard, + name: Option, + mode: RestoreMode, + ) -> Result<(), StoreError> { + use crate::relational::dump::Metadata; + + let metadata_path = dir.join("metadata.json"); + let metadata = Metadata::from_file(&metadata_path)?; + + // Validate that the target shard exists before making any DB changes + self.stores + .get(&shard) + .ok_or_else(|| StoreError::UnknownShard(shard.to_string()))?; + + // Resolve the subgraph name for deployment rule matching. If not + // supplied, look up an existing name from the DB; error if none. + let name = match name { + Some(n) => n, + None => { + let names = self + .mirror + .subgraphs_by_deployment_hash(metadata.deployment.as_str()) + .await?; + let (name, _) = names.into_iter().next().ok_or_else(|| { + StoreError::InternalError( + "no subgraph name found for this deployment; use --name to specify one" + .into(), + ) + })?; + SubgraphName::new(name).map_err(|n| { + StoreError::InternalError(format!("invalid subgraph name `{n}` in database")) + })? + } + }; + + // Use deployment rules to determine which node should index this + // deployment. The rules also return candidate shards, but we ignore + // those since the shard is user-specified for restore. + let node = self.node_for_deployment(&name, &metadata.network).await?; + + let mut pconn = self.primary_conn().await?; + let action = pconn + .plan_restore(&shard, &metadata.deployment, &mode) + .await?; + + // Determine schema_version the same way allocate_site does + let schema_version = match metadata.graft_base.as_ref() { + Some(graft_base) => { + let base_site = pconn.find_active_site(graft_base).await?.ok_or_else(|| { + StoreError::DeploymentNotFound("graft_base not found".to_string()) + })?; + base_site.schema_version + } + None => DeploymentSchemaVersion::LATEST, + }; + + let site = match action { + RestoreAction::Create { active } => { + pconn + .create_site( + shard, + metadata.deployment.clone(), + metadata.network.clone(), + schema_version, + active, + ) + .await? + } + RestoreAction::Replace { existing } => { + let was_active = existing.active; + let existing = Arc::new(existing); + let store = self.for_site(&existing)?; + store.drop_deployment(&existing).await?; + pconn.drop_site(&existing).await?; + // Drop and re-acquire the primary connection to avoid pool + // deadlock: drop_deployment above used a separate connection + // from the same pool, and create_site below needs one too. + drop(pconn); + let mut pconn = self.primary_conn().await?; + pconn + .create_site( + shard, + metadata.deployment.clone(), + metadata.network.clone(), + schema_version, + was_active, + ) + .await? + } + }; + + let site = Arc::new(site); + let store = self.for_site(&site)?; + store.restore(site.cheap_clone(), dir, &metadata).await?; + + // Assign the restored deployment to the node determined by + // deployment rules + let mut pconn = self.primary_conn().await?; + let changes = pconn.assign_subgraph(&site, &node).await?; + let event = StoreEvent::new(changes); + pconn.send_store_event(&self.sender, &event).await?; + + Ok(()) + } } const STATE_ENS_NOT_CHECKED: u8 = 0; From 2350b7af0a3f910f2b5b7a0fafd34f2049e28d01 Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Mon, 23 Feb 2026 15:11:16 -0800 Subject: [PATCH 19/20] node: Add graphman restore command Add `graphman restore` CLI with options: --directory Path to dump directory --shard Target shard (default: primary) --name Subgraph name for deployment rule matching --replace Drop and recreate if exists in target shard --add Create copy in a different shard --force Restore regardless of current state --- node/src/bin/manager.rs | 42 ++++++++++++++++++++++ node/src/manager/commands/mod.rs | 1 + node/src/manager/commands/restore.rs | 54 ++++++++++++++++++++++++++++ 3 files changed, 97 insertions(+) create mode 100644 node/src/manager/commands/restore.rs diff --git a/node/src/bin/manager.rs b/node/src/bin/manager.rs index 5ac90b73300..13e09805601 100644 --- a/node/src/bin/manager.rs +++ b/node/src/bin/manager.rs @@ -342,6 +342,33 @@ pub enum Command { /// The name of the directory to dump to directory: String, }, + + /// Restore a subgraph deployment from a dump directory + /// + /// EXPERIMENTAL - NOT FOR PRODUCTION USE + /// + /// Restore a subgraph deployment from a dump created with the `dump` + /// command. + Restore { + /// Path to the dump directory + directory: String, + /// The database shard to restore into (default: primary) + #[clap(long)] + shard: Option, + /// Subgraph name for deployment rule matching and node assignment. + /// If omitted, uses an existing name from the database; errors if none found. + #[clap(long)] + name: Option, + /// Drop and recreate if the deployment already exists in the target shard + #[clap(long, conflicts_with_all = ["add", "force"])] + replace: bool, + /// Create a copy in a shard that doesn't have this deployment (requires --shard) + #[clap(long, conflicts_with_all = ["replace", "force"])] + add: bool, + /// Restore no matter what: replace if exists in target shard, add if not + #[clap(long, conflicts_with_all = ["replace", "add"])] + force: bool, + }, } impl Command { @@ -1757,6 +1784,21 @@ async fn main() -> anyhow::Result<()> { commands::dump::run(subgraph_store, primary_pool, deployment, directory).await } + + Restore { + directory, + shard, + name, + replace, + add, + force, + } => { + let store = ctx.store().await; + let subgraph_store = store.subgraph_store(); + + commands::restore::run(subgraph_store, directory, shard, name, replace, add, force) + .await + } } } diff --git a/node/src/manager/commands/mod.rs b/node/src/manager/commands/mod.rs index e2f413c1f1c..9e5ba8e0b95 100644 --- a/node/src/manager/commands/mod.rs +++ b/node/src/manager/commands/mod.rs @@ -14,6 +14,7 @@ pub mod provider_checks; pub mod prune; pub mod query; pub mod remove; +pub mod restore; pub mod rewind; pub mod run; pub mod stats; diff --git a/node/src/manager/commands/restore.rs b/node/src/manager/commands/restore.rs new file mode 100644 index 00000000000..84eae8f7cbf --- /dev/null +++ b/node/src/manager/commands/restore.rs @@ -0,0 +1,54 @@ +use std::sync::Arc; + +use graph::{bail, prelude::anyhow::Result}; + +use graph::prelude::SubgraphName; +use graph_store_postgres::{RestoreMode, Shard, SubgraphStore, PRIMARY_SHARD}; + +pub async fn run( + subgraph_store: Arc, + directory: String, + shard: Option, + name: Option, + replace: bool, + add: bool, + force: bool, +) -> Result<()> { + if add && shard.is_none() { + bail!("--add requires --shard"); + } + + let directory = std::path::Path::new(&directory).canonicalize()?; + let stat = std::fs::metadata(&directory)?; + + if !stat.is_dir() { + bail!( + "The path `{}` is not a directory", + directory.to_string_lossy() + ); + } + + let shard = match shard { + Some(s) => Shard::new(s)?, + None => PRIMARY_SHARD.clone(), + }; + + let name = name + .map(|n| SubgraphName::new(n.clone()).map_err(|_| anyhow::anyhow!("invalid name `{n}`"))) + .transpose()?; + + let mode = if replace { + RestoreMode::Replace + } else if add { + RestoreMode::Add + } else if force { + RestoreMode::Force + } else { + RestoreMode::Default + }; + + subgraph_store + .restore(&directory, shard, name, mode) + .await?; + Ok(()) +} From 8f6b0dc8eca7bb1e92bc86e4a1a01f030c74b7dc Mon Sep 17 00:00:00 2001 From: David Lutterkort Date: Mon, 23 Feb 2026 15:52:57 -0800 Subject: [PATCH 20/20] store: Use REPEATABLE READ transaction for consistent dump Wrap the dump's data-reading operations in a REPEATABLE READ READ ONLY transaction to get a consistent MVCC snapshot. This prevents head block vs entity data mismatches, cross-table inconsistency, and missing or phantom rows caused by concurrent indexing or pruning. Add a TransactionBuilder for PermittedConnection since diesel-async's TransactionBuilder requires TransactionManager = AnsiTransactionManager, which pool-wrapped connection types don't satisfy. --- store/postgres/src/deployment_store.rs | 23 +++++-- store/postgres/src/pool/mod.rs | 94 ++++++++++++++++++++++++++ 2 files changed, 112 insertions(+), 5 deletions(-) diff --git a/store/postgres/src/deployment_store.rs b/store/postgres/src/deployment_store.rs index 2238aab8ae5..b92ea15425c 100644 --- a/store/postgres/src/deployment_store.rs +++ b/store/postgres/src/deployment_store.rs @@ -900,13 +900,26 @@ impl DeploymentStore { pub(crate) async fn dump(&self, site: Arc, dir: PathBuf) -> Result<(), StoreError> { let mut conn = self.pool.get_permitted().await?; + // Layout and index list are schema metadata — safe to load outside + // the snapshot transaction let layout = self.layout(&mut conn, site.cheap_clone()).await?; - let entity_count = crate::detail::entity_count(&mut conn, &site).await?; - // Loading the IndexList should happen inside dump, but the - // interface does not allow it; should be changed let index_list = IndexList::load(&mut conn, site.cheap_clone(), self.clone()).await?; - layout - .dump(&mut conn, index_list, dir, &site.network, entity_count) + + // Use REPEATABLE READ to get a consistent MVCC snapshot for the + // entire dump. All queries inside see the same database state, + // regardless of concurrent indexing or pruning. + conn.build_transaction() + .repeatable_read() + .read_only() + .run(|conn| { + async move { + let entity_count = crate::detail::entity_count(conn, &site).await?; + layout + .dump(conn, index_list, dir, &site.network, entity_count) + .await + } + .scope_boxed() + }) .await } diff --git a/store/postgres/src/pool/mod.rs b/store/postgres/src/pool/mod.rs index fc0b0cd7388..9d881ca7ebd 100644 --- a/store/postgres/src/pool/mod.rs +++ b/store/postgres/src/pool/mod.rs @@ -19,6 +19,7 @@ use graph::slog::warn; use graph::util::timed_rw_lock::TimedMutex; use tokio::sync::OwnedSemaphorePermit; +use diesel_async::scoped_futures::ScopedBoxFuture; use std::collections::HashMap; use std::fmt::{self}; use std::ops::{Deref, DerefMut}; @@ -65,6 +66,99 @@ impl DerefMut for PermittedConnection { } } +impl PermittedConnection { + /// Build a transaction with custom isolation level and read mode. + /// + /// This is analogous to `diesel_async::pg::TransactionBuilder` but + /// works with the pool-wrapped connection type. The closure receives + /// `&mut PermittedConnection`, keeping the full wrapper type available + /// so callers can pass it to functions that expect `&mut AsyncPgConnection` + /// (the pool alias, not the raw diesel type). + pub fn build_transaction(&mut self) -> TransactionBuilder<'_> { + TransactionBuilder::new(self) + } +} + +/// Builder for a PostgreSQL transaction with configurable isolation level +/// and read mode. Created via [`PermittedConnection::build_transaction`]. +/// +/// We can't use diesel-async's `TransactionBuilder` because it requires +/// `C: AsyncConnection`. Our +/// connection types don't satisfy that: the blanket deref impl in +/// diesel-async wraps the transaction manager at each deref level, so +/// `Object` gets `PoolTransactionManager` +/// and `PermittedConnection` gets +/// `PoolTransactionManager>`. +/// Neither matches `AnsiTransactionManager`. +#[must_use = "Transaction builder does nothing unless you call `run` on it"] +pub struct TransactionBuilder<'a> { + conn: &'a mut PermittedConnection, + isolation_level: Option<&'static str>, + read_only: bool, +} + +impl<'a> TransactionBuilder<'a> { + fn new(conn: &'a mut PermittedConnection) -> Self { + Self { + conn, + isolation_level: None, + read_only: false, + } + } + + /// Set the transaction isolation level to `REPEATABLE READ`. + pub fn repeatable_read(mut self) -> Self { + self.isolation_level = Some("REPEATABLE READ"); + self + } + + /// Set the transaction isolation level to `SERIALIZABLE`. + pub fn serializable(mut self) -> Self { + self.isolation_level = Some("SERIALIZABLE"); + self + } + + /// Make the transaction `READ ONLY`. + pub fn read_only(mut self) -> Self { + self.read_only = true; + self + } + + /// Execute `f` inside the configured transaction. Commits on `Ok`, + /// rolls back on `Err`. + /// + /// The closure must return a `ScopedBoxFuture` (use `.scope_boxed()` + /// from `ScopedFutureExt`). + pub async fn run<'b, T, E, F>(self, f: F) -> Result + where + F: for<'r> FnOnce(&'r mut PermittedConnection) -> ScopedBoxFuture<'b, 'r, Result> + + Send + + 'a, + T: 'b, + E: From + 'b, + { + let mut sql = String::from("BEGIN TRANSACTION"); + if let Some(level) = self.isolation_level { + sql.push_str(" ISOLATION LEVEL "); + sql.push_str(level); + } + if self.read_only { + sql.push_str(" READ ONLY"); + } + self.conn.batch_execute(&sql).await?; + match f(self.conn).await { + Ok(value) => { + self.conn.batch_execute("COMMIT").await?; + Ok(value) + } + Err(e) => { + self.conn.batch_execute("ROLLBACK").await.ok(); + Err(e) + } + } + } +} + /// The namespace under which the `PRIMARY_TABLES` are mapped into each /// shard pub(crate) const PRIMARY_PUBLIC: &str = "primary_public";