changed gtfs_feed_downloads to a sensor instead of cron
This commit is contained in:
parent
37f23fda18
commit
97aa28e003
5 changed files with 176 additions and 17 deletions
|
|
@ -20,7 +20,7 @@ logger = logging.getLogger(__name__)
|
|||
|
||||
@asset(
|
||||
deps=["agency_list"],
|
||||
group_name="gtfs_rt_metadata",
|
||||
group_name="gtfs_rt_vehicles_metadata",
|
||||
automation_condition=AutomationCondition.eager()
|
||||
)
|
||||
def gtfs_rt_vehicles_metadata(
|
||||
|
|
@ -150,7 +150,7 @@ gtfs_rt_vehicles_partitions_def = DynamicPartitionsDefinition(name="gtfs_rt_vehi
|
|||
|
||||
@asset(
|
||||
deps=["gtfs_rt_vehicles_metadata"],
|
||||
group_name="gtfs_rt_metadata",
|
||||
group_name="gtfs_rt_vehicles_metadata",
|
||||
automation_condition=AutomationCondition.eager()
|
||||
)
|
||||
def gtfs_rt_vehicles_partitions(
|
||||
|
|
@ -196,7 +196,7 @@ class GTFSRTDownloadConfig(Config):
|
|||
@asset(
|
||||
partitions_def=gtfs_rt_vehicles_partitions_def,
|
||||
deps=[gtfs_rt_vehicles_partitions],
|
||||
group_name="gtfs_rt_downloads",
|
||||
group_name="gtfs_rt_vehicles_downloads",
|
||||
)
|
||||
def gtfs_rt_vehicles_downloads(
|
||||
context: AssetExecutionContext,
|
||||
|
|
|
|||
|
|
@ -5,6 +5,7 @@ from dagster import (
|
|||
MetadataValue,
|
||||
AutomationCondition,
|
||||
DynamicPartitionsDefinition,
|
||||
Config
|
||||
)
|
||||
from dagster_duckdb import DuckDBResource
|
||||
from resources import MobilityDatabaseAPI
|
||||
|
|
@ -14,9 +15,14 @@ from pathlib import Path
|
|||
from datetime import datetime
|
||||
import logging
|
||||
|
||||
|
||||
logger = logging.getLogger(__name__)
|
||||
|
||||
|
||||
class GTFSDownloadConfig(Config):
|
||||
provider: str
|
||||
producer_url: str
|
||||
|
||||
|
||||
@asset(
|
||||
deps=["agency_list"],
|
||||
group_name="gtfs_metadata",
|
||||
|
|
@ -191,10 +197,10 @@ def gtfs_feed_partitions(
|
|||
partitions_def=gtfs_feeds_partitions_def,
|
||||
deps=["gtfs_feed_partitions"],
|
||||
group_name="gtfs_downloads",
|
||||
automation_condition=AutomationCondition.on_cron("0 * * * *") | AutomationCondition.eager(),
|
||||
)
|
||||
def gtfs_feed_downloads(
|
||||
context: AssetExecutionContext,
|
||||
config: GTFSDownloadConfig,
|
||||
duckdb: DuckDBResource,
|
||||
) -> Output[None]:
|
||||
"""
|
||||
|
|
@ -204,19 +210,10 @@ def gtfs_feed_downloads(
|
|||
Runs on the hour and whenever new partitions are added.
|
||||
"""
|
||||
feed_id = context.partition_key
|
||||
download_url = config.producer_url
|
||||
provider = config.provider
|
||||
|
||||
with duckdb.get_connection() as conn:
|
||||
# Get the download URL for this feed
|
||||
feed_info = conn.execute("""
|
||||
SELECT feed_id, provider, producer_url
|
||||
FROM gtfs_feed_metadata
|
||||
WHERE feed_id = ?
|
||||
""", [feed_id]).fetchone()
|
||||
|
||||
if not feed_info:
|
||||
raise ValueError(f"Feed {feed_id} not found in metadata")
|
||||
|
||||
feed_id, provider, download_url = feed_info
|
||||
|
||||
if not download_url:
|
||||
context.log.warning(f"No download URL for {feed_id}")
|
||||
|
|
|
|||
|
|
@ -11,6 +11,7 @@ from dagster_duckdb import DuckDBResource
|
|||
from assets import config, gtfs_static, gtfs_realtime
|
||||
|
||||
# Import sensor modules
|
||||
from sensors import gtfs_static as gtfs_static_sensors
|
||||
from sensors import gtfs_realtime as gtfs_rt_sensors
|
||||
|
||||
from resources import MobilityDatabaseAPI
|
||||
|
|
@ -26,6 +27,8 @@ defs = Definitions(
|
|||
target="*",
|
||||
default_status=DefaultSensorStatus.RUNNING,
|
||||
),
|
||||
gtfs_static_sensors.gtfs_static_hourly_sensor,
|
||||
gtfs_static_sensors.gtfs_static_partition_update_sensor,
|
||||
gtfs_rt_sensors.gtfs_rt_vehicles_sensor,
|
||||
],
|
||||
resources={
|
||||
|
|
|
|||
|
|
@ -1,8 +1,11 @@
|
|||
"""
|
||||
"""""
|
||||
GTFS data pipeline sensors.
|
||||
"""
|
||||
from .gtfs_static import *
|
||||
from .gtfs_realtime import *
|
||||
|
||||
__all__ = [
|
||||
"gtfs_static_hourly_sensor",
|
||||
"gtfs_static_partition_update_sensor",
|
||||
"gtfs_rt_vehicles_sensor",
|
||||
]
|
||||
|
|
|
|||
156
user_code/sensors/gtfs_static.py
Normal file
156
user_code/sensors/gtfs_static.py
Normal file
|
|
@ -0,0 +1,156 @@
|
|||
from dagster import (
|
||||
sensor,
|
||||
RunRequest,
|
||||
SkipReason,
|
||||
SensorEvaluationContext,
|
||||
DefaultSensorStatus,
|
||||
SensorResult,
|
||||
EventLogEntry,
|
||||
)
|
||||
from dagster_duckdb import DuckDBResource
|
||||
|
||||
|
||||
@sensor(
|
||||
name="gtfs_static_hourly_sensor",
|
||||
minimum_interval_seconds=3600, # 60 minutes
|
||||
asset_selection=["gtfs_feed_downloads"],
|
||||
default_status=DefaultSensorStatus.RUNNING
|
||||
)
|
||||
def gtfs_static_hourly_sensor(
|
||||
context: SensorEvaluationContext,
|
||||
duckdb: DuckDBResource,
|
||||
) -> list[RunRequest] | SkipReason:
|
||||
"""
|
||||
Sensor that triggers gtfs_feed_downloads every 60 minutes.
|
||||
Fetches feed metadata once and passes it to each partition run.
|
||||
"""
|
||||
with duckdb.get_connection() as conn:
|
||||
# Get all active feeds with their metadata in one query
|
||||
feeds = conn.execute("""
|
||||
SELECT feed_id, provider, producer_url
|
||||
FROM gtfs_feed_metadata
|
||||
WHERE producer_url IS NOT NULL AND producer_url != ''
|
||||
ORDER BY feed_id
|
||||
""").fetchall()
|
||||
|
||||
if not feeds:
|
||||
return SkipReason("No GTFS feeds configured")
|
||||
|
||||
# Create a RunRequest for each partition with metadata
|
||||
run_requests = [
|
||||
RunRequest(
|
||||
partition_key=feed_id,
|
||||
run_config={
|
||||
"ops": {
|
||||
"gtfs_feed_downloads": {
|
||||
"config": {
|
||||
"provider": provider,
|
||||
"producer_url": producer_url,
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
tags={
|
||||
"feed_id": feed_id,
|
||||
"sensor": "gtfs_static_sensor"
|
||||
}
|
||||
)
|
||||
for feed_id, provider, producer_url in feeds
|
||||
]
|
||||
|
||||
context.log.info(f"Triggering downloads for {len(run_requests)} GTFS feeds")
|
||||
|
||||
return run_requests
|
||||
|
||||
@sensor(
|
||||
name="gtfs_static_partition_update_sensor",
|
||||
asset_selection=["gtfs_feed_downloads"],
|
||||
default_status=DefaultSensorStatus.RUNNING
|
||||
)
|
||||
def gtfs_static_partition_update_sensor(
|
||||
context: SensorEvaluationContext,
|
||||
duckdb: DuckDBResource,
|
||||
) -> list[RunRequest] | SkipReason:
|
||||
"""
|
||||
Sensor that triggers gtfs_feed_downloads when gtfs_feed_partitions is materialized.
|
||||
This runs whenever new partitions are added.
|
||||
"""
|
||||
# Get the asset key for gtfs_feed_partitions
|
||||
from dagster import AssetKey
|
||||
|
||||
partition_asset_key = AssetKey("gtfs_feed_partitions")
|
||||
|
||||
# Get the last materialization of gtfs_feed_partitions
|
||||
last_materialization = context.instance.get_latest_materialization_event(
|
||||
partition_asset_key
|
||||
)
|
||||
|
||||
if not last_materialization:
|
||||
return SkipReason("gtfs_feed_partitions has not been materialized yet")
|
||||
|
||||
# Get cursor (last processed materialization timestamp)
|
||||
cursor = context.cursor
|
||||
last_run_timestamp = float(cursor) if cursor else 0
|
||||
|
||||
# Get the timestamp of the latest materialization
|
||||
materialization_timestamp = last_materialization.timestamp
|
||||
|
||||
# Check if there's a new materialization since last sensor run
|
||||
if materialization_timestamp <= last_run_timestamp:
|
||||
return SkipReason(
|
||||
f"No new materialization of gtfs_feed_partitions since last check "
|
||||
f"(last: {last_run_timestamp}, current: {materialization_timestamp})"
|
||||
)
|
||||
|
||||
context.log.info(
|
||||
f"Detected new materialization of gtfs_feed_partitions at {materialization_timestamp}"
|
||||
)
|
||||
|
||||
# Get all feeds from the database
|
||||
with duckdb.get_connection() as conn:
|
||||
feeds = conn.execute("""
|
||||
SELECT feed_id, provider, producer_url
|
||||
FROM gtfs_feed_metadata
|
||||
WHERE producer_url IS NOT NULL AND producer_url != ''
|
||||
ORDER BY feed_id
|
||||
""").fetchall()
|
||||
|
||||
if not feeds:
|
||||
# Update cursor even if no feeds to avoid re-triggering
|
||||
return SensorResult(
|
||||
skip_reason="No GTFS feeds configured",
|
||||
cursor=str(materialization_timestamp)
|
||||
)
|
||||
|
||||
run_requests = [
|
||||
RunRequest(
|
||||
partition_key=feed_id,
|
||||
run_config={
|
||||
"ops": {
|
||||
"gtfs_feed_downloads": {
|
||||
"config": {
|
||||
"provider": provider,
|
||||
"producer_url": producer_url,
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
tags={
|
||||
"feed_id": feed_id,
|
||||
"sensor": "gtfs_static_partition_update_sensor",
|
||||
"trigger": "partition_update"
|
||||
}
|
||||
)
|
||||
for feed_id, provider, producer_url in feeds
|
||||
]
|
||||
|
||||
context.log.info(
|
||||
f"Triggering downloads for {len(run_requests)} GTFS feeds "
|
||||
f"due to partition update"
|
||||
)
|
||||
|
||||
# Return with updated cursor
|
||||
return SensorResult(
|
||||
run_requests=run_requests,
|
||||
cursor=str(materialization_timestamp)
|
||||
)
|
||||
Loading…
Add table
Add a link
Reference in a new issue