diff --git a/python_modules/dagster/dagster/_core/asset_graph_view/asset_graph_view.py b/python_modules/dagster/dagster/_core/asset_graph_view/asset_graph_view.py index f4860f3992071..56d2aa7398b30 100644 --- a/python_modules/dagster/dagster/_core/asset_graph_view/asset_graph_view.py +++ b/python_modules/dagster/dagster/_core/asset_graph_view/asset_graph_view.py @@ -250,8 +250,6 @@ def compute_mapped_subset( from_partitions_def = self.asset_graph.get(from_key).partitions_def to_partitions_def = self.asset_graph.get(to_key).partitions_def - partition_mapping = self.asset_graph.get_partition_mapping(from_key, to_key) - if direction == "down": if from_partitions_def is None or to_partitions_def is None: return ( @@ -259,6 +257,11 @@ def compute_mapped_subset( if from_subset.is_empty else self.get_full_subset(key=to_key) ) + + child_key = to_key + parent_key = from_key + partition_mapping = self.asset_graph.get_partition_mapping(child_key, parent_key) + to_partitions_subset = partition_mapping.get_downstream_partitions_for_partitions( upstream_partitions_subset=from_subset.get_internal_subset_value(), upstream_partitions_def=from_partitions_def, @@ -273,6 +276,11 @@ def compute_mapped_subset( if from_subset.is_empty else self.get_full_subset(key=to_key) ) + + child_key = from_key + parent_key = to_key + partition_mapping = self.asset_graph.get_partition_mapping(child_key, parent_key) + to_partitions_subset = ( partition_mapping.get_upstream_mapped_partitions_result_for_partitions( downstream_partitions_subset=from_subset.get_internal_subset_value() diff --git a/python_modules/dagster/dagster_tests/asset_defs_tests/asset_graph_view_tests/test_basic_asset_graph_view.py b/python_modules/dagster/dagster_tests/asset_defs_tests/asset_graph_view_tests/test_basic_asset_graph_view.py index dc5827e81f621..43696c2141fd6 100644 --- a/python_modules/dagster/dagster_tests/asset_defs_tests/asset_graph_view_tests/test_basic_asset_graph_view.py +++ b/python_modules/dagster/dagster_tests/asset_defs_tests/asset_graph_view_tests/test_basic_asset_graph_view.py @@ -62,12 +62,11 @@ def down_letters(): ... # from full up to down up_subset = asset_graph_view_t0.get_full_subset(key=up_numbers.key) assert up_subset.expensively_compute_partition_keys() == {"1", "2", "3"} - assert ( - up_subset.compute_child_subset(down_letters.key).expensively_compute_partition_keys() - == set() - ) + assert up_subset.compute_child_subset( + down_letters.key + ).expensively_compute_partition_keys() == {"a", "b", "c"} - # from full up to down + # from full down to up down_subset = asset_graph_view_t0.get_full_subset(key=down_letters.key) assert down_subset.expensively_compute_partition_keys() == {"a", "b", "c"} assert down_subset.compute_parent_subset( @@ -98,12 +97,9 @@ def down_letters(): ... ) # subset of up to subset of down - assert ( - up_subset.compute_intersection_with_partition_keys({"2"}) - .compute_child_subset(down_letters.key) - .expensively_compute_partition_keys() - == set() - ) + assert up_subset.compute_intersection_with_partition_keys({"2"}).compute_child_subset( + down_letters.key + ).expensively_compute_partition_keys() == {"b"} # subset of down to subset of up assert down_subset.compute_intersection_with_partition_keys({"b"}).compute_parent_subset(