Ask AI

Dagster instance#

This guide is applicable to Dagster Open Source (OSS) deployments. For Dagster+, refer to the Dagster+ 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.

Default local behavior#

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:

$DAGSTER_HOME
├── dagster.yaml
├── history
│   ├── runs
│   │   ├── 00636713-98a9-461c-a9ac-d049407059cd.db
│   │   └── ...
│   └── runs.db
└── storage
    ├── 00636713-98a9-461c-a9ac-d049407059cd
    │   └── compute_logs
    │       └── ...
    └── ...

Here's a breakdown of the files and directories that are generated:

File/DirectoryDescription
history/A directory containing historical information for runs.
history/runs.dbSQLite database file that contains information about runs.
history/[run_id].dbSQLite database file that contains per-run event logs.
storage/A directory a list of subdirectories, one for each run.
storage/[run_id]/compute_logsA directory specific to the run that contains the stdout and stderr logs from the execution of the compute functions of each op.

Configuration reference#

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:

NameKeyDescription
Dagster storagestorageControls how job and asset history is persisted. This includes run, event log, and schedule/sensor tick metadata, as well as other useful data.
Run launcherrun_launcherDetermines where runs are executed.
Run coordinatorrun_coordinatorDetermines the policy used to set prioritization rules and concurrency limits for runs.
Compute log storagecompute_logsControls the capture and persistence of raw stdout and stderr ext logs.
Local artifact storagelocal_artifact_storageConfigures storage for artifacts that require a local disk or when using the filesystem I/O manager (FilesytemIOManager).
TelemetrytelemetryUsed to opt in/out of Dagster collecting anonymized usage statistics.
gRPC serverscode_serversConfigures how Dagster loads the code in a code location.
Data retentiondata_retentionControls how long Dagster retains certain types of data that have diminishing value over time, such as schedule/sensor tick data.
Sensor evaluationsensorsControls how sensors are evaluated.
Schedule evaluationschedulesControls how schedules are evaluated.
Auto-materializeauto_materializeControls how assets are auto-materialized.

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.

Dagster storage#

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.

SQLite storage (default)#

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 variable
storage:
  sqlite:
    base_dir:
      env: SQLITE_STORAGE_BASE_DIR

Run launcher#

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.

DefaultRunLauncher#

The DefaultRunLauncher spawns a new process in the same node as a job's code location.

run_launcher:
  module: dagster.core.launcher
  class: DefaultRunLauncher

Run coordinator#

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.

DefaultRunCoordinator (default)#

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

Compute log storage#

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.

LocalComputeLogManager#

Used by default, the LocalComputeLogManager writes stdout and 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

Local artifact storage#

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

Telemetry#

The telemetry key allows you to opt in or out of Dagster collecting anonymized usage statistics. This is set to true by default.

# Allows opting out of Dagster collecting usage statistics.
telemetry:
  enabled: false

Refer to the Telemetry documentation for more info.

gRPC servers#

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

Data retention#

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 data
retention:
  schedule:
    purge_after_days: 90 # sets retention policy for schedule ticks of all types
  sensor:
    purge_after_days:
      skipped: 7
      failure: 30
      success: -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.

Sensor evaluation#

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: true
  num_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.

Schedule evaluation#

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: true
  num_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.

Auto-materialize#

The auto_materialize key allows you to adjust configuration related to auto-materializing assets.

auto_materialize:
  run_tags:
    key: value