-
Notifications
You must be signed in to change notification settings - Fork 1.5k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[dagster-airlift][rfc] Proxy operator launches partitioned runs (#25324)
## Summary & Motivation Adds a pluggable implementation to BaseAssetsOperator which handles mapping the current airflow run to a partitioned run in Dagster. By default, we do the same thing that we do in the sensor - we attempt to map the logical date directly to a partition. Important points to note: - I make the simplifying assumption that all assets within a given task share the same partitions definition. This makes it so that we can keep to the "one run" constraint from a previous PR. - There's two points of pluggability that I think make sense to expose. The first is the method get_partition_key(context, partition_keys), which allows users to pick a partition key from the list to use. The second is a pluggable default implementation translate_logical_date_to_partition_key, which takes a list of partition key formats. This is to support TimeWindowPartitionsDefinitions that use a custom format / cron schedule without needing to do a full reimplementation. All they would do is override get_partition_key to call translate_logical_date_to_partition_key with their custom format. ## How I Tested These Changes Added a new test which takes a daily dag and constructs a daily partitioned materialization. Might be worth testing all the other formatting cases, as well as pluggability. ## Changelog NOCHANGELOG
- Loading branch information
Showing
15 changed files
with
306 additions
and
8 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -34,6 +34,13 @@ | |
} | ||
} | ||
} | ||
isPartitioned | ||
partitionDefinition { | ||
type | ||
name | ||
fmt | ||
} | ||
partitionKeys | ||
} | ||
} | ||
""" | ||
|
124 changes: 124 additions & 0 deletions
124
examples/experimental/dagster-airlift/dagster_airlift/in_airflow/partition_utils.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,124 @@ | ||
from datetime import ( | ||
datetime, | ||
timezone as tz, | ||
) | ||
from enum import Enum | ||
from typing import Any, Mapping, NamedTuple, Optional, Sequence | ||
|
||
PARTITION_NAME_TAG = "dagster/partition" | ||
|
||
|
||
class PartitionDefinitionType(Enum): | ||
TIME_WINDOW = "TIME_WINDOW" | ||
STATIC = "STATIC" | ||
MULTIPARTITIONED = "MULTIPARTITIONED" | ||
DYNAMIC = "DYNAMIC" | ||
|
||
|
||
class TimeWindowPartitioningInformation(NamedTuple): | ||
fmt: str | ||
|
||
|
||
class PartitioningInformation(NamedTuple): | ||
partitioning_type: PartitionDefinitionType | ||
partition_keys: Sequence[str] | ||
# Eventually we can add more of these for different partitioning types | ||
additional_info: Optional[TimeWindowPartitioningInformation] | ||
|
||
@staticmethod | ||
def from_asset_node_graphql( | ||
asset_nodes: Sequence[Mapping[str, Any]], | ||
) -> Optional["PartitioningInformation"]: | ||
assets_partitioned = [_asset_is_partitioned(asset_node) for asset_node in asset_nodes] | ||
if any(assets_partitioned) and not all(assets_partitioned): | ||
raise Exception( | ||
"Found some unpartitioned assets and some partitioned assets in the same task. " | ||
"For a given task, all assets must have the same partitions definition. " | ||
) | ||
partition_keys_per_asset = [ | ||
set(asset_node["partitionKeys"]) | ||
for asset_node in asset_nodes | ||
if asset_node["isPartitioned"] | ||
] | ||
if not all_sets_equal(partition_keys_per_asset): | ||
raise Exception( | ||
"Found differing partition keys across assets in this task. " | ||
"For a given task, all assets must have the same partitions definition. " | ||
) | ||
# Now we can proceed with the assumption that all assets are partitioned and have the same partition keys. | ||
# This, we only look at the first asset node. | ||
asset_node = next(iter(asset_nodes)) | ||
if not asset_node["isPartitioned"]: | ||
return None | ||
partitioning_type = PartitionDefinitionType(asset_node["partitionDefinition"]["type"]) | ||
return PartitioningInformation( | ||
partitioning_type=partitioning_type, | ||
partition_keys=asset_node["partitionKeys"], | ||
additional_info=_build_additional_info_for_type(asset_node, partitioning_type), | ||
) | ||
|
||
@property | ||
def time_window_partitioning_info(self) -> TimeWindowPartitioningInformation: | ||
if self.partitioning_type != PartitionDefinitionType.TIME_WINDOW: | ||
raise Exception( | ||
f"Partitioning type is {self.partitioning_type}, but expected {PartitionDefinitionType.TIME_WINDOW}" | ||
) | ||
if self.additional_info is None: | ||
raise Exception( | ||
f"Partitioning type is {self.partitioning_type}, but no additional info was provided." | ||
) | ||
return self.additional_info | ||
|
||
|
||
def _build_additional_info_for_type( | ||
asset_node: Mapping[str, Any], partitioning_type: PartitionDefinitionType | ||
) -> Optional[TimeWindowPartitioningInformation]: | ||
if partitioning_type != PartitionDefinitionType.TIME_WINDOW: | ||
return None | ||
return TimeWindowPartitioningInformation(fmt=asset_node["partitionDefinition"]["fmt"]) | ||
|
||
|
||
def all_sets_equal(list_of_sets): | ||
if not list_of_sets: | ||
return True | ||
return len(set.union(*list_of_sets)) == len(set.intersection(*list_of_sets)) | ||
|
||
|
||
def translate_logical_date_to_partition_key( | ||
logical_date: datetime, partitioning_info: PartitioningInformation | ||
) -> str: | ||
if not partitioning_info.partitioning_type == PartitionDefinitionType.TIME_WINDOW: | ||
raise Exception( | ||
"Only time-window partitioned assets or non-partitioned assets are supported out of the box." | ||
) | ||
fmt = partitioning_info.time_window_partitioning_info.fmt | ||
partitions_and_datetimes = [ | ||
(_get_partition_datetime(partition_key, fmt), partition_key) | ||
for partition_key in partitioning_info.partition_keys | ||
] | ||
matching_partition = next( | ||
( | ||
partition_key | ||
for datetime, partition_key in partitions_and_datetimes | ||
if datetime.timestamp() == logical_date.timestamp() | ||
), | ||
None, | ||
) | ||
if matching_partition is None: | ||
raise Exception(f"No partition key found for logical date {logical_date}") | ||
return matching_partition | ||
|
||
|
||
def _asset_is_partitioned(asset_node: Mapping[str, Any]) -> bool: | ||
return asset_node["isPartitioned"] | ||
|
||
|
||
def _get_partition_datetime(partition_key: str, fmt: str) -> datetime: | ||
try: | ||
return _add_default_utc_timezone_if_none(datetime.strptime(partition_key, fmt)) | ||
except ValueError: | ||
raise Exception(f"Could not parse partition key {partition_key} with format {fmt}.") | ||
|
||
|
||
def _add_default_utc_timezone_if_none(dt: datetime) -> datetime: | ||
return dt.replace(tzinfo=tz.utc) if dt.tzinfo is None else dt |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
35 changes: 35 additions & 0 deletions
35
...l/dagster-airlift/examples/kitchen-sink/kitchen_sink/airflow_dags/migrated_partitioned.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,35 @@ | ||
from datetime import timedelta | ||
from pathlib import Path | ||
|
||
from airflow import DAG | ||
from airflow.operators.python import PythonOperator | ||
from dagster._time import get_current_datetime_midnight | ||
from dagster_airlift.in_airflow import proxying_to_dagster | ||
from dagster_airlift.in_airflow.proxied_state import load_proxied_state_from_yaml | ||
|
||
|
||
def print_hello() -> None: | ||
print("Hello") # noqa: T201 | ||
|
||
|
||
default_args = { | ||
"owner": "airflow", | ||
"depends_on_past": False, | ||
"retries": 0, | ||
} | ||
|
||
with DAG( | ||
dag_id="migrated_daily_interval_dag", | ||
default_args=default_args, | ||
schedule="@daily", | ||
start_date=get_current_datetime_midnight() - timedelta(days=1), | ||
# We pause this dag upon creation to avoid running it immediately | ||
is_paused_upon_creation=True, | ||
) as minute_dag: | ||
PythonOperator(task_id="my_task", python_callable=print_hello) | ||
|
||
|
||
proxying_to_dagster( | ||
proxied_state=load_proxied_state_from_yaml(Path(__file__).parent / "proxied_state"), | ||
global_vars=globals(), | ||
) |
3 changes: 3 additions & 0 deletions
3
...les/kitchen-sink/kitchen_sink/airflow_dags/proxied_state/migrated_daily_interval_dag.yaml
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,3 @@ | ||
tasks: | ||
- id: my_task | ||
proxied: True |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.