Skip to content

Commit

Permalink
[cli] Set DAGSTER_IS_DEFS_VALIDATION_CLI in `dagster definitions va…
Browse files Browse the repository at this point in the history
…lidate` (#25624)

## Summary & Motivation

Our design partners would like to have more control on the code and its
behavior when in validation mode: different credentials, etc.

This PR updates `dagster definitions validate` to set the env var
`DAGSTER_IS_DEFS_VALIDATION_CLI` when the command is running , like we
do so `DAGSTER_IS_DEV_CLI` and `dagster dev`. This will allow users to
control the code using the env var like:

```
import os

if os.getenv("DAGSTER_IS_DEFS_VALIDATION_CLI"):
    ...
```

## How I Tested These Changes

Additional unit test
  • Loading branch information
maximearmstrong authored Oct 30, 2024
1 parent 1b790db commit 4270e2e
Show file tree
Hide file tree
Showing 3 changed files with 24 additions and 0 deletions.
3 changes: 3 additions & 0 deletions python_modules/dagster/dagster/_cli/definitions.py
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
import logging
import os
import sys

import click
Expand Down Expand Up @@ -60,6 +61,8 @@ def validate_command_options(f):
""",
)
def definitions_validate_command(log_level: str, log_format: str, **kwargs: ClickArgValue):
os.environ["DAGSTER_IS_DEFS_VALIDATION_CLI"] = "1"

configure_loggers(formatter=log_format, log_level=log_level.upper())
logger = logging.getLogger("dagster")

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,12 @@
import os

from dagster import Definitions, asset, define_asset_job

if os.getenv("DAGSTER_IS_DEFS_VALIDATION_CLI"):

@asset
def my_gated_asset() -> None: ...

my_gated_job = define_asset_job(name="my_gated_job", selection="my_gated_asset")

defs = Definitions(assets=[my_gated_asset], jobs=[my_gated_job])
Original file line number Diff line number Diff line change
Expand Up @@ -70,3 +70,12 @@ def test_invalid_project(options, monkeypatch):
assert result.exit_code == 1
assert "Validation failed" in result.output
assert "Duplicate asset key: AssetKey(['my_asset'])" in result.output


def test_env_var(monkeypatch):
with monkeypatch.context() as m:
m.chdir(VALID_PROJECT_PATH)
# Definitions in `gated_definitions.py` are gated by the "DAGSTER_IS_DEFS_VALIDATION_CLI" environment variable
result = invoke_validate(options=["-f", "valid_project/gated_definitions.py"])
assert result.exit_code == 0
assert "Validation successful for code location gated_definitions.py." in result.output

0 comments on commit 4270e2e

Please sign in to comment.