Skip to content

Commit

Permalink
Set RunRequest.partition_key when backfilling with policy with `max…
Browse files Browse the repository at this point in the history
…_partitions_per_run=1`
  • Loading branch information
smackesey committed Jun 12, 2024
1 parent 1c7ba44 commit 695dac3
Show file tree
Hide file tree
Showing 2 changed files with 39 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -421,7 +421,8 @@ def _build_run_request_for_partition_key_range(
ASSET_PARTITION_RANGE_START_TAG: partition_range_start,
ASSET_PARTITION_RANGE_END_TAG: partition_range_end,
}
return RunRequest(asset_selection=asset_keys, tags=tags)
partition_key = partition_range_start if partition_range_start == partition_range_end else None
return RunRequest(asset_selection=asset_keys, partition_key=partition_key, tags=tags)


def get_auto_observe_run_requests(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1072,6 +1072,43 @@ def foo_child():
]


def test_max_partitions_per_range_1_sets_run_request_partition_key():
@asset(
partitions_def=DailyPartitionsDefinition("2023-10-01"),
backfill_policy=BackfillPolicy.multi_run(1),
)
def foo():
pass

asset_graph = get_asset_graph({"repo1": [foo]})

asset_backfill_data = AssetBackfillData.from_asset_partitions(
asset_graph=asset_graph,
partition_names=[
"2023-10-05",
"2023-10-06",
],
asset_selection=[foo.key],
dynamic_partitions_store=MagicMock(),
all_partitions=False,
backfill_start_timestamp=create_datetime(2023, 10, 7, 0, 0, 0).timestamp(),
)

result = execute_asset_backfill_iteration_consume_generator(
"apple", asset_backfill_data, asset_graph, DagsterInstance.ephemeral()
)

assert [run_request.partition_key for run_request in result.run_requests] == [
"2023-10-05",
"2023-10-06",
]

assert [run_request.partition_key_range for run_request in result.run_requests] == [
PartitionKeyRange("2023-10-05", "2023-10-05"),
PartitionKeyRange("2023-10-06", "2023-10-06"),
]


# 0 turns off batching
# 2 will require multiple batches to fulfill the backfill
# 10 will require a single to fulfill the backfill
Expand Down

0 comments on commit 695dac3

Please sign in to comment.