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:
- Report an
AssetMaterialization
event for an external asset to Dagster - Report an
asset_check
evaluation for an external asset to Dagster - Report an
AssetObservation
event for an external asset to Dagster
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:
Location | URL | Example request URL |
---|---|---|
Local webserver | Defaults to localhost:3000 | localhost:3000/report_asset_check |
Dagster+ | {ORGANIZATION}.dagster.plus/{DEPLOYMENT_NAME} | https://my-org.dagster.plus/prod/report_asset_check |
Open source deployment | URL of the Dagster webserver | https://dagster.my-org.com/report_asset_check |
Available APIs
Endpoint | Description |
---|---|
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
Method | POST |
Resource URL | /report_asset_materialization/ |
Authentication | Required only for Dagster+. The request header must
contain the |
Request header | The request header must specify the following:
|
Request body | If included, the request body must be valid JSON. |
Parameters
Parameters can be passed in multiple ways and will be considered in the following order:
- URL (
asset_key
only) - Request body, which must be valid JSON
- Query parameter
Name | Required/Optional | Description |
---|---|---|
asset_key | Required | The key of the materialized asset. May be passed as: |
metadata | Optional | Arbitrary metadata about the asset, specified as key-value pairs. May be passed as:
|
data_version | Optional | 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
|
description | Optional | A human-readable description of the materialized value. May be passed in
JSON body or as a query parameter; value is passed to the
|
partition | Optional | The name of the partition that was materialized. May be passed in JSON
body or as a query parameter; value is passed to the
|
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 anerror
object:{"error": ...}
Examples
- Local webserver
- Dagster+
- Open source deployment
Local webserver
Report an asset materialization against locally running webserver:
curl -X POST localhost:3000/report_asset_materialization/{ASSET_KEY}
Dagster+
Report an asset materialization against Dagster+ with a JSON body via cURL:
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
},
}'
Report an asset materialization against Dagster+ in Python using requests
:
import requests
url = "https://{ORGANIZATION}.dagster.cloud/{DEPLOYMENT_NAME}/report_asset_materialization/"
payload = {
"asset_key": "ASSET_KEY",
"metadata": {"rows": 10},
}
headers = {
"Content-Type": "application/json",
"Dagster-Cloud-Api-Token": "TOKEN"
}
response = requests.request("POST", url, json=payload, headers=headers)
response.raise_for_status()
Open source deployment
Report an asset materialization against an open source deployment (hosted at DAGSTER_WEBSERVER_HOST
) in Python using requests
:
import requests
url = f"{DAGSTER_WEBSERVER_HOST}/report_asset_materialization/{ASSET_KEY}"
response = requests.request("POST", url)
response.raise_for_status()
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
Method | POST |
Resource URL | /report_asset_check/ |
Authentication | Required only for Dagster+. The request header must
contain the |
Request header | The request header must specify the following:
|
Request body | If included, the request body must be valid JSON. |
Parameters
Parameters can be passed in multiple ways and will be considered in the following order:
- URL (
asset_key
only) - Request body, which must be valid JSON
- Query parameter
Name | Required/Optional | Description |
---|---|---|
asset_key | Required | The key of the checked asset. May be passed as: |
passed | Required | The pass/fail result of the check. May be passed as:
|
check_name | Required | The name of the check. May be passed in JSON body or as a query
parameter; value is passed to the |
metadata | Optional | Arbitrary metadata about the check, specified as key-value pairs. May be passed as:
|
severity | Optional | The severity of the check. Accepted values are:
May be passed in JSON body or as a query parameter; value is passed to
the |
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 anerror
object:{"error": ...}
Examples
- Local webserver
- Dagster+
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
Dagster+
Report a failed asset check (check_null_ids
) against Dagster+ with a JSON body via cURL:
curl --request POST \
--url https://{ORGANIZATION}.dagster.cloud/{DEPLOYMENT_NAME}/report_asset_check/ \
--header 'Content-Type: application/json' \
--header 'Dagster-Cloud-Api-Token: {TOKEN}' \
--data '{
"asset_key": "{ASSET_KEY}",
"check_name": "check_null_ids",
"passed": false,
"metadata": {
"null_rows": 3
},
}'
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
Method | POST |
Resource URL | /report_asset_observation/ |
Authentication | Required only for Dagster+. The request header must
contain the |
Request header | The request header must specify the following:
|
Request body | If included, the request body must be valid JSON. |
Parameters
Parameters can be passed in multiple ways and will be considered in the following order:
- URL (
asset_key
only) - Request body, which must be valid JSON
- Query parameter
Name | Required/Optional | Description |
---|---|---|
asset_key | Required | The key of the observed asset. May be passed as: |
metadata | Optional | Arbitrary metadata about the asset, specified as key-value pairs. May be passed as:
|
data_version | Optional | The data version of the observed asset. May be passed in JSON body or as
a query parameter; value is passed to
|
description | Optional | 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
|
partition | Optional | The name of the partition that was observed. May be passed in JSON body
or as a query parameter; value is passed to the
|
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 anerror
object:{"error": ...}
Examples
- Local webserver
- Dagster+
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}
Dagster+
Report an asset observation against Dagster+ with a JSON body via cURL:
curl --request POST \
--url https://{ORGANIZATION}.dagster.cloud/{DEPLOYMENT_NAME}/report_asset_observation/ \
--header 'Content-Type: application/json' \
--header 'Dagster-Cloud-Api-Token: {TOKEN}' \
--data '{
"asset_key": "{ASSET_KEY}",
"metadata": {
"rows": 10
},
"data_version": "{VERSION}",
}'
Instance API
Refer to the External assets instance API doc for information on the instance API.