From 4f24b40d91158db1a0df052059ab6f6f22890692 Mon Sep 17 00:00:00 2001 From: rickychen-infinirc Date: Tue, 30 Dec 2025 10:11:20 +0800 Subject: [PATCH 1/2] Refactor airflow-core/src to use SQLA2 --- .pre-commit-config.yaml | 3 +++ .../airflow/dag_processing/bundles/manager.py | 2 +- .../0015_2_9_0_update_trigger_kwargs_type.py | 5 ++-- airflow-core/src/airflow/utils/db_cleanup.py | 27 ++++++++++--------- 4 files changed, 21 insertions(+), 16 deletions(-) diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index 897ee78b18ea6..e5957656a2212 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -424,6 +424,9 @@ repos: (?x) ^airflow-ctl.*\.py$| ^airflow-core/src/airflow/models/.*\.py$| + ^airflow-core/src/airflow/dag_processing/.*\.py$| + ^airflow-core/src/airflow/migrations/.*\.py$| + ^airflow-core/src/airflow/utils/.*\.py$| ^airflow-core/src/airflow/api_fastapi/execution_api/routes/task_instances.py$| ^airflow-core/tests/unit/api_fastapi/core_api/routes/public/test_assets.py$| ^airflow-core/tests/unit/models/test_serialized_dag.py$| diff --git a/airflow-core/src/airflow/dag_processing/bundles/manager.py b/airflow-core/src/airflow/dag_processing/bundles/manager.py index 738e0785e19ed..d13fb027e626e 100644 --- a/airflow-core/src/airflow/dag_processing/bundles/manager.py +++ b/airflow-core/src/airflow/dag_processing/bundles/manager.py @@ -230,7 +230,7 @@ def _extract_and_sign_template(bundle_name: str) -> tuple[str | None, dict]: self.log.debug("Signed URL template for bundle %s", bundle_name) return new_template_, new_params_ - stored = {b.name: b for b in session.query(DagBundleModel).all()} + stored = {b.name: b for b in session.scalars(select(DagBundleModel)).all()} bundle_to_team = { bundle.name: bundle.teams[0].name if len(bundle.teams) == 1 else None for bundle in stored.values() diff --git a/airflow-core/src/airflow/migrations/versions/0015_2_9_0_update_trigger_kwargs_type.py b/airflow-core/src/airflow/migrations/versions/0015_2_9_0_update_trigger_kwargs_type.py index 08843afb0e3b2..adfb615ee7d79 100644 --- a/airflow-core/src/airflow/migrations/versions/0015_2_9_0_update_trigger_kwargs_type.py +++ b/airflow-core/src/airflow/migrations/versions/0015_2_9_0_update_trigger_kwargs_type.py @@ -32,6 +32,7 @@ import sqlalchemy as sa from alembic import context, op +from sqlalchemy import select from sqlalchemy.orm import lazyload from airflow.models.trigger import Trigger @@ -60,7 +61,7 @@ def upgrade(): if not context.is_offline_mode(): session = get_session() try: - for trigger in session.query(Trigger).options(lazyload(Trigger.task_instance)): + for trigger in session.scalars(select(Trigger).options(lazyload(Trigger.task_instance))): trigger.kwargs = trigger.kwargs session.commit() finally: @@ -81,7 +82,7 @@ def downgrade(): else: session = get_session() try: - for trigger in session.query(Trigger).options(lazyload(Trigger.task_instance)): + for trigger in session.scalars(select(Trigger).options(lazyload(Trigger.task_instance))): trigger.encrypted_kwargs = json.dumps(BaseSerialization.serialize(trigger.kwargs)) session.commit() finally: diff --git a/airflow-core/src/airflow/utils/db_cleanup.py b/airflow-core/src/airflow/utils/db_cleanup.py index 558146fc84d23..60b4a8c5d475d 100644 --- a/airflow-core/src/airflow/utils/db_cleanup.py +++ b/airflow-core/src/airflow/utils/db_cleanup.py @@ -47,7 +47,8 @@ if TYPE_CHECKING: from pendulum import DateTime - from sqlalchemy.orm import Query, Session + from sqlalchemy.orm import Session + from sqlalchemy.sql.selectable import Select from airflow.models import Base @@ -163,8 +164,8 @@ def readable_config(self): config_dict: dict[str, _TableConfig] = {x.orm_model.name: x for x in sorted(config_list)} -def _check_for_rows(*, query: Query, print_rows: bool = False) -> int: - num_entities = query.count() +def _check_for_rows(*, query: Select, print_rows: bool = False, session: Session) -> int: + num_entities = session.execute(select(func.count()).select_from(query.subquery())).scalar() print(f"Found {num_entities} rows meeting deletion criteria.") if not print_rows or num_entities == 0: return num_entities @@ -172,8 +173,8 @@ def _check_for_rows(*, query: Query, print_rows: bool = False) -> int: max_rows_to_print = 100 print(f"Printing first {max_rows_to_print} rows.") logger.debug("print entities query: %s", query) - for entry in query.limit(max_rows_to_print): - print(entry.__dict__) + for entry in session.execute(query.limit(max_rows_to_print)): + print(entry._asdict()) return num_entities @@ -193,7 +194,7 @@ def _dump_table_to_file(*, target_table: str, file_path: str, export_format: str def _do_delete( - *, query: Query, orm_model: Base, skip_archive: bool, session: Session, batch_size: int | None + *, query: Select, orm_model: Base, skip_archive: bool, session: Session, batch_size: int | None ) -> None: import itertools import re @@ -204,7 +205,7 @@ def _do_delete( while True: limited_query = query.limit(batch_size) if batch_size else query - if limited_query.count() == 0: # nothing left to delete + if session.execute(select(func.count()).select_from(limited_query.subquery())).scalar() == 0: break batch_no = next(batch_counter) @@ -233,7 +234,7 @@ def _do_delete( logger.debug("insert statement:\n%s", insert_stm.compile()) session.execute(insert_stm) else: - stmt = CreateTableAs(target_table_name, limited_query.selectable) + stmt = CreateTableAs(target_table_name, limited_query) logger.debug("ctas query:\n%s", stmt.compile()) session.execute(stmt) session.commit() @@ -309,10 +310,10 @@ def _build_query( clean_before_timestamp: DateTime, session: Session, **kwargs, -) -> Query: +) -> Select: base_table_alias = "base" base_table = aliased(orm_model, name=base_table_alias) - query = session.query(base_table).with_entities(text(f"{base_table_alias}.*")) + query = select(text(f"{base_table_alias}.*")).select_from(base_table) base_table_recency_col = base_table.c[recency_column.name] conditions = [base_table_recency_col < clean_before_timestamp] if keep_last: @@ -325,7 +326,7 @@ def _build_query( max_date_colname=max_date_col_name, session=session, ) - query = query.select_from(base_table).outerjoin( + query = query.outerjoin( subquery, and_( *[base_table.c[x] == subquery.c[x] for x in keep_last_group_by], # type: ignore[attr-defined] @@ -364,9 +365,9 @@ def _cleanup_table( clean_before_timestamp=clean_before_timestamp, session=session, ) - logger.debug("old rows query:\n%s", query.selectable.compile()) + logger.debug("old rows query:\n%s", query.compile()) print(f"Checking table {orm_model.name}") - num_rows = _check_for_rows(query=query, print_rows=False) + num_rows = _check_for_rows(query=query, print_rows=False, session=session) if num_rows and not dry_run: _do_delete( From ddc0b4929f6de8954fad93945d453b78587bcf7d Mon Sep 17 00:00:00 2001 From: rickychen-infinirc Date: Tue, 30 Dec 2025 16:58:40 +0800 Subject: [PATCH 2/2] Update ERD diagram --- airflow-core/docs/img/airflow_erd.sha256 | 2 +- airflow-core/docs/img/airflow_erd.svg | 4330 +++++++++++----------- 2 files changed, 2166 insertions(+), 2166 deletions(-) diff --git a/airflow-core/docs/img/airflow_erd.sha256 b/airflow-core/docs/img/airflow_erd.sha256 index 98f8adadd86af..ec09c9173a6fc 100644 --- a/airflow-core/docs/img/airflow_erd.sha256 +++ b/airflow-core/docs/img/airflow_erd.sha256 @@ -1 +1 @@ -1c79db933fe961f2d23605e1dcf73125923ea39bbd719c800e8adc1aa5bedb52 \ No newline at end of file +b12c8039b770d25cf01b3d23b66a1e9578f82403e62794aeb835ea151c161b12 \ No newline at end of file diff --git a/airflow-core/docs/img/airflow_erd.svg b/airflow-core/docs/img/airflow_erd.svg index fd0e44ec76b95..17abb3cafaa15 100644 --- a/airflow-core/docs/img/airflow_erd.svg +++ b/airflow-core/docs/img/airflow_erd.svg @@ -4,2627 +4,2627 @@ - - + + %3 - + dag_priority_parsing_request - -dag_priority_parsing_request - -id - - [VARCHAR(32)] - NOT NULL - -bundle_name - - [VARCHAR(250)] - NOT NULL - -relative_fileloc - - [VARCHAR(2000)] - NOT NULL + +dag_priority_parsing_request + +id + + [VARCHAR(32)] + NOT NULL + +bundle_name + + [VARCHAR(250)] + NOT NULL + +relative_fileloc + + [VARCHAR(2000)] + NOT NULL job - -job - -id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - -end_date - - [TIMESTAMP] - -executor_class - - [VARCHAR(500)] - -hostname - - [VARCHAR(500)] - -job_type - - [VARCHAR(30)] - -latest_heartbeat - - [TIMESTAMP] - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -unixname - - [VARCHAR(1000)] + +job + +id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + +end_date + + [TIMESTAMP] + +executor_class + + [VARCHAR(500)] + +hostname + + [VARCHAR(500)] + +job_type + + [VARCHAR(30)] + +latest_heartbeat + + [TIMESTAMP] + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +unixname + + [VARCHAR(1000)] log - -log - -id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - -dttm - - [TIMESTAMP] - -event - - [VARCHAR(60)] - NOT NULL - -extra - - [TEXT] - -logical_date - - [TIMESTAMP] - -map_index - - [INTEGER] - -owner - - [VARCHAR(500)] - -owner_display_name - - [VARCHAR(500)] - -run_id - - [VARCHAR(250)] - -task_id - - [VARCHAR(250)] - -try_number - - [INTEGER] + +log + +id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + +dttm + + [TIMESTAMP] + +event + + [VARCHAR(60)] + NOT NULL + +extra + + [TEXT] + +logical_date + + [TIMESTAMP] + +map_index + + [INTEGER] + +owner + + [VARCHAR(500)] + +owner_display_name + + [VARCHAR(500)] + +run_id + + [VARCHAR(250)] + +task_id + + [VARCHAR(250)] + +try_number + + [INTEGER] partitioned_asset_key_log - -partitioned_asset_key_log - -id - - [INTEGER] - NOT NULL - -asset_event_id - - [INTEGER] - NOT NULL - -asset_id - - [INTEGER] - NOT NULL - -asset_partition_dag_run_id - - [INTEGER] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -source_partition_key - - [VARCHAR(250)] - NOT NULL - -target_dag_id - - [VARCHAR(250)] - NOT NULL - -target_partition_key - - [VARCHAR(250)] - NOT NULL + +partitioned_asset_key_log + +id + + [INTEGER] + NOT NULL + +asset_event_id + + [INTEGER] + NOT NULL + +asset_id + + [INTEGER] + NOT NULL + +asset_partition_dag_run_id + + [INTEGER] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +source_partition_key + + [VARCHAR(250)] + NOT NULL + +target_dag_id + + [VARCHAR(250)] + NOT NULL + +target_partition_key + + [VARCHAR(250)] + NOT NULL import_error - -import_error - -id - - [INTEGER] - NOT NULL - -bundle_name - - [VARCHAR(250)] - -filename - - [VARCHAR(1024)] - -stacktrace - - [TEXT] - -timestamp - - [TIMESTAMP] + +import_error + +id + + [INTEGER] + NOT NULL + +bundle_name + + [VARCHAR(250)] + +filename + + [VARCHAR(1024)] + +stacktrace + + [TEXT] + +timestamp + + [TIMESTAMP] dag_bundle - -dag_bundle - -name - - [VARCHAR(250)] - NOT NULL - -active - - [BOOLEAN] - -last_refreshed - - [TIMESTAMP] - -signed_url_template - - [VARCHAR(200)] - -template_params - - [JSON] - -version - - [VARCHAR(200)] + +dag_bundle + +name + + [VARCHAR(250)] + NOT NULL + +active + + [BOOLEAN] + +last_refreshed + + [TIMESTAMP] + +signed_url_template + + [VARCHAR(200)] + +template_params + + [JSON] + +version + + [VARCHAR(200)] dag_bundle_team - -dag_bundle_team - -dag_bundle_name - - [VARCHAR(250)] - NOT NULL - -team_name - - [VARCHAR(50)] - NOT NULL + +dag_bundle_team + +dag_bundle_name + + [VARCHAR(250)] + NOT NULL + +team_name + + [VARCHAR(50)] + NOT NULL dag_bundle:name--dag_bundle_team:dag_bundle_name - -0..N -1 + +0..N +1 dag - -dag - -dag_id - - [VARCHAR(250)] - NOT NULL - -asset_expression - - [JSON] - -bundle_name - - [VARCHAR(250)] - NOT NULL - -bundle_version - - [VARCHAR(200)] - -dag_display_name - - [VARCHAR(2000)] - -deadline - - [JSON] - -description - - [TEXT] - -fail_fast - - [BOOLEAN] - NOT NULL - -fileloc - - [VARCHAR(2000)] - -has_import_errors - - [BOOLEAN] - -has_task_concurrency_limits - - [BOOLEAN] - NOT NULL - -is_paused - - [BOOLEAN] - -is_stale - - [BOOLEAN] - NOT NULL - -last_expired - - [TIMESTAMP] - -last_parse_duration - - [DOUBLE_PRECISION] - -last_parsed_time - - [TIMESTAMP] - -max_active_runs - - [INTEGER] - -max_active_tasks - - [INTEGER] - NOT NULL - -max_consecutive_failed_dag_runs - - [INTEGER] - NOT NULL - -next_dagrun - - [TIMESTAMP] - -next_dagrun_create_after - - [TIMESTAMP] - -next_dagrun_data_interval_end - - [TIMESTAMP] - -next_dagrun_data_interval_start - - [TIMESTAMP] - -owners - - [VARCHAR(2000)] - -relative_fileloc - - [VARCHAR(2000)] - -timetable_description - - [VARCHAR(1000)] - -timetable_summary - - [TEXT] + +dag + +dag_id + + [VARCHAR(250)] + NOT NULL + +asset_expression + + [JSON] + +bundle_name + + [VARCHAR(250)] + NOT NULL + +bundle_version + + [VARCHAR(200)] + +dag_display_name + + [VARCHAR(2000)] + +deadline + + [JSON] + +description + + [TEXT] + +fail_fast + + [BOOLEAN] + NOT NULL + +fileloc + + [VARCHAR(2000)] + +has_import_errors + + [BOOLEAN] + +has_task_concurrency_limits + + [BOOLEAN] + NOT NULL + +is_paused + + [BOOLEAN] + +is_stale + + [BOOLEAN] + NOT NULL + +last_expired + + [TIMESTAMP] + +last_parse_duration + + [DOUBLE_PRECISION] + +last_parsed_time + + [TIMESTAMP] + +max_active_runs + + [INTEGER] + +max_active_tasks + + [INTEGER] + NOT NULL + +max_consecutive_failed_dag_runs + + [INTEGER] + NOT NULL + +next_dagrun + + [TIMESTAMP] + +next_dagrun_create_after + + [TIMESTAMP] + +next_dagrun_data_interval_end + + [TIMESTAMP] + +next_dagrun_data_interval_start + + [TIMESTAMP] + +owners + + [VARCHAR(2000)] + +relative_fileloc + + [VARCHAR(2000)] + +timetable_description + + [VARCHAR(1000)] + +timetable_summary + + [TEXT] dag_bundle:name--dag:bundle_name - -0..N -1 + +0..N +1 team - -team - -name - - [VARCHAR(50)] - NOT NULL + +team + +name + + [VARCHAR(50)] + NOT NULL team:name--dag_bundle_team:team_name - -0..N -1 + +0..N +1 connection - -connection - -id - - [INTEGER] - NOT NULL - -conn_id - - [VARCHAR(250)] - NOT NULL - -conn_type - - [VARCHAR(500)] - NOT NULL - -description - - [TEXT] - -extra - - [TEXT] - -host - - [VARCHAR(500)] - -is_encrypted - - [BOOLEAN] - -is_extra_encrypted - - [BOOLEAN] - -login - - [TEXT] - -password - - [TEXT] - -port - - [INTEGER] - -schema - - [VARCHAR(500)] - -team_name - - [VARCHAR(50)] + +connection + +id + + [INTEGER] + NOT NULL + +conn_id + + [VARCHAR(250)] + NOT NULL + +conn_type + + [VARCHAR(500)] + NOT NULL + +description + + [TEXT] + +extra + + [TEXT] + +host + + [VARCHAR(500)] + +is_encrypted + + [BOOLEAN] + +is_extra_encrypted + + [BOOLEAN] + +login + + [TEXT] + +password + + [TEXT] + +port + + [INTEGER] + +schema + + [VARCHAR(500)] + +team_name + + [VARCHAR(50)] team:name--connection:team_name - -0..N -{0,1} + +0..N +{0,1} slot_pool - -slot_pool - -id - - [INTEGER] - NOT NULL - -description - - [TEXT] - -include_deferred - - [BOOLEAN] - NOT NULL - -pool - - [VARCHAR(256)] - -slots - - [INTEGER] - -team_name - - [VARCHAR(50)] + +slot_pool + +id + + [INTEGER] + NOT NULL + +description + + [TEXT] + +include_deferred + + [BOOLEAN] + NOT NULL + +pool + + [VARCHAR(256)] + +slots + + [INTEGER] + +team_name + + [VARCHAR(50)] team:name--slot_pool:team_name - -0..N -{0,1} + +0..N +{0,1} variable - -variable - -id - - [INTEGER] - NOT NULL - -description - - [TEXT] - -is_encrypted - - [BOOLEAN] - -key - - [VARCHAR(250)] - -team_name - - [VARCHAR(50)] - -val - - [TEXT] + +variable + +id + + [INTEGER] + NOT NULL + +description + + [TEXT] + +is_encrypted + + [BOOLEAN] + +key + + [VARCHAR(250)] + +team_name + + [VARCHAR(50)] + +val + + [TEXT] team:name--variable:team_name - -0..N -{0,1} + +0..N +{0,1} trigger - -trigger - -id - - [INTEGER] - NOT NULL - -classpath - - [VARCHAR(1000)] - NOT NULL - -created_date - - [TIMESTAMP] - NOT NULL - -kwargs - - [TEXT] - NOT NULL - -triggerer_id - - [INTEGER] + +trigger + +id + + [INTEGER] + NOT NULL + +classpath + + [VARCHAR(1000)] + NOT NULL + +created_date + + [TIMESTAMP] + NOT NULL + +kwargs + + [TEXT] + NOT NULL + +triggerer_id + + [INTEGER] callback - -callback - -id - - [UUID] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -data - - [JSONB] - NOT NULL - -fetch_method - - [VARCHAR(20)] - NOT NULL - -output - - [TEXT] - -priority_weight - - [INTEGER] - NOT NULL - -state - - [VARCHAR(10)] - -trigger_id - - [INTEGER] - -type - - [VARCHAR(20)] - NOT NULL + +callback + +id + + [UUID] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +data + + [JSONB] + NOT NULL + +fetch_method + + [VARCHAR(20)] + NOT NULL + +output + + [TEXT] + +priority_weight + + [INTEGER] + NOT NULL + +state + + [VARCHAR(10)] + +trigger_id + + [INTEGER] + +type + + [VARCHAR(20)] + NOT NULL trigger:id--callback:trigger_id - -0..N -{0,1} + +0..N +{0,1} asset_watcher - -asset_watcher - -asset_id - - [INTEGER] - NOT NULL - -trigger_id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(1500)] - NOT NULL + +asset_watcher + +asset_id + + [INTEGER] + NOT NULL + +trigger_id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(1500)] + NOT NULL - + trigger:id--asset_watcher:trigger_id - -0..N -1 + +0..N +1 task_instance - -task_instance - -id - - [UUID] - NOT NULL - -context_carrier - - [JSONB] - -custom_operator_name - - [VARCHAR(1000)] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -duration - - [DOUBLE_PRECISION] - -end_date - - [TIMESTAMP] - -executor - - [VARCHAR(1000)] - -executor_config - - [BYTEA] - -external_executor_id - - [VARCHAR(250)] - -hostname - - [VARCHAR(1000)] - -last_heartbeat_at - - [TIMESTAMP] - -map_index - - [INTEGER] - NOT NULL - -max_tries - - [INTEGER] - -next_kwargs - - [JSONB] - -next_method - - [VARCHAR(1000)] - -operator - - [VARCHAR(1000)] - -pid - - [INTEGER] - -pool - - [VARCHAR(256)] - NOT NULL - -pool_slots - - [INTEGER] - NOT NULL - -priority_weight - - [INTEGER] - -queue - - [VARCHAR(256)] - -queued_by_job_id - - [INTEGER] - -queued_dttm - - [TIMESTAMP] - -rendered_map_index - - [VARCHAR(250)] - -run_id - - [VARCHAR(250)] - NOT NULL - -scheduled_dttm - - [TIMESTAMP] - -span_status - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -task_display_name - - [VARCHAR(2000)] - -task_id - - [VARCHAR(250)] - NOT NULL - -trigger_id - - [INTEGER] - -trigger_timeout - - [TIMESTAMP] - -try_number - - [INTEGER] - -unixname - - [VARCHAR(1000)] - -updated_at - - [TIMESTAMP] + +task_instance + +id + + [UUID] + NOT NULL + +context_carrier + + [JSONB] + +custom_operator_name + + [VARCHAR(1000)] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +duration + + [DOUBLE_PRECISION] + +end_date + + [TIMESTAMP] + +executor + + [VARCHAR(1000)] + +executor_config + + [BYTEA] + +external_executor_id + + [VARCHAR(250)] + +hostname + + [VARCHAR(1000)] + +last_heartbeat_at + + [TIMESTAMP] + +map_index + + [INTEGER] + NOT NULL + +max_tries + + [INTEGER] + +next_kwargs + + [JSONB] + +next_method + + [VARCHAR(1000)] + +operator + + [VARCHAR(1000)] + +pid + + [INTEGER] + +pool + + [VARCHAR(256)] + NOT NULL + +pool_slots + + [INTEGER] + NOT NULL + +priority_weight + + [INTEGER] + +queue + + [VARCHAR(256)] + +queued_by_job_id + + [INTEGER] + +queued_dttm + + [TIMESTAMP] + +rendered_map_index + + [VARCHAR(250)] + +run_id + + [VARCHAR(250)] + NOT NULL + +scheduled_dttm + + [TIMESTAMP] + +span_status + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +task_display_name + + [VARCHAR(2000)] + +task_id + + [VARCHAR(250)] + NOT NULL + +trigger_id + + [INTEGER] + +trigger_timeout + + [TIMESTAMP] + +try_number + + [INTEGER] + +unixname + + [VARCHAR(1000)] + +updated_at + + [TIMESTAMP] - + trigger:id--task_instance:trigger_id - -0..N -{0,1} + +0..N +{0,1} deadline - -deadline - -id - - [UUID] - NOT NULL - -callback_id - - [UUID] - NOT NULL - -dagrun_id - - [INTEGER] - -deadline_time - - [TIMESTAMP] - NOT NULL - -missed - - [BOOLEAN] - NOT NULL + +deadline + +id + + [UUID] + NOT NULL + +callback_id + + [UUID] + NOT NULL + +dagrun_id + + [INTEGER] + +deadline_time + + [TIMESTAMP] + NOT NULL + +missed + + [BOOLEAN] + NOT NULL - + callback:id--deadline:callback_id - -0..N -1 + +0..N +1 asset_alias - -asset_alias - -id - - [INTEGER] - NOT NULL - -group - - [VARCHAR(1500)] - NOT NULL - -name - - [VARCHAR(1500)] - NOT NULL + +asset_alias + +id + + [INTEGER] + NOT NULL + +group + + [VARCHAR(1500)] + NOT NULL + +name + + [VARCHAR(1500)] + NOT NULL asset_alias_asset - -asset_alias_asset - -alias_id - - [INTEGER] - NOT NULL - -asset_id - - [INTEGER] - NOT NULL + +asset_alias_asset + +alias_id + + [INTEGER] + NOT NULL + +asset_id + + [INTEGER] + NOT NULL - + asset_alias:id--asset_alias_asset:alias_id - -0..N -1 + +0..N +1 asset_alias_asset_event - -asset_alias_asset_event - -alias_id - - [INTEGER] - NOT NULL - -event_id - - [INTEGER] - NOT NULL + +asset_alias_asset_event + +alias_id + + [INTEGER] + NOT NULL + +event_id + + [INTEGER] + NOT NULL - + asset_alias:id--asset_alias_asset_event:alias_id - -0..N -1 + +0..N +1 dag_schedule_asset_alias_reference - -dag_schedule_asset_alias_reference - -alias_id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +dag_schedule_asset_alias_reference + +alias_id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL - + asset_alias:id--dag_schedule_asset_alias_reference:alias_id - -0..N -1 + +0..N +1 asset - -asset - -id - - [INTEGER] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -extra - - [JSON] - NOT NULL - -group - - [VARCHAR(1500)] - NOT NULL - -name - - [VARCHAR(1500)] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -uri - - [VARCHAR(1500)] - NOT NULL + +asset + +id + + [INTEGER] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +extra + + [JSON] + NOT NULL + +group + + [VARCHAR(1500)] + NOT NULL + +name + + [VARCHAR(1500)] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +uri + + [VARCHAR(1500)] + NOT NULL - + asset:id--asset_alias_asset:asset_id - -0..N -1 + +0..N +1 - + asset:id--asset_watcher:asset_id - -0..N -1 + +0..N +1 asset_active - -asset_active - -name - - [VARCHAR(1500)] - NOT NULL - -uri - - [VARCHAR(1500)] - NOT NULL + +asset_active + +name + + [VARCHAR(1500)] + NOT NULL + +uri + + [VARCHAR(1500)] + NOT NULL asset:name--asset_active:name - -1 -1 + +1 +1 asset:uri--asset_active:uri - -1 -1 + +1 +1 dag_schedule_asset_reference - -dag_schedule_asset_reference - -asset_id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +dag_schedule_asset_reference + +asset_id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL - + asset:id--dag_schedule_asset_reference:asset_id - -0..N -1 + +0..N +1 task_outlet_asset_reference - -task_outlet_asset_reference - -asset_id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +task_outlet_asset_reference + +asset_id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL asset:id--task_outlet_asset_reference:asset_id - -0..N -1 + +0..N +1 task_inlet_asset_reference - -task_inlet_asset_reference - -asset_id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +task_inlet_asset_reference + +asset_id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL - + asset:id--task_inlet_asset_reference:asset_id - -0..N -1 + +0..N +1 asset_dag_run_queue - -asset_dag_run_queue - -asset_id - - [INTEGER] - NOT NULL - -target_dag_id - - [VARCHAR(250)] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL + +asset_dag_run_queue + +asset_id + + [INTEGER] + NOT NULL + +target_dag_id + + [VARCHAR(250)] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL asset:id--asset_dag_run_queue:asset_id - -0..N -1 + +0..N +1 asset_event - -asset_event - -id - - [INTEGER] - NOT NULL - -asset_id - - [INTEGER] - NOT NULL - -extra - - [JSON] - NOT NULL - -partition_key - - [VARCHAR(250)] - -source_dag_id - - [VARCHAR(250)] - -source_map_index - - [INTEGER] - -source_run_id - - [VARCHAR(250)] - -source_task_id - - [VARCHAR(250)] - -timestamp - - [TIMESTAMP] - NOT NULL + +asset_event + +id + + [INTEGER] + NOT NULL + +asset_id + + [INTEGER] + NOT NULL + +extra + + [JSON] + NOT NULL + +partition_key + + [VARCHAR(250)] + +source_dag_id + + [VARCHAR(250)] + +source_map_index + + [INTEGER] + +source_run_id + + [VARCHAR(250)] + +source_task_id + + [VARCHAR(250)] + +timestamp + + [TIMESTAMP] + NOT NULL - + asset_event:id--asset_alias_asset_event:event_id - -0..N -1 + +0..N +1 dagrun_asset_event - -dagrun_asset_event - -dag_run_id - - [INTEGER] - NOT NULL - -event_id - - [INTEGER] - NOT NULL + +dagrun_asset_event + +dag_run_id + + [INTEGER] + NOT NULL + +event_id + + [INTEGER] + NOT NULL asset_event:id--dagrun_asset_event:event_id - -0..N -1 + +0..N +1 dag_version - -dag_version - -id - - [UUID] - NOT NULL - -bundle_name - - [VARCHAR(250)] - -bundle_version - - [VARCHAR(250)] - -created_at - - [TIMESTAMP] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -last_updated - - [TIMESTAMP] - NOT NULL - -version_number - - [INTEGER] - NOT NULL + +dag_version + +id + + [UUID] + NOT NULL + +bundle_name + + [VARCHAR(250)] + +bundle_version + + [VARCHAR(250)] + +created_at + + [TIMESTAMP] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +last_updated + + [TIMESTAMP] + NOT NULL + +version_number + + [INTEGER] + NOT NULL dag:dag_id--dag_version:dag_id - -0..N -1 + +0..N +1 dag_schedule_asset_name_reference - -dag_schedule_asset_name_reference - -dag_id - - [VARCHAR(250)] - NOT NULL - -name - - [VARCHAR(1500)] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL + +dag_schedule_asset_name_reference + +dag_id + + [VARCHAR(250)] + NOT NULL + +name + + [VARCHAR(1500)] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL dag:dag_id--dag_schedule_asset_name_reference:dag_id - -0..N -1 + +0..N +1 dag_schedule_asset_uri_reference - -dag_schedule_asset_uri_reference - -dag_id - - [VARCHAR(250)] - NOT NULL - -uri - - [VARCHAR(1500)] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL + +dag_schedule_asset_uri_reference + +dag_id + + [VARCHAR(250)] + NOT NULL + +uri + + [VARCHAR(1500)] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL dag:dag_id--dag_schedule_asset_uri_reference:dag_id - -0..N -1 + +0..N +1 - + dag:dag_id--dag_schedule_asset_alias_reference:dag_id - -0..N -1 + +0..N +1 - + dag:dag_id--dag_schedule_asset_reference:dag_id - -0..N -1 + +0..N +1 dag:dag_id--task_outlet_asset_reference:dag_id - -0..N -1 + +0..N +1 - + dag:dag_id--task_inlet_asset_reference:dag_id - -0..N -1 + +0..N +1 dag:dag_id--asset_dag_run_queue:target_dag_id - -0..N -1 + +0..N +1 dag_tag - -dag_tag - -dag_id - - [VARCHAR(250)] - NOT NULL - -name - - [VARCHAR(100)] - NOT NULL + +dag_tag + +dag_id + + [VARCHAR(250)] + NOT NULL + +name + + [VARCHAR(100)] + NOT NULL dag:dag_id--dag_tag:dag_id - -0..N -1 + +0..N +1 dag_owner_attributes - -dag_owner_attributes - -dag_id - - [VARCHAR(250)] - NOT NULL - -owner - - [VARCHAR(500)] - NOT NULL - -link - - [VARCHAR(500)] - NOT NULL + +dag_owner_attributes + +dag_id + + [VARCHAR(250)] + NOT NULL + +owner + + [VARCHAR(500)] + NOT NULL + +link + + [VARCHAR(500)] + NOT NULL dag:dag_id--dag_owner_attributes:dag_id - -0..N -1 + +0..N +1 dag_warning - -dag_warning - -dag_id - - [VARCHAR(250)] - NOT NULL - -warning_type - - [VARCHAR(50)] - NOT NULL - -message - - [TEXT] - NOT NULL - -timestamp - - [TIMESTAMP] - NOT NULL + +dag_warning + +dag_id + + [VARCHAR(250)] + NOT NULL + +warning_type + + [VARCHAR(50)] + NOT NULL + +message + + [TEXT] + NOT NULL + +timestamp + + [TIMESTAMP] + NOT NULL dag:dag_id--dag_warning:dag_id - -0..N -1 + +0..N +1 dag_favorite - -dag_favorite - -dag_id - - [VARCHAR(250)] - NOT NULL - -user_id - - [VARCHAR(250)] - NOT NULL + +dag_favorite + +dag_id + + [VARCHAR(250)] + NOT NULL + +user_id + + [VARCHAR(250)] + NOT NULL dag:dag_id--dag_favorite:dag_id - -0..N -1 + +0..N +1 dag_run - -dag_run - -id - - [INTEGER] - NOT NULL - -backfill_id - - [INTEGER] - -bundle_version - - [VARCHAR(250)] - -clear_number - - [INTEGER] - NOT NULL - -conf - - [JSONB] - -context_carrier - - [JSONB] - -created_dag_version_id - - [UUID] - -creating_job_id - - [INTEGER] - -dag_id - - [VARCHAR(250)] - NOT NULL - -data_interval_end - - [TIMESTAMP] - -data_interval_start - - [TIMESTAMP] - -end_date - - [TIMESTAMP] - -last_scheduling_decision - - [TIMESTAMP] - -log_template_id - - [INTEGER] - -logical_date - - [TIMESTAMP] - -partition_key - - [VARCHAR(250)] - -queued_at - - [TIMESTAMP] - -run_after - - [TIMESTAMP] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -run_type - - [VARCHAR(50)] - NOT NULL - -scheduled_by_job_id - - [INTEGER] - -span_status - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(50)] - -triggered_by - - [VARCHAR(50)] - -triggering_user_name - - [VARCHAR(512)] - -updated_at - - [TIMESTAMP] + +dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + +bundle_version + + [VARCHAR(250)] + +clear_number + + [INTEGER] + NOT NULL + +conf + + [JSONB] + +context_carrier + + [JSONB] + +created_dag_version_id + + [UUID] + +creating_job_id + + [INTEGER] + +dag_id + + [VARCHAR(250)] + NOT NULL + +data_interval_end + + [TIMESTAMP] + +data_interval_start + + [TIMESTAMP] + +end_date + + [TIMESTAMP] + +last_scheduling_decision + + [TIMESTAMP] + +log_template_id + + [INTEGER] + +logical_date + + [TIMESTAMP] + +partition_key + + [VARCHAR(250)] + +queued_at + + [TIMESTAMP] + +run_after + + [TIMESTAMP] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +run_type + + [VARCHAR(50)] + NOT NULL + +scheduled_by_job_id + + [INTEGER] + +span_status + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(50)] + +triggered_by + + [VARCHAR(50)] + +triggering_user_name + + [VARCHAR(512)] + +updated_at + + [TIMESTAMP] - + dag_version:id--dag_run:created_dag_version_id - -0..N -{0,1} + +0..N +{0,1} dag_code - -dag_code - -id - - [UUID] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - NOT NULL - -fileloc - - [VARCHAR(2000)] - NOT NULL - -last_updated - - [TIMESTAMP] - NOT NULL - -source_code - - [TEXT] - NOT NULL - -source_code_hash - - [VARCHAR(32)] - NOT NULL + +dag_code + +id + + [UUID] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + NOT NULL + +fileloc + + [VARCHAR(2000)] + NOT NULL + +last_updated + + [TIMESTAMP] + NOT NULL + +source_code + + [TEXT] + NOT NULL + +source_code_hash + + [VARCHAR(32)] + NOT NULL dag_version:id--dag_code:dag_version_id - -0..N -1 + +0..N +1 serialized_dag - -serialized_dag - -id - - [UUID] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -dag_hash - - [VARCHAR(32)] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - NOT NULL - -data - - [JSONB] - -data_compressed - - [BYTEA] - -last_updated - - [TIMESTAMP] - NOT NULL + +serialized_dag + +id + + [UUID] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +dag_hash + + [VARCHAR(32)] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + NOT NULL + +data + + [JSONB] + +data_compressed + + [BYTEA] + +last_updated + + [TIMESTAMP] + NOT NULL dag_version:id--serialized_dag:dag_version_id - -0..N -1 + +0..N +1 dag_version:id--task_instance:dag_version_id - -0..N -{0,1} + +0..N +{0,1} log_template - -log_template - -id - - [INTEGER] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -elasticsearch_id - - [TEXT] - NOT NULL - -filename - - [TEXT] - NOT NULL + +log_template + +id + + [INTEGER] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +elasticsearch_id + + [TEXT] + NOT NULL + +filename + + [TEXT] + NOT NULL log_template:id--dag_run:log_template_id - -0..N -{0,1} + +0..N +{0,1} - + dag_run:id--deadline:dagrun_id - -0..N -{0,1} + +0..N +{0,1} dag_run:id--dagrun_asset_event:dag_run_id - -0..N -1 + +0..N +1 asset_partition_dag_run - -asset_partition_dag_run - -id - - [INTEGER] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -created_dag_run_id - - [INTEGER] - -partition_key - - [VARCHAR(250)] - NOT NULL - -target_dag_id - - [VARCHAR(250)] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +asset_partition_dag_run + +id + + [INTEGER] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +created_dag_run_id + + [INTEGER] + +partition_key + + [VARCHAR(250)] + NOT NULL + +target_dag_id + + [VARCHAR(250)] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL dag_run:id--asset_partition_dag_run:created_dag_run_id - -0..N -{0,1} + +0..N +{0,1} backfill_dag_run - -backfill_dag_run - -id - - [INTEGER] - NOT NULL - -backfill_id - - [INTEGER] - NOT NULL - -dag_run_id - - [INTEGER] - -exception_reason - - [VARCHAR(250)] - -logical_date - - [TIMESTAMP] - NOT NULL - -sort_ordinal - - [INTEGER] - NOT NULL + +backfill_dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + NOT NULL + +dag_run_id + + [INTEGER] + +exception_reason + + [VARCHAR(250)] + +logical_date + + [TIMESTAMP] + NOT NULL + +sort_ordinal + + [INTEGER] + NOT NULL dag_run:id--backfill_dag_run:dag_run_id - -0..N -{0,1} + +0..N +{0,1} -dag_run:run_id--task_instance:run_id - -0..N -1 +dag_run:dag_id--task_instance:dag_id + +0..N +1 - -dag_run:dag_id--task_instance:dag_id - -0..N -1 + +dag_run:run_id--task_instance:run_id + +0..N +1 dag_run_note - -dag_run_note - -dag_run_id - - [INTEGER] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [VARCHAR(128)] + +dag_run_note + +dag_run_id + + [INTEGER] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [VARCHAR(128)] dag_run:id--dag_run_note:dag_run_id - -1 -1 + +1 +1 backfill - -backfill - -id - - [INTEGER] - NOT NULL - -completed_at - - [TIMESTAMP] - -created_at - - [TIMESTAMP] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_run_conf - - [JSON] - NOT NULL - -from_date - - [TIMESTAMP] - NOT NULL - -is_paused - - [BOOLEAN] - -max_active_runs - - [INTEGER] - NOT NULL - -reprocess_behavior - - [VARCHAR(250)] - NOT NULL - -to_date - - [TIMESTAMP] - NOT NULL - -triggering_user_name - - [VARCHAR(512)] - -updated_at - - [TIMESTAMP] - NOT NULL + +backfill + +id + + [INTEGER] + NOT NULL + +completed_at + + [TIMESTAMP] + +created_at + + [TIMESTAMP] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_run_conf + + [JSON] + NOT NULL + +from_date + + [TIMESTAMP] + NOT NULL + +is_paused + + [BOOLEAN] + +max_active_runs + + [INTEGER] + NOT NULL + +reprocess_behavior + + [VARCHAR(250)] + NOT NULL + +to_date + + [TIMESTAMP] + NOT NULL + +triggering_user_name + + [VARCHAR(512)] + +updated_at + + [TIMESTAMP] + NOT NULL - + backfill:id--dag_run:backfill_id - -0..N -{0,1} + +0..N +{0,1} backfill:id--backfill_dag_run:backfill_id - -0..N -1 + +0..N +1 hitl_detail - -hitl_detail - -ti_id - - [UUID] - NOT NULL - -assignees - - [JSON] - -body - - [TEXT] - -chosen_options - - [JSON] - -created_at - - [TIMESTAMP] - NOT NULL - -defaults - - [JSON] - -multiple - - [BOOLEAN] - -options - - [JSON] - NOT NULL - -params - - [JSON] - NOT NULL - -params_input - - [JSON] - NOT NULL - -responded_at - - [TIMESTAMP] - -responded_by - - [JSON] - -subject - - [TEXT] - NOT NULL + +hitl_detail + +ti_id + + [UUID] + NOT NULL + +assignees + + [JSON] + +body + + [TEXT] + +chosen_options + + [JSON] + +created_at + + [TIMESTAMP] + NOT NULL + +defaults + + [JSON] + +multiple + + [BOOLEAN] + +options + + [JSON] + NOT NULL + +params + + [JSON] + NOT NULL + +params_input + + [JSON] + NOT NULL + +responded_at + + [TIMESTAMP] + +responded_by + + [JSON] + +subject + + [TEXT] + NOT NULL task_instance:id--hitl_detail:ti_id - -1 -1 + +1 +1 task_map - -task_map - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -keys - - [JSONB] - -length - - [INTEGER] - NOT NULL + +task_map + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +keys + + [JSONB] + +length + + [INTEGER] + NOT NULL -task_instance:map_index--task_map:map_index - -0..N -1 +task_instance:task_id--task_map:task_id + +0..N +1 -task_instance:task_id--task_map:task_id - -0..N -1 +task_instance:map_index--task_map:map_index + +0..N +1 task_instance:dag_id--task_map:dag_id - -0..N -1 + +0..N +1 task_instance:run_id--task_map:run_id - -0..N -1 + +0..N +1 task_reschedule - -task_reschedule - -id - - [INTEGER] - NOT NULL - -duration - - [INTEGER] - NOT NULL - -end_date - - [TIMESTAMP] - NOT NULL - -reschedule_date - - [TIMESTAMP] - NOT NULL - -start_date - - [TIMESTAMP] - NOT NULL - -ti_id - - [UUID] - NOT NULL + +task_reschedule + +id + + [INTEGER] + NOT NULL + +duration + + [INTEGER] + NOT NULL + +end_date + + [TIMESTAMP] + NOT NULL + +reschedule_date + + [TIMESTAMP] + NOT NULL + +start_date + + [TIMESTAMP] + NOT NULL + +ti_id + + [UUID] + NOT NULL task_instance:id--task_reschedule:ti_id - -0..N -1 + +0..N +1 xcom - -xcom - -dag_run_id - - [INTEGER] - NOT NULL - -key - - [VARCHAR(512)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -timestamp - - [TIMESTAMP] - NOT NULL - -value - - [JSONB] + +xcom + +dag_run_id + + [INTEGER] + NOT NULL + +key + + [VARCHAR(512)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +timestamp + + [TIMESTAMP] + NOT NULL + +value + + [JSONB] -task_instance:task_id--xcom:task_id - -0..N -1 +task_instance:dag_id--xcom:dag_id + +0..N +1 -task_instance:run_id--xcom:run_id - -0..N -1 +task_instance:map_index--xcom:map_index + +0..N +1 -task_instance:dag_id--xcom:dag_id - -0..N -1 +task_instance:task_id--xcom:task_id + +0..N +1 -task_instance:map_index--xcom:map_index - -0..N -1 +task_instance:run_id--xcom:run_id + +0..N +1 task_instance_note - -task_instance_note - -ti_id - - [UUID] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [VARCHAR(128)] + +task_instance_note + +ti_id + + [UUID] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [VARCHAR(128)] task_instance:id--task_instance_note:ti_id - -1 -1 + +1 +1 task_instance_history - -task_instance_history - -task_instance_id - - [UUID] - NOT NULL - -context_carrier - - [JSONB] - -custom_operator_name - - [VARCHAR(1000)] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -duration - - [DOUBLE_PRECISION] - -end_date - - [TIMESTAMP] - -executor - - [VARCHAR(1000)] - -executor_config - - [BYTEA] - -external_executor_id - - [VARCHAR(250)] - -hostname - - [VARCHAR(1000)] - -map_index - - [INTEGER] - NOT NULL - -max_tries - - [INTEGER] - -next_kwargs - - [JSONB] - -next_method - - [VARCHAR(1000)] - -operator - - [VARCHAR(1000)] - -pid - - [INTEGER] - -pool - - [VARCHAR(256)] - NOT NULL - -pool_slots - - [INTEGER] - NOT NULL - -priority_weight - - [INTEGER] - -queue - - [VARCHAR(256)] - -queued_by_job_id - - [INTEGER] - -queued_dttm - - [TIMESTAMP] - -rendered_map_index - - [VARCHAR(250)] - -run_id - - [VARCHAR(250)] - NOT NULL - -scheduled_dttm - - [TIMESTAMP] - -span_status - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -task_display_name - - [VARCHAR(2000)] - -task_id - - [VARCHAR(250)] - NOT NULL - -trigger_id - - [INTEGER] - -trigger_timeout - - [TIMESTAMP] - -try_number - - [INTEGER] - NOT NULL - -unixname - - [VARCHAR(1000)] - -updated_at - - [TIMESTAMP] + +task_instance_history + +task_instance_id + + [UUID] + NOT NULL + +context_carrier + + [JSONB] + +custom_operator_name + + [VARCHAR(1000)] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +duration + + [DOUBLE_PRECISION] + +end_date + + [TIMESTAMP] + +executor + + [VARCHAR(1000)] + +executor_config + + [BYTEA] + +external_executor_id + + [VARCHAR(250)] + +hostname + + [VARCHAR(1000)] + +map_index + + [INTEGER] + NOT NULL + +max_tries + + [INTEGER] + +next_kwargs + + [JSONB] + +next_method + + [VARCHAR(1000)] + +operator + + [VARCHAR(1000)] + +pid + + [INTEGER] + +pool + + [VARCHAR(256)] + NOT NULL + +pool_slots + + [INTEGER] + NOT NULL + +priority_weight + + [INTEGER] + +queue + + [VARCHAR(256)] + +queued_by_job_id + + [INTEGER] + +queued_dttm + + [TIMESTAMP] + +rendered_map_index + + [VARCHAR(250)] + +run_id + + [VARCHAR(250)] + NOT NULL + +scheduled_dttm + + [TIMESTAMP] + +span_status + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +task_display_name + + [VARCHAR(2000)] + +task_id + + [VARCHAR(250)] + NOT NULL + +trigger_id + + [INTEGER] + +trigger_timeout + + [TIMESTAMP] + +try_number + + [INTEGER] + NOT NULL + +unixname + + [VARCHAR(1000)] + +updated_at + + [TIMESTAMP] -task_instance:task_id--task_instance_history:task_id - -0..N -1 +task_instance:run_id--task_instance_history:run_id + +0..N +1 -task_instance:run_id--task_instance_history:run_id - -0..N -1 +task_instance:map_index--task_instance_history:map_index + +0..N +1 -task_instance:map_index--task_instance_history:map_index - -0..N -1 +task_instance:task_id--task_instance_history:task_id + +0..N +1 task_instance:dag_id--task_instance_history:dag_id - -0..N -1 + +0..N +1 rendered_task_instance_fields - -rendered_task_instance_fields - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -k8s_pod_yaml - - [JSON] - -rendered_fields - - [JSON] - NOT NULL + +rendered_task_instance_fields + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +k8s_pod_yaml + + [JSON] + +rendered_fields + + [JSON] + NOT NULL -task_instance:dag_id--rendered_task_instance_fields:dag_id - -0..N -1 +task_instance:run_id--rendered_task_instance_fields:run_id + +0..N +1 -task_instance:run_id--rendered_task_instance_fields:run_id - -0..N -1 +task_instance:map_index--rendered_task_instance_fields:map_index + +0..N +1 task_instance:task_id--rendered_task_instance_fields:task_id - -0..N -1 + +0..N +1 -task_instance:map_index--rendered_task_instance_fields:map_index - -0..N -1 +task_instance:dag_id--rendered_task_instance_fields:dag_id + +0..N +1 hitl_detail_history - -hitl_detail_history - -ti_history_id - - [UUID] - NOT NULL - -assignees - - [JSON] - -body - - [TEXT] - -chosen_options - - [JSON] - -created_at - - [TIMESTAMP] - NOT NULL - -defaults - - [JSON] - -multiple - - [BOOLEAN] - -options - - [JSON] - NOT NULL - -params - - [JSON] - NOT NULL - -params_input - - [JSON] - NOT NULL - -responded_at - - [TIMESTAMP] - -responded_by - - [JSON] - -subject - - [TEXT] - NOT NULL + +hitl_detail_history + +ti_history_id + + [UUID] + NOT NULL + +assignees + + [JSON] + +body + + [TEXT] + +chosen_options + + [JSON] + +created_at + + [TIMESTAMP] + NOT NULL + +defaults + + [JSON] + +multiple + + [BOOLEAN] + +options + + [JSON] + NOT NULL + +params + + [JSON] + NOT NULL + +params_input + + [JSON] + NOT NULL + +responded_at + + [TIMESTAMP] + +responded_by + + [JSON] + +subject + + [TEXT] + NOT NULL task_instance_history:task_instance_id--hitl_detail_history:ti_history_id - -1 -1 + +1 +1 alembic_version - -alembic_version - -version_num - - [VARCHAR(32)] - NOT NULL + +alembic_version + +version_num + + [VARCHAR(32)] + NOT NULL