diff --git a/python_modules/dagster-graphql/dagster_graphql/cli.py b/python_modules/dagster-graphql/dagster_graphql/cli.py index a07feb4e283cf..fa80a2b486dcb 100644 --- a/python_modules/dagster-graphql/dagster_graphql/cli.py +++ b/python_modules/dagster-graphql/dagster_graphql/cli.py @@ -8,7 +8,7 @@ import dagster._seven as seven import requests from dagster._cli.utils import get_instance_for_cli, get_temporary_instance_for_cli -from dagster._cli.workspace import workspace_target_argument +from dagster._cli.workspace import workspace_target_options from dagster._cli.workspace.cli_target import ( WORKSPACE_TARGET_WARNING, get_workspace_process_context_from_kwargs, @@ -127,7 +127,7 @@ def execute_query_against_remote(host, query, variables): } -@workspace_target_argument +@workspace_target_options @click.command( name="ui", help=( diff --git a/python_modules/dagster-webserver/dagster_webserver/cli.py b/python_modules/dagster-webserver/dagster_webserver/cli.py index d26fb9d075770..71a362d51c251 100644 --- a/python_modules/dagster-webserver/dagster_webserver/cli.py +++ b/python_modules/dagster-webserver/dagster_webserver/cli.py @@ -14,7 +14,7 @@ from dagster._cli.utils import get_possibly_temporary_instance_for_cli from dagster._cli.workspace import ( get_workspace_process_context_from_kwargs, - workspace_target_argument, + workspace_target_options, ) from dagster._cli.workspace.cli_target import WORKSPACE_TARGET_WARNING, ClickArgValue from dagster._core.instance import InstanceRef @@ -75,7 +75,7 @@ def create_dagster_webserver_cli(): """ ), ) -@workspace_target_argument +@workspace_target_options @click.option( "--host", "-h", diff --git a/python_modules/dagster/dagster/_cli/api.py b/python_modules/dagster/dagster/_cli/api.py index f6813ff172daa..2f26ba113e5b4 100644 --- a/python_modules/dagster/dagster/_cli/api.py +++ b/python_modules/dagster/dagster/_cli/api.py @@ -15,7 +15,7 @@ from dagster._cli.utils import get_instance_for_cli from dagster._cli.workspace.cli_target import ( get_working_directory_from_kwargs, - python_origin_target_argument, + python_origin_target_options, ) from dagster._core.definitions.metadata import MetadataValue from dagster._core.errors import DagsterExecutionInterruptedError @@ -601,7 +601,7 @@ def _execute_step_command_body( ), envvar="DAGSTER_LAZY_LOAD_USER_CODE", ) -@python_origin_target_argument +@python_origin_target_options @click.option( "--use-python-environment-entry-point", is_flag=True, diff --git a/python_modules/dagster/dagster/_cli/asset.py b/python_modules/dagster/dagster/_cli/asset.py index 554080c186f50..0add19010cf99 100644 --- a/python_modules/dagster/dagster/_cli/asset.py +++ b/python_modules/dagster/dagster/_cli/asset.py @@ -7,8 +7,8 @@ from dagster._cli.utils import get_instance_for_cli, get_possibly_temporary_instance_for_cli from dagster._cli.workspace.cli_target import ( get_repository_python_origin_from_kwargs, - python_job_config_argument, - python_origin_target_argument, + python_job_config_option, + python_origin_target_options, ) from dagster._core.definitions.asset_selection import AssetSelection from dagster._core.definitions.backfill_policy import BackfillPolicyType @@ -32,7 +32,7 @@ def asset_cli(): @asset_cli.command(name="materialize", help="Execute a run to materialize a selection of assets") -@python_origin_target_argument +@python_origin_target_options @click.option("--select", help="Comma-separated Asset selection to target", required=True) @click.option("--partition", help="Asset partition to target", required=False) @click.option( @@ -40,7 +40,7 @@ def asset_cli(): help="Asset partition range to target i.e. ...", required=False, ) -@python_job_config_argument("materialize") +@python_job_config_option(command_name="materialize") @click.option( "--config-json", type=click.STRING, @@ -157,7 +157,7 @@ def execute_materialize_command(instance: DagsterInstance, kwargs: Mapping[str, @asset_cli.command(name="list", help="List assets") -@python_origin_target_argument +@python_origin_target_options @click.option("--select", help="Asset selection to target", required=False) def asset_list_command(**kwargs): repository_origin = get_repository_python_origin_from_kwargs(kwargs) diff --git a/python_modules/dagster/dagster/_cli/code_server.py b/python_modules/dagster/dagster/_cli/code_server.py index 8174276f7b618..78e7b4216e066 100644 --- a/python_modules/dagster/dagster/_cli/code_server.py +++ b/python_modules/dagster/dagster/_cli/code_server.py @@ -11,7 +11,7 @@ import dagster._seven as seven from dagster._cli.workspace.cli_target import ( get_working_directory_from_kwargs, - python_origin_target_argument, + python_origin_target_options, ) from dagster._core.instance import InstanceRef from dagster._core.types.loadable_target_origin import LoadableTargetOrigin @@ -66,7 +66,7 @@ def code_server_cli(): help="Maximum number of (threaded) workers to use in the code server", envvar="DAGSTER_CODE_SERVER_MAX_WORKERS", ) -@python_origin_target_argument +@python_origin_target_options @click.option( "--use-python-environment-entry-point", is_flag=True, diff --git a/python_modules/dagster/dagster/_cli/definitions.py b/python_modules/dagster/dagster/_cli/definitions.py index d72cc566f3cae..e6f3c11ced86e 100644 --- a/python_modules/dagster/dagster/_cli/definitions.py +++ b/python_modules/dagster/dagster/_cli/definitions.py @@ -9,10 +9,10 @@ from dagster._cli.utils import get_possibly_temporary_instance_for_cli from dagster._cli.workspace.cli_target import ( ClickArgValue, + generate_module_name_option, + generate_python_file_option, + generate_workspace_option, get_workspace_from_kwargs, - python_file_option, - python_module_option, - workspace_option, ) from dagster._utils.log import configure_loggers @@ -25,9 +25,9 @@ def definitions_cli(): def validate_command_options(f): return apply_click_params( f, - workspace_option(), - python_file_option(allow_multiple=True), - python_module_option(allow_multiple=True), + generate_workspace_option(), + generate_python_file_option(allow_multiple=True), + generate_module_name_option(allow_multiple=True), ) diff --git a/python_modules/dagster/dagster/_cli/dev.py b/python_modules/dagster/dagster/_cli/dev.py index 13328ec548008..eeab51eb91131 100644 --- a/python_modules/dagster/dagster/_cli/dev.py +++ b/python_modules/dagster/dagster/_cli/dev.py @@ -18,12 +18,12 @@ from dagster._cli.utils import get_possibly_temporary_instance_for_cli from dagster._cli.workspace.cli_target import ( ClickArgValue, + generate_grpc_server_target_options, + generate_module_name_option, + generate_python_file_option, + generate_working_directory_option, + generate_workspace_option, get_workspace_load_target, - grpc_server_target_click_options, - python_file_option, - python_module_option, - working_directory_option, - workspace_option, ) from dagster._core.instance import DagsterInstance from dagster._core.workspace.context import WorkspaceProcessContext @@ -39,11 +39,11 @@ def dev_command_options(f): return apply_click_params( f, - workspace_option(), - python_file_option(allow_multiple=True), - python_module_option(allow_multiple=True), - working_directory_option(), - *grpc_server_target_click_options(hidden=True), + generate_workspace_option(), + generate_python_file_option(allow_multiple=True), + generate_module_name_option(allow_multiple=True), + generate_working_directory_option(), + *generate_grpc_server_target_options(hidden=True), ) diff --git a/python_modules/dagster/dagster/_cli/job.py b/python_modules/dagster/dagster/_cli/job.py index 4c3aa7ac3c3ef..5d485cad3ead2 100644 --- a/python_modules/dagster/dagster/_cli/job.py +++ b/python_modules/dagster/dagster/_cli/job.py @@ -25,10 +25,10 @@ get_remote_repository_from_kwargs, get_run_config_from_file_list, get_workspace_from_kwargs, - job_repository_target_argument, - job_target_argument, - python_job_config_argument, - python_job_target_argument, + job_repository_target_options, + job_target_options, + python_job_config_option, + python_job_target_options, ) from dagster._core.definitions import JobDefinition from dagster._core.definitions.reconstruct import ReconstructableJob @@ -84,7 +84,7 @@ def apply_click_params(command: T_Callable, *click_params: ClickOption) -> T_Cal name="list", help=f"List the jobs in a repository. {WORKSPACE_TARGET_WARNING}", ) -@job_repository_target_argument +@job_repository_target_options def job_list_command(**kwargs): return execute_list_command(kwargs, click.echo) @@ -141,7 +141,7 @@ def get_job_instructions(command_name): help="Print a job.\n\n{instructions}".format(instructions=get_job_instructions("print")), ) @click.option("--verbose", is_flag=True) -@job_target_argument +@job_target_options def job_print_command(verbose, **cli_args): with get_possibly_temporary_instance_for_cli("``dagster job print``") as instance: return execute_print_command(instance, verbose, cli_args, click.echo) @@ -244,8 +244,8 @@ def print_op( instructions=get_job_in_same_python_env_instructions("execute") ), ) -@python_job_target_argument -@python_job_config_argument("execute") +@python_job_target_options +@python_job_config_option(command_name="execute") @click.option("--tags", type=click.STRING, help="JSON string of tags to use for this job run") @click.option( "-o", @@ -342,8 +342,8 @@ def do_execute_command( ) ), ) -@job_target_argument -@python_job_config_argument("launch") +@job_target_options +@python_job_config_option(command_name="launch") @click.option( "--config-json", type=click.STRING, @@ -497,7 +497,7 @@ def _check_execute_remote_job_args( instructions=get_job_in_same_python_env_instructions("scaffold_config") ), ) -@python_job_target_argument +@python_job_target_options @click.option("--print-only-required", default=False, is_flag=True) def job_scaffold_command(**kwargs): execute_scaffold_command(kwargs, click.echo) @@ -530,7 +530,7 @@ def do_scaffold_command( instructions=get_job_instructions("backfill") ), ) -@job_target_argument +@job_target_options @click.option( "--partitions", type=click.STRING, diff --git a/python_modules/dagster/dagster/_cli/run.py b/python_modules/dagster/dagster/_cli/run.py index 189519cdb4ebe..fe911d1d618ef 100644 --- a/python_modules/dagster/dagster/_cli/run.py +++ b/python_modules/dagster/dagster/_cli/run.py @@ -5,7 +5,7 @@ from dagster import __version__ as dagster_version from dagster._cli.utils import get_instance_for_cli -from dagster._cli.workspace.cli_target import get_remote_job_from_kwargs, job_target_argument +from dagster._cli.workspace.cli_target import get_remote_job_from_kwargs, job_target_options @click.group(name="run") @@ -85,7 +85,7 @@ def run_wipe_command(force: bool) -> None: "from_label", help="The repository from which to migrate (format: @)", ) -@job_target_argument +@job_target_options def run_migrate_command(from_label: str, **kwargs: Any) -> None: from dagster._core.storage.dagster_run import RunsFilter from dagster._core.storage.runs.sql_run_storage import SqlRunStorage diff --git a/python_modules/dagster/dagster/_cli/schedule.py b/python_modules/dagster/dagster/_cli/schedule.py index 1e9d7f5386903..844061bbab716 100644 --- a/python_modules/dagster/dagster/_cli/schedule.py +++ b/python_modules/dagster/dagster/_cli/schedule.py @@ -13,7 +13,7 @@ from dagster._cli.utils import get_instance_for_cli from dagster._cli.workspace.cli_target import ( get_remote_repository_from_kwargs, - repository_target_argument, + repository_target_options, ) from dagster._core.definitions.run_request import InstigatorType from dagster._core.instance import DagsterInstance @@ -140,7 +140,7 @@ def check_repo_and_scheduler(repository: RemoteRepository, instance: DagsterInst @schedule_cli.command( name="preview", help="Preview changes that will be performed by `dagster schedule up`." ) -@repository_target_argument +@repository_target_options def schedule_preview_command(**kwargs): return execute_preview_command(kwargs, click.echo) @@ -159,7 +159,7 @@ def execute_preview_command(cli_args, print_fn): name="list", help="List all schedules that correspond to a repository.", ) -@repository_target_argument +@repository_target_options @click.option("--running", help="Filter for running schedules", is_flag=True, default=False) @click.option("--stopped", help="Filter for stopped schedules", is_flag=True, default=False) @click.option("--name", help="Only display schedule schedule names", is_flag=True, default=False) @@ -230,7 +230,7 @@ def extract_schedule_name(schedule_name: Optional[Union[str, Sequence[str]]]) -> @schedule_cli.command(name="start", help="Start an existing schedule.") @click.argument("schedule_name", nargs=-1) # , required=True) @click.option("--start-all", help="start all schedules", is_flag=True, default=False) -@repository_target_argument +@repository_target_options def schedule_start_command(schedule_name, start_all, **kwargs): schedule_name = extract_schedule_name(schedule_name) if schedule_name is None and start_all is False: @@ -270,7 +270,7 @@ def execute_start_command(schedule_name, all_flag, cli_args, print_fn): @schedule_cli.command(name="stop", help="Stop an existing schedule.") @click.argument("schedule_name", nargs=-1) -@repository_target_argument +@repository_target_options def schedule_stop_command(schedule_name, **kwargs): schedule_name = extract_schedule_name(schedule_name) return execute_stop_command(schedule_name, kwargs, click.echo) @@ -298,7 +298,7 @@ def execute_stop_command(schedule_name, cli_args, print_fn, instance=None): @schedule_cli.command(name="logs", help="Get logs for a schedule.") @click.argument("schedule_name", nargs=-1) -@repository_target_argument +@repository_target_options def schedule_logs_command(schedule_name, **kwargs): schedule_name = extract_schedule_name(schedule_name) if schedule_name is None: @@ -370,7 +370,7 @@ def execute_logs_command(schedule_name, cli_args, print_fn, instance=None): is_flag=True, default=False, ) -@repository_target_argument +@repository_target_options def schedule_restart_command(schedule_name, restart_all_running, **kwargs): schedule_name = extract_schedule_name(schedule_name) return execute_restart_command(schedule_name, restart_all_running, kwargs, click.echo) diff --git a/python_modules/dagster/dagster/_cli/sensor.py b/python_modules/dagster/dagster/_cli/sensor.py index 995b20dfcb45f..22ecc19dce09d 100644 --- a/python_modules/dagster/dagster/_cli/sensor.py +++ b/python_modules/dagster/dagster/_cli/sensor.py @@ -14,7 +14,7 @@ get_code_location_from_kwargs, get_remote_repository_from_code_location, get_remote_repository_from_kwargs, - repository_target_argument, + repository_target_options, ) from dagster._core.definitions.run_request import InstigatorType from dagster._core.instance import DagsterInstance @@ -123,7 +123,7 @@ def extract_sensor_name(sensor_name): name="list", help="List all sensors that correspond to a repository.", ) -@repository_target_argument +@repository_target_options @click.option("--running", help="Filter for running sensors", is_flag=True, default=False) @click.option("--stopped", help="Filter for stopped sensors", is_flag=True, default=False) @click.option("--name", help="Only display sensor sensor names", is_flag=True, default=False) @@ -183,7 +183,7 @@ def execute_list_command(running_filter, stopped_filter, name_filter, cli_args, @sensor_cli.command(name="start", help="Start an existing sensor.") @click.argument("sensor_name", nargs=-1) # , required=True) @click.option("--start-all", help="start all sensors", is_flag=True, default=False) -@repository_target_argument +@repository_target_options def sensor_start_command(sensor_name, start_all, **kwargs): if not start_all: sensor_name = extract_sensor_name(sensor_name) @@ -217,7 +217,7 @@ def execute_start_command(sensor_name, all_flag, cli_args, print_fn): @sensor_cli.command(name="stop", help="Stop an existing sensor.") @click.argument("sensor_name", nargs=-1) -@repository_target_argument +@repository_target_options def sensor_stop_command(sensor_name, **kwargs): sensor_name = extract_sensor_name(sensor_name) return execute_stop_command(sensor_name, kwargs, click.echo) @@ -259,7 +259,7 @@ def execute_stop_command(sensor_name, cli_args, print_fn): help="Set the cursor value for the sensor context", default=None, ) -@repository_target_argument +@repository_target_options def sensor_preview_command(sensor_name, since, last_run_key, cursor, **kwargs): sensor_name = extract_sensor_name(sensor_name) if since: @@ -332,7 +332,7 @@ def execute_preview_command( @click.option( "--delete", help="Delete the existing cursor value for the sensor context", is_flag=True ) -@repository_target_argument +@repository_target_options def sensor_cursor_command(sensor_name, **kwargs): sensor_name = extract_sensor_name(sensor_name) return execute_cursor_command(sensor_name, kwargs, click.echo) diff --git a/python_modules/dagster/dagster/_cli/workspace/__init__.py b/python_modules/dagster/dagster/_cli/workspace/__init__.py index ebbd0e8d18eda..474dc6d183b5f 100644 --- a/python_modules/dagster/dagster/_cli/workspace/__init__.py +++ b/python_modules/dagster/dagster/_cli/workspace/__init__.py @@ -1,4 +1,4 @@ from dagster._cli.workspace.cli_target import ( get_workspace_process_context_from_kwargs as get_workspace_process_context_from_kwargs, - workspace_target_argument as workspace_target_argument, + workspace_target_options as workspace_target_options, ) diff --git a/python_modules/dagster/dagster/_cli/workspace/cli_target.py b/python_modules/dagster/dagster/_cli/workspace/cli_target.py index 919c570ed11f3..7ee9e90258565 100644 --- a/python_modules/dagster/dagster/_cli/workspace/cli_target.py +++ b/python_modules/dagster/dagster/_cli/workspace/cli_target.py @@ -284,7 +284,15 @@ def get_workspace_from_kwargs( yield workspace_process_context.create_request_context() -def python_file_option(allow_multiple: bool) -> ClickOption: +# ######################## +# ##### OPTION GENERATORS +# ######################## + +# These are named as generate_*_option(s) and return a ClickOption or list of Click Options. These +# cannot be directly applied to click commands/groups as decorators. + + +def generate_python_file_option(allow_multiple: bool) -> ClickOption: return click.option( "--python-file", "-f", @@ -303,7 +311,7 @@ def python_file_option(allow_multiple: bool) -> ClickOption: ) -def workspace_option() -> ClickOption: +def generate_workspace_option() -> ClickOption: return click.option( "--workspace", "-w", @@ -313,7 +321,7 @@ def workspace_option() -> ClickOption: ) -def python_module_option(allow_multiple: bool) -> ClickOption: +def generate_module_name_option(allow_multiple: bool) -> ClickOption: return click.option( "--module-name", "-m", @@ -329,7 +337,7 @@ def python_module_option(allow_multiple: bool) -> ClickOption: ) -def working_directory_option() -> ClickOption: +def generate_working_directory_option() -> ClickOption: return click.option( "--working-directory", "-d", @@ -338,11 +346,54 @@ def working_directory_option() -> ClickOption: ) -def python_target_click_options(allow_multiple_python_targets: bool) -> Sequence[ClickOption]: +def generate_job_name_option() -> ClickOption: + return click.option( + "--job", + "-j", + "job_name", + help="Job within the repository, necessary if more than one job is present.", + ) + + +def generate_repository_name_option() -> ClickOption: + return click.option( + "--repository", + "-r", + help=( + "Name of the repository, necessary if more than one repository is present in the" + " code location." + ), + ) + + +def generate_run_config_option(command_name: str) -> ClickOption: + return click.option( + "-c", + "--config", + type=click.Path(exists=True), + multiple=True, + help=( + "Specify one or more run config files. These can also be file patterns. " + "If more than one run config file is captured then those files are merged. " + "Files listed first take precedence. They will smash the values of subsequent " + "files at the key-level granularity. If the file is a pattern then you must " + "enclose it in double quotes" + "\n\nExample: " + f"dagster job {command_name} -f hello_world.py -j pandas_hello_world " + '-c "pandas_hello_world/*.yaml"' + "\n\nYou can also specify multiple files:" + "\n\nExample: " + f"dagster job {command_name} -f hello_world.py -j pandas_hello_world " + "-c pandas_hello_world/ops.yaml -c pandas_hello_world/env.yaml" + ), + ) + + +def generate_python_target_options(allow_multiple_python_targets: bool) -> Sequence[ClickOption]: return [ - working_directory_option(), - python_file_option(allow_multiple=allow_multiple_python_targets), - python_module_option(allow_multiple=allow_multiple_python_targets), + generate_working_directory_option(), + generate_python_file_option(allow_multiple=allow_multiple_python_targets), + generate_module_name_option(allow_multiple=allow_multiple_python_targets), click.option( "--package-name", help="Specify Python package where repository or job function lives", @@ -360,7 +411,7 @@ def python_target_click_options(allow_multiple_python_targets: bool) -> Sequence ] -def grpc_server_target_click_options(hidden=False) -> Sequence[ClickOption]: +def generate_grpc_server_target_options(hidden=False) -> Sequence[ClickOption]: return [ click.option( "--grpc-port", @@ -393,132 +444,114 @@ def grpc_server_target_click_options(hidden=False) -> Sequence[ClickOption]: ] -def workspace_target_click_options() -> Sequence[ClickOption]: +def generate_workspace_target_options() -> Sequence[ClickOption]: return [ click.option("--empty-workspace", is_flag=True, help="Allow an empty workspace"), - workspace_option(), - *python_target_click_options(allow_multiple_python_targets=True), - *grpc_server_target_click_options(), + generate_workspace_option(), + *generate_python_target_options(allow_multiple_python_targets=True), + *generate_grpc_server_target_options(), ] -def python_job_target_click_options() -> Sequence[ClickOption]: +def generate_python_job_target_options() -> Sequence[ClickOption]: return [ - *python_target_click_options(allow_multiple_python_targets=False), + *generate_python_target_options(allow_multiple_python_targets=False), click.option( "--repository", "-r", help="Repository name, necessary if more than one repository is present.", ), - job_option(), + generate_job_name_option(), ] -def target_with_config_option(command_name: str) -> ClickOption: - return click.option( - "-c", - "--config", - type=click.Path(exists=True), - multiple=True, - help=( - "Specify one or more run config files. These can also be file patterns. " - "If more than one run config file is captured then those files are merged. " - "Files listed first take precedence. They will smash the values of subsequent " - "files at the key-level granularity. If the file is a pattern then you must " - "enclose it in double quotes" - "\n\nExample: " - f"dagster job {command_name} -f hello_world.py -j pandas_hello_world " - '-c "pandas_hello_world/*.yaml"' - "\n\nYou can also specify multiple files:" - "\n\nExample: " - f"dagster job {command_name} -f hello_world.py -j pandas_hello_world " - "-c pandas_hello_world/ops.yaml -c pandas_hello_world/env.yaml" +def generate_repository_identifier_options() -> Sequence[ClickOption]: + return [ + generate_repository_name_option(), + click.option( + "--location", + "-l", + help="Name of the code location, necessary if more than one location is present.", ), - ) + ] + + +# ######################## +# ##### USABLE AS CLICK DECORATORS +# ######################## +# These are named as *_options and can be directly applied to click commands/groups as decorators. +# They contain various subsets from the generate_* -def python_job_config_argument(command_name: str) -> Callable[[T_Callable], T_Callable]: + +def python_job_config_option(*, command_name: str) -> Callable[[T_Callable], T_Callable]: def wrap(f: T_Callable) -> T_Callable: - return target_with_config_option(command_name)(f) + from dagster._cli.job import apply_click_params + + return apply_click_params(f, generate_run_config_option(command_name)) return wrap -def python_job_target_argument(f): +def python_job_target_options(f: T_Callable) -> T_Callable: from dagster._cli.job import apply_click_params - return apply_click_params(f, *python_job_target_click_options()) + return apply_click_params(f, *generate_python_job_target_options()) -def workspace_target_argument(f): +def workspace_target_options(f: T_Callable) -> T_Callable: from dagster._cli.job import apply_click_params - return apply_click_params(f, *workspace_target_click_options()) + return apply_click_params(f, *generate_workspace_target_options()) -def job_workspace_target_argument(f): +def job_workspace_target_options(f: T_Callable) -> T_Callable: from dagster._cli.job import apply_click_params - return apply_click_params(f, *workspace_target_click_options()) + return apply_click_params(f, *generate_workspace_target_options()) -def grpc_server_origin_target_argument(f): +def grpc_server_origin_target_options(f: T_Callable) -> T_Callable: from dagster._cli.job import apply_click_params - options = grpc_server_target_click_options() - return apply_click_params(f, *options) + return apply_click_params(f, *generate_grpc_server_target_options()) -def python_origin_target_argument(f): +def python_origin_target_options(f: T_Callable) -> T_Callable: from dagster._cli.job import apply_click_params - options = python_target_click_options(allow_multiple_python_targets=False) - return apply_click_params(f, *options) - - -def repository_click_options() -> Sequence[ClickOption]: - return [ - click.option( - "--repository", - "-r", - help=( - "Name of the repository, necessary if more than one repository is present in the" - " code location." - ), - ), - click.option( - "--location", - "-l", - help="Name of the code location, necessary if more than one location is present.", - ), - ] + return apply_click_params( + f, *generate_python_target_options(allow_multiple_python_targets=False) + ) -def repository_target_argument(f): +def repository_target_options(f: T_Callable) -> T_Callable: from dagster._cli.job import apply_click_params - return apply_click_params(workspace_target_argument(f), *repository_click_options()) + return apply_click_params( + f, *generate_workspace_target_options(), *generate_repository_identifier_options() + ) -def job_repository_target_argument(f: T_Callable) -> T_Callable: +def job_repository_target_options(f: T_Callable) -> T_Callable: from dagster._cli.job import apply_click_params - return apply_click_params(job_workspace_target_argument(f), *repository_click_options()) - - -def job_option() -> ClickOption: - return click.option( - "--job", - "-j", - "job_name", - help="Job within the repository, necessary if more than one job is present.", - ) + options = [ + *generate_workspace_target_options(), + *generate_repository_identifier_options(), + ] + return apply_click_params(f, *options) -def job_target_argument(f: T_Callable) -> T_Callable: +def job_target_options(f: T_Callable) -> T_Callable: from dagster._cli.job import apply_click_params - return apply_click_params(job_repository_target_argument(f), job_option()) + options = [ + *generate_workspace_target_options(), + *generate_repository_identifier_options(), + generate_job_name_option(), + ] + return apply_click_params(f, *options) def get_job_python_origin_from_kwargs(kwargs: ClickArgMapping) -> JobPythonOrigin: diff --git a/python_modules/dagster/dagster/_daemon/cli/__init__.py b/python_modules/dagster/dagster/_daemon/cli/__init__.py index d46bb0bd60a64..f70963d2347ed 100644 --- a/python_modules/dagster/dagster/_daemon/cli/__init__.py +++ b/python_modules/dagster/dagster/_daemon/cli/__init__.py @@ -10,7 +10,7 @@ ClickArgMapping, ClickArgValue, get_workspace_load_target, - workspace_target_argument, + workspace_target_options, ) from dagster._core.instance import DagsterInstance, InstanceRef from dagster._core.telemetry import telemetry_wrapper @@ -67,7 +67,7 @@ def _get_heartbeat_tolerance(): required=False, hidden=True, ) -@workspace_target_argument +@workspace_target_options def run_command( code_server_log_level: str, log_level: str, diff --git a/python_modules/dagster/dagster_tests/cli_tests/workspace_tests/test_job_load.py b/python_modules/dagster/dagster_tests/cli_tests/workspace_tests/test_job_load.py index 7932097625bc3..9dab40dd017b0 100644 --- a/python_modules/dagster/dagster_tests/cli_tests/workspace_tests/test_job_load.py +++ b/python_modules/dagster/dagster_tests/cli_tests/workspace_tests/test_job_load.py @@ -1,7 +1,7 @@ import click import pytest from click.testing import CliRunner -from dagster._cli.workspace.cli_target import get_remote_job_from_kwargs, job_target_argument +from dagster._cli.workspace.cli_target import get_remote_job_from_kwargs, job_target_options from dagster._core.instance import DagsterInstance from dagster._core.remote_representation import RemoteJob from dagster._core.test_utils import instance_for_test @@ -12,7 +12,7 @@ def load_pipeline_via_cli_runner(cli_args): capture_result = {"external_pipeline": None} @click.command(name="test_pipeline_command") - @job_target_argument + @job_target_options def command(**kwargs): with get_remote_job_from_kwargs(DagsterInstance.get(), "", kwargs) as remote_job: capture_result["external_pipeline"] = remote_job # pyright: ignore[reportArgumentType] diff --git a/python_modules/dagster/dagster_tests/cli_tests/workspace_tests/test_repository_load.py b/python_modules/dagster/dagster_tests/cli_tests/workspace_tests/test_repository_load.py index 0291e596ab924..62867b923e346 100644 --- a/python_modules/dagster/dagster_tests/cli_tests/workspace_tests/test_repository_load.py +++ b/python_modules/dagster/dagster_tests/cli_tests/workspace_tests/test_repository_load.py @@ -4,7 +4,7 @@ from dagster._cli.workspace.cli_target import ( get_remote_repository_from_kwargs, get_workspace_from_kwargs, - repository_target_argument, + repository_target_options, ) from dagster._core.instance import DagsterInstance from dagster._core.remote_representation import RemoteRepository @@ -15,7 +15,7 @@ def load_repository_via_cli_runner(cli_args, repo_assert_fn=None): @click.command(name="test_repository_command") - @repository_target_argument + @repository_target_options def command(**kwargs): with get_remote_repository_from_kwargs( DagsterInstance.get(), @@ -34,7 +34,7 @@ def command(**kwargs): def load_workspace_via_cli_runner(cli_args, workspace_assert_fn=None): @click.command(name="test_workspace_command") - @repository_target_argument + @repository_target_options def command(**kwargs): with get_workspace_from_kwargs( DagsterInstance.get(), diff --git a/python_modules/libraries/dagster-sigma/dagster_sigma/cli.py b/python_modules/libraries/dagster-sigma/dagster_sigma/cli.py index f234e51c6c991..5d51e45523b01 100644 --- a/python_modules/libraries/dagster-sigma/dagster_sigma/cli.py +++ b/python_modules/libraries/dagster-sigma/dagster_sigma/cli.py @@ -2,7 +2,7 @@ from dagster import _check as check from dagster._cli.workspace.cli_target import ( get_repository_python_origin_from_kwargs, - python_origin_target_argument, + python_origin_target_options, ) from dagster._core.definitions.definitions_load_context import ( DefinitionsLoadContext, @@ -24,7 +24,7 @@ def app(): @app.command(name="snapshot", help="Snapshot sigma instance data") -@python_origin_target_argument +@python_origin_target_options @click.option("--output-path", "-o", help="Path to save the snapshot to", required=True) def sigma_snapshot_command(**kwargs) -> None: experimental_warning("The `dagster-sigma snapshot` command")