From 2ab41c8ef9aa6f65553b10208028dc3e4712e26d Mon Sep 17 00:00:00 2001 From: JamieDeMaria Date: Fri, 27 Oct 2023 12:18:19 -0400 Subject: [PATCH] partition keys prop --- .../_core/execution/context/compute.py | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/python_modules/dagster/dagster/_core/execution/context/compute.py b/python_modules/dagster/dagster/_core/execution/context/compute.py index edaccaa2f041c..3c85a33637fc9 100644 --- a/python_modules/dagster/dagster/_core/execution/context/compute.py +++ b/python_modules/dagster/dagster/_core/execution/context/compute.py @@ -308,6 +308,47 @@ def my_asset(context: AssetExecutionContext): """ return self._step_execution_context.partition_key + @public + @property + def partition_keys(self) -> Sequence[str]: + """Returns a list of the partition keys for the current run. + + If you want to write your asset to support running a backfill of several partitions in a single run, + you can use ``partition_keys`` to get all of the partitions being materialized + by the backfill. + + Raises an error if the asset is a multi-asset. In a multi-asset, use ``asset_partition_keys_for_output`` + + Examples: + .. code-block:: python + + partitions_def = DailyPartitionsDefinition("2023-08-20") + + @asset( + partitions_def=partitions_def + ) + def an_asset(context: AssetExecutionContext): + context.log.info(context.partition_keys) + + + # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log: + # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"] + + + @asset( + partitions_def=partitions_def, + ins={ + "self_dependent_asset": AssetIn(partition_mapping=TimeWindowPartitionMapping(start_offset=-1, end_offset=-1)) + } + ) + def self_dependent_asset(context: AssetExecutionContext, self_dependent_asset): + context.log.info(context.partition_keys) + + # running a backfill of the 2023-08-21 through 2023-08-25 partitions of this asset will log: + # ["2023-08-21", "2023-08-22", "2023-08-23", "2023-08-24", "2023-08-25"] + """ + return self.asset_partition_keys_for_output() + @deprecated(breaking_version="2.0", additional_warn_text="Use `partition_key_range` instead.") @public @property