From 5f1de9b9f7c44608d4943fe74aa3af6eb043bc27 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 27 Aug 2024 12:49:53 -0700 Subject: [PATCH] make job_name param on PartitionArgs and PartitionNames args optional --- python_modules/dagster/dagster/_grpc/types.py | 12 +++++----- .../api_tests/test_api_snapshot_partition.py | 22 +++++++++++++++++++ 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/python_modules/dagster/dagster/_grpc/types.py b/python_modules/dagster/dagster/_grpc/types.py index 8feacb565a487..c7a23bf37b168 100644 --- a/python_modules/dagster/dagster/_grpc/types.py +++ b/python_modules/dagster/dagster/_grpc/types.py @@ -408,10 +408,10 @@ class PartitionArgs( "_PartitionArgs", [ ("repository_origin", RemoteRepositoryOrigin), - ("job_name", str), # This is here for backcompat. it's expected to always be f"{job_name}_partition_set". ("partition_set_name", str), ("partition_name", str), + ("job_name", Optional[str]), ("instance_ref", Optional[InstanceRef]), # This is introduced in the same release that we're making it possible for an asset job # to target assets with different PartitionsDefinitions. Prior user code versions can @@ -424,9 +424,9 @@ class PartitionArgs( def __new__( cls, repository_origin: RemoteRepositoryOrigin, - job_name: str, partition_set_name: str, partition_name: str, + job_name: Optional[str] = None, instance_ref: Optional[InstanceRef] = None, selected_asset_keys: Optional[AbstractSet[AssetKey]] = None, ): @@ -438,7 +438,7 @@ def __new__( RemoteRepositoryOrigin, ), partition_set_name=check.str_param(partition_set_name, "partition_set_name"), - job_name=check.str_param(job_name, "job_name"), + job_name=check.opt_str_param(job_name, "job_name"), partition_name=check.str_param(partition_name, "partition_name"), instance_ref=check.opt_inst_param(instance_ref, "instance_ref", InstanceRef), selected_asset_keys=check.opt_nullable_set_param( @@ -459,13 +459,13 @@ class PartitionNamesArgs( "_PartitionNamesArgs", [ ("repository_origin", RemoteRepositoryOrigin), - ("job_name", str), # This is here for backcompat. it's expected to always be f"{job_name}_partition_set". ("partition_set_name", str), # This is introduced in the same release that we're making it possible for an asset job # to target assets with different PartitionsDefinitions. Prior user code versions can # (and do) safely ignore this parameter, because, in those versions, the job name on its # own is enough to specify which PartitionsDefinition to use. + ("job_name", Optional[str]), ("selected_asset_keys", Optional[AbstractSet[AssetKey]]), ], ) @@ -473,8 +473,8 @@ class PartitionNamesArgs( def __new__( cls, repository_origin: RemoteRepositoryOrigin, - job_name: str, partition_set_name: str, + job_name: Optional[str] = None, selected_asset_keys: Optional[AbstractSet[AssetKey]] = None, ): return super(PartitionNamesArgs, cls).__new__( @@ -482,7 +482,7 @@ def __new__( repository_origin=check.inst_param( repository_origin, "repository_origin", RemoteRepositoryOrigin ), - job_name=check.str_param(job_name, "job_name"), + job_name=check.opt_str_param(job_name, "job_name"), partition_set_name=check.str_param(partition_set_name, "partition_set_name"), selected_asset_keys=check.opt_nullable_set_param( selected_asset_keys, "selected_asset_keys", of_type=AssetKey diff --git a/python_modules/dagster/dagster_tests/api_tests/test_api_snapshot_partition.py b/python_modules/dagster/dagster_tests/api_tests/test_api_snapshot_partition.py index 5187bb7336355..83ba2fb038207 100644 --- a/python_modules/dagster/dagster_tests/api_tests/test_api_snapshot_partition.py +++ b/python_modules/dagster/dagster_tests/api_tests/test_api_snapshot_partition.py @@ -340,3 +340,25 @@ def test_dynamic_partition_set_grpc(instance: DagsterInstance): ) assert isinstance(data, ExternalPartitionSetExecutionParamData) assert data.partition_data == [] + + +def test_external_partition_tags_grpc_backcompat_no_job_name(instance: DagsterInstance): + with get_bar_repo_code_location(instance) as code_location: + repository_handle = code_location.get_repository("bar_repo").handle + + api_client = code_location.client + + result = deserialize_value( + api_client.external_partition_tags( + partition_args=PartitionArgs( + repository_origin=repository_handle.get_external_origin(), + partition_set_name="baz_partition_set", + partition_name="c", + instance_ref=instance.get_ref(), + ) + ) + ) + + assert isinstance(result, ExternalPartitionTagsData) + assert result.tags + assert result.tags["foo"] == "bar"