Skip to content

Commit

Permalink
[components] Move global options to subcommands (#26640)
Browse files Browse the repository at this point in the history
## Summary & Motivation

This changes the options scheme in `dagster-dg` so that options are no
longer hierarchically processed and passed down the command tree,
_except_ in the special case of `dg component generate` subcommands,
where global options are defined on the containing group.

Now, all leaves of the tree (except for `dg component generate`
subcommands) have the same set of "global options" appended to their
options list. These are visually separated in the help message. We
correspondingly simplify the Usage message to have a single `[OPTIONS]`
input spot.

There are also now fewer apparent global options because two of the old
ones, `--clear-cache` and `--rebuild-component-registry`, must be
executed as standalone pseudo-commands (e.g. `dg --clear-cache`) rather
than used as modifiers to other commands.

Before (`dg component-type info --help`):

```
Usage: dg [OPTIONS] component-type info [OPTIONS] COMPONENT_TYPE

  Get detailed information on a registered Dagster component type.

Options:
  --description
  --generate-params-schema
  --component-params-schema
  -h, --help                 Show this message and exit.

Options (dg):
  --builtin-component-lib TEXT  Specify a builitin component library to use.
  --verbose                     Enable verbose output for debugging.
  --disable-cache               Disable caching of component registry data.
  --clear-cache                 Clear the cache before running the command.
  --rebuild-component-registry  Recompute and cache the set of available component types for the current environment.
                                Note that this also happens automatically whenever the cache is detected to be stale.
  --cache-dir PATH              Specify a directory to use for the cache.
```

After (`dg component-type info --help`):

```
Usage: dg component-type info [OPTIONS] COMPONENT_TYPE

  Get detailed information on a registered Dagster component type.

Options:
  --description
  --generate-params-schema
  --component-params-schema
  -h, --help                 Show this message and exit.

Global options:
  --builtin-component-lib TEXT  Specify a builitin component library to use.
  --verbose                     Enable verbose output for debugging.
  --disable-cache               Disable the cache..
  --cache-dir PATH              Specify a directory to use for the cache.
```

---

Before (`dg generate component dagster_components.dbt_project --help`):

```
Usage: dg [OPTIONS] component generate dagster_components.dbt_project [OPTIONS] COMPONENT_NAME

Options:
  --json-params TEXT   JSON string of component parameters.
  --init BOOLEAN       init
  --project-path TEXT  project_path
  -h, --help           Show this message and exit.

Options (dg):
  --builtin-component-lib TEXT  Specify a builitin component library to use.
  --verbose                     Enable verbose output for debugging.
  --disable-cache               Disable caching of component registry data.
  --clear-cache                 Clear the cache before running the command.
  --rebuild-component-registry  Recompute and cache the set of available component types for the current environment.
                                Note that this also happens automatically whenever the cache is detected to be stale.
  --cache-dir PATH              Specify a directory to use for the cache.
```

After (`dg generate component dagster_components.dbt_project --help`):

```
Usage: dg component generate [GLOBAL OPTIONS] dagster_components.dbt_project [OPTIONS] COMPONENT_NAME

Options:
  --json-params TEXT   JSON string of component parameters.
  --init BOOLEAN       init
  --project-path TEXT  project_path
  -h, --help           Show this message and exit.

Global options:
  --builtin-component-lib TEXT  Specify a builitin component library to use.
  --verbose                     Enable verbose output for debugging.
  --disable-cache               Disable the cache..
  --cache-dir PATH              Specify a directory to use for the cache.
```

## How I Tested These Changes

Modified unit tests, played with it on the command line.
  • Loading branch information
smackesey authored Dec 22, 2024
1 parent 0099d83 commit cee5d3e
Show file tree
Hide file tree
Showing 15 changed files with 439 additions and 333 deletions.
80 changes: 24 additions & 56 deletions python_modules/libraries/dagster-dg/dagster_dg/cli/__init__.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,12 +3,11 @@

import click

from dagster_dg.cache import DgCache
from dagster_dg.cli.code_location import code_location_group
from dagster_dg.cli.component import component_group
from dagster_dg.cli.component_type import component_type_group
from dagster_dg.cli.deployment import deployment_group
from dagster_dg.config import DgConfig, set_config_on_cli_context
from dagster_dg.cli.global_options import dg_global_options
from dagster_dg.context import (
DgContext,
ensure_uv_lock,
Expand All @@ -20,44 +19,30 @@
from dagster_dg.utils import DgClickGroup
from dagster_dg.version import __version__

DG_CLI_MAX_OUTPUT_WIDTH = 120

def create_dg_cli():
commands = {
"code-location": code_location_group,
"deployment": deployment_group,
"component": component_group,
"component-type": component_type_group,
}

# Defaults are defined on the DgConfig object.
def create_dg_cli():
@click.group(
commands=commands,
context_settings={"max_content_width": 120, "help_option_names": ["-h", "--help"]},
name="dg",
commands={
"code-location": code_location_group,
"deployment": deployment_group,
"component": component_group,
"component-type": component_type_group,
},
context_settings={
"max_content_width": DG_CLI_MAX_OUTPUT_WIDTH,
"help_option_names": ["-h", "--help"],
},
invoke_without_command=True,
cls=DgClickGroup,
)
@click.option(
"--builtin-component-lib",
type=str,
default=DgConfig.builtin_component_lib,
help="Specify a builitin component library to use.",
)
@click.option(
"--verbose",
is_flag=True,
default=DgConfig.verbose,
help="Enable verbose output for debugging.",
)
@click.option(
"--disable-cache",
is_flag=True,
default=DgConfig.disable_cache,
help="Disable caching of component registry data.",
)
@dg_global_options
@click.option(
"--clear-cache",
is_flag=True,
help="Clear the cache before running the command.",
help="Clear the cache.",
default=False,
)
@click.option(
Expand All @@ -69,33 +54,15 @@ def create_dg_cli():
),
default=False,
)
@click.option(
"--cache-dir",
type=Path,
default=DgConfig.cache_dir,
help="Specify a directory to use for the cache.",
)
@click.version_option(__version__, "--version", "-v")
@click.pass_context
def group(
context: click.Context,
builtin_component_lib: str,
verbose: bool,
disable_cache: bool,
cache_dir: Path,
clear_cache: bool,
rebuild_component_registry: bool,
**global_options: object,
):
"""CLI tools for working with Dagster components."""
context.ensure_object(dict)
config = DgConfig(
builtin_component_lib=builtin_component_lib,
verbose=verbose,
disable_cache=disable_cache,
cache_dir=cache_dir,
)
set_config_on_cli_context(context, config)

"""CLI for working with Dagster components."""
if clear_cache and rebuild_component_registry:
click.echo(
click.style(
Expand All @@ -104,35 +71,36 @@ def group(
)
sys.exit(1)
elif clear_cache:
DgCache.from_config(config).clear_all()
dg_context = DgContext.from_cli_global_options(global_options)
dg_context.cache.clear_all()
if context.invoked_subcommand is None:
context.exit(0)
elif rebuild_component_registry:
dg_context = DgContext.from_cli_global_options(global_options)
if context.invoked_subcommand is not None:
click.echo(
click.style(
"Cannot specify --rebuild-component-registry with a subcommand.", fg="red"
)
)
sys.exit(1)
_rebuild_component_registry(context)
_rebuild_component_registry(dg_context)
elif context.invoked_subcommand is None:
click.echo(context.get_help())
context.exit(0)

return group


def _rebuild_component_registry(cli_context: click.Context):
dg_context = DgContext.from_cli_context(cli_context)
def _rebuild_component_registry(dg_context: DgContext):
if not is_inside_code_location_directory(Path.cwd()):
click.echo(
click.style(
"This command must be run inside a Dagster code location directory.", fg="red"
)
)
sys.exit(1)
if not dg_context.cache:
if not dg_context.has_cache:
click.echo(
click.style("Cache is disabled. This command cannot be run without a cache.", fg="red")
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@

import click

from dagster_dg.cli.global_options import dg_global_options
from dagster_dg.context import DeploymentDirectoryContext, DgContext, is_inside_deployment_directory
from dagster_dg.generate import generate_code_location
from dagster_dg.utils import DgClickCommand, DgClickGroup
Expand Down Expand Up @@ -40,9 +41,12 @@ def code_location_group():
default=False,
help="Do not create a virtual environment for the code location.",
)
@click.pass_context
@dg_global_options
def code_location_generate_command(
cli_context: click.Context, name: str, use_editable_dagster: Optional[str], skip_venv: bool
name: str,
use_editable_dagster: Optional[str],
skip_venv: bool,
**global_options: object,
) -> None:
"""Generate a Dagster code location file structure and a uv-managed virtual environment scoped
to the code location.
Expand All @@ -53,6 +57,7 @@ def code_location_generate_command(
The code location file structure defines a Python package with some pre-existing internal
structure:
\b
├── <name>
│ ├── __init__.py
│ ├── components
Expand All @@ -66,8 +71,8 @@ def code_location_generate_command(
The `<name>.components` directory holds components (which can be created with `dg generate
component`). The `<name>.lib` directory holds custom component types scoped to the code
location (which can be created with `dg component-type generate`).
"""
dg_context = DgContext.from_cli_context(cli_context)
""" # noqa: D301
dg_context = DgContext.from_cli_global_options(global_options)
if is_inside_deployment_directory(Path.cwd()):
context = DeploymentDirectoryContext.from_path(Path.cwd(), dg_context)
if context.has_code_location(name):
Expand Down Expand Up @@ -101,10 +106,10 @@ def code_location_generate_command(


@code_location_group.command(name="list", cls=DgClickCommand)
@click.pass_context
def code_location_list_command(cli_context: click.Context) -> None:
@dg_global_options
def code_location_list_command(**global_options: object) -> None:
"""List code locations in the current deployment."""
dg_context = DgContext.from_cli_context(cli_context)
dg_context = DgContext.from_cli_global_options(global_options)
if not is_inside_deployment_directory(Path.cwd()):
click.echo(
click.style("This command must be run inside a Dagster deployment directory.", fg="red")
Expand Down
90 changes: 79 additions & 11 deletions python_modules/libraries/dagster-dg/dagster_dg/cli/component.py
Original file line number Diff line number Diff line change
@@ -1,11 +1,18 @@
import sys
from pathlib import Path
from typing import Any, Mapping, Optional
from typing import Any, List, Mapping, Optional

import click
from click.core import ParameterSource

from dagster_dg.cli.global_options import dg_global_options
from dagster_dg.component import RemoteComponentType
from dagster_dg.config import (
DgConfig,
get_config_from_cli_context,
has_config_on_cli_context,
set_config_on_cli_context,
)
from dagster_dg.context import (
CodeLocationDirectoryContext,
DgContext,
Expand All @@ -16,6 +23,7 @@
DgClickCommand,
DgClickGroup,
json_schema_property_to_click_option,
not_none,
parse_json_option,
)

Expand Down Expand Up @@ -48,14 +56,17 @@ def get_command(self, cli_context: click.Context, cmd_name: str) -> Optional[cli
self._define_commands(cli_context)
return super().get_command(cli_context, cmd_name)

def list_commands(self, cli_context):
def list_commands(self, cli_context: click.Context) -> List[str]:
if not self._commands_defined:
self._define_commands(cli_context)
return super().list_commands(cli_context)

def _define_commands(self, cli_context: click.Context) -> None:
"""Dynamically define a command for each registered component type."""
app_context = DgContext.from_cli_context(cli_context)
if not has_config_on_cli_context(cli_context):
cli_context.invoke(not_none(self.callback), **cli_context.params)
config = get_config_from_cli_context(cli_context)
dg_context = DgContext.from_config(config)

if not is_inside_code_location_directory(Path.cwd()):
click.echo(
Expand All @@ -65,21 +76,77 @@ def _define_commands(self, cli_context: click.Context) -> None:
)
sys.exit(1)

context = CodeLocationDirectoryContext.from_path(Path.cwd(), app_context)
context = CodeLocationDirectoryContext.from_path(Path.cwd(), dg_context)
for key, component_type in context.iter_component_types():
command = _create_component_generate_subcommand(key, component_type)
self.add_command(command)


@component_group.group(name="generate", cls=ComponentGenerateGroup)
def component_generate_group() -> None:
class ComponentGenerateSubCommand(DgClickCommand):
def format_usage(self, context: click.Context, formatter: click.HelpFormatter) -> None:
if not isinstance(self, click.Command):
raise ValueError("This mixin is only intended for use with click.Command instances.")
arg_pieces = self.collect_usage_pieces(context)
command_parts = context.command_path.split(" ")
command_parts.insert(-1, "[GLOBAL OPTIONS]")
return formatter.write_usage(" ".join(command_parts), " ".join(arg_pieces))

def format_options(self, context: click.Context, formatter: click.HelpFormatter) -> None:
# This will not produce any global options since there are none defined on component
# generate subcommands.
super().format_options(context, formatter)

# Get the global options off the parent group.
parent_context = not_none(context.parent)
parent_command = not_none(context.parent).command
if not isinstance(parent_command, DgClickGroup):
raise ValueError("Parent command must be a DgClickGroup.")
_, global_opts = parent_command.get_partitioned_opts(context)

with formatter.section("Global options"):
records = [not_none(p.get_help_record(parent_context)) for p in global_opts]
formatter.write_dl(records)


# We have to override the usual Click processing of `--help` here. The issue is
# that click will process this option before processing anything else, but because we are
# dynamically generating subcommands based on the content of other options, the output of --help
# actually depends on these other options. So we opt out of Click's short-circuiting
# behavior of `--help` by setting `help_option_names=[]`, ensuring that we can process the other
# options first and generate the correct subcommands. We then add a custom `--help` option that
# gets invoked inside the callback.
@component_group.group(
name="generate",
cls=ComponentGenerateGroup,
invoke_without_command=True,
context_settings={"help_option_names": []},
)
@click.option("-h", "--help", "help_", is_flag=True, help="Show this message and exit.")
@dg_global_options
@click.pass_context
def component_generate_group(context: click.Context, help_: bool, **global_options: object) -> None:
"""Generate a scaffold of a Dagster component."""
# Click attempts to resolve subcommands BEFORE it invokes this callback.
# Therefore we need to manually invoke this callback during subcommand generation to make sure
# it runs first. It will be invoked again later by Click. We make it idempotent to deal with
# that.
if not has_config_on_cli_context(context):
set_config_on_cli_context(context, DgConfig.from_cli_global_options(global_options))
if help_:
click.echo(context.get_help())
context.exit(0)


def _create_component_generate_subcommand(
component_key: str, component_type: RemoteComponentType
) -> DgClickCommand:
@click.command(name=component_key, cls=DgClickCommand)
# We need to "reset" the help option names to the default ones because we inherit the parent
# value of context settings from the parent group, which has been customized.
@click.command(
name=component_key,
cls=ComponentGenerateSubCommand,
context_settings={"help_option_names": ["-h", "--help"]},
)
@click.argument("component_name", type=str)
@click.option(
"--json-params",
Expand Down Expand Up @@ -112,7 +179,8 @@ def generate_component_command(
It is an error to pass both --json-params and key-value pairs as options.
"""
dg_context = DgContext.from_cli_context(cli_context)
config = get_config_from_cli_context(cli_context)
dg_context = DgContext.from_config(config)
if not is_inside_code_location_directory(Path.cwd()):
click.echo(
click.style(
Expand Down Expand Up @@ -185,10 +253,10 @@ def generate_component_command(


@component_group.command(name="list", cls=DgClickCommand)
@click.pass_context
def component_list_command(cli_context: click.Context) -> None:
@dg_global_options
def component_list_command(**global_options: object) -> None:
"""List Dagster component instances defined in the current code location."""
dg_context = DgContext.from_cli_context(cli_context)
dg_context = DgContext.from_cli_global_options(global_options)
if not is_inside_code_location_directory(Path.cwd()):
click.echo(
click.style(
Expand Down
Loading

0 comments on commit cee5d3e

Please sign in to comment.