Ask AI

Using environment variables and secrets#

Environment variables, which are key-value pairs configured outside your source code, allow you to dynamically modify application behavior depending on environment.

Using environment variables, you can define various configuration options for your Dagster application and securely set up secrets. For example, instead of hard-coding database credentials - which is bad practice and cumbersome for development - you can use environment variables to supply user details. This allows you to parameterize your pipeline without modifying code or insecurely storing sensitive data.


Declaring environment variables#

How environment variables are declared depends on whether you're developing locally or have already deployed your Dagster project.

Local development#

As of Dagster 1.1.0, Using .env files is supported for loading environment variables into local environments. A .env file is a text file containing key-value pairs that is used locally, but not checked into source control. Using a .env file allows you to develop and test locally without putting sensitive info at risk. For example:

# .env

DATABASE_NAME=staging
DATABASE_SCHEMA=sales
DATABASE_USERNAME=salesteam
DATABASE_PASSWORD=supersecretstagingpassword

If Dagster detects a .env file in the same folder where dagster-webserver or dagster-daemon is launched, it will automatically load the environment variables in the file. This also applies to variables exported from Dagster Cloud.

When using a .env file, keep the following in mind:

  • The .env file must be in the same folder where dagster-webserver or dagster-daemon is launched
  • Any time the .env file is modified, the workspace must be re-loaded to make the Dagster webserver/UI aware of the changes

Accessing environment variables#

In this section, we'll demonstrate how to access environment variables once they've been declared. There are two ways to do this:

In Python code#

To access environment variables in your Dagster code, you can use os.getenv:

import os

database_name = os.getenv("DATABASE_NAME")

This approach also works for accessing built-in environment variables for Dagster Cloud:

import os

deployment_name = os.getenv("DAGSTER_CLOUD_DEPLOYMENT_NAME")

Refer to the Dagster Cloud Branch Deployments example for a real-world example.

From Dagster configuration#

Configurable Dagster objects - such as ops, assets, resources, I/O managers, and so on - can accept configuration from environment variables. Dagster provides a native way to specify environment variables in your configuration. These environment variables are retrieved at launch time, rather than on initialization as with os.getenv. Refer to the next section for more info.

In Python code#

To access an environment variable as part of a Dagster configuration in Python code, you may use the following special syntax:

"PARAMETER_NAME": EnvVar("ENVIRONMENT_VARIABLE_NAME")

For example:

"access_token": EnvVar("GITHUB_ACCESS_TOKEN")

Using EnvVar vs os.getenv#

We just covered two different ways to access environment variables in Dagster. So, which one should you use? When choosing an approach, keep the following in mind:

  • When os.getenv is used, the variable's value is retrieved when Dagster loads the code location and will be visible in the UI.
  • When EnvVar is used, the variable's value is retrieved at runtime and won't be visible in the UI.

Using the EnvVar approach has a few unique benefits:

  • Improved observability. The UI will display information about configuration values sourced from environment variables.
  • Secret values are hidden in the UI. Secret values are hidden in the Launchpaid, Resources page, and other places where configuration is displayed.
  • Simplified testing. Because you can provide string values directly to configuration rather than environment variables, testing may be easier.

Handling secrets#

Using environment variables to provide secrets ensures sensitive info won't be visible in your code or the launchpad in the UI. In Dagster, best practice for handling secrets uses configuration and resources.

A resource is typically used to connect to an external service or system, such as a database. Resources can be configured separately from the rest of your app, allowing you to define it once and reuse it as needed.

Let's take a look at an example from the Dagster Crash Course, which creates a GitHub resource and supplies it to assets. Let's start by looking at the resource:

## resources.py

from dagster import StringSource, resource
from github import Github

class GithubClientResource(ConfigurableResource):
  access_token: str

  def get_client(self) -> Github:
    return Github(self.access_token)

Let's review what's happening here:

  • This code creates a GitHub resource named GithubClientResource
  • By subclassing ConfigurableResource and specifying the access_token field, we're telling Dagster that we want to be able to configure the resource with an access_token parameter
  • Since access_token is a string value, this config parameter can either be:
    • An environment variable, or
    • Provided directly in the configuration

