Skip to main content

External assets REST API reference

As Dagster doesn't control scheduling or materializing external assets, it's up to you to keep their metadata updated. Use the endpoints described in this reference to report updates for external assets back to Dagster.

API functionality

Using the External Asset APIs, you can:

Authentication

Authentication is required only if requests are being made against a Dagster+ instance. To authenticate, provide a valid Dagster+ user token using the Dagster-Cloud-Api-Token header:

curl --request POST \
--url https://{ORGANIZATION}.dagster.cloud/{deployment_name}/report_asset_materialization/ \
--header 'Content-Type: application/json' \
--header 'Dagster-Cloud-Api-Token: {TOKEN}' \
--data '{
"asset_key": "{ASSET_KEY}",
"metadata": {
"rows": 10
},
}'

Constructing request URLs

The full URL you send requests to will vary depending on how you access your Dagster instance:

LocationURLExample request URL
Local webserverDefaults to localhost:3000localhost:3000/report_asset_check
Dagster+{ORGANIZATION}.dagster.plus/{DEPLOYMENT_NAME}https://my-org.dagster.plus/prod/report_asset_check
Open source deploymentURL of the Dagster webserverhttps://dagster.my-org.com/report_asset_check

Available APIs

EndpointDescription
POST /report_asset_materialization/Records an AssetMaterialization event for an external asset.
POST /report_asset_check/Records an asset_check evaluation for an external asset.
POST /report_asset_observation/Records an AssetObservation event for an external asset.

Report an asset materialization

Records an AssetMaterialization event for an external asset. This event type notifies the Dagster framework that a materialized value has been produced for an asset.

Using this endpoint, you could report to Dagster that an external asset has been updated and include metadata about the materialization. For example, the number of updated rows, the time the update occurred, and so on.

Resources

MethodPOST
Resource URL/report_asset_materialization/
Authentication

Required only for Dagster+. The request header must contain the Dagster-Cloud-Api-Token header and a valid user token.

Request header

The request header must specify the following:

  • Dagster-Cloud-Api-Token - Required if using Dagster+, e.g. Dagster-Cloud-Api-Token: [USER_TOKEN]

  • Content-Type - Required if the request contains a JSON body, specified as Content-Type: application/json

Request bodyIf included, the request body must be valid JSON.

Parameters

Parameters can be passed in multiple ways and will be considered in the following order:

  1. URL (asset_key only)
  2. Request body, which must be valid JSON
  3. Query parameter

Name

Required/Optional

Description
asset_keyRequired

The key of the materialized asset. May be passed as:

  • URL path - Specified as path components after /report_asset_materialization/, where each / delimits parts of a multipart AssetKey.

  • JSON body - Value is passed to the AssetKey constructor.

  • Query parameter - Accepts string or JSON encoded array for multipart keys.

metadataOptional

Arbitrary metadata about the asset, specified as key-value pairs. May be passed as:

  • JSON body - Value is passed to the AssetMaterialization constructor.

  • Query parameter - Accepts a JSON encoded object.

data_versionOptional

The data version of the asset associated with the materialization. May be passed in JSON body or as a query parameter; value is passed to AssetMaterialization via tags.

descriptionOptional

A human-readable description of the materialized value. May be passed in JSON body or as a query parameter; value is passed to the AssetMaterialization constructor.

partitionOptional

The name of the partition that was materialized. May be passed in JSON body or as a query parameter; value is passed to the AssetMaterialization constructor.

Returns

The API will return JSON, whether the request succeeds or fails.

  • 200 OK - Response body contains an empty object: {}
  • 400 Bad request - Response body contains an error object: {"error": ...}

Examples

Local webserver

Report an asset materialization against locally running webserver:

curl -X POST localhost:3000/report_asset_materialization/{ASSET_KEY}

Report an asset check evaluation

Records an AssetCheckEvaluation event for an external asset. This event type notifies the Dagster framework of the results of an executed check.

Using this endpoint, you could report to Dagster that an asset check has been executed and include metadata about the check. For example, if the check looks for null values in an id column, you could include the number of records with null IDs.

Resources

MethodPOST
Resource URL/report_asset_check/
Authentication

Required only for Dagster+. The request header must contain the Dagster-Cloud-Api-Token header and a valid user token.

