The Dagster instance

The DagsterInstance defines everything Dagster needs for a single deployment: 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.

Locally, an instance usually corresponds to a single Dagit process. In production, you'll want all of your processes to share a single instance config so they can effectively share information.

NOTE: many important configurations like execution parallelism are set on a per-pipeline-run basis, please check the section below on pipeline configuration.

Default local behavior

When you use Dagster, like running dagit from the command line, Dagster checks whether the environment variable DAGSTER_HOME is set. If it is, Dagster will look for an instance config file at $DAGSTER_HOME/dagster.yaml.

This file contains configuration settings that tell Dagster where and how to store local artifacts like intermediates, schedules, stdout and stderr logs from solid compute functions, and information about past runs and the structured events.

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
├── schedules
│   ├── my_repository
│   │   ├── my_pipeline_4c9ddc62-eeab-4937-a429-fdbc8832013c.json
│   │   └── ...
│   └── ...
└── storage
    ├── 00636713-98a9-461c-a9ac-d049407059cd
    │   ├── compute_logs
    │   │   ├── my_solid.compute.complete
    │   │   ├── my_solid.compute.err
    │   │   ├── my_solid.compute.out
    │   │   └── ...
    │   └── intermediates
    │       ├── my_solid.compute
    │       │   ├── output_one
    │       │   └── ...
    │       └── ...
    └── ...

The runs.db and {run_id}.db files are SQLite database files recording information about pipeline runs and per-run event logs respectively. The {pipeline_name}_{run_id}.json files store information about schedules associated with pipelines. 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. And the intermediates directories contain serialized representations of the outputs of each solid, enabling incremental recomputation of pipeline subsets.

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.

Instance Configuration YAML

In persistent Dagster deployments, you will typically want to configure some or all of this behavior. When you configure an instance, you can select the the implementation of its constituent components.

For example, you may want to use a Postgres instance to store runs and the corresponding event logs, to store intermediates and other local artifacts on an NFS mount, 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:

dagster.yaml
# ==================================================================================================
# 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 will control timed, repeated execution of pipeline runs.
# Currently, only SystemCronScheduler is supported.
scheduler:
  module: dagster_cron.cron_scheduler
  class: SystemCronScheduler

# ==================================================================================================
# 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

# ==================================================================================================
# 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, and
# storing intermediates when using the default filesystem intermediates storage (intermediates
# storage is not configured on the instance, but instead is configured in pipeline run config).
#
# Currently only supports LocalArtifactStorage.
local_artifact_storage:
  module: dagster.core.storage.root
  class: LocalArtifactStorage
  config:
    base_dir: "/var/shared/dagster"

# ==================================================================================================
# Opt-Ins
# ==================================================================================================
# Enable experimental Dagster features. Currently available features:
#
# Setting local_servers to true will create a gRPC server for each of the Python repository
# locations in your workspace and use it to load repositories and execute pipelines. The servers
# will automatically shut down along with Dagster once they have finished any runs being executed.
opt_in:
  local_servers: true

# ==================================================================================================
# 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

Event Log Storage

Compute Log Manager

Local Artifact Storage

Scheduler (disabled by default)

Scheduler Storage

Run Launcher

Pipeline Run Configuration YAML

Along with instance-level configuration, Dagster supports configuring several properties on a per-pipeline-run basis. An example is shown below:

pipeline_run.yaml
# ==================================================================================================
# Execution
# ==================================================================================================
# Configure whether to use single-process or multi-process execution, or use custom executors like
# Celery. Custom executors can be defined with the @executor decorator.
#
# **NOTE**: setting executors globally on the Dagster instance is not currently supported!
#
# Currently available executors:
# - in_process (default)
# - multiprocess
# - celery (provided by dagster_celery)
# - celery-k8s (provided by dagster_celery)
# - dask (provided by dagster_dask)
execution:
  multiprocess:
    config:
      # Note that max_concurrent: 0 is equivalent to multiprocessing.cpu_count() - see:
      # https://docs.python.org/3/library/multiprocessing.html#multiprocessing.cpu_count
      max_concurrent: 4

# ==================================================================================================
# Storage
# ==================================================================================================
# The storage key in pipeline run config is used to set where inputs/outputs are stored during
# pipeline execution. Custom storage backends can be defined with the @system_storage decorator.
#
# **NOTE**: setting pipeline run storage globally on the Dagster instance is not currently
# supported!
#
# You will need to configure persistent intermediates storage for all of your pipeline runs that you
# would like to run on multiprocess or distributed executors. This is because those executors use
# intermediates storage to exchange input and output values -- you should ensure that whatever
# storage you use is accessible by all processes/nodes involved in execution.
#
# Generally, filesystem storage is suitable for local in_process/multiprocess execution, and object/
# distributed storage like S3, GCS, or NFS is suitable for distributed execution.
#
# Currently available storage types:
# - in_memory (default)
# - filesystem
# - s3 (provided by dagster_aws.s3)
# - gcs (provided by dagster_gcp)
storage:
  filesystem:

# ==================================================================================================
# Loggers
# ==================================================================================================
# The loggers key in pipeline run config is used to set up customized loggers. Custom loggers can be
# defined with the @logger decorator.
#
# **NOTE**: setting custom loggers globally on the Dagster instance is not currently supported!
#
# Currently available loggers:
# - colored_console_logger (default)
# - json_console_logger
# - cloudwatch_logger (provided by dagster_azure.cloudwatch)
# - papertrail_logger (provided by dagster_papertrail)
loggers:
  console:
    config:
      log_level: DEBUG