This guide is applicable to Dagster Open Source (OSS) deployments. For Dagster Cloud, refer to the Dagster Cloud documentation.
The Dagster instance defines the configuration that Dagster needs for a single deployment - for example, where to store the history of past runs and their associated logs, where to stream the raw logs from op compute functions, and how to launch new runs.
All of the processes and services that make up your Dagster deployment should share a single instance config file, named dagster.yaml, so that they can effectively share information.
Some important configuration, like execution parallelism, is set on a per-job basis rather than on the instance.
When a Dagster process like the Dagster webserver or Dagster CLI commands are launched, Dagster tries to load your instance. If the environment variable DAGSTER_HOME is set, Dagster looks for an instance config file at $DAGSTER_HOME/dagster.yaml. This file contains the configuration settings that make up the instance.
If DAGSTER_HOME isn't set, Dagster tools will use a temporary directory for storage that is cleaned up when the process exits. This can be useful when using Dagster for temporary local development or testing, when you don't care about the results being persisted.
If DAGSTER_HOME is set but dagster.yaml isn't present or is empty, Dagster will persist data on the local filesystem, structured like the following:
In persistent Dagster deployments, you'll typically want to configure many of the components on the instance. For example, you may want to use a Postgres instance to store runs and the corresponding event logs, but stream compute logs to an Amazon S3 bucket.
To do this, provide a $DAGSTER_HOME/dagster.yaml file, which the webserver and all other Dagster tools will look for on startup. In this file, you can configure different aspects of your Dagster instance, including:
Note: Environment variables in YAML configuration are supported by using an env: key instead of a literal string value. Sample configurations in this reference include examples using environment variables.
The storage key allows you to configure how job and asset history is persisted. This includes metadata on runs, event logs, schedule/sensor ticks, and other useful data.
Refer to the following tabs for available options and sample configuration.
To use a SQLite database for storage, configure storage.sqlite in dagster.yaml:
# there are two ways to set storage to SqliteStorage# this config manually sets the directory (`base_dir`) for Sqlite to store data in:storage:sqlite:base_dir: /path/to/dir
# and this config grabs the directory from an environment variablestorage:sqlite:base_dir:env: SQLITE_STORAGE_BASE_DIR
To use Postgres storage, you'll need to install the dagster-postgres library.
To use a PostgreSQL database (DagsterPostgresStorage) for storage, configure storage.postgres in dagster.yaml:
# Postgres storage can be set using either credentials or a connection string. This requires that# the `dagster-postgres` library be installed.# this config manually sets the Postgres credentialsstorage:postgres:postgres_db:username:{ DAGSTER_PG_USERNAME }password:{ DAGSTER_PG_PASSWORD }hostname:{ DAGSTER_PG_HOSTNAME }db_name:{ DAGSTER_PG_DB }port:5432# and this config grabs the database credentials from environment variablesstorage:postgres:postgres_db:username:env: DAGSTER_PG_USERNAME
password:env: DAGSTER_PG_PASSWORD
hostname:env: DAGSTER_PG_HOST
db_name:env: DAGSTER_PG_DB
port:5432# and this config sets the credentials via DB connection string / url:storage:postgres:postgres_url:{ PG_DB_CONN_STRING }# This config gets the DB connection string / url via environment variables:storage:postgres:postgres_url:env: PG_DB_CONN_STRING
To use MySQL storage, you'll need to install the dagster-postgres library.
To use a MySQL database (DagsterMySQLStorage) for storage, configure storage.mysql in dagster.yaml:
# MySQL storage can be set using either credentials or a connection string. This requires that the# `dagster-mysql` library be installed.# this config manually sets the MySQL credentialsstorage:mysql:mysql_db:username:{ DAGSTER_MYSQL_USERNAME }password:{ DAGSTER_MYSQL_PASSWORD }hostname:{ DAGSTER_MYSQL_HOSTNAME }db_name:{ DAGSTER_MYSQL_DB }port:3306# and this config grabs the database credentials from environment variablesstorage:mysql:mysql_db:username:env: DAGSTER_MYSQL_USERNAME
password:env: DAGSTER_MYSQL_PASSWORD
hostname:env: DAGSTER_MYSQL_HOSTNAME
db_name:env: DAGSTER_MYSQL_DB
port:3306# and this config sets the credentials via DB connection string / url:storage:mysql:mysql_url:{ MYSQL_DB_CONN_STRING }# this config grabs the MySQL connection string from environment variablesstorage:mysql:mysql_url:env: MYSQL_DB_CONN_STRING
The run_launcher key allows you to configure the run launcher for your instance. Run launchers determine where runs are executed. You can use one of the Dagster-provided options or write your own custom run launcher. Refer to the Run launcher documentation for more info.
Refer to the following tabs for available options and sample configuration.
# there are multiple ways to configure the K8sRunLauncher# you can set the follow configuration values directlyrun_launcher:module: dagster_k8s.launcher
class: K8sRunLauncher
config:service_account_name: pipeline_run_service_account
job_image: my_project/dagster_image:latest
instance_config_map: dagster-instance
postgres_password_secret: dagster-postgresql-secret
# alternatively, you can grab any of these config values from environment variables:run_launcher:module: dagster_k8s.launcher
class: K8sRunLauncher
config:service_account_name:env: PIPELINE_RUN_SERVICE_ACCOUNT
job_image:env: DAGSTER_IMAGE_NAME
instance_config_map:env: DAGSTER_INSTANCE_CONFIG_MAP
postgres_password_secret:env: DAGSTER_POSTGRES_SECRET
The run_coordinator key allows you to configure the run coordinator for your instance. Run coordinators determine the policy used to set the prioritization rules and concurrency limits for runs. Refer to the Run coordinator documentation for more information and troubleshooting help.
Refer to the following tabs for available options and sample configuration.
The default run coordinator, the DefaultRunCoordinator immediately sends runs to the run launcher. There isn't a notion of Queued runs.
# Since DefaultRunCoordinator is the default option, omitting the `run_coordinator` key will also suffice,# but if you would like to set it explicitly:run_coordinator:module: dagster.core.run_coordinator
class: DefaultRunCoordinator
The QueuedRunCoordinator allows you to set limits on the number of runs that can be executed at once. Note This requires an active dagster-daemon process to launch the runs.
This run coordinator supports both limiting the overall number of concurrent runs and specific limits based on run tags. For example, to avoid throttling, you can specify a concurrency limit for runs that interact with a specific cloud service.
# There are a few ways to configure the QueuedRunCoordinator:# this first option has concurrency limits set to default valuesrun_coordinator:module: dagster.core.run_coordinator
class: QueuedRunCoordinator
# this second option manually specifies limits:run_coordinator:module: dagster.core.run_coordinator
class: QueuedRunCoordinator
config:max_concurrent_runs:25tag_concurrency_limits:-key:"database"value:"redshift"limit:4-key:"dagster/backfill"limit:10# as always, some or all of these values can be obtained from environment variables:run_coordinator:module: dagster.core.run_coordinator
class: QueuedRunCoordinator
config:max_concurrent_runs:env: DAGSTER_OVERALL_CONCURRENCY_LIMIT
tag_concurrency_limits:-key:"database"value:"redshift"limit:env: DAGSTER_REDSHIFT_CONCURRENCY_LIMIT
-key:"dagster/backfill"limit:env: DAGSTER_BACKFILL_CONCURRENCY_LIMIT
# for higher dequeue throughput, threading can be enabled:run_coordinator:module: dagster.core.run_coordinator
class: QueuedRunCoordinator
config:dequeue_use_threads:truedequeue_num_workers:8
The compute_logs key allows you to configure compute log storage. Compute log storage controls the capture and persistence of raw stdout and stderr text logs.
Refer to the following tabs for available options and sample configuration.
# there are two ways to set the directory that the LocalComputeLogManager writes# stdout & stderr logs to# You could directly set the `base_dir` keycompute_logs:module: dagster.core.storage.local_compute_log_manager
class: LocalComputeLogManager
config:base_dir: /path/to/directory
# Alternatively, you could set the `base_dir` key to an environment variablecompute_logs:module: dagster.core.storage.local_compute_log_manager
class: LocalComputeLogManager
config:base_dir:env: LOCAL_COMPUTE_LOG_MANAGER_DIRECTORY
# there are multiple ways to configure the AzureBlobComputeLogManager# you can set the necessary configuration values directly:compute_logs:module: dagster_azure.blob.compute_log_manager
class: AzureBlobComputeLogManager
config:storage_account: mycorp-dagster
container: compute-logs
secret_key: foo
local_dir: /tmp/bar
prefix: dagster-test-# alternatively, you can obtain any of these config values from environment variablescompute_logs:module: dagster_azure.blob.compute_log_manager
class: AzureBlobComputeLogManager
config:storage_account:env: MYCORP_DAGSTER_STORAGE_ACCOUNT_NAME
container:env: CONTAINER_NAME
secret_key:env: SECRET_KEY
local_dir:env: LOCAL_DIR_PATH
prefix:env: DAGSTER_COMPUTE_LOG_PREFIX
# there are multiple ways to configure the GCSComputeLogManager# you can set the necessary configuration values directly:compute_logs:module: dagster_gcp.gcs.compute_log_manager
class: GCSComputeLogManager
config:bucket: mycorp-dagster-compute-logs
prefix: dagster-test-# alternatively, you can obtain any of these config values from environment variablescompute_logs:module: dagster_gcp.gcs.compute_log_manager
class: GCSComputeLogManager
config:bucket:env: MYCORP_DAGSTER_COMPUTE_LOGS_BUCKET
prefix:env: DAGSTER_COMPUTE_LOG_PREFIX
The S3ComputeLogManager writes stdout and stderr to an Amazon Web Services S3 bucket.
# there are multiple ways to configure the S3ComputeLogManager# you can set the config values directly:compute_logs:module: dagster_aws.s3.compute_log_manager
class: S3ComputeLogManager
config:bucket:"mycorp-dagster-compute-logs"prefix:"dagster-test-"# or grab some or all of them from environment variablescompute_logs:module: dagster_aws.s3.compute_log_manager
class: S3ComputeLogManager
config:bucket:env: MYCORP_DAGSTER_COMPUTE_LOGS_BUCKET
prefix:env: DAGSTER_COMPUTE_LOG_PREFIX
The local_artifact_storage key allows you to configure local artifact storage. Local artifact storage is used to:
Configure storage for artifacts that require a local disk, or
Store inputs and outputs when using the filesystem I/O manager (FilesytemIOManager). Refer to the I/O managers documentation for more information on how other I/O managers store artifacts.
Note: LocalArtifactStorage is currently the only option for local artifact storage. This option configures the directory used by the default filesystem I/O Manager, as well as any artifacts that require a local disk.
# there are two possible ways to configure LocalArtifactStorage# example local_artifact_storage setup pointing to /var/shared/dagster directorylocal_artifact_storage:module: dagster.core.storage.root
class: LocalArtifactStorage
config:base_dir:"/path/to/dir"# alternatively, `base_dir` can be set to an environment variablelocal_artifact_storage:module: dagster.core.storage.root
class: LocalArtifactStorage
config:base_dir:env: DAGSTER_LOCAL_ARTIFACT_STORAGE_DIR
The code_servers key allows you to configure how Dagster loads the code in a code location.
When you aren't running your own gRPC server, the webserver and the Dagster daemon load your code from a gRPC server running in a subprocess. By default, if your code takes more than 60 seconds to load, Dagster assumes that it's hanging and stops waiting for it to load.
If you expect that your code will take longer than 60 seconds to load, set the code_servers.local_startup_timeout key. The value should be an integer that indicates the maximum timeout, in seconds.
# Configures how long Dagster waits for code locations# to load before timing out.code_servers:local_startup_timeout:120
The retention key allows you to configure how long Dagster retains certain types of data. Specifically, data that has diminishing value over time, such as schedule/sensor tick data. Cleaning up old ticks can help minimize storage concerns and improve query performance.
By default, Dagster retains skipped sensor ticks for seven days and all other tick types indefinitely. To customize the retention policies for schedule and sensor ticks, use the purge_after_days key:
# Configures how long Dagster keeps sensor / schedule tick dataretention:schedule:purge_after_days:90# sets retention policy for schedule ticks of all typessensor:purge_after_days:skipped:7failure:30success:-1# keep success ticks indefinitely
The purge_after_days key accepts either:
A single integer that indicates how long, in days, to retain ticks of all types. Note: A value of -1 retains ticks indefinitely.
A mapping of tick types (skipped, failure, success) to integers. The integers indicate how long, in days, to retain the tick type.
The sensors key allows you to configure how sensors are evaluated. To evaluate multiple sensors in parallel simultaneously, set the use_threads and num_workers keys:
sensors:use_threads:truenum_workers:8
You can also set the optional num_submit_workers key to evaluate multiple run requests from the same sensor tick in parallel, which can help decrease latency when a single sensor tick returns many run requests.
The schedules key allows you to configure how schedules are evaluated. By default, Dagster evaluates schedules one at a time.
To evaluate multiple schedules in parallel simultaneously, set the use_threads and num_workers keys:
schedules:use_threads:truenum_workers:8
You can also set the optional num_submit_workers key to evaluate multiple run requests from the same schedule tick in parallel, which can help decrease latency when a single schedule tick returns many run requests.