Request header

The request header must specify the following:

  • Dagster-Cloud-Api-Token - Required if using Dagster+, e.g. Dagster-Cloud-Api-Token: [USER_TOKEN]

  • Content-Type - Required if the request contains a JSON body, specified as Content-Type: application/json

Request bodyIf included, the request body must be valid JSON.

Parameters

Parameters can be passed in multiple ways and will be considered in the following order:

  1. URL (asset_key only)
  2. Request body, which must be valid JSON
  3. Query parameter

Name

Required/Optional

Description
asset_keyRequired

The key of the checked asset. May be passed as:

  • URL path - Specified as path components after /report_asset_check/, where each / delimits parts of a multipart AssetKey.

  • JSON body - Value is passed to the AssetKey constructor.

  • Query parameter - Accepts string or JSON encoded array for multipart keys.

passedRequired

The pass/fail result of the check. May be passed as:

  • JSON body - Value is passed to the AssetCheckEvaluation constructor.

  • Query parameter - Accepts a JSON encoded boolean: true or false.

check_nameRequired

The name of the check. May be passed in JSON body or as a query parameter; value is passed to the AssetCheckEvaluation constructor.

metadataOptional

Arbitrary metadata about the check, specified as key-value pairs. May be passed as:

  • JSON body - Value is passed to the AssetCheckEvaluation constructor.

  • Query parameter - Accepts a JSON encoded object.

severityOptional

The severity of the check. Accepted values are:

  • WARN
  • ERROR (default)

May be passed in JSON body or as a query parameter; value is passed to the AssetCheckEvaluation constructor.

Returns

The API will return JSON, whether the request succeeds or fails.

  • 200 OK - Response body contains an empty object: {}
  • 400 Bad request - Response body contains an error object: {"error": ...}

Examples

Local webserver

Report a successful asset check (check_null_ids) against a locally running webserver:

curl -X POST localhost:3000/report_asset_check/{ASSET_KEY}?check_name=check_null_ids&passed=true

Report an asset observation

Records an AssetObservation event for an external asset. This event type captures metadata about an asset at a point in time and provides it to the Dagster framework. Refer to the Asset observation documentation for more information.

Resources

MethodPOST
Resource URL/report_asset_observation/
Authentication

Required only for Dagster+. The request header must contain the Dagster-Cloud-Api-Token header and a valid user token.

Request header

The request header must specify the following:

  • Dagster-Cloud-Api-Token - Required if using Dagster+, e.g. Dagster-Cloud-Api-Token: [USER_TOKEN]

  • Content-Type - Required if the request contains a JSON body, specified as Content-Type: application/json

Request bodyIf included, the request body must be valid JSON.

Parameters

Parameters can be passed in multiple ways and will be considered in the following order:

  1. URL (asset_key only)
  2. Request body, which must be valid JSON
  3. Query parameter

Name

Required/Optional

Description
asset_keyRequired

The key of the observed asset. May be passed as:

  • URL path - Specified as path components after /report_asset_observation/, where each / delimits parts of a multipart AssetKey.

  • JSON body - Value is passed to the AssetKey constructor.

  • Query parameter - Accepts string or JSON encoded array for multipart keys.

metadataOptional

Arbitrary metadata about the asset, specified as key-value pairs. May be passed as:

  • JSON body - Value is passed to the AssetObservation constructor.

  • Query parameter - Accepts a JSON encoded object.

data_versionOptional

The data version of the observed asset. May be passed in JSON body or as a query parameter; value is passed to AssetObservation via tags.

descriptionOptional

A human-readable description of the asset or observation. May be passed in JSON body or as a query parameter; value is passed to the AssetObservation constructor.

partitionOptional

The name of the partition that was observed. May be passed in JSON body or as a query parameter; value is passed to the AssetObservation constructor.

Returns

The API will return JSON, whether the request succeeds or fails.

  • 200 OK - Response body contains an empty object: {}
  • 400 Bad request - Response body contains an error object: {"error": ...}

Examples

Local webserver

Report an asset observation with a data version against a locally running webserver:

curl -X POST localhost:3000/report_asset_observation/{ASSET_KEY}?data_version={VERSION}

Instance API

Refer to the External assets instance API doc for information on the instance API.