As storing secrets in configuration is bad practice, we'll opt for using an environment variable. In this code, we're configuring the resource supplying it to our assets:

# __init__.py

from my_dagster_project import assets
from my_dagster_project.resources import GithubClientResource

from dagster import Definitions, EnvVar, load_assets_from_package_module

defs = Definitions(
    assets=load_assets_from_package_module(assets),
    resources={
        "github_api": GithubClientResource(access_token=EnvVar("GITHUB_ACCESS_TOKEN"))
    },
)

Let's review what's happening here:

  • We pass configuration info to the resource when we construct it. In this example, we're telling Dagster to load the access_token from the GITHUB_ACCESS_TOKEN environment variable by wrapping it in EnvVar.
  • We're adding that resource to our Definitions object so it's available for our assets.

Parameterizing pipeline behavior#

Using environment variables, you define how your code should execute at runtime.

Example 1: Per-environment configuration#

In this example, we'll demonstrate how to use different I/O manager configurations for local and production environments using configuration (specifically the configured API) and resources.

This example is adapted from the Transitioning data pipelines from development to production guide:

# __init__.py

resources = {
    "local": {
        "snowflake_io_manager": SnowflakePandasIOManager(
            account="abc1234.us-east-1",
            user=EnvVar("DEV_SNOWFLAKE_USER"),
            password=EnvVar("DEV_SNOWFLAKE_PASSWORD"),
            database="LOCAL",
            schema=EnvVar("DEV_SNOWFLAKE_SCHEMA"),
        ),
    },
    "production": {
        "snowflake_io_manager": SnowflakePandasIOManager(
            account="abc1234.us-east-1",
            user="system@company.com",
            password=EnvVar("SYSTEM_SNOWFLAKE_PASSWORD"),
            database="PRODUCTION",
            schema="HACKER_NEWS",
        ),
    },
}

deployment_name = os.getenv("DAGSTER_DEPLOYMENT", "local")

defs = Definitions(
    assets=[items, comments, stories], resources=resources[deployment_name]
)

Let's review what's happening here:

  • We've created a dictionary of resource definitions, resources, named after our local and production environments. In this example, we're using a Pandas Snowflake I/O manager.
  • For both local and production, we constructed the I/O manager using environment-specific run configuration. Note the differences in configuration between local and production, specifically where environment variables were used.
  • Following the resources dictionary, we define the deployment_name variable, which determines the current executing environment. This variable defaults to local, ensuring that DAGSTER_DEPLOYMENT=PRODUCTION must be set to use the production configuration.

Example 2: Dagster Cloud Branch Deployments#

This section is applicable only to Dagster Cloud.

Adapted from the Testing against production with Dagster Cloud Branch Deployments guide, this example demonstrates how to determine the current deployment type at runtime - Branch Deployment or full deployment - without using resources or configuration.

Let's look at a function that determines the current deployment using the DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT environment variable:

def get_current_env():
  is_branch_depl = os.getenv("DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT") == "1"
  assert is_branch_depl != None  # env var must be set
  return "branch" if is_branch_depl else "prod"

This function checks the value of DAGSTER_CLOUD_IS_BRANCH_DEPLOYMENT and, if equal to 1, returns a variable with the value of branch. This indicates that the current deployment is a Branch Deployment. Otherwise, the deployment is a full deployment and is_branch_depl will be returned with a value of prod.

Using this info, we can write code that executes differently when in a Branch Deployment or a full deployment.


Troubleshooting#

ErrorDescription and resolution
You have attempted to fetch the environment variable "[variable]" which is not set. In order for this execution to succeed it must be set in this environment.Surfacing when a run is launched in the UI, this error means that an environment variable set using StringSource could not be found in the executing environment.

Verify that the environment variable is named correctly and accessible in the environment.
  • If developing locally and using a .env file, try re-loading the workspace in the UI. The workspace must be re-loaded any time this file is modified for the UI to be aware of the changes.
  • If using Dagster Cloud:
No environment variables in .env fileDagster located and attempted to load a local .env file while launching dagster-webserver, but couldn't find any environment variables in the file.

If this is unexpected, verify that your .env is correctly formatted and located in the same folder where you're runningdagster-webserver.