From 1b2eefe004a8103f0094899a47b83ca3b28207d6 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 14 Apr 2025 15:45:06 -0400 Subject: [PATCH 01/11] [gql] latest event id resolver for Asset so that we can order by id --- .../schema/pipelines/pipeline.py | 8 ++++++++ .../dagster/_core/storage/event_log/base.py | 18 ++++++++++++++++++ 2 files changed, 26 insertions(+) diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py index 83d564ae42600..5ca3bf2445417 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py @@ -21,6 +21,7 @@ RunRecord, RunsFilter, ) +from dagster._core.storage.event_log.base import AssetRecord from dagster._core.storage.tags import REPOSITORY_LABEL_TAG, RUN_METRIC_TAGS, TagType, get_tag_type from dagster._core.workspace.permissions import Permissions from dagster._utils.tags import get_boolean_tag_value @@ -247,6 +248,7 @@ class GrapheneAsset(graphene.ObjectType): cursor=graphene.String(), ) definition = graphene.Field("dagster_graphql.schema.asset_graph.GrapheneAssetNode") + latestEventId = graphene.Field(graphene.Int) class Meta: name = "Asset" @@ -389,6 +391,12 @@ def resolve_assetObservations( ) ] + async def resolve_LatestEventId(self, graphene_info): + asset_record = await AssetRecord.gen(graphene_info.context, self.key) + if asset_record: + return asset_record.asset_entry.last_event_storage_id + return None + class GrapheneEventConnection(graphene.ObjectType): class Meta: diff --git a/python_modules/dagster/dagster/_core/storage/event_log/base.py b/python_modules/dagster/dagster/_core/storage/event_log/base.py index d1c173941c02a..cb401e80060a7 100644 --- a/python_modules/dagster/dagster/_core/storage/event_log/base.py +++ b/python_modules/dagster/dagster/_core/storage/event_log/base.py @@ -133,6 +133,12 @@ def last_observation(self) -> Optional["EventLogEntry"]: return None return self.last_observation_record.event_log_entry + @property + def last_observation_storage_id(self) -> Optional[int]: + if self.last_observation_record is None: + return None + return self.last_observation_record.storage_id + @property def last_materialization_storage_id(self) -> Optional[int]: if self.last_materialization_record is None: @@ -151,6 +157,18 @@ def last_failed_to_materialize_storage_id(self) -> Optional[int]: return None return self.last_failed_to_materialize_record.storage_id + @property + def last_event_storage_id(self) -> Optional[int]: + """Get the storage id of the latest event for this asset.""" + event_ids = [ + self.last_materialization_storage_id, + self.last_observation_storage_id, + self.last_failed_to_materialize_storage_id, + self.last_planned_materialization_storage_id, + ] + event_ids = [event_id for event_id in event_ids if event_id is not None] + return max(event_ids) if event_ids else None + class AssetRecord( NamedTuple("_NamedTuple", [("storage_id", int), ("asset_entry", AssetEntry)]), From eb9a9752dc88fb24841061167e924a29062c4c8d Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 15 Apr 2025 10:41:07 -0400 Subject: [PATCH 02/11] update name --- .../dagster-ui/packages/ui-core/src/graphql/schema.graphql | 1 + js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts | 5 +++++ .../dagster_graphql/schema/pipelines/pipeline.py | 4 ++-- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql b/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql index c504c0e1d80f5..fef19e0522240 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql @@ -627,6 +627,7 @@ type Asset { cursor: String ): MaterializationHistoryConnection! definition: AssetNode + latestEventSortKey: Int } type MaterializationHistoryConnection { diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts b/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts index ad0a6f9e8ee4c..245e319d82d58 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts @@ -106,6 +106,7 @@ export type Asset = { definition: Maybe; id: Scalars['String']['output']; key: AssetKey; + latestEventSortKey: Maybe; }; export type AssetAssetMaterializationHistoryArgs = { @@ -6201,6 +6202,10 @@ export const buildAsset = ( : relationshipsToOmit.has('AssetKey') ? ({} as AssetKey) : buildAssetKey({}, relationshipsToOmit), + latestEventSortKey: + overrides && overrides.hasOwnProperty('latestEventSortKey') + ? overrides.latestEventSortKey! + : 6449, }; }; diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py index 5ca3bf2445417..a9938d283ed13 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py @@ -248,7 +248,7 @@ class GrapheneAsset(graphene.ObjectType): cursor=graphene.String(), ) definition = graphene.Field("dagster_graphql.schema.asset_graph.GrapheneAssetNode") - latestEventId = graphene.Field(graphene.Int) + latestEventSortKey = graphene.Field(graphene.Int) class Meta: name = "Asset" @@ -391,7 +391,7 @@ def resolve_assetObservations( ) ] - async def resolve_LatestEventId(self, graphene_info): + async def resolve_latestEventSortKey(self, graphene_info): asset_record = await AssetRecord.gen(graphene_info.context, self.key) if asset_record: return asset_record.asset_entry.last_event_storage_id From 6696f3a73ac6723423e495e8eb8574869230c7b9 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Tue, 15 Apr 2025 11:58:47 -0400 Subject: [PATCH 03/11] tests --- .../dagster_graphql_tests/graphql/repo.py | 30 +++ .../graphql/test_assets.py | 203 +++++++++++++++++- 2 files changed, 223 insertions(+), 10 deletions(-) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py index 77804bf5ef5ae..d786e8e51ac0e 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/repo.py @@ -1766,6 +1766,31 @@ def ungrouped_asset_5(): return 1 +@asset(key_prefix="grouping_prefix") +def asset_with_prefix_1(): + return 1 + + +@asset(key_prefix="grouping_prefix") +def asset_with_prefix_2(): + return 1 + + +@asset(key_prefix="grouping_prefix") +def asset_with_prefix_3(): + return 1 + + +@asset(key_prefix="grouping_prefix") +def asset_with_prefix_4(): + return 1 + + +@asset(key_prefix="grouping_prefix") +def asset_with_prefix_5(): + return 1 + + @multi_asset(outs={"int_asset": AssetOut(), "str_asset": AssetOut()}) def typed_multi_asset() -> tuple[int, str]: return (1, "yay") @@ -2219,6 +2244,11 @@ def define_assets(): concurrency_asset, concurrency_graph_asset, concurrency_multi_asset, + asset_with_prefix_1, + asset_with_prefix_2, + asset_with_prefix_3, + asset_with_prefix_4, + asset_with_prefix_5, ] diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py index 497bb56afb5c1..5ed2d358bc1ac 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py @@ -24,9 +24,14 @@ AssetMaterializationFailure, AssetMaterializationFailureReason, AssetMaterializationFailureType, + AssetObservation, ) from dagster._core.definitions.multi_dimensional_partitions import MultiPartitionKey -from dagster._core.events import StepMaterializationData +from dagster._core.events import ( + AssetMaterializationPlannedData, + AssetObservationData, + StepMaterializationData, +) from dagster._core.events.log import EventLogEntry from dagster._core.storage.dagster_run import DagsterRunStatus from dagster._core.test_utils import instance_for_test, poll_for_finished_run @@ -909,6 +914,24 @@ } """ +GET_ASSET_SORT_KEY = """ + query AssetSortKeyQuery($prefix: [String!]!) { + assetsOrError(prefix: $prefix) { + ... on AssetConnection { + nodes { + key { + path + } + latestEventSortKey + } + } + ... on PythonError { + message + } + } + } +""" + def _create_run( graphql_context: WorkspaceRequestContext, @@ -1268,7 +1291,10 @@ def test_asset_asof_timestamp(self, graphql_context: WorkspaceRequestContext): result = execute_dagster_graphql( graphql_context, GET_ASSET_MATERIALIZATION_AFTER_TIMESTAMP, - variables={"assetKey": {"path": ["a"]}, "afterTimestamp": str(after_timestamp)}, + variables={ + "assetKey": {"path": ["a"]}, + "afterTimestamp": str(after_timestamp), + }, ) assert result.data assert result.data["assetOrError"] @@ -1281,7 +1307,10 @@ def test_asset_asof_timestamp(self, graphql_context: WorkspaceRequestContext): result = execute_dagster_graphql( graphql_context, GET_ASSET_MATERIALIZATION_AFTER_TIMESTAMP, - variables={"assetKey": {"path": ["a"]}, "afterTimestamp": str(after_timestamp)}, + variables={ + "assetKey": {"path": ["a"]}, + "afterTimestamp": str(after_timestamp), + }, ) assert result.data assert result.data["assetOrError"] @@ -2835,7 +2864,11 @@ def test_automation_condition(self, graphql_context: WorkspaceRequestContext): assert len(custom_automation_condition_asset) == 1 condition = custom_automation_condition_asset[0]["automationCondition"] assert condition["label"] is None - assert condition["expandedLabel"] == ["(some_custom_name)", "SINCE", "(handled)"] + assert condition["expandedLabel"] == [ + "(some_custom_name)", + "SINCE", + "(handled)", + ] def test_tags(self, graphql_context: WorkspaceRequestContext): result = execute_dagster_graphql( @@ -2901,7 +2934,12 @@ def test_has_asset_checks(self, graphql_context: WorkspaceRequestContext): assert result.data["assetNodes"] for a in result.data["assetNodes"]: - if a["assetKey"]["path"] in [["asset_1"], ["one"], ["check_in_op_asset"], ["asset_3"]]: + if a["assetKey"]["path"] in [ + ["asset_1"], + ["one"], + ["check_in_op_asset"], + ["asset_3"], + ]: assert a["hasAssetChecks"] is True else: assert a["hasAssetChecks"] is False, f"Asset {a['assetKey']} has asset checks" @@ -3007,6 +3045,135 @@ def test_get_partition_mapping(self, graphql_context: WorkspaceRequestContext): == "Maps a downstream partition to any upstream partition with an overlapping time window." ) + def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): + """Tests that the asset event sort key is correct, and based on the latest event for an asset + when querying for asset events. + """ + # TestAssetAwareEventLog::test_asset_event_sort_key[sqlite_with_default_run_launcher_managed_grpc_env] + # log events for some assets, make sure the time order of the events doesn't match the + # alpahbetical order of the asset keys so that the default alphabetical sorting doesnt + asset_prefix = "grouping_prefix" + query_keys = [ + AssetKey([asset_prefix, "asset_with_prefix_1"]), + AssetKey([asset_prefix, "asset_with_prefix_2"]), + AssetKey([asset_prefix, "asset_with_prefix_3"]), + AssetKey([asset_prefix, "asset_with_prefix_4"]), + AssetKey([asset_prefix, "asset_with_prefix_5"]), + ] + storage = graphql_context.instance.event_log_storage + test_run_id = make_new_run_id() + storage.store_event( + EventLogEntry( + error_info=None, + user_message="", + level="debug", + run_id=test_run_id, + timestamp=1.0, + dagster_event=DagsterEvent( + DagsterEventType.ASSET_MATERIALIZATION.value, + "nonce", + event_specific_data=StepMaterializationData( + materialization=AssetMaterialization(asset_key=query_keys[3]), + ), + ), + ) + ) + storage.store_event( + EventLogEntry( + error_info=None, + user_message="", + level="debug", + run_id=test_run_id, + timestamp=2.0, + dagster_event=DagsterEvent( + DagsterEventType.ASSET_MATERIALIZATION.value, + "nonce", + event_specific_data=StepMaterializationData( + materialization=AssetMaterialization(asset_key=query_keys[0]), + ), + ), + ) + ) + storage.store_event( + EventLogEntry( + error_info=None, + user_message="", + level="debug", + run_id=test_run_id, + timestamp=3.0, + dagster_event=DagsterEvent( + DagsterEventType.ASSET_MATERIALIZATION_PLANNED.value, + "nonce", + event_specific_data=AssetMaterializationPlannedData(asset_key=query_keys[1]), + ), + ) + ) + storage.store_event( + EventLogEntry( + error_info=None, + user_message="", + level="debug", + run_id=test_run_id, + timestamp=4.0, + dagster_event=DagsterEvent( + DagsterEventType.ASSET_OBSERVATION.value, + "nonce", + event_specific_data=AssetObservationData( + AssetObservation(asset_key=query_keys[2]) + ), + ), + ) + ) + storage.store_event( + EventLogEntry( + error_info=None, + user_message="", + level="debug", + run_id=test_run_id, + timestamp=5.0, + dagster_event=DagsterEvent.build_asset_failed_to_materialize_event( + DagsterEventType.ASSET_FAILED_TO_MATERIALIZE.value, + "nonce", + asset_materialization_failure=AssetMaterializationFailure( + asset_key=query_keys[4], + failure_type=AssetMaterializationFailureType.MATERIALIZATION, + reason=AssetMaterializationFailureReason.FAILED_TO_MATERIALIZE, + ), + ), + ) + ) + if storage.asset_records_have_last_planned_and_failed_materializations: + expected_order = { + query_keys[0]: 2, + query_keys[1]: 3, + query_keys[2]: 4, + query_keys[3]: 1, + query_keys[4]: 5, + } + else: + expected_order = { + query_keys[0]: 2, + query_keys[1]: None, + query_keys[2]: None, + query_keys[3]: 1, + query_keys[4]: None, + } + + result = execute_dagster_graphql( + graphql_context, + GET_ASSET_SORT_KEY, + variables={ + "prefix": [asset_prefix], + }, + ) + assert result + assert len(result.data["assetsOrError"]["nodes"]) == 4 + + for list_item in result.data["assetsOrError"]["nodes"]: + asset_key = AssetKey.from_graphql_input(list_item["key"]) + assert asset_key in expected_order.keys() + assert list_item["latestEventSortKey"] == expected_order[asset_key] + # This is factored out of TestAssetAwareEventLog because there is a separate implementation for plus # graphql tests. @@ -3696,14 +3863,26 @@ def test_2d_subset_backcompat(): assert ranges[0]["primaryDimStartKey"] == "2022-03-03" assert ranges[0]["primaryDimEndKey"] == "2022-03-04" - assert set(ranges[0]["secondaryDim"]["materializedPartitions"]) == {"a", "c"} - assert set(ranges[0]["secondaryDim"]["unmaterializedPartitions"]) == {"b", "d"} + assert set(ranges[0]["secondaryDim"]["materializedPartitions"]) == { + "a", + "c", + } + assert set(ranges[0]["secondaryDim"]["unmaterializedPartitions"]) == { + "b", + "d", + } assert ranges[1]["primaryDimStartKey"] == "2022-03-06" assert ranges[1]["primaryDimEndKey"] == "2022-03-06" assert len(ranges[1]["secondaryDim"]["materializedPartitions"]) == 2 - assert set(ranges[1]["secondaryDim"]["materializedPartitions"]) == {"a", "c"} - assert set(ranges[1]["secondaryDim"]["unmaterializedPartitions"]) == {"b", "d"} + assert set(ranges[1]["secondaryDim"]["materializedPartitions"]) == { + "a", + "c", + } + assert set(ranges[1]["secondaryDim"]["unmaterializedPartitions"]) == { + "b", + "d", + } def test_concurrency_assets(graphql_context: WorkspaceRequestContext): @@ -3789,7 +3968,11 @@ def test_asset_materialization_history(self, graphql_context: WorkspaceRequestCo result = execute_dagster_graphql( graphql_context, GET_ASSET_MATERIALIZATION_HISTORY, - variables={"assetKey": {"path": ["asset_1"]}, "eventTypeSelector": "ALL", "limit": 2}, + variables={ + "assetKey": {"path": ["asset_1"]}, + "eventTypeSelector": "ALL", + "limit": 2, + }, ) assert result.data From 7ec7e97479be488201ea056e8390bcabb705fadd Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 16 Apr 2025 15:29:14 -0400 Subject: [PATCH 04/11] snapshots --- .../__snapshots__/test_all_snapshot_ids.ambr | 1276 ++++++++++------- .../graphql/__snapshots__/test_assets.ambr | 417 +----- .../graphql/__snapshots__/test_solids.ambr | 80 ++ .../graphql/test_assets.py | 4 +- 4 files changed, 875 insertions(+), 902 deletions(-) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_all_snapshot_ids.ambr b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_all_snapshot_ids.ambr index 2c1e445ba054f..eff036c081709 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_all_snapshot_ids.ambr +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_all_snapshot_ids.ambr @@ -466,387 +466,7 @@ "scalar_kind": null, "type_param_keys": null }, - "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "base_dir", - "type_key": "StringSourceType" - } - ], - "given_name": null, - "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.31ebade63b77a8095e9f78414e75445567b90463": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "concurrency_op_1", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "concurrency_op_2", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.31ebade63b77a8095e9f78414e75445567b90463", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.35d3e42b53e66506c5867f04644849cd03763bc6": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"hanging_op\": {}, \"my_op\": {}, \"never_runs_op\": {}}", - "description": null, - "is_required": false, - "name": "ops", - "type_key": "Shape.811a60b4c43530c3d6100304f377dbd2d3045291" - } - ], - "given_name": null, - "key": "Shape.35d3e42b53e66506c5867f04644849cd03763bc6", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "[DEPRECATED]", - "is_required": false, - "name": "marker_to_close", - "type_key": "String" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"enabled\": {}}", - "description": "Whether retries are enabled or not. By default, retries are enabled.", - "is_required": false, - "name": "retries", - "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" - } - ], - "given_name": null, - "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "config", - "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" - } - ], - "given_name": null, - "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.4797b8c4f5fe1c2adf1051241f981d32ce3b9178": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", - "description": "Configure how steps are executed within a run.", - "is_required": false, - "name": "execution", - "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": "Configure how loggers emit messages within a run.", - "is_required": false, - "name": "loggers", - "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"asset_1\": {}, \"asset_1_my_check\": {}, \"asset_2\": {}, \"asset_3\": {}, \"asset_3_asset_3_check\": {}, \"asset_3_asset_3_other_check\": {}, \"asset_one\": {}, \"asset_two\": {}, \"asset_with_automation_condition\": {}, \"asset_with_compute_storage_kinds\": {\"config\": {}}, \"asset_with_custom_automation_condition\": {}, \"asset_yields_observation\": {}, \"bar\": {}, \"baz\": {}, \"check_in_op_asset\": {}, \"concurrency_asset\": {}, \"concurrency_graph_asset\": {\"ops\": {\"concurrency_op_1\": {}, \"concurrency_op_2\": {}}}, \"concurrency_multi_asset\": {\"config\": {}}, \"downstream_asset\": {}, \"downstream_dynamic_partitioned_asset\": {}, \"downstream_static_partitioned_asset\": {}, \"downstream_time_partitioned_asset\": {}, \"downstream_weekly_partitioned_asset\": {}, \"dynamic_in_multipartitions_fail\": {}, \"dynamic_in_multipartitions_success\": {}, \"executable_asset\": {}, \"fail_partition_materialization\": {}, \"first_asset\": {}, \"foo\": {}, \"foo_bar\": {}, \"fresh_diamond_bottom\": {}, \"fresh_diamond_left\": {}, \"fresh_diamond_right\": {}, \"fresh_diamond_top\": {}, \"grouped_asset_1\": {}, \"grouped_asset_2\": {}, \"grouped_asset_4\": {}, \"hanging_asset\": {}, \"hanging_graph\": {\"ops\": {\"hanging_op\": {}, \"my_op\": {}, \"never_runs_op\": {}}}, \"hanging_partition_asset\": {}, \"integers_asset\": {}, \"middle_static_partitioned_asset_1\": {}, \"middle_static_partitioned_asset_2\": {}, \"multi_asset_with_kinds\": {\"config\": {}}, \"multi_run_backfill_policy_asset\": {}, \"multipartitions_1\": {}, \"multipartitions_2\": {}, \"multipartitions_fail\": {}, \"never_runs_asset\": {}, \"no_multipartitions_1\": {}, \"not_included_asset\": {}, \"observable_asset_same_version\": {}, \"output_then_hang_asset\": {}, \"single_run_backfill_policy_asset\": {}, \"subsettable_checked_multi_asset\": {\"config\": {}}, \"typed_asset\": {}, \"typed_multi_asset\": {\"config\": {}}, \"unconnected\": {}, \"ungrouped_asset_3\": {}, \"ungrouped_asset_5\": {}, \"unpartitioned_upstream_of_partitioned\": {}, \"untyped_asset\": {}, \"upstream_daily_partitioned_asset\": {}, \"upstream_dynamic_partitioned_asset\": {}, \"upstream_static_partitioned_asset\": {}, \"upstream_time_partitioned_asset\": {}, \"yield_partition_materialization\": {}}", - "description": "Configure runtime parameters for ops or assets.", - "is_required": false, - "name": "ops", - "type_key": "Shape.d629a5c706111e6e7c83e6a450c536a149a9f9b3" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": "Configure how shared resources are implemented within a run.", - "is_required": true, - "name": "resources", - "type_key": "Shape.7493b137e48f8d4b013bce61e92617ff1bc51f7f" - } - ], - "given_name": null, - "key": "Shape.4797b8c4f5fe1c2adf1051241f981d32ce3b9178", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "path", - "type_key": "String" - } - ], - "given_name": null, - "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.62edccaf30696e25335ae92685bdc41e204e30e6": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "config", - "type_key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709" - } - ], - "given_name": null, - "key": "Shape.62edccaf30696e25335ae92685bdc41e204e30e6", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - } - ], - "given_name": null, - "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.7493b137e48f8d4b013bce61e92617ff1bc51f7f": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "dummy_io_manager", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "hanging_asset_resource", - "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{\"config\": {}}", - "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", - "is_required": false, - "name": "io_manager", - "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" - } - ], - "given_name": null, - "key": "Shape.7493b137e48f8d4b013bce61e92617ff1bc51f7f", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.811a60b4c43530c3d6100304f377dbd2d3045291": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "hanging_op", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "my_op", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - }, - { - "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", - "description": null, - "is_required": false, - "name": "never_runs_op", - "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" - } - ], - "given_name": null, - "key": "Shape.811a60b4c43530c3d6100304f377dbd2d3045291", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "file", - "type_key": "String" - } - ], - "given_name": null, - "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { - "__class__": "ConfigTypeSnap", - "description": null, - "enum_values": null, - "fields": [ - { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": true, - "name": "config", - "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" - } - ], - "given_name": null, - "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", - "kind": { - "__enum__": "ConfigTypeKind.STRICT_SHAPE" - }, - "scalar_kind": null, - "type_param_keys": null - }, - "Shape.d629a5c706111e6e7c83e6a450c536a149a9f9b3": { + "Shape.154e6e9f9db308c19f4d75b8c4c2209607a3de8f": { "__class__": "ConfigTypeSnap", "description": null, "enum_values": null, @@ -1184,6 +804,51 @@ "name": "grouped_asset_4", "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "grouping_prefix__asset_with_prefix_1", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "grouping_prefix__asset_with_prefix_2", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "grouping_prefix__asset_with_prefix_3", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "grouping_prefix__asset_with_prefix_4", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "grouping_prefix__asset_with_prefix_5", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, { "__class__": "ConfigFieldSnap", "default_provided": true, @@ -1456,7 +1121,387 @@ } ], "given_name": null, - "key": "Shape.d629a5c706111e6e7c83e6a450c536a149a9f9b3", + "key": "Shape.154e6e9f9db308c19f4d75b8c4c2209607a3de8f", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "base_dir", + "type_key": "StringSourceType" + } + ], + "given_name": null, + "key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.31ebade63b77a8095e9f78414e75445567b90463": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "concurrency_op_1", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "concurrency_op_2", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.31ebade63b77a8095e9f78414e75445567b90463", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.35d3e42b53e66506c5867f04644849cd03763bc6": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"hanging_op\": {}, \"my_op\": {}, \"never_runs_op\": {}}", + "description": null, + "is_required": false, + "name": "ops", + "type_key": "Shape.811a60b4c43530c3d6100304f377dbd2d3045291" + } + ], + "given_name": null, + "key": "Shape.35d3e42b53e66506c5867f04644849cd03763bc6", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "[DEPRECATED]", + "is_required": false, + "name": "marker_to_close", + "type_key": "String" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"enabled\": {}}", + "description": "Whether retries are enabled or not. By default, retries are enabled.", + "is_required": false, + "name": "retries", + "type_key": "Selector.1bfb167aea90780aa679597800c71bd8c65ed0b2" + } + ], + "given_name": null, + "key": "Shape.44f24ac55059da1634e84af6c1bf7e0ed332251c", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.44f2a71367507edd1b8e64f739222c4312b3691b": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.18b2faaf1efd505374f7f25fcb61ed59bd5be851" + } + ], + "given_name": null, + "key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "path", + "type_key": "String" + } + ], + "given_name": null, + "key": "Shape.4b53b73df342381d0d05c5f36183dc99cb9676e2", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.62edccaf30696e25335ae92685bdc41e204e30e6": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709" + } + ], + "given_name": null, + "key": "Shape.62edccaf30696e25335ae92685bdc41e204e30e6", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.743e47901855cb245064dd633e217bfcb49a11a7": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + } + ], + "given_name": null, + "key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.7493b137e48f8d4b013bce61e92617ff1bc51f7f": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "dummy_io_manager", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "hanging_asset_resource", + "type_key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {}}", + "description": "Built-in filesystem IO manager that stores and retrieves values using pickling.", + "is_required": false, + "name": "io_manager", + "type_key": "Shape.44f2a71367507edd1b8e64f739222c4312b3691b" + } + ], + "given_name": null, + "key": "Shape.7493b137e48f8d4b013bce61e92617ff1bc51f7f", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.811a60b4c43530c3d6100304f377dbd2d3045291": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "hanging_op", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "my_op", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "never_runs_op", + "type_key": "Shape.743e47901855cb245064dd633e217bfcb49a11a7" + } + ], + "given_name": null, + "key": "Shape.811a60b4c43530c3d6100304f377dbd2d3045291", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.87c41db52dfcd85309626a6f11959e38b97a1a10": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"config\": {\"retries\": {\"enabled\": {}}}}", + "description": "Configure how steps are executed within a run.", + "is_required": false, + "name": "execution", + "type_key": "Shape.09d73f0755bf4752d3f121837669c8660dcf451e" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": "Configure how loggers emit messages within a run.", + "is_required": false, + "name": "loggers", + "type_key": "Shape.e895d95ee6d0eff1b884c76f44a2ab7089f0c49b" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{\"asset_1\": {}, \"asset_1_my_check\": {}, \"asset_2\": {}, \"asset_3\": {}, \"asset_3_asset_3_check\": {}, \"asset_3_asset_3_other_check\": {}, \"asset_one\": {}, \"asset_two\": {}, \"asset_with_automation_condition\": {}, \"asset_with_compute_storage_kinds\": {\"config\": {}}, \"asset_with_custom_automation_condition\": {}, \"asset_yields_observation\": {}, \"bar\": {}, \"baz\": {}, \"check_in_op_asset\": {}, \"concurrency_asset\": {}, \"concurrency_graph_asset\": {\"ops\": {\"concurrency_op_1\": {}, \"concurrency_op_2\": {}}}, \"concurrency_multi_asset\": {\"config\": {}}, \"downstream_asset\": {}, \"downstream_dynamic_partitioned_asset\": {}, \"downstream_static_partitioned_asset\": {}, \"downstream_time_partitioned_asset\": {}, \"downstream_weekly_partitioned_asset\": {}, \"dynamic_in_multipartitions_fail\": {}, \"dynamic_in_multipartitions_success\": {}, \"executable_asset\": {}, \"fail_partition_materialization\": {}, \"first_asset\": {}, \"foo\": {}, \"foo_bar\": {}, \"fresh_diamond_bottom\": {}, \"fresh_diamond_left\": {}, \"fresh_diamond_right\": {}, \"fresh_diamond_top\": {}, \"grouped_asset_1\": {}, \"grouped_asset_2\": {}, \"grouped_asset_4\": {}, \"grouping_prefix__asset_with_prefix_1\": {}, \"grouping_prefix__asset_with_prefix_2\": {}, \"grouping_prefix__asset_with_prefix_3\": {}, \"grouping_prefix__asset_with_prefix_4\": {}, \"grouping_prefix__asset_with_prefix_5\": {}, \"hanging_asset\": {}, \"hanging_graph\": {\"ops\": {\"hanging_op\": {}, \"my_op\": {}, \"never_runs_op\": {}}}, \"hanging_partition_asset\": {}, \"integers_asset\": {}, \"middle_static_partitioned_asset_1\": {}, \"middle_static_partitioned_asset_2\": {}, \"multi_asset_with_kinds\": {\"config\": {}}, \"multi_run_backfill_policy_asset\": {}, \"multipartitions_1\": {}, \"multipartitions_2\": {}, \"multipartitions_fail\": {}, \"never_runs_asset\": {}, \"no_multipartitions_1\": {}, \"not_included_asset\": {}, \"observable_asset_same_version\": {}, \"output_then_hang_asset\": {}, \"single_run_backfill_policy_asset\": {}, \"subsettable_checked_multi_asset\": {\"config\": {}}, \"typed_asset\": {}, \"typed_multi_asset\": {\"config\": {}}, \"unconnected\": {}, \"ungrouped_asset_3\": {}, \"ungrouped_asset_5\": {}, \"unpartitioned_upstream_of_partitioned\": {}, \"untyped_asset\": {}, \"upstream_daily_partitioned_asset\": {}, \"upstream_dynamic_partitioned_asset\": {}, \"upstream_static_partitioned_asset\": {}, \"upstream_time_partitioned_asset\": {}, \"yield_partition_materialization\": {}}", + "description": "Configure runtime parameters for ops or assets.", + "is_required": false, + "name": "ops", + "type_key": "Shape.154e6e9f9db308c19f4d75b8c4c2209607a3de8f" + }, + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": "Configure how shared resources are implemented within a run.", + "is_required": true, + "name": "resources", + "type_key": "Shape.7493b137e48f8d4b013bce61e92617ff1bc51f7f" + } + ], + "given_name": null, + "key": "Shape.87c41db52dfcd85309626a6f11959e38b97a1a10", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "file", + "type_key": "String" + } + ], + "given_name": null, + "key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e", + "kind": { + "__enum__": "ConfigTypeKind.STRICT_SHAPE" + }, + "scalar_kind": null, + "type_param_keys": null + }, + "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3": { + "__class__": "ConfigTypeSnap", + "description": null, + "enum_values": null, + "fields": [ + { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": true, + "name": "config", + "type_key": "Shape.9a3a315bff2146cca750edbec49c6b4b4d0ce58e" + } + ], + "given_name": null, + "key": "Shape.b13a6c5637084590cc1538f9522324bfeb4b46b3", "kind": { "__enum__": "ConfigTypeKind.STRICT_SHAPE" }, @@ -2240,6 +2285,46 @@ "solid_name": "grouped_asset_4", "tags": {} }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "grouping_prefix__asset_with_prefix_1", + "solid_name": "grouping_prefix__asset_with_prefix_1", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "grouping_prefix__asset_with_prefix_2", + "solid_name": "grouping_prefix__asset_with_prefix_2", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "grouping_prefix__asset_with_prefix_3", + "solid_name": "grouping_prefix__asset_with_prefix_3", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "grouping_prefix__asset_with_prefix_4", + "solid_name": "grouping_prefix__asset_with_prefix_4", + "tags": {} + }, + { + "__class__": "SolidInvocationSnap", + "input_dep_snaps": [], + "is_dynamic_mapped": false, + "solid_def_name": "grouping_prefix__asset_with_prefix_5", + "solid_name": "grouping_prefix__asset_with_prefix_5", + "tags": {} + }, { "__class__": "SolidInvocationSnap", "input_dep_snaps": [ @@ -2667,7 +2752,7 @@ "name": "io_manager" } ], - "root_config_key": "Shape.4797b8c4f5fe1c2adf1051241f981d32ce3b9178" + "root_config_key": "Shape.87c41db52dfcd85309626a6f11959e38b97a1a10" } ], "name": "__ASSET_JOB", @@ -3262,10 +3347,171 @@ "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "foo_bar" + "name": "foo_bar" + } + ], + "name": "baz", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "check_in_op_asset", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + }, + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "check_in_op_asset_my_check" + } + ], + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "concurrency_asset", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "pool": "foo", + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": true, + "default_value_as_json_str": "{}", + "description": null, + "is_required": false, + "name": "config", + "type_key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709" + }, + "description": null, + "input_def_snaps": [], + "name": "concurrency_multi_asset", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Nothing", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "concurrency_multi_asset_1" + }, + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Nothing", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "concurrency_multi_asset_2" + } + ], + "pool": "buzz", + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "concurrency_op_1", + "output_def_snaps": [ + { + "__class__": "OutputDefSnap", + "dagster_type_key": "Any", + "description": null, + "is_dynamic": false, + "is_required": true, + "name": "result" + } + ], + "pool": "bar", + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "input_1" } ], - "name": "baz", + "name": "concurrency_op_2", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3276,6 +3522,7 @@ "name": "result" } ], + "pool": "baz", "required_resource_keys": [], "tags": {} }, @@ -3291,24 +3538,23 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [], - "name": "check_in_op_asset", - "output_def_snaps": [ + "input_def_snaps": [ { - "__class__": "OutputDefSnap", + "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "is_dynamic": false, - "is_required": true, - "name": "result" - }, + "name": "hanging_graph" + } + ], + "name": "downstream_asset", + "output_def_snaps": [ { "__class__": "OutputDefSnap", "dagster_type_key": "Any", "description": null, "is_dynamic": false, "is_required": true, - "name": "check_in_op_asset_my_check" + "name": "result" } ], "required_resource_keys": [], @@ -3326,8 +3572,15 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [], - "name": "concurrency_asset", + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "upstream_dynamic_partitioned_asset" + } + ], + "name": "downstream_dynamic_partitioned_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3338,7 +3591,6 @@ "name": "result" } ], - "pool": "foo", "required_resource_keys": [], "tags": {} }, @@ -3346,52 +3598,29 @@ "__class__": "SolidDefSnap", "config_field_snap": { "__class__": "ConfigFieldSnap", - "default_provided": true, - "default_value_as_json_str": "{}", + "default_provided": false, + "default_value_as_json_str": null, "description": null, "is_required": false, "name": "config", - "type_key": "Shape.da39a3ee5e6b4b0d3255bfef95601890afd80709" + "type_key": "Any" }, "description": null, - "input_def_snaps": [], - "name": "concurrency_multi_asset", - "output_def_snaps": [ + "input_def_snaps": [ { - "__class__": "OutputDefSnap", - "dagster_type_key": "Nothing", + "__class__": "InputDefSnap", + "dagster_type_key": "Any", "description": null, - "is_dynamic": false, - "is_required": true, - "name": "concurrency_multi_asset_1" + "name": "middle_static_partitioned_asset_1" }, { - "__class__": "OutputDefSnap", - "dagster_type_key": "Nothing", + "__class__": "InputDefSnap", + "dagster_type_key": "Any", "description": null, - "is_dynamic": false, - "is_required": true, - "name": "concurrency_multi_asset_2" + "name": "middle_static_partitioned_asset_2" } ], - "pool": "buzz", - "required_resource_keys": [], - "tags": {} - }, - { - "__class__": "SolidDefSnap", - "config_field_snap": { - "__class__": "ConfigFieldSnap", - "default_provided": false, - "default_value_as_json_str": null, - "description": null, - "is_required": false, - "name": "config", - "type_key": "Any" - }, - "description": null, - "input_def_snaps": [], - "name": "concurrency_op_1", + "name": "downstream_static_partitioned_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3402,7 +3631,6 @@ "name": "result" } ], - "pool": "bar", "required_resource_keys": [], "tags": {} }, @@ -3423,10 +3651,10 @@ "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "input_1" + "name": "upstream_time_partitioned_asset" } ], - "name": "concurrency_op_2", + "name": "downstream_time_partitioned_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3437,7 +3665,6 @@ "name": "result" } ], - "pool": "baz", "required_resource_keys": [], "tags": {} }, @@ -3458,10 +3685,10 @@ "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "hanging_graph" + "name": "upstream_daily_partitioned_asset" } ], - "name": "downstream_asset", + "name": "downstream_weekly_partitioned_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3492,10 +3719,10 @@ "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "upstream_dynamic_partitioned_asset" + "name": "dynamic_in_multipartitions_success" } ], - "name": "downstream_dynamic_partitioned_asset", + "name": "dynamic_in_multipartitions_fail", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3521,21 +3748,8 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "middle_static_partitioned_asset_1" - }, - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "middle_static_partitioned_asset_2" - } - ], - "name": "downstream_static_partitioned_asset", + "input_def_snaps": [], + "name": "dynamic_in_multipartitions_success", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3566,14 +3780,14 @@ "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "upstream_time_partitioned_asset" + "name": "unexecutable_asset" } ], - "name": "downstream_time_partitioned_asset", + "name": "executable_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "Any", + "dagster_type_key": "Nothing", "description": null, "is_dynamic": false, "is_required": true, @@ -3595,15 +3809,8 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "upstream_daily_partitioned_asset" - } - ], - "name": "downstream_weekly_partitioned_asset", + "input_def_snaps": [], + "name": "fail_partition_materialization", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3634,10 +3841,10 @@ "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "dynamic_in_multipartitions_success" + "name": "dummy_source_asset" } ], - "name": "dynamic_in_multipartitions_fail", + "name": "first_asset", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3664,7 +3871,7 @@ }, "description": null, "input_def_snaps": [], - "name": "dynamic_in_multipartitions_success", + "name": "foo", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3695,14 +3902,20 @@ "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "unexecutable_asset" + "name": "bar" + }, + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "foo" } ], - "name": "executable_asset", + "name": "foo_bar", "output_def_snaps": [ { "__class__": "OutputDefSnap", - "dagster_type_key": "Nothing", + "dagster_type_key": "Any", "description": null, "is_dynamic": false, "is_required": true, @@ -3724,8 +3937,21 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [], - "name": "fail_partition_materialization", + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "fresh_diamond_left" + }, + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "fresh_diamond_right" + } + ], + "name": "fresh_diamond_bottom", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3756,10 +3982,10 @@ "__class__": "InputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "dummy_source_asset" + "name": "fresh_diamond_top" } ], - "name": "first_asset", + "name": "fresh_diamond_left", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3785,8 +4011,15 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [], - "name": "foo", + "input_def_snaps": [ + { + "__class__": "InputDefSnap", + "dagster_type_key": "Any", + "description": null, + "name": "fresh_diamond_top" + } + ], + "name": "fresh_diamond_right", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3815,18 +4048,12 @@ "input_def_snaps": [ { "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "bar" - }, - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", + "dagster_type_key": "Nothing", "description": null, - "name": "foo" + "name": "diamond_source" } ], - "name": "foo_bar", + "name": "fresh_diamond_top", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3852,21 +4079,8 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "fresh_diamond_left" - }, - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "fresh_diamond_right" - } - ], - "name": "fresh_diamond_bottom", + "input_def_snaps": [], + "name": "grouped_asset_1", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3892,15 +4106,8 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Any", - "description": null, - "name": "fresh_diamond_top" - } - ], - "name": "fresh_diamond_left", + "input_def_snaps": [], + "name": "grouped_asset_2", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3926,15 +4133,35 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [ + "input_def_snaps": [], + "name": "grouped_asset_4", + "output_def_snaps": [ { - "__class__": "InputDefSnap", + "__class__": "OutputDefSnap", "dagster_type_key": "Any", "description": null, - "name": "fresh_diamond_top" + "is_dynamic": false, + "is_required": true, + "name": "result" } ], - "name": "fresh_diamond_right", + "required_resource_keys": [], + "tags": {} + }, + { + "__class__": "SolidDefSnap", + "config_field_snap": { + "__class__": "ConfigFieldSnap", + "default_provided": false, + "default_value_as_json_str": null, + "description": null, + "is_required": false, + "name": "config", + "type_key": "Any" + }, + "description": null, + "input_def_snaps": [], + "name": "grouping_prefix__asset_with_prefix_1", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3960,15 +4187,8 @@ "type_key": "Any" }, "description": null, - "input_def_snaps": [ - { - "__class__": "InputDefSnap", - "dagster_type_key": "Nothing", - "description": null, - "name": "diamond_source" - } - ], - "name": "fresh_diamond_top", + "input_def_snaps": [], + "name": "grouping_prefix__asset_with_prefix_2", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -3995,7 +4215,7 @@ }, "description": null, "input_def_snaps": [], - "name": "grouped_asset_1", + "name": "grouping_prefix__asset_with_prefix_3", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -4022,7 +4242,7 @@ }, "description": null, "input_def_snaps": [], - "name": "grouped_asset_2", + "name": "grouping_prefix__asset_with_prefix_4", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -4049,7 +4269,7 @@ }, "description": null, "input_def_snaps": [], - "name": "grouped_asset_4", + "name": "grouping_prefix__asset_with_prefix_5", "output_def_snaps": [ { "__class__": "OutputDefSnap", @@ -35535,7 +35755,7 @@ 'd9f6d85793df3d9df94d4aedb21bb659c1202bda' # --- # name: test_all_snapshot_ids[1] - 'e84b4d71da08be361c3ab5c4769786d7f17ee93f' + '06194a2bfd72c3683124ac3696e9ea4834bbfd7e' # --- # name: test_all_snapshot_ids[20] ''' diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_assets.ambr b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_assets.ambr index 35b14f5610094..117731d2ed729 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_assets.ambr +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_assets.ambr @@ -333,6 +333,51 @@ ]), }), }), + dict({ + 'id': 'test_location.test_repo.["grouping_prefix", "asset_with_prefix_1"]', + 'key': dict({ + 'path': list([ + 'grouping_prefix', + 'asset_with_prefix_1', + ]), + }), + }), + dict({ + 'id': 'test_location.test_repo.["grouping_prefix", "asset_with_prefix_2"]', + 'key': dict({ + 'path': list([ + 'grouping_prefix', + 'asset_with_prefix_2', + ]), + }), + }), + dict({ + 'id': 'test_location.test_repo.["grouping_prefix", "asset_with_prefix_3"]', + 'key': dict({ + 'path': list([ + 'grouping_prefix', + 'asset_with_prefix_3', + ]), + }), + }), + dict({ + 'id': 'test_location.test_repo.["grouping_prefix", "asset_with_prefix_4"]', + 'key': dict({ + 'path': list([ + 'grouping_prefix', + 'asset_with_prefix_4', + ]), + }), + }), + dict({ + 'id': 'test_location.test_repo.["grouping_prefix", "asset_with_prefix_5"]', + 'key': dict({ + 'path': list([ + 'grouping_prefix', + 'asset_with_prefix_5', + ]), + }), + }), dict({ 'id': 'test_location.test_repo.["hanging_asset"]', 'key': dict({ @@ -631,378 +676,6 @@ }), }) # --- -# name: TestAssetAwareEventLog.test_freshness_info[0] - dict({ - 'assetNodes': list([ - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["asset_1"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["asset_2"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["asset_3"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["asset_one"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["asset_two"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["asset_with_automation_condition"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["asset_with_custom_automation_condition"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["asset_yields_observation"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["bar"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["baz"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["check_in_op_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["concurrency_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["concurrency_graph_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["concurrency_multi_asset_1"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["concurrency_multi_asset_2"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["diamond_source"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["downstream_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["downstream_dynamic_partitioned_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["downstream_static_partitioned_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["downstream_time_partitioned_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["downstream_weekly_partitioned_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["dummy_source_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["dynamic_in_multipartitions_fail"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["dynamic_in_multipartitions_success"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["executable_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["fail_partition_materialization"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["first_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["first_kinds_key"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["foo"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["foo_bar"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["fourth_kinds_key"]', - }), - dict({ - 'freshnessInfo': dict({ - 'currentMinutesLate': 0.0, - 'latestMaterializationMinutesLate': None, - }), - 'freshnessPolicy': dict({ - 'cronSchedule': None, - 'maximumLagMinutes': 30.0, - }), - 'id': 'test_location.test_repo.["fresh_diamond_bottom"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["fresh_diamond_left"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["fresh_diamond_right"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["fresh_diamond_top"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["grouped_asset_1"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["grouped_asset_2"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["grouped_asset_4"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["hanging_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["hanging_graph"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["hanging_partition_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["int_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["integers_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["middle_static_partitioned_asset_1"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["middle_static_partitioned_asset_2"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["multi_run_backfill_policy_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["multipartitions_1"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["multipartitions_2"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["multipartitions_fail"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["never_runs_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["no_multipartitions_1"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["not_included_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["observable_asset_same_version"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["one"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["output_then_hang_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["second_kinds_key"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["single_run_backfill_policy_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["str_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["third_kinds_key"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["two"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["typed_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["unconnected"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["unexecutable_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["ungrouped_asset_3"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["ungrouped_asset_5"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["unpartitioned_upstream_of_partitioned"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["untyped_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["upstream_daily_partitioned_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["upstream_dynamic_partitioned_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["upstream_static_partitioned_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["upstream_time_partitioned_asset"]', - }), - dict({ - 'freshnessInfo': None, - 'freshnessPolicy': None, - 'id': 'test_location.test_repo.["yield_partition_materialization"]', - }), - ]), - }) -# --- # name: TestAssetAwareEventLog.test_get_asset_key_materialization[0] dict({ 'assetOrError': dict({ diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_solids.ambr b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_solids.ambr index 3a02dffaee68d..22791847d4b0b 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_solids.ambr +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_solids.ambr @@ -1291,6 +1291,86 @@ }), ]), }), + dict({ + '__typename': 'UsedSolid', + 'definition': dict({ + 'name': 'grouping_prefix__asset_with_prefix_1', + }), + 'invocations': list([ + dict({ + 'pipeline': dict({ + 'name': '__ASSET_JOB', + }), + 'solidHandle': dict({ + 'handleID': 'grouping_prefix__asset_with_prefix_1', + }), + }), + ]), + }), + dict({ + '__typename': 'UsedSolid', + 'definition': dict({ + 'name': 'grouping_prefix__asset_with_prefix_2', + }), + 'invocations': list([ + dict({ + 'pipeline': dict({ + 'name': '__ASSET_JOB', + }), + 'solidHandle': dict({ + 'handleID': 'grouping_prefix__asset_with_prefix_2', + }), + }), + ]), + }), + dict({ + '__typename': 'UsedSolid', + 'definition': dict({ + 'name': 'grouping_prefix__asset_with_prefix_3', + }), + 'invocations': list([ + dict({ + 'pipeline': dict({ + 'name': '__ASSET_JOB', + }), + 'solidHandle': dict({ + 'handleID': 'grouping_prefix__asset_with_prefix_3', + }), + }), + ]), + }), + dict({ + '__typename': 'UsedSolid', + 'definition': dict({ + 'name': 'grouping_prefix__asset_with_prefix_4', + }), + 'invocations': list([ + dict({ + 'pipeline': dict({ + 'name': '__ASSET_JOB', + }), + 'solidHandle': dict({ + 'handleID': 'grouping_prefix__asset_with_prefix_4', + }), + }), + ]), + }), + dict({ + '__typename': 'UsedSolid', + 'definition': dict({ + 'name': 'grouping_prefix__asset_with_prefix_5', + }), + 'invocations': list([ + dict({ + 'pipeline': dict({ + 'name': '__ASSET_JOB', + }), + 'solidHandle': dict({ + 'handleID': 'grouping_prefix__asset_with_prefix_5', + }), + }), + ]), + }), dict({ '__typename': 'UsedSolid', 'definition': dict({ diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py index 5ed2d358bc1ac..24fdf9ad40484 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py @@ -3136,7 +3136,7 @@ def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): "nonce", asset_materialization_failure=AssetMaterializationFailure( asset_key=query_keys[4], - failure_type=AssetMaterializationFailureType.MATERIALIZATION, + failure_type=AssetMaterializationFailureType.FAILED, reason=AssetMaterializationFailureReason.FAILED_TO_MATERIALIZE, ), ), @@ -3167,7 +3167,7 @@ def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): }, ) assert result - assert len(result.data["assetsOrError"]["nodes"]) == 4 + assert len(result.data["assetsOrError"]["nodes"]) == 5 for list_item in result.data["assetsOrError"]["nodes"]: asset_key = AssetKey.from_graphql_input(list_item["key"]) From 12d171f91617025bf4afae8c0b6a8489312d009b Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 16 Apr 2025 16:30:48 -0400 Subject: [PATCH 05/11] snap --- .../graphql/__snapshots__/test_assets.ambr | 397 ++++++++++++++++++ 1 file changed, 397 insertions(+) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_assets.ambr b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_assets.ambr index 117731d2ed729..7fc24868ae1fc 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_assets.ambr +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/__snapshots__/test_assets.ambr @@ -676,6 +676,403 @@ }), }) # --- +# name: TestAssetAwareEventLog.test_freshness_info[0] + dict({ + 'assetNodes': list([ + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["asset_1"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["asset_2"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["asset_3"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["asset_one"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["asset_two"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["asset_with_automation_condition"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["asset_with_custom_automation_condition"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["asset_yields_observation"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["bar"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["baz"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["check_in_op_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["concurrency_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["concurrency_graph_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["concurrency_multi_asset_1"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["concurrency_multi_asset_2"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["diamond_source"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["downstream_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["downstream_dynamic_partitioned_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["downstream_static_partitioned_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["downstream_time_partitioned_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["downstream_weekly_partitioned_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["dummy_source_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["dynamic_in_multipartitions_fail"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["dynamic_in_multipartitions_success"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["executable_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["fail_partition_materialization"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["first_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["first_kinds_key"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["foo"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["foo_bar"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["fourth_kinds_key"]', + }), + dict({ + 'freshnessInfo': dict({ + 'currentMinutesLate': 0.0, + 'latestMaterializationMinutesLate': None, + }), + 'freshnessPolicy': dict({ + 'cronSchedule': None, + 'maximumLagMinutes': 30.0, + }), + 'id': 'test_location.test_repo.["fresh_diamond_bottom"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["fresh_diamond_left"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["fresh_diamond_right"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["fresh_diamond_top"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["grouped_asset_1"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["grouped_asset_2"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["grouped_asset_4"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["grouping_prefix", "asset_with_prefix_1"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["grouping_prefix", "asset_with_prefix_2"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["grouping_prefix", "asset_with_prefix_3"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["grouping_prefix", "asset_with_prefix_4"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["grouping_prefix", "asset_with_prefix_5"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["hanging_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["hanging_graph"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["hanging_partition_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["int_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["integers_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["middle_static_partitioned_asset_1"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["middle_static_partitioned_asset_2"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["multi_run_backfill_policy_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["multipartitions_1"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["multipartitions_2"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["multipartitions_fail"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["never_runs_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["no_multipartitions_1"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["not_included_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["observable_asset_same_version"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["one"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["output_then_hang_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["second_kinds_key"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["single_run_backfill_policy_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["str_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["third_kinds_key"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["two"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["typed_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["unconnected"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["unexecutable_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["ungrouped_asset_3"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["ungrouped_asset_5"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["unpartitioned_upstream_of_partitioned"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["untyped_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["upstream_daily_partitioned_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["upstream_dynamic_partitioned_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["upstream_static_partitioned_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["upstream_time_partitioned_asset"]', + }), + dict({ + 'freshnessInfo': None, + 'freshnessPolicy': None, + 'id': 'test_location.test_repo.["yield_partition_materialization"]', + }), + ]), + }) +# --- # name: TestAssetAwareEventLog.test_get_asset_key_materialization[0] dict({ 'assetOrError': dict({ From c201abe30feec4d02cd3ff5424cdbc66261278b3 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 16 Apr 2025 17:07:39 -0400 Subject: [PATCH 06/11] wip to make the test better --- .../graphql/test_assets.py | 68 +++++++++++++++---- 1 file changed, 55 insertions(+), 13 deletions(-) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py index 24fdf9ad40484..5c9d8084725fa 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py @@ -3060,14 +3060,14 @@ def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): AssetKey([asset_prefix, "asset_with_prefix_4"]), AssetKey([asset_prefix, "asset_with_prefix_5"]), ] + asset_keys_to_run_ids = {query_key: make_new_run_id() for query_key in query_keys} storage = graphql_context.instance.event_log_storage - test_run_id = make_new_run_id() storage.store_event( EventLogEntry( error_info=None, user_message="", level="debug", - run_id=test_run_id, + run_id=asset_keys_to_run_ids[query_keys[3]], timestamp=1.0, dagster_event=DagsterEvent( DagsterEventType.ASSET_MATERIALIZATION.value, @@ -3083,7 +3083,7 @@ def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): error_info=None, user_message="", level="debug", - run_id=test_run_id, + run_id=asset_keys_to_run_ids[query_keys[0]], timestamp=2.0, dagster_event=DagsterEvent( DagsterEventType.ASSET_MATERIALIZATION.value, @@ -3099,7 +3099,7 @@ def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): error_info=None, user_message="", level="debug", - run_id=test_run_id, + run_id=asset_keys_to_run_ids[query_keys[1]], timestamp=3.0, dagster_event=DagsterEvent( DagsterEventType.ASSET_MATERIALIZATION_PLANNED.value, @@ -3113,7 +3113,7 @@ def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): error_info=None, user_message="", level="debug", - run_id=test_run_id, + run_id=asset_keys_to_run_ids[query_keys[2]], timestamp=4.0, dagster_event=DagsterEvent( DagsterEventType.ASSET_OBSERVATION.value, @@ -3129,7 +3129,7 @@ def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): error_info=None, user_message="", level="debug", - run_id=test_run_id, + run_id=asset_keys_to_run_ids[query_keys[4]], timestamp=5.0, dagster_event=DagsterEvent.build_asset_failed_to_materialize_event( DagsterEventType.ASSET_FAILED_TO_MATERIALIZE.value, @@ -3144,18 +3144,60 @@ def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): ) if storage.asset_records_have_last_planned_and_failed_materializations: expected_order = { - query_keys[0]: 2, - query_keys[1]: 3, - query_keys[2]: 4, - query_keys[3]: 1, - query_keys[4]: 5, + query_keys[0]: storage.get_records_for_run( + run_id=asset_keys_to_run_ids[query_keys[0]], + of_type=DagsterEventType.ASSET_MATERIALIZATION, + limit=1, + ) + .records[0] + .storage_id, + query_keys[1]: storage.get_records_for_run( + run_id=asset_keys_to_run_ids[query_keys[1]], + of_type=DagsterEventType.ASSET_MATERIALIZATION_PLANNED, + limit=1, + ) + .records[0] + .storage_id, + query_keys[2]: storage.get_records_for_run( + run_id=asset_keys_to_run_ids[query_keys[2]], + of_type=DagsterEventType.ASSET_OBSERVATION, + limit=1, + ) + .records[0] + .storage_id, + query_keys[3]: storage.get_records_for_run( + run_id=asset_keys_to_run_ids[query_keys[3]], + of_type=DagsterEventType.ASSET_MATERIALIZATION, + limit=1, + ) + .records[0] + .storage_id, + query_keys[4]: storage.get_records_for_run( + run_id=asset_keys_to_run_ids[query_keys[4]], + of_type=DagsterEventType.ASSET_FAILED_TO_MATERIALIZE, + limit=1, + ) + .records[0] + .storage_id, } else: expected_order = { - query_keys[0]: 2, + query_keys[0]: storage.get_records_for_run( + run_id=asset_keys_to_run_ids[query_keys[0]], + of_type=DagsterEventType.ASSET_MATERIALIZATION, + limit=1, + ) + .records[0] + .storage_id, query_keys[1]: None, query_keys[2]: None, - query_keys[3]: 1, + query_keys[3]: storage.get_records_for_run( + run_id=asset_keys_to_run_ids[query_keys[3]], + of_type=DagsterEventType.ASSET_MATERIALIZATION, + limit=1, + ) + .records[0] + .storage_id, query_keys[4]: None, } From 4b1410aeb8c8ce9ee0fd33d13f14221b7ed9636f Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Wed, 16 Apr 2025 17:25:07 -0400 Subject: [PATCH 07/11] update test --- .../graphql/test_assets.py | 223 ++++++------------ 1 file changed, 75 insertions(+), 148 deletions(-) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py index 5c9d8084725fa..23f0828f6b598 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py @@ -24,7 +24,6 @@ AssetMaterializationFailure, AssetMaterializationFailureReason, AssetMaterializationFailureType, - AssetObservation, ) from dagster._core.definitions.multi_dimensional_partitions import MultiPartitionKey from dagster._core.events import ( @@ -3049,157 +3048,85 @@ def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): """Tests that the asset event sort key is correct, and based on the latest event for an asset when querying for asset events. """ - # TestAssetAwareEventLog::test_asset_event_sort_key[sqlite_with_default_run_launcher_managed_grpc_env] - # log events for some assets, make sure the time order of the events doesn't match the - # alpahbetical order of the asset keys so that the default alphabetical sorting doesnt - asset_prefix = "grouping_prefix" - query_keys = [ - AssetKey([asset_prefix, "asset_with_prefix_1"]), - AssetKey([asset_prefix, "asset_with_prefix_2"]), - AssetKey([asset_prefix, "asset_with_prefix_3"]), - AssetKey([asset_prefix, "asset_with_prefix_4"]), - AssetKey([asset_prefix, "asset_with_prefix_5"]), - ] - asset_keys_to_run_ids = {query_key: make_new_run_id() for query_key in query_keys} storage = graphql_context.instance.event_log_storage - storage.store_event( - EventLogEntry( - error_info=None, - user_message="", - level="debug", - run_id=asset_keys_to_run_ids[query_keys[3]], - timestamp=1.0, - dagster_event=DagsterEvent( - DagsterEventType.ASSET_MATERIALIZATION.value, - "nonce", - event_specific_data=StepMaterializationData( - materialization=AssetMaterialization(asset_key=query_keys[3]), - ), - ), - ) - ) - storage.store_event( - EventLogEntry( - error_info=None, - user_message="", - level="debug", - run_id=asset_keys_to_run_ids[query_keys[0]], - timestamp=2.0, - dagster_event=DagsterEvent( - DagsterEventType.ASSET_MATERIALIZATION.value, - "nonce", - event_specific_data=StepMaterializationData( - materialization=AssetMaterialization(asset_key=query_keys[0]), - ), - ), - ) - ) - storage.store_event( - EventLogEntry( - error_info=None, - user_message="", - level="debug", - run_id=asset_keys_to_run_ids[query_keys[1]], - timestamp=3.0, - dagster_event=DagsterEvent( - DagsterEventType.ASSET_MATERIALIZATION_PLANNED.value, - "nonce", - event_specific_data=AssetMaterializationPlannedData(asset_key=query_keys[1]), - ), - ) - ) - storage.store_event( - EventLogEntry( - error_info=None, - user_message="", - level="debug", - run_id=asset_keys_to_run_ids[query_keys[2]], - timestamp=4.0, - dagster_event=DagsterEvent( - DagsterEventType.ASSET_OBSERVATION.value, - "nonce", - event_specific_data=AssetObservationData( - AssetObservation(asset_key=query_keys[2]) + + event_type_to_event_specific_data_mapping = { + DagsterEventType.ASSET_MATERIALIZATION.value: StepMaterializationData, + DagsterEventType.ASSET_MATERIALIZATION_PLANNED.value: AssetMaterializationPlannedData, + DagsterEventType.ASSET_OBSERVATION.value: AssetObservationData, + DagsterEventType.ASSET_FAILED_TO_MATERIALIZE.value: AssetMaterializationFailure, + } + + def build_event_specific_data(event_type: DagsterEventType, asset_key: AssetKey): + event_specific_data_cls = event_type_to_event_specific_data_mapping[event_type.value] + if event_type == DagsterEventType.ASSET_FAILED_TO_MATERIALIZE.value: + return event_specific_data_cls( + asset_key=asset_key, + failure_type=AssetMaterializationFailureType.FAILED, + reason=AssetMaterializationFailureReason.FAILED_TO_MATERIALIZE, + ) + else: + return event_specific_data_cls(asset_key=asset_key) + + asset_prefix = "grouping_prefix" + asset_keys_to_event_type_and_run_id = { + AssetKey([asset_prefix, "asset_with_prefix_1"]): ( + DagsterEventType.ASSET_MATERIALIZATION, + make_new_run_id(), + ), + AssetKey([asset_prefix, "asset_with_prefix_2"]): ( + DagsterEventType.ASSET_MATERIALIZATION_PLANNED, + make_new_run_id(), + ), + AssetKey([asset_prefix, "asset_with_prefix_3"]): ( + DagsterEventType.ASSET_OBSERVATION, + make_new_run_id(), + ), + AssetKey([asset_prefix, "asset_with_prefix_4"]): ( + DagsterEventType.ASSET_MATERIALIZATION, + make_new_run_id(), + ), + AssetKey([asset_prefix, "asset_with_prefix_5"]): ( + DagsterEventType.ASSET_FAILED_TO_MATERIALIZE, + make_new_run_id(), + ), + } + + for i, (asset_key, (event_type, run_id)) in enumerate( + asset_keys_to_event_type_and_run_id.items() + ): + storage.store_event( + EventLogEntry( + error_info=None, + user_message="", + level="debug", + run_id=run_id, + timestamp=float(i), + dagster_event=DagsterEvent( + event_type, + "nonce", + event_specific_data=build_event_specific_data(event_type, asset_key), ), - ), + ) ) - ) - storage.store_event( - EventLogEntry( - error_info=None, - user_message="", - level="debug", - run_id=asset_keys_to_run_ids[query_keys[4]], - timestamp=5.0, - dagster_event=DagsterEvent.build_asset_failed_to_materialize_event( - DagsterEventType.ASSET_FAILED_TO_MATERIALIZE.value, - "nonce", - asset_materialization_failure=AssetMaterializationFailure( - asset_key=query_keys[4], - failure_type=AssetMaterializationFailureType.FAILED, - reason=AssetMaterializationFailureReason.FAILED_TO_MATERIALIZE, - ), - ), + + expected_order: dict[AssetKey, Optional[int]] = {} + + expected_order = { + asset_key: storage.get_records_for_run( + run_id=run_id, + of_type=event_type, + limit=1, ) - ) - if storage.asset_records_have_last_planned_and_failed_materializations: - expected_order = { - query_keys[0]: storage.get_records_for_run( - run_id=asset_keys_to_run_ids[query_keys[0]], - of_type=DagsterEventType.ASSET_MATERIALIZATION, - limit=1, - ) - .records[0] - .storage_id, - query_keys[1]: storage.get_records_for_run( - run_id=asset_keys_to_run_ids[query_keys[1]], - of_type=DagsterEventType.ASSET_MATERIALIZATION_PLANNED, - limit=1, - ) - .records[0] - .storage_id, - query_keys[2]: storage.get_records_for_run( - run_id=asset_keys_to_run_ids[query_keys[2]], - of_type=DagsterEventType.ASSET_OBSERVATION, - limit=1, - ) - .records[0] - .storage_id, - query_keys[3]: storage.get_records_for_run( - run_id=asset_keys_to_run_ids[query_keys[3]], - of_type=DagsterEventType.ASSET_MATERIALIZATION, - limit=1, - ) - .records[0] - .storage_id, - query_keys[4]: storage.get_records_for_run( - run_id=asset_keys_to_run_ids[query_keys[4]], - of_type=DagsterEventType.ASSET_FAILED_TO_MATERIALIZE, - limit=1, - ) - .records[0] - .storage_id, - } - else: - expected_order = { - query_keys[0]: storage.get_records_for_run( - run_id=asset_keys_to_run_ids[query_keys[0]], - of_type=DagsterEventType.ASSET_MATERIALIZATION, - limit=1, - ) - .records[0] - .storage_id, - query_keys[1]: None, - query_keys[2]: None, - query_keys[3]: storage.get_records_for_run( - run_id=asset_keys_to_run_ids[query_keys[3]], - of_type=DagsterEventType.ASSET_MATERIALIZATION, - limit=1, - ) - .records[0] - .storage_id, - query_keys[4]: None, - } + .records[0] + .storage_id + for asset_key, (event_type, run_id) in asset_keys_to_event_type_and_run_id.items() + } + + if not storage.asset_records_have_last_planned_and_failed_materializations: + for asset_key, (run_id, event_type) in asset_keys_to_event_type_and_run_id.items(): + if event_type != DagsterEventType.ASSET_MATERIALIZATION: + expected_order[asset_key] = None result = execute_dagster_graphql( graphql_context, From ba55e71984084fdd2da8420964ac3f312f725dbe Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Thu, 17 Apr 2025 10:41:06 -0400 Subject: [PATCH 08/11] fix it --- .../graphql/test_assets.py | 91 ++++++++----------- 1 file changed, 40 insertions(+), 51 deletions(-) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py index 23f0828f6b598..cda0d5d3f184d 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py @@ -24,9 +24,12 @@ AssetMaterializationFailure, AssetMaterializationFailureReason, AssetMaterializationFailureType, + AssetObservation, ) from dagster._core.definitions.multi_dimensional_partitions import MultiPartitionKey +from dagster._core.event_api import EventRecordsFilter from dagster._core.events import ( + AssetFailedToMaterializeData, AssetMaterializationPlannedData, AssetObservationData, StepMaterializationData, @@ -3045,67 +3048,57 @@ def test_get_partition_mapping(self, graphql_context: WorkspaceRequestContext): ) def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): - """Tests that the asset event sort key is correct, and based on the latest event for an asset - when querying for asset events. - """ + """Tests that the asset event sort key is correct, and based on the latest event for an asset.""" storage = graphql_context.instance.event_log_storage event_type_to_event_specific_data_mapping = { - DagsterEventType.ASSET_MATERIALIZATION.value: StepMaterializationData, - DagsterEventType.ASSET_MATERIALIZATION_PLANNED.value: AssetMaterializationPlannedData, - DagsterEventType.ASSET_OBSERVATION.value: AssetObservationData, - DagsterEventType.ASSET_FAILED_TO_MATERIALIZE.value: AssetMaterializationFailure, - } - - def build_event_specific_data(event_type: DagsterEventType, asset_key: AssetKey): - event_specific_data_cls = event_type_to_event_specific_data_mapping[event_type.value] - if event_type == DagsterEventType.ASSET_FAILED_TO_MATERIALIZE.value: - return event_specific_data_cls( + DagsterEventType.ASSET_MATERIALIZATION.value: lambda asset_key: StepMaterializationData( + AssetMaterialization(asset_key=asset_key) + ), + DagsterEventType.ASSET_MATERIALIZATION_PLANNED.value: lambda asset_key: AssetMaterializationPlannedData( + asset_key=asset_key + ), + DagsterEventType.ASSET_OBSERVATION.value: lambda asset_key: AssetObservationData( + AssetObservation(asset_key=asset_key) + ), + DagsterEventType.ASSET_FAILED_TO_MATERIALIZE.value: lambda asset_key: AssetFailedToMaterializeData( + AssetMaterializationFailure( asset_key=asset_key, failure_type=AssetMaterializationFailureType.FAILED, reason=AssetMaterializationFailureReason.FAILED_TO_MATERIALIZE, ) - else: - return event_specific_data_cls(asset_key=asset_key) + ), + } asset_prefix = "grouping_prefix" - asset_keys_to_event_type_and_run_id = { - AssetKey([asset_prefix, "asset_with_prefix_1"]): ( - DagsterEventType.ASSET_MATERIALIZATION, - make_new_run_id(), - ), - AssetKey([asset_prefix, "asset_with_prefix_2"]): ( - DagsterEventType.ASSET_MATERIALIZATION_PLANNED, - make_new_run_id(), - ), - AssetKey([asset_prefix, "asset_with_prefix_3"]): ( - DagsterEventType.ASSET_OBSERVATION, - make_new_run_id(), - ), - AssetKey([asset_prefix, "asset_with_prefix_4"]): ( - DagsterEventType.ASSET_MATERIALIZATION, - make_new_run_id(), - ), - AssetKey([asset_prefix, "asset_with_prefix_5"]): ( - DagsterEventType.ASSET_FAILED_TO_MATERIALIZE, - make_new_run_id(), - ), + asset_keys_to_event_type = { + AssetKey([asset_prefix, "asset_with_prefix_1"]): DagsterEventType.ASSET_MATERIALIZATION, + AssetKey( + [asset_prefix, "asset_with_prefix_2"] + ): DagsterEventType.ASSET_MATERIALIZATION_PLANNED, + AssetKey([asset_prefix, "asset_with_prefix_3"]): DagsterEventType.ASSET_OBSERVATION, + AssetKey([asset_prefix, "asset_with_prefix_4"]): DagsterEventType.ASSET_MATERIALIZATION, + AssetKey( + [asset_prefix, "asset_with_prefix_5"] + ): DagsterEventType.ASSET_FAILED_TO_MATERIALIZE, } - for i, (asset_key, (event_type, run_id)) in enumerate( - asset_keys_to_event_type_and_run_id.items() - ): + test_run_id = make_new_run_id() + + for i, (asset_key, event_type) in enumerate(asset_keys_to_event_type.items()): storage.store_event( EventLogEntry( error_info=None, user_message="", level="debug", - run_id=run_id, + run_id=test_run_id, timestamp=float(i), dagster_event=DagsterEvent( - event_type, + event_type.value, "nonce", - event_specific_data=build_event_specific_data(event_type, asset_key), + event_specific_data=event_type_to_event_specific_data_mapping[ + event_type.value + ](asset_key), ), ) ) @@ -3113,18 +3106,15 @@ def build_event_specific_data(event_type: DagsterEventType, asset_key: AssetKey) expected_order: dict[AssetKey, Optional[int]] = {} expected_order = { - asset_key: storage.get_records_for_run( - run_id=run_id, - of_type=event_type, + asset_key: storage.get_event_records( + event_records_filter=EventRecordsFilter(event_type=event_type, asset_key=asset_key), limit=1, - ) - .records[0] - .storage_id - for asset_key, (event_type, run_id) in asset_keys_to_event_type_and_run_id.items() + )[0].storage_id + for asset_key, event_type in asset_keys_to_event_type.items() } if not storage.asset_records_have_last_planned_and_failed_materializations: - for asset_key, (run_id, event_type) in asset_keys_to_event_type_and_run_id.items(): + for asset_key, event_type in asset_keys_to_event_type.items(): if event_type != DagsterEventType.ASSET_MATERIALIZATION: expected_order[asset_key] = None @@ -3136,7 +3126,6 @@ def build_event_specific_data(event_type: DagsterEventType, asset_key: AssetKey) }, ) assert result - assert len(result.data["assetsOrError"]["nodes"]) == 5 for list_item in result.data["assetsOrError"]["nodes"]: asset_key = AssetKey.from_graphql_input(list_item["key"]) From 0c7ecb7ac07c5498cbe2b47dd014e65cc9994d83 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Thu, 17 Apr 2025 11:41:48 -0400 Subject: [PATCH 09/11] make bigint --- .../dagster-ui/packages/ui-core/src/graphql/schema.graphql | 4 +++- js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts | 5 +++-- .../dagster_graphql/schema/pipelines/pipeline.py | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql b/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql index fef19e0522240..7e665f70ebf9c 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql @@ -627,7 +627,7 @@ type Asset { cursor: String ): MaterializationHistoryConnection! definition: AssetNode - latestEventSortKey: Int + latestEventSortKey: BigInt } type MaterializationHistoryConnection { @@ -1098,6 +1098,8 @@ type AssetCheckNeedsAgentUpgradeError implements Error { union Instigator = Schedule | Sensor +scalar BigInt + enum EvaluationErrorReason { RUNTIME_TYPE_MISMATCH MISSING_REQUIRED_FIELD diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts b/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts index 245e319d82d58..78b488dc39e13 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts @@ -16,6 +16,7 @@ export type Scalars = { Boolean: {input: boolean; output: boolean}; Int: {input: number; output: number}; Float: {input: number; output: number}; + BigInt: {input: any; output: any}; GenericScalar: {input: any; output: any}; JSONString: {input: any; output: any}; RunConfigData: {input: any; output: any}; @@ -106,7 +107,7 @@ export type Asset = { definition: Maybe; id: Scalars['String']['output']; key: AssetKey; - latestEventSortKey: Maybe; + latestEventSortKey: Maybe; }; export type AssetAssetMaterializationHistoryArgs = { @@ -6205,7 +6206,7 @@ export const buildAsset = ( latestEventSortKey: overrides && overrides.hasOwnProperty('latestEventSortKey') ? overrides.latestEventSortKey! - : 6449, + : 'facilis', }; }; diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py index a9938d283ed13..9129c07efabb0 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py @@ -248,7 +248,7 @@ class GrapheneAsset(graphene.ObjectType): cursor=graphene.String(), ) definition = graphene.Field("dagster_graphql.schema.asset_graph.GrapheneAssetNode") - latestEventSortKey = graphene.Field(graphene.Int) + latestEventSortKey = graphene.Field(graphene.BigInt) class Meta: name = "Asset" From 5b018154cb0cb4f4de3b79467ac4f44b1e9dbb7f Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 18 Apr 2025 14:53:37 -0400 Subject: [PATCH 10/11] use id --- .../dagster-ui/packages/ui-core/src/graphql/schema.graphql | 4 +--- js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts | 5 ++--- .../dagster_graphql/schema/pipelines/pipeline.py | 2 +- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql b/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql index 7e665f70ebf9c..2f1999cfef22b 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/schema.graphql @@ -627,7 +627,7 @@ type Asset { cursor: String ): MaterializationHistoryConnection! definition: AssetNode - latestEventSortKey: BigInt + latestEventSortKey: ID } type MaterializationHistoryConnection { @@ -1098,8 +1098,6 @@ type AssetCheckNeedsAgentUpgradeError implements Error { union Instigator = Schedule | Sensor -scalar BigInt - enum EvaluationErrorReason { RUNTIME_TYPE_MISMATCH MISSING_REQUIRED_FIELD diff --git a/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts b/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts index 78b488dc39e13..1664c2d4cbc97 100644 --- a/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts +++ b/js_modules/dagster-ui/packages/ui-core/src/graphql/types.ts @@ -16,7 +16,6 @@ export type Scalars = { Boolean: {input: boolean; output: boolean}; Int: {input: number; output: number}; Float: {input: number; output: number}; - BigInt: {input: any; output: any}; GenericScalar: {input: any; output: any}; JSONString: {input: any; output: any}; RunConfigData: {input: any; output: any}; @@ -107,7 +106,7 @@ export type Asset = { definition: Maybe; id: Scalars['String']['output']; key: AssetKey; - latestEventSortKey: Maybe; + latestEventSortKey: Maybe; }; export type AssetAssetMaterializationHistoryArgs = { @@ -6206,7 +6205,7 @@ export const buildAsset = ( latestEventSortKey: overrides && overrides.hasOwnProperty('latestEventSortKey') ? overrides.latestEventSortKey! - : 'facilis', + : 'b9e5eeed-491e-4839-9bbf-1dedd727f77b', }; }; diff --git a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py index 9129c07efabb0..a2ccfd3681e3d 100644 --- a/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py +++ b/python_modules/dagster-graphql/dagster_graphql/schema/pipelines/pipeline.py @@ -248,7 +248,7 @@ class GrapheneAsset(graphene.ObjectType): cursor=graphene.String(), ) definition = graphene.Field("dagster_graphql.schema.asset_graph.GrapheneAssetNode") - latestEventSortKey = graphene.Field(graphene.BigInt) + latestEventSortKey = graphene.Field(graphene.ID) class Meta: name = "Asset" From b43b82fb4f38267f0e5073b4685dd19429f56ae6 Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Mon, 21 Apr 2025 09:43:25 -0400 Subject: [PATCH 11/11] fix test --- .../dagster_graphql_tests/graphql/test_assets.py | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py index cda0d5d3f184d..cb8b077aa08a3 100644 --- a/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py +++ b/python_modules/dagster-graphql/dagster_graphql_tests/graphql/test_assets.py @@ -3103,13 +3103,17 @@ def test_asset_event_sort_key(self, graphql_context: WorkspaceRequestContext): ) ) - expected_order: dict[AssetKey, Optional[int]] = {} + expected_order: dict[AssetKey, Optional[str]] = {} expected_order = { - asset_key: storage.get_event_records( - event_records_filter=EventRecordsFilter(event_type=event_type, asset_key=asset_key), - limit=1, - )[0].storage_id + asset_key: str( + storage.get_event_records( + event_records_filter=EventRecordsFilter( + event_type=event_type, asset_key=asset_key + ), + limit=1, + )[0].storage_id + ) for asset_key, event_type in asset_keys_to_event_type.items() }