The DagsterInstance
defines all of 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 solid compute functions, how to store local artifacts on disk, 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 so that they can effectively share information.
Some important configuration, like execution parallelism, is also set on a per-pipeline-run basis rather than on the instance. See Pipeline Runs for pipeline configuration.
When you launch a Dagster process, like Dagit or the Dagster CLI commands, Dagster attempts to load your instance. If the environment variable DAGSTER_HOME
is set, Dagster will look for an instance config file at $DAGSTER_HOME/dagster.yaml
. This file contains each of the configuration settings that make up the instance.
By default (if dagster.yaml
is not present or nothing is specified in that file), Dagster will store this information on the local filesystem, laid out like this:
$DAGSTER_HOME
├── dagster.yaml
├── history
│ ├── runs
│ │ ├── 00636713-98a9-461c-a9ac-d049407059cd.db
│ │ └── ...
│ └── runs.db
└── storage
├── 00636713-98a9-461c-a9ac-d049407059cd
│ └── compute_logs
│ ├── my_solid.compute.complete
│ ├── my_solid.compute.err
│ ├── my_solid.compute.out
│ └── ...
└── ...
The runs.db
and {run_id}.db
files are SQLite database files recording information about pipeline runs and per-run event logs respectively. The compute_logs
directories (one per pipeline run) contain the stdout and stderr logs from the execution of the compute functions of each solid in a pipeline.
If DAGSTER_HOME
is not set, the Dagster tools will use an ephemeral instance for execution. In this case, the run and event log storages will be in-memory rather than persisted to disk, and filesystem storage will use a temporary directory that is cleaned up when the process exits. This is useful for tests and is the default when invoking Python APIs such as execute_pipeline
directly.
In persistent Dagster deployments, you will 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, and to stream compute logs to an S3 bucket.
To do this, provide a $DAGSTER_HOME/dagster.yaml
file. Dagit and all Dagster tools will look for this file on startup. In the dagster.yaml
file, you can configure many different aspects of your Dagster Instance, all of which are detailed below.
Note that Dagster supports retrieving instance YAML values from environment variables, using an env:
key instead of a string literal value. Examples of using env:
are included in the sample configurations below:
This configures how the history of runs is persisted - this includes metadata on a given run's status, relevant timestamps, and other useful data to inspect a run's history.
To configure Run Storage, you should set the run_storage
attribute in your dagster.yaml
. There are three available options:
SqliteRunStorage
uses a Sqlite DB as the backing run storage.# there are two ways to set run_storage to SqliteRunStorage
# this config manually sets the directory (`base_dir`) for Sqlite to store run information in:
run_storage:
module: dagster.core.storage.runs
class: SqliteRunStorage
config:
base_dir: /path/to/dir
# and this config grabs the directory from an environment variable
run_storage:
module: dagster.core.storage.runs
class: SqliteRunStorage
config:
base_dir:
env: SQLITE_RUN_STORAGE_BASE_DIR
PostgresRunStorage
uses a Postgres DB as the backing run storage solution.# There are three ways to set run_storage to PostgresRunStorage
# this config manually sets the Postgres credentials
run_storage:
module: dagster_postgres.run_storage
class: PostgresRunStorage
config:
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 variables
run_storage:
module: dagster_postgres.run_storage
class: PostgresRunStorage
config:
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:
run_storage:
module: dagster_postgres.run_storage
class: PostgresRunStorage
config:
postgres_url: { PG_DB_CONN_STRING }
# This config gets the DB connection string / url via environment variables:
run_storage:
module: dagster_postgres.run_storage
class: PostgresRunStorage
config:
postgres_url:
env: PG_DB_CONN_STRING
MySQLRunStorage
uses a MySQL DB as the backing run storage solution.# There are three ways to set run_storage to MySQLRunStorage
# this config manually sets the MySQL credentials
run_storage:
module: dagster_mysql.run_storage
class: MySQLRunStorage
config:
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 variables
run_storage:
module: dagster_mysql.run_storage
class: MySQLRunStorage
config:
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:
run_storage:
module: dagster_mysql.run_storage
class: MySQLRunStorage
config:
mysql_url: { MYSQL_DB_CONN_STRING }
# this config grabs the MySQL connection string from environment variables
run_storage:
module: dagster_mysql.run_storage
class: MySQLRunStorage
config:
mysql_url:
env: MYSQL_DB_CONN_STRING
This controls how the structured event logs produced by each run are persisted - this includes events related to solids starting/running/completing, pipeline events, and useful event-related data.
To configure Event Log Storage, you should set the event_log_storage
attribute in your dagster.yaml
. There are four available options:
SqliteEventLogStorage
uses a Sqlite database as the backing storage for event logs. However, it utilizes a seperate Sqlite database (i.e. directory) for each run. In other words, the storage is _run-sharded_. This means that certain useful features, like the Assets page, are not usable since Dagster can't perform cross-run queries.Note that by leaving your event_log_storage
attribute blank, Run Storage will be set to Sqlite by default. If you would like to manually specify event_log_storage
, add the following snippet to your dagster.yaml
:
# there are two ways to set `event_log_storage` to SqliteEventLogStorage
# the first manually sets the directory (`base_dir`) to write event log data to:
event_log_storage:
module: dagster.core.storage.event_log
class: SqliteEventLogStorage
config:
base_dir: /path/to/dir
# and the second grabs the directory from an environment variable
event_log_storage:
module: dagster.core.storage.event_log
class: SqliteEventLogStorage
config:
base_dir:
env: SQLITE_EVENT_LOG_STORAGE_BASE_DIR
ConsolidatedSqliteEventLogStorage
is meant to imitate the behavior of Postgres or MySQL-based event log storages, as it is not _run sharded_. That is, all runs are persisted to the same Sqlite DB, which allows for cross-run queries (& thus use of certain Dagster features, like the Assets page).# there are two ways to set `event_log_storage` to ConsolidatedSqliteEventLogStorage
# the first manually sets the directory (`base_dir`) to write event log data to:
event_log_storage:
module: dagster.core.storage.event_log
class: ConsolidatedSqliteEventLogStorage
config:
base_dir: /path/to/dir
# and the second grabs the directory from an environment variable
event_log_storage:
module: dagster.core.storage.event_log
class: ConsolidatedSqliteEventLogStorage
config:
base_dir:
env: CONSOLIDATED_SQLITE_EVENT_LOG_STORAGE_BASE_DIR
PostgresEventLogStorage
utilizes a Postgres Database as the backing Event Log Storage solution.# There are four ways to set event_log_storage to PostgresEventLogStorage
# this config manually sets the Postgres credentials
event_log_storage:
module: dagster_postgres.event_log_storage
class: PostgresEventLogStorage
config:
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 variables
event_log_storage:
module: dagster_postgres.event_log_storage
class: PostgresEventLogStorage
config:
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:
event_log_storage:
module: dagster_postgres.event_log_storage
class: PostgresEventLogStorage
config:
postgres_url: { PG_DB_CONN_STRING }
# This config gets the DB connection string / url via environment variables:
event_log_storage:
module: dagster_postgres.event_log_storage
class: PostgresEventLogStorage
config:
postgres_url:
env: PG_DB_CONN_STRING
MySQLEventLogStorage
uses a MySQL DB as the backing event log storage solution.# There are four ways to set event_log_storage to MySQLRunStorage
# this config manually sets the MySQL credentials
event_log_storage:
module: dagster_mysql.event_log_storage
class: MySQLEventLogStorage
config:
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 variables
event_log_storage:
module: dagster_mysql.event_log_storage
class: MySQLEventLogStorage
config:
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:
event_log_storage:
module: dagster_mysql.event_log_storage
class: MySQLEventLogStorage
config:
mysql_url: { MYSQL_DB_CONN_STRING }
# this config grabs the MySQL connection string from environment variables
event_log_storage:
module: dagster_mysql.event_log_storage
class: MySQLEventLogStorage
config:
mysql_url:
env: MYSQL_DB_CONN_STRING
This provides an optional scheduler which controls execution of pipeline runs at regular intervals.
To configure the scheduler, set scheduler
in your dagster.yaml
. If you don't set scheduler
, no Scheduler is configured by default.
There are three options for the Scheduler: DagsterDaemonScheduler
, SystemCronScheduler
, and K8sScheduler
. We recommend using DagsterDaemonScheduler
.
SystemCronScheduler
and K8sScheduler
are deprecated and will be removed in the 0.12.0 Dagster release.
DagsterDaemonScheduler
submits runs from the dagster-daemon (long-lived) process. It periodically checks each running schedule for execution times that don’t have runs yet and launches them.Please also see the Daemon Deployment Docs for more context on deployment.
scheduler:
module: dagster.core.scheduler
class: DagsterDaemonScheduler
This option controls the backing storage used by the scheduler to manage the state of schedules (for example, tracking Queued
runs) and persist records of attempts-related data.
To configure Schedule Storage, you should set the schedule_storage
key in your dagster.yaml
. There are three available options:
SqliteScheduleStorage
uses a Sqlite DB as the backing schedule storage solution.# there are two ways to set `schedule_storage` to SqliteScheduleStorage
# the first manually sets the directory (`base_dir`) to write schedule-related data to:
schedule_storage:
module: dagster.core.storage.schedules
class: SqliteScheduleStorage
config:
base_dir: /path/to/dir
# the second grabs the directory from an environment variable
schedule_storage:
module: dagster.core.storage.schedules
class: SqliteScheduleStorage
config:
base_dir:
env: SQLITE_SCHEDULE_STORAGE_DIRECTORY
PostgresScheduleStorage
uses a Postgres DB as the backing schedule storage solution.# There are three ways to set schedule_storage to PostgresScheduleStorage
# this config manually sets the Postgres credentials
schedule_storage:
module: dagster_postgres.schedule_storage
class: PostgresScheduleStorage
config:
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 variables
schedule_storage:
module: dagster_postgres.schedule_storage
class: PostgresScheduleStorage
config:
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:
schedule_storage:
module: dagster_postgres.schedule_storage
class: PostgresScheduleStorage
config:
postgres_url: { PG_DB_CONN_STRING }
# This config gets the DB connection string / url via environment variables:
schedule_storage:
module: dagster_postgres.schedule_storage
class: PostgresScheduleStorage
config:
postgres_url:
env: PG_DB_CONN_STRING
MySQLScheduleStorage
uses a MySQL DB as the backing schedule storage solution.# There are three ways to set schedule_storage to MySQLScheduleStorage
# this config manually sets the MySQL credentials
schedule_storage:
module: dagster_mysql.schedule_storage
class: MySQLScheduleStorage
config:
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 variables
schedule_storage:
module: dagster_mysql.schedule_storage
class: MySQLScheduleStorage
config:
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:
schedule_storage:
module: dagster_mysql.schedule_storage
class: MySQLScheduleStorage
config:
mysql_url: { MYSQL_DB_CONN_STRING }
# this config grabs the MySQL connection string from environment variables
schedule_storage:
module: dagster_mysql.schedule_storage
class: MySQLScheduleStorage
config:
mysql_url:
env: MYSQL_DB_CONN_STRING
This determines where runs are executed.
There are two Dagster-provided options for the Run Launcher; users also can write custom run launchers. Check out the Run Launcher page for more information.
To configure the Run Launcher, set run_launcher
in your dagster.yaml
in one of the following ways:
The DefaultRunLauncher
spawns a new process in the same node as a pipeline's repository location. Please see the Run Launcher Docs for deployment information.
run_launcher:
module: dagster.core.launcher
class: DefaultRunLauncher
The K8sRunLauncher
allocates a Kubernetes Job per run. Please see the Run Launcher Docs for deployment information.
# there are multiple ways to configure the K8sRunLauncher
# you can set the follow configuration values directly
run_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
This determines the policy used to determine the prioritization rules nad concurrency limits for runs. Please see the Run Coordinator Docs for more information and for troubleshooting help.
To configure the Run Coordinator, set the run_coodinator
key in your dagster.yaml
. There are two options:
The DefaultRunCoordinator
immediately sends runs to the run launcher (no notion of Queued
runs).
See the Run Coordinator Docs for more information.
# 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 executing at once. This sends runs to a Dequeuer process (i.e. a Scheduler / Daemon) via run storage. Note that this requires a daemon process to be active to actually launch the runs.
This option has several configuration possibilities which allow for both limit the overall number of concurrent runs and more specific limits based on run tags - e.g. perhaps only 4 runs interacting with a cloud service should run concurrently to avoid being throttled.
For more information, see the Run Coordinator Docs.
# There are a few ways to configure the QueuedRunCoordinator:
# this first option has concurrency limits set to default values
run_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: 25
tag_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
Compute log storage controls the capture and persistence of raw stdout & stderr text logs.
To configure Compute Log Storage, set the compute_logs
key in your dagster.yaml
.
LocalComputeLogManager
writes stdout & stderr logs to disk.# there are two ways to set the directory that the LocalComputeLogManager writes
# stdout & stderr logs to
# You could directly set the `base_dir` key
compute_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 variable
compute_logs:
module: dagster.core.storage.local_compute_log_manager
class: LocalComputeLogManager
config:
base_dir:
env: LOCAL_COMPUTE_LOG_MANAGER_DIRECTORY
AzureBlobComputeLogManager
writes stdout & stderr logs to Azure Blob Storage.# 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 variables
compute_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
S3ComputeLogManager
writes stdout & stderr logs to AWS S3.# 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 variables
compute_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 is used to configure storage for any artifacts that require a local disk, such as schedules, or when using the filesystem system storage to manage files and intermediates.
Note that pipeline intermediates storage is itself not configured this way, but instead is configured on the pipeline level (i.e. via IO Managers)
To configure Local Artifact Storage, set local_artifact_storage
as follows in your dagster.yaml
:
LocalArtifactStorage
is currently the only option for Local Artifact Storage. This configures the directory used by the default filesystem intermediates storage, as well as any schedule-related artifacts that require a local disk.# there are two possible ways to configure LocalArtifactStorage
# example local_artifact_storage setup pointing to /var/shared/dagster directory
local_artifact_storage:
module: dagster.core.storage.root
class: LocalArtifactStorage
config:
base_dir: "/path/to/dir"
# alternatively, `base_dir` can be set to an environment variable
local_artifact_storage:
module: dagster.core.storage.root
class: LocalArtifactStorage
config:
base_dir:
env: DAGSTER_LOCAL_ARTIFACT_STORAGE_DIR
This allows opting in/out (set to true
by default) of Dagster collecting anonymized usage statistics.
To configure Telemetry, set the telemetry
key in your dagster.yaml
.
For more information on how and why we use telemetry, please visit the Telemetry Docs.
# Allows opting out of Dagster collecting usage statistics.
telemetry:
enabled: false