def asset_partitions_for_job_partition( job_partition_key: str, ) -> Mapping[AssetKey, PartitionKeyRange]: return { asset_key: PartitionKeyRange(job_partition_key, job_partition_key) for assets_def in assets for asset_key in assets_def.asset_keys if assets_def.partitions_def }
def get_downstream_partitions_for_partition_range( self, upstream_partition_key_range: PartitionKeyRange, downstream_partitions_def: PartitionsDefinition, # pylint: disable=unused-argument upstream_partitions_def: PartitionsDefinition, # pylint: disable=unused-argument ) -> PartitionKeyRange: return PartitionKeyRange( upstream_partition_key_range.start.split("|")[-1], upstream_partition_key_range.end.split("|")[-1], )
def get_upstream_partitions_for_partition_range( self, downstream_partition_key_range: PartitionKeyRange, downstream_partitions_def: PartitionsDefinition, # pylint: disable=unused-argument upstream_partitions_def: PartitionsDefinition, # pylint: disable=unused-argument ) -> PartitionKeyRange: return PartitionKeyRange( f"{self.hemisphere}|{downstream_partition_key_range.start}", f"{self.hemisphere}|{downstream_partition_key_range.end}", )
def get_upstream_partitions_for_partition_range( self, downstream_partition_key_range: PartitionKeyRange, downstream_partitions_def: PartitionsDefinition, upstream_partitions_def: PartitionsDefinition, ) -> PartitionKeyRange: assert downstream_partitions_def assert upstream_partitions_def start, end = downstream_partition_key_range return PartitionKeyRange(str(max(1, int(start) - 1)), end)
def get_upstream_partitions_for_partition_range( self, downstream_partition_key_range: PartitionKeyRange, downstream_partitions_def: PartitionsDefinition, upstream_partitions_def: PartitionsDefinition, ) -> PartitionKeyRange: del downstream_partitions_def, upstream_partitions_def start, end = downstream_partition_key_range assert start == "2020-01-02" assert end == "2020-01-02" return PartitionKeyRange("2020-01-01", "2020-01-02")
def asset_partition_key_range_for_output(self, output_name: str) -> PartitionKeyRange: op_config = self.op_config if op_config is not None and "assets" in op_config: all_output_asset_partitions = op_config["assets"].get("output_partitions") if all_output_asset_partitions is not None: this_output_asset_partitions = all_output_asset_partitions.get(output_name) if this_output_asset_partitions is not None: return PartitionKeyRange( this_output_asset_partitions["start"], this_output_asset_partitions["end"] ) check.failed("The output has no asset partitions")
def test_assets_with_same_partitioning(): partitions_def = StaticPartitionsDefinition(["a", "b", "c", "d"]) @asset(partitions_def=partitions_def) def upstream_asset(): pass @asset(partitions_def=partitions_def) def downstream_asset(upstream_asset): assert upstream_asset assert get_upstream_partitions_for_partition_range( downstream_asset, upstream_asset, AssetKey("upstream_asset"), PartitionKeyRange("a", "c"), ) == PartitionKeyRange("a", "c") assert get_downstream_partitions_for_partition_range( downstream_asset, upstream_asset, AssetKey("upstream_asset"), PartitionKeyRange("a", "c"), ) == PartitionKeyRange("a", "c")
def _map_partitions( self, from_partitions_def: PartitionsDefinition, to_partitions_def: PartitionsDefinition, from_partition_key_range: PartitionKeyRange, ) -> PartitionKeyRange: if not isinstance(from_partitions_def, TimeWindowPartitionsDefinition) or not isinstance( from_partitions_def, TimeWindowPartitionsDefinition): raise DagsterInvalidDefinitionError( "TimeWindowPartitionMappings can only operate on TimeWindowPartitionsDefinitions" ) from_partitions_def = cast(TimeWindowPartitionsDefinition, from_partitions_def) to_partitions_def = cast(TimeWindowPartitionsDefinition, to_partitions_def) if to_partitions_def.timezone != from_partitions_def.timezone: raise DagsterInvalidDefinitionError("Timezones don't match") to_period = to_partitions_def.schedule_type from_period = from_partitions_def.schedule_type from_start_dt = datetime.strptime(from_partition_key_range.start, from_partitions_def.fmt) from_end_dt = datetime.strptime(from_partition_key_range.end, from_partitions_def.fmt) if to_period > from_period: to_start_dt = round_datetime_to_period(from_start_dt, to_period) to_end_dt = round_datetime_to_period(from_end_dt, to_period) elif to_period < from_period: to_start_dt = from_start_dt to_end_dt = (from_end_dt + from_period.delta) - to_period.delta else: to_start_dt = from_start_dt to_end_dt = from_end_dt return PartitionKeyRange( to_start_dt.strftime(to_partitions_def.fmt), to_end_dt.strftime(to_partitions_def.fmt), )
def test_filter_mapping_partitions_dep(): downstream_partitions = ["john", "ringo", "paul", "george"] upstream_partitions = [ f"{hemisphere}|{beatle}" for beatle in downstream_partitions for hemisphere in ["southern", "northern"] ] downstream_partitions_def = StaticPartitionsDefinition( downstream_partitions) upstream_partitions_def = StaticPartitionsDefinition(upstream_partitions) class HemisphereFilteringPartitionMapping(PartitionMapping): def __init__(self, hemisphere: str): self.hemisphere = hemisphere def get_upstream_partitions_for_partition_range( self, downstream_partition_key_range: PartitionKeyRange, downstream_partitions_def: PartitionsDefinition, # pylint: disable=unused-argument upstream_partitions_def: PartitionsDefinition, # pylint: disable=unused-argument ) -> PartitionKeyRange: return PartitionKeyRange( f"{self.hemisphere}|{downstream_partition_key_range.start}", f"{self.hemisphere}|{downstream_partition_key_range.end}", ) def get_downstream_partitions_for_partition_range( self, upstream_partition_key_range: PartitionKeyRange, downstream_partitions_def: PartitionsDefinition, # pylint: disable=unused-argument upstream_partitions_def: PartitionsDefinition, # pylint: disable=unused-argument ) -> PartitionKeyRange: return PartitionKeyRange( upstream_partition_key_range.start.split("|")[-1], upstream_partition_key_range.end.split("|")[-1], ) @asset(partitions_def=upstream_partitions_def) def upstream_asset(): pass @asset( partitions_def=downstream_partitions_def, partition_mappings={ "upstream_asset": HemisphereFilteringPartitionMapping("southern") }, ) def downstream_asset(upstream_asset): assert upstream_asset assert get_upstream_partitions_for_partition_range( downstream_asset, upstream_asset, AssetKey("upstream_asset"), PartitionKeyRange("ringo", "paul"), ) == PartitionKeyRange("southern|ringo", "southern|paul") assert get_downstream_partitions_for_partition_range( downstream_asset, upstream_asset, AssetKey("upstream_asset"), PartitionKeyRange("southern|ringo", "southern|paul"), ) == PartitionKeyRange("ringo", "paul")