Skip to content

Commit

Permalink
small
Browse files Browse the repository at this point in the history
  • Loading branch information
jamiedemaria committed Nov 12, 2024
1 parent 5a2987f commit 514d595
Show file tree
Hide file tree
Showing 3 changed files with 7 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -245,7 +245,7 @@ def __repr__(self) -> str:
)

@classmethod
def empty():
def empty(cls):
return AssetGraphSubset(
partitions_subsets_by_asset_key={}, non_partitioned_asset_keys=set()
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1822,9 +1822,12 @@ def _get_failed_asset_partitions(
start=run.tags[ASSET_PARTITION_RANGE_START_TAG],
end=run.tags[ASSET_PARTITION_RANGE_END_TAG],
)
asset_partition_candidates = []
for asset_key in failed_asset_keys:
asset_partition_candidates = asset_graph.get_partitions_in_range(
asset_key, partition_range, instance_queryer
asset_partition_candidates.extend(
asset_graph.get_partitions_in_range(
asset_key, partition_range, instance_queryer
)
)
else:
# a regular backfill run that run on a single partition
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3040,6 +3040,7 @@ def test_asset_backfill_fails_if_retries_fail(
list(execute_backfill_iteration(workspace_context, get_default_daemon_logger("BackfillDaemon")))
backfill = instance.get_backfill(backfill_id)
assert backfill
assert backfill.asset_backfill_data
assert backfill.asset_backfill_data.all_targeted_partitions_have_materialization_status()
assert backfill.status == BulkActionStatus.REQUESTED

Expand Down

0 comments on commit 514d595

Please sign in to comment.