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. Note that Dagster supports retrieving instance YAML values from environment
variables, using env:
instead of a string literal. An example dagster.yaml
is below:
# ==================================================================================================
# Run Storage
# ==================================================================================================
# Controls how the history of runs is persisted. Can be set to SqliteRunStorage (default) or
# PostgresRunStorage.
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
# ==================================================================================================
# Event Log Storage
# ==================================================================================================
# Controls how the structured event logs produced by each run are persisted. Can be set to
# SqliteEventLogStorage (default) or PostgresEventLogStorage.
event_log_storage:
module: dagster_postgres.event_log
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
# ==================================================================================================
# Scheduler
# ==================================================================================================
# Provides an optional scheduler which controls execution of pipeline runs at regular intervals.
# We recommend using the default DagsterDaemonScheduler - SystemCronScheduler and K8sScheduler are
# also available but are deprecated.
scheduler:
module: dagster.core.scheduler
class: DagsterDaemonScheduler
# ==================================================================================================
# Schedule Storage
# ==================================================================================================
# Controls the backing storage used by the scheduler to manage the state of schedules and persist
# records of attempts.
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
# ==================================================================================================
# Run Launcher
# ==================================================================================================
# Component that determines where runs are executed.
run_launcher:
module: dagster.core.launcher
class: DefaultRunLauncher
# ==================================================================================================
# Run Coordinator
# ==================================================================================================
# Determines the policy used to determine the prioritization rules and concurrency limits for runs.
# Can be set to DefaultRunCoordinator (default) or QueuedRunCoordinator when you want to maintain
# limits on the number of runs that can be executing at once.
run_coordinator:
module: dagster.core.run_coordinator
class: QueuedRunCoordinator
# ==================================================================================================
# Compute Log Storage
# ==================================================================================================
# Controls the capture and persistence of raw stdout & stderr text logs.
#
# Currently supported:
# - LocalComputeLogManager (default)
# - AzureBlobComputeLogManager
# - S3ComputeLogManager
compute_logs:
module: dagster_aws.s3.compute_log_manager
class: S3ComputeLogManager
config:
bucket: "mycorp-dagster-compute-logs"
prefix: "dagster-test-"
# ==================================================================================================
# Local Artifact Storage
# ==================================================================================================
# Controls where to store data on the local disk. Primarily used for storing schedule metadata.
#
# Currently only supports LocalArtifactStorage.
local_artifact_storage:
module: dagster.core.storage.root
class: LocalArtifactStorage
config:
base_dir: "/var/shared/dagster"
# ==================================================================================================
# Telemetry
# ==================================================================================================
# Allows opting out of Dagster collecting usage statistics. See
# https://docs.dagster.io/install#telemetry for more details.
telemetry:
enabled: false
The API docs for available options are linked below:
Run Storage
SqliteRunStorage
PostgresRunStorage
Event Log Storage
Compute Log Manager
Local Artifact Storage
LocalArtifactStorage
Scheduler
DagsterDaemonScheduler
SystemCronScheduler
Scheduler Storage
Run Launcher
DefaultRunLauncher
Run Coordinator