From 8817056559e062a68bd7b8a4ccbfd8852d4c779c Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 1 Aug 2024 12:56:54 -0700 Subject: [PATCH] [docs] AssetSpec IO managers via metadata (#23279) ## Summary & Motivation With `SourceAsset`s being deprecated, posting this PR to kick off a discussion around what we tell users who have been relying on its `io_manager_key` attribute. **Another related option** would be to expose an `IOManagerMetadataSet` instead of a constant. ```python from dagster import AssetSpec, IOManagerMetadataSet my_source_asset = AssetSpec( key=AssetKey("my_source_asset"), metadata=**IOManagerMetadataSet(io_manager_key="s3_io_manager") ) ``` **A third option** would be to expose an `AssetSpecWithIOManager` subclass that internally uses the metadata scheme above. **A fourth option** would be a `with_io_manager` static method on `AssetSpec` that constructs an `AssetSpec` using the metadata schema bove. **A fifth option** would be to deprecate the use of IO managers on non-executable assets. I am not a fan of this option, because I think a big part of the beauty of the IO manager system is being able to depend on an asset and treat the process that generates that asset as a black box. Having one set of capabilities for loading executable assets and another for non-executable assets seems arbitrary. ## How I Tested These Changes --- docs/content/concepts/io-management/io-managers.mdx | 12 ++++++++---- .../concepts/io_management/source_asset.py | 10 +++++++--- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/docs/content/concepts/io-management/io-managers.mdx b/docs/content/concepts/io-management/io-managers.mdx index 9f3013bd4b9c5..ab58035d99902 100644 --- a/docs/content/concepts/io-management/io-managers.mdx +++ b/docs/content/concepts/io-management/io-managers.mdx @@ -240,21 +240,25 @@ defs = Definitions( ### Using I/O managers to load source data -Asset definitions often depend on data assets that are generated outside of Dagster, or in a different code location within Dagster, and it's often useful to use I/O managers to load the data from these assets. To represent one of these assets, you can create an asset definition that includes an I/O manager, but no materialization function. Your other assets can then depend on it and load data from it, just as they would with a materializable asset. +Asset definitions often depend on data assets that are generated outside of Dagster, or in a different code location within Dagster, and it's often useful to use I/O managers to load the data from these assets. You can use an to define an asset with no materialization function, and you can assign an I/O manager to it by including an entry in its metadata dictionary with the `"dagster/io_manager_key"` key. Your other assets can then depend on it and load data from it, just as they would with a materializable asset. For example: ```python file=/concepts/io_management/source_asset.py -from dagster import AssetKey, SourceAsset, asset +from dagster import AssetKey, AssetSpec, Definitions, asset -my_source_asset = SourceAsset( - key=AssetKey("my_source_asset"), io_manager_key="s3_io_manager" +my_source_asset = AssetSpec( + key=AssetKey("my_source_asset"), + metadata={"dagster/io_manager_key": "s3_io_manager"}, ) @asset def my_derived_asset(my_source_asset): return my_source_asset + [4] + + +defs = Definitions(assets=[my_source_asset, my_derived_asset]) ``` ### Asset input I/O managers diff --git a/examples/docs_snippets/docs_snippets/concepts/io_management/source_asset.py b/examples/docs_snippets/docs_snippets/concepts/io_management/source_asset.py index 9280a5078ac49..4043207daf349 100644 --- a/examples/docs_snippets/docs_snippets/concepts/io_management/source_asset.py +++ b/examples/docs_snippets/docs_snippets/concepts/io_management/source_asset.py @@ -1,10 +1,14 @@ -from dagster import AssetKey, SourceAsset, asset +from dagster import AssetKey, AssetSpec, Definitions, asset -my_source_asset = SourceAsset( - key=AssetKey("my_source_asset"), io_manager_key="s3_io_manager" +my_source_asset = AssetSpec( + key=AssetKey("my_source_asset"), + metadata={"dagster/io_manager_key": "s3_io_manager"}, ) @asset def my_derived_asset(my_source_asset): return my_source_asset + [4] + + +defs = Definitions(assets=[my_source_asset, my_derived_asset])