From 3f619fb3265f9cf6bd1f03fcabc6e82fb8160def Mon Sep 17 00:00:00 2001 From: Ephraim Anierobi <splendidzigy24@gmail.com> Date: Tue, 5 Nov 2024 15:19:24 +0100 Subject: [PATCH] AIP-65: Add DAG versioning support (#42913) * AIP-65: Add DAG versioning support This commit introduces versioning for DAGs Changes: - Introduced DagVersion model to handle versioning of DAGs. - Added version_name field to DAG for use in tracking the dagversion by users - Added support for version retrieval in the get_dag_source API endpoint - Modified DAG execution logic to reference dag_version_id instead of the dag_hash to ensure DAG runs are linked to specific versions. Fix tests revert RESTAPI changes * fixup! AIP-65: Add DAG versioning support * fixup! fixup! AIP-65: Add DAG versioning support * fix migration * fix test * more test fixes * update query count * fix static checks * Fix query and add created_at to dag_version table * improve code * Change to using UUID for primary keys * DagCode.bulk_write_code is no longer used * fixup! Change to using UUID for primary keys * fix tests * fixup! fix tests * use uuid for version_name * fixup! use uuid for version_name * use row lock when writing dag version * use row lock when writing dag version * fixup! use row lock when writing dag version * deactivating dag should not remove serialized dags * save version_name as string not uuid * Make dag_version_id unique * fixup! Make dag_version_id unique * Fix tests * Use uuid7 * fix test * fixup! fix test * use binary=False for uuid field to fix sqlite issue * apply suggestions from code review * Remove unnecessary version_name on dagmodel * Fix sqlalchemy 2 warning * Fix conflicts * Apply suggestions from code review Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> * fixup! Apply suggestions from code review * fixup! fixup! Apply suggestions from code review * add test for dagversion model and make version_name, number and dag_id unique * Remove commented test as serdag can no longer disappear * Add SQLAlchemy-utils to requirements * mark test_dag_version.py as db_test * make version_name nullable * Apply suggestions from code review * fixup! Apply suggestions from code review * remove file_updater * Use dag_version for creating dagruns instead of dag_version_id * fix conflicts * use if TYPE_CHECKING * Add docstrings to methods * Move getting latest serdags to SerializedDagModel --- airflow/api/common/trigger_dag.py | 5 +- .../endpoints/dag_run_endpoint.py | 5 +- airflow/dag_processing/manager.py | 15 - .../example_dags/plugins/event_listener.py | 5 +- airflow/jobs/scheduler_job_runner.py | 22 +- .../versions/0047_3_0_0_add_dag_versioning.py | 151 + airflow/models/__init__.py | 1 + airflow/models/backfill.py | 6 +- airflow/models/dag.py | 26 +- airflow/models/dag_version.py | 167 + airflow/models/dagbag.py | 3 +- airflow/models/dagcode.py | 117 +- airflow/models/dagrun.py | 22 +- airflow/models/serialized_dag.py | 141 +- airflow/models/taskinstance.py | 14 +- airflow/models/taskinstancehistory.py | 2 + airflow/serialization/pydantic/dag_run.py | 2 +- airflow/serialization/schema.json | 1 + airflow/utils/db.py | 2 +- airflow/www/views.py | 4 +- docs/apache-airflow/img/airflow_erd.sha256 | 2 +- docs/apache-airflow/img/airflow_erd.svg | 3942 +++++++++-------- docs/apache-airflow/migrations-ref.rst | 4 +- hatch_build.py | 1 + .../api_endpoints/test_dag_run_endpoint.py | 1 + .../pre_commit/check_ti_vs_tis_attributes.py | 1 + task_sdk/src/airflow/sdk/definitions/dag.py | 6 + .../endpoints/test_dag_run_endpoint.py | 5 +- .../endpoints/test_task_endpoint.py | 6 +- tests/cli/commands/test_task_command.py | 2 +- tests/dag_processing/test_job_runner.py | 24 +- tests/dag_processing/test_processor.py | 1 + tests/jobs/test_scheduler_job.py | 219 +- tests/models/test_dag.py | 24 +- tests/models/test_dag_version.py | 113 + tests/models/test_dagbag.py | 14 +- tests/models/test_dagcode.py | 110 +- tests/models/test_dagrun.py | 1 + tests/models/test_serialized_dag.py | 111 +- tests/models/test_taskinstance.py | 4 +- tests/operators/test_trigger_dagrun.py | 13 +- tests/sensors/test_external_task_sensor.py | 3 +- tests/utils/test_db_cleanup.py | 1 + tests/www/views/test_views_tasks.py | 36 +- tests_common/pytest_plugin.py | 59 +- tests_common/test_utils/db.py | 1 + 46 files changed, 3049 insertions(+), 2366 deletions(-) create mode 100644 airflow/migrations/versions/0047_3_0_0_add_dag_versioning.py create mode 100644 airflow/models/dag_version.py create mode 100644 tests/models/test_dag_version.py diff --git a/airflow/api/common/trigger_dag.py b/airflow/api/common/trigger_dag.py index b18957261f3a0..44beae3f1f78c 100644 --- a/airflow/api/common/trigger_dag.py +++ b/airflow/api/common/trigger_dag.py @@ -25,6 +25,7 @@ from airflow.api_internal.internal_api_call import internal_api_call from airflow.exceptions import DagNotFound, DagRunAlreadyExists from airflow.models import DagBag, DagModel, DagRun +from airflow.models.dag_version import DagVersion from airflow.utils import timezone from airflow.utils.session import NEW_SESSION, provide_session from airflow.utils.state import DagRunState @@ -92,14 +93,14 @@ def _trigger_dag( run_conf = None if conf: run_conf = conf if isinstance(conf, dict) else json.loads(conf) - + dag_version = DagVersion.get_latest_version(dag.dag_id) dag_run = dag.create_dagrun( run_id=run_id, execution_date=execution_date, state=DagRunState.QUEUED, conf=run_conf, external_trigger=True, - dag_hash=dag_bag.dags_hash.get(dag_id), + dag_version=dag_version, data_interval=data_interval, triggered_by=triggered_by, ) diff --git a/airflow/api_connexion/endpoints/dag_run_endpoint.py b/airflow/api_connexion/endpoints/dag_run_endpoint.py index 8ebb2b44e2bb3..6a38eb27ff45c 100644 --- a/airflow/api_connexion/endpoints/dag_run_endpoint.py +++ b/airflow/api_connexion/endpoints/dag_run_endpoint.py @@ -61,6 +61,7 @@ from airflow.auth.managers.models.resource_details import DagAccessEntity from airflow.exceptions import ParamValidationError from airflow.models import DagModel, DagRun +from airflow.models.dag_version import DagVersion from airflow.timetables.base import DataInterval from airflow.utils.airflow_flask_app import get_airflow_app from airflow.utils.api_migration import mark_fastapi_migration_done @@ -341,7 +342,7 @@ def post_dag_run(*, dag_id: str, session: Session = NEW_SESSION) -> APIResponse: ) else: data_interval = dag.timetable.infer_manual_data_interval(run_after=logical_date) - + dag_version = DagVersion.get_latest_version(dag.dag_id) dag_run = dag.create_dagrun( run_type=DagRunType.MANUAL, run_id=run_id, @@ -350,7 +351,7 @@ def post_dag_run(*, dag_id: str, session: Session = NEW_SESSION) -> APIResponse: state=DagRunState.QUEUED, conf=post_body.get("conf"), external_trigger=True, - dag_hash=get_airflow_app().dag_bag.dags_hash.get(dag_id), + dag_version=dag_version, session=session, triggered_by=DagRunTriggeredByType.REST_API, ) diff --git a/airflow/dag_processing/manager.py b/airflow/dag_processing/manager.py index 0f3441a5d4d13..3bc467e2f7063 100644 --- a/airflow/dag_processing/manager.py +++ b/airflow/dag_processing/manager.py @@ -50,7 +50,6 @@ from airflow.models.dagwarning import DagWarning from airflow.models.db_callback_request import DbCallbackRequest from airflow.models.errors import ParseImportError -from airflow.models.serialized_dag import SerializedDagModel from airflow.secrets.cache import SecretCache from airflow.stats import Stats from airflow.traces.tracer import Trace, add_span @@ -539,10 +538,6 @@ def deactivate_stale_dags( if deactivated: cls.logger().info("Deactivated %i DAGs which are no longer present in file.", deactivated) - for dag_id in to_deactivate: - SerializedDagModel.remove_dag(dag_id) - cls.logger().info("Deleted DAG %s in serialized_dag table", dag_id) - def _run_parsing_loop(self): # In sync mode we want timeout=None -- wait forever until a message is received if self._async_mode: @@ -819,20 +814,10 @@ def _iter_dag_filelocs(fileloc: str) -> Iterator[str]: dag_filelocs = {full_loc for path in self._file_paths for full_loc in _iter_dag_filelocs(path)} - from airflow.models.dagcode import DagCode - - SerializedDagModel.remove_deleted_dags( - alive_dag_filelocs=dag_filelocs, - processor_subdir=self.get_dag_directory(), - ) DagModel.deactivate_deleted_dags( dag_filelocs, processor_subdir=self.get_dag_directory(), ) - DagCode.remove_deleted_code( - dag_filelocs, - processor_subdir=self.get_dag_directory(), - ) return True return False diff --git a/airflow/example_dags/plugins/event_listener.py b/airflow/example_dags/plugins/event_listener.py index 4b9be307c4e7e..6d9fe2ff11735 100644 --- a/airflow/example_dags/plugins/event_listener.py +++ b/airflow/example_dags/plugins/event_listener.py @@ -164,9 +164,10 @@ def on_dag_run_running(dag_run: DagRun, msg: str): """ print("Dag run in running state") queued_at = dag_run.queued_at - dag_hash_info = dag_run.dag_hash - print(f"Dag information Queued at: {queued_at} hash info: {dag_hash_info}") + version = dag_run.dag_version.version + + print(f"Dag information Queued at: {queued_at} version: {version}") # [END howto_listen_dagrun_running_task] diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index fb85a4a73cc32..ffa250fc81441 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -54,10 +54,10 @@ ) from airflow.models.backfill import Backfill from airflow.models.dag import DAG, DagModel +from airflow.models.dag_version import DagVersion from airflow.models.dagbag import DagBag from airflow.models.dagrun import DagRun from airflow.models.dagwarning import DagWarning, DagWarningType -from airflow.models.serialized_dag import SerializedDagModel from airflow.models.taskinstance import SimpleTaskInstance, TaskInstance from airflow.stats import Stats from airflow.ti_deps.dependencies_states import EXECUTION_STATES @@ -1338,7 +1338,7 @@ def _create_dag_runs(self, dag_models: Collection[DagModel], session: Session) - self.log.error("DAG '%s' not found in serialized_dag table", dag_model.dag_id) continue - dag_hash = self.dagbag.dags_hash.get(dag.dag_id) + latest_dag_version = DagVersion.get_latest_version(dag.dag_id, session=session) data_interval = dag.get_next_data_interval(dag_model) # Explicitly check if the DagRun already exists. This is an edge case @@ -1358,7 +1358,7 @@ def _create_dag_runs(self, dag_models: Collection[DagModel], session: Session) - data_interval=data_interval, external_trigger=False, session=session, - dag_hash=dag_hash, + dag_version=latest_dag_version, creating_job_id=self.job.id, triggered_by=DagRunTriggeredByType.TIMETABLE, ) @@ -1417,7 +1417,7 @@ def _create_dag_runs_asset_triggered( ) continue - dag_hash = self.dagbag.dags_hash.get(dag.dag_id) + latest_dag_version = DagVersion.get_latest_version(dag.dag_id, session=session) # Explicitly check if the DagRun already exists. This is an edge case # where a Dag Run is created but `DagModel.next_dagrun` and `DagModel.next_dagrun_create_after` @@ -1472,7 +1472,7 @@ def _create_dag_runs_asset_triggered( state=DagRunState.QUEUED, external_trigger=False, session=session, - dag_hash=dag_hash, + dag_version=latest_dag_version, creating_job_id=self.job.id, triggered_by=DagRunTriggeredByType.ASSET, ) @@ -1750,18 +1750,20 @@ def _verify_integrity_if_dag_changed(self, dag_run: DagRun, session: Session) -> Return True if we determine that DAG still exists. """ - latest_version = SerializedDagModel.get_latest_version_hash(dag_run.dag_id, session=session) - if dag_run.dag_hash == latest_version: + latest_dag_version = DagVersion.get_latest_version(dag_run.dag_id, session=session) + if TYPE_CHECKING: + assert latest_dag_version + if dag_run.dag_version_id == latest_dag_version.id: self.log.debug("DAG %s not changed structure, skipping dagrun.verify_integrity", dag_run.dag_id) return True - dag_run.dag_hash = latest_version - # Refresh the DAG dag_run.dag = self.dagbag.get_dag(dag_id=dag_run.dag_id, session=session) if not dag_run.dag: return False + dag_run.dag_version = latest_dag_version + # Verify integrity also takes care of session.flush dag_run.verify_integrity(session=session) return True @@ -2041,7 +2043,6 @@ def _cleanup_stale_dags(self, session: Session = NEW_SESSION) -> None: In case one of DagProcessors is stopped (in case there are multiple of them for different dag folders), its dags are never marked as inactive. - Also remove dags from SerializedDag table. Executed on schedule only if [scheduler]standalone_dag_processor is True. """ self.log.debug("Checking dags not parsed within last %s seconds.", self._dag_stale_not_seen_duration) @@ -2056,7 +2057,6 @@ def _cleanup_stale_dags(self, session: Session = NEW_SESSION) -> None: self.log.info("Found (%d) stales dags not parsed after %s.", len(stale_dags), limit_lpt) for dag in stale_dags: dag.is_active = False - SerializedDagModel.remove_dag(dag_id=dag.dag_id, session=session) session.flush() @provide_session diff --git a/airflow/migrations/versions/0047_3_0_0_add_dag_versioning.py b/airflow/migrations/versions/0047_3_0_0_add_dag_versioning.py new file mode 100644 index 0000000000000..12ffc174c750a --- /dev/null +++ b/airflow/migrations/versions/0047_3_0_0_add_dag_versioning.py @@ -0,0 +1,151 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +""" +add dag versioning. + +Revision ID: 2b47dc6bc8df +Revises: d03e4a635aa3 +Create Date: 2024-10-09 05:44:04.670984 + +""" + +from __future__ import annotations + +import sqlalchemy as sa +from alembic import op +from sqlalchemy_utils import UUIDType + +from airflow.migrations.db_types import StringID +from airflow.models.base import naming_convention +from airflow.utils import timezone +from airflow.utils.sqlalchemy import UtcDateTime + +# revision identifiers, used by Alembic. +revision = "2b47dc6bc8df" +down_revision = "d03e4a635aa3" +branch_labels = None +depends_on = None +airflow_version = "3.0.0" + + +def upgrade(): + """Apply add dag versioning.""" + op.create_table( + "dag_version", + sa.Column("id", UUIDType(binary=False), nullable=False), + sa.Column("version_number", sa.Integer(), nullable=False), + sa.Column("version_name", StringID()), + sa.Column("dag_id", StringID(), nullable=False), + sa.Column("created_at", UtcDateTime(), nullable=False, default=timezone.utcnow), + sa.ForeignKeyConstraint( + ("dag_id",), ["dag.dag_id"], name=op.f("dag_version_dag_id_fkey"), ondelete="CASCADE" + ), + sa.PrimaryKeyConstraint("id", name=op.f("dag_version_pkey")), + sa.UniqueConstraint("dag_id", "version_number", name="dag_id_v_name_v_number_unique_constraint"), + ) + with op.batch_alter_table("dag_code", recreate="always", naming_convention=naming_convention) as batch_op: + batch_op.drop_constraint("dag_code_pkey", type_="primary") + batch_op.add_column( + sa.Column("id", UUIDType(binary=False), primary_key=True), insert_before="fileloc_hash" + ) + batch_op.create_primary_key("dag_code_pkey", ["id"]) + batch_op.add_column(sa.Column("dag_version_id", UUIDType(binary=False), nullable=False)) + batch_op.create_foreign_key( + batch_op.f("dag_code_dag_version_id_fkey"), + "dag_version", + ["dag_version_id"], + ["id"], + ondelete="CASCADE", + ) + batch_op.create_unique_constraint("dag_code_dag_version_id_uq", ["dag_version_id"]) + + with op.batch_alter_table( + "serialized_dag", recreate="always", naming_convention=naming_convention + ) as batch_op: + batch_op.drop_constraint("serialized_dag_pkey", type_="primary") + batch_op.add_column(sa.Column("id", UUIDType(binary=False), primary_key=True)) + batch_op.drop_index("idx_fileloc_hash") + batch_op.drop_column("fileloc_hash") + batch_op.drop_column("fileloc") + batch_op.create_primary_key("serialized_dag_pkey", ["id"]) + batch_op.add_column(sa.Column("dag_version_id", UUIDType(binary=False), nullable=False)) + batch_op.create_foreign_key( + batch_op.f("serialized_dag_dag_version_id_fkey"), + "dag_version", + ["dag_version_id"], + ["id"], + ondelete="CASCADE", + ) + batch_op.create_unique_constraint("serialized_dag_dag_version_id_uq", ["dag_version_id"]) + + with op.batch_alter_table("task_instance", schema=None) as batch_op: + batch_op.add_column(sa.Column("dag_version_id", UUIDType(binary=False))) + batch_op.create_foreign_key( + batch_op.f("task_instance_dag_version_id_fkey"), + "dag_version", + ["dag_version_id"], + ["id"], + ondelete="CASCADE", + ) + + with op.batch_alter_table("task_instance_history", schema=None) as batch_op: + batch_op.add_column(sa.Column("dag_version_id", UUIDType(binary=False))) + + with op.batch_alter_table("dag_run", schema=None) as batch_op: + batch_op.add_column(sa.Column("dag_version_id", UUIDType(binary=False))) + batch_op.create_foreign_key( + batch_op.f("dag_run_dag_version_id_fkey"), + "dag_version", + ["dag_version_id"], + ["id"], + ondelete="CASCADE", + ) + batch_op.drop_column("dag_hash") + + +def downgrade(): + """Unapply add dag versioning.""" + with op.batch_alter_table("task_instance_history", schema=None) as batch_op: + batch_op.drop_column("dag_version_id") + + with op.batch_alter_table("task_instance", schema=None) as batch_op: + batch_op.drop_constraint(batch_op.f("task_instance_dag_version_id_fkey"), type_="foreignkey") + batch_op.drop_column("dag_version_id") + + with op.batch_alter_table("dag_code", schema=None) as batch_op: + batch_op.drop_column("id") + batch_op.drop_constraint(batch_op.f("dag_code_dag_version_id_fkey"), type_="foreignkey") + batch_op.drop_column("dag_version_id") + batch_op.create_primary_key("dag_code_pkey", ["fileloc_hash"]) + + with op.batch_alter_table("serialized_dag", schema=None, naming_convention=naming_convention) as batch_op: + batch_op.drop_column("id") + batch_op.add_column(sa.Column("fileloc", sa.String(length=2000), autoincrement=False, nullable=False)) + batch_op.add_column(sa.Column("fileloc_hash", sa.BIGINT(), autoincrement=False, nullable=False)) + batch_op.create_index("idx_fileloc_hash", ["fileloc_hash"], unique=False) + batch_op.create_primary_key("serialized_dag_pkey", ["dag_id"]) + batch_op.drop_constraint(batch_op.f("serialized_dag_dag_version_id_fkey"), type_="foreignkey") + batch_op.drop_column("dag_version_id") + + with op.batch_alter_table("dag_run", schema=None) as batch_op: + batch_op.add_column(sa.Column("dag_hash", sa.String(length=32), autoincrement=False, nullable=True)) + batch_op.drop_constraint(batch_op.f("dag_run_dag_version_id_fkey"), type_="foreignkey") + batch_op.drop_column("dag_version_id") + + op.drop_table("dag_version") diff --git a/airflow/models/__init__.py b/airflow/models/__init__.py index 1ab4e5584c976..6d8803410532a 100644 --- a/airflow/models/__init__.py +++ b/airflow/models/__init__.py @@ -57,6 +57,7 @@ def import_all_models(): import airflow.models.asset import airflow.models.backfill + import airflow.models.dag_version import airflow.models.dagwarning import airflow.models.errors import airflow.models.serialized_dag diff --git a/airflow/models/backfill.py b/airflow/models/backfill.py index 648b35c5bdebe..11d677542fc0e 100644 --- a/airflow/models/backfill.py +++ b/airflow/models/backfill.py @@ -43,6 +43,7 @@ from airflow.api_connexion.exceptions import NotFound from airflow.exceptions import AirflowException from airflow.models.base import Base, StringID +from airflow.models.dag_version import DagVersion from airflow.settings import json from airflow.utils import timezone from airflow.utils.session import create_session @@ -200,7 +201,7 @@ def _create_backfill_dag_run( ) ) return - + dag_version = DagVersion.get_latest_version(dag.dag_id, session=session) dr = dag.create_dagrun( triggered_by=DagRunTriggeredByType.BACKFILL, execution_date=info.logical_date, @@ -213,6 +214,7 @@ def _create_backfill_dag_run( creating_job_id=None, session=session, backfill_id=backfill_id, + dag_version=dag_version, ) session.add( BackfillDagRun( @@ -253,7 +255,7 @@ def _create_backfill( from airflow.models.serialized_dag import SerializedDagModel with create_session() as session: - serdag = session.get(SerializedDagModel, dag_id) + serdag = session.scalar(SerializedDagModel.latest_item_select_object(dag_id)) if not serdag: raise NotFound(f"Could not find dag {dag_id}") # todo: if dag has no schedule, raise diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 337fc5c8163e1..e6a67c6ad7e5e 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -85,7 +85,7 @@ ) from airflow.models.base import Base, StringID from airflow.models.baseoperator import BaseOperator -from airflow.models.dagcode import DagCode +from airflow.models.dag_version import DagVersion from airflow.models.dagrun import RUN_ID_REGEX, DagRun from airflow.models.taskinstance import ( Context, @@ -257,7 +257,7 @@ def _create_orm_dagrun( conf, state, run_type, - dag_hash, + dag_version, creating_job_id, data_interval, backfill_id, @@ -273,7 +273,7 @@ def _create_orm_dagrun( conf=conf, state=state, run_type=run_type, - dag_hash=dag_hash, + dag_version=dag_version, creating_job_id=creating_job_id, data_interval=data_interval, triggered_by=triggered_by, @@ -424,6 +424,7 @@ class DAG(TaskSDKDag, LoggingMixin): **Warning**: A fail stop dag can only have tasks with the default trigger rule ("all_success"). An exception will be thrown if any task in a fail stop dag has a non default trigger rule. :param dag_display_name: The display name of the DAG which appears on the UI. + :param version_name: The version name to use in storing the dag to the DB. """ partial: bool = False @@ -1708,7 +1709,7 @@ def create_dagrun( conf: dict | None = None, run_type: DagRunType | None = None, session: Session = NEW_SESSION, - dag_hash: str | None = None, + dag_version: DagVersion | None = None, creating_job_id: int | None = None, data_interval: tuple[datetime, datetime] | None = None, backfill_id: int | None = None, @@ -1728,7 +1729,7 @@ def create_dagrun( :param conf: Dict containing configuration/parameters to pass to the DAG :param creating_job_id: id of the job creating this DagRun :param session: database session - :param dag_hash: Hash of Serialized DAG + :param dag_version: The DagVersion object for this run :param data_interval: Data interval of the DagRun :param backfill_id: id of the backfill run if one exists """ @@ -1800,7 +1801,7 @@ def create_dagrun( conf=conf, state=state, run_type=run_type, - dag_hash=dag_hash, + dag_version=dag_version, creating_job_id=creating_job_id, backfill_id=backfill_id, data_interval=data_interval, @@ -1833,7 +1834,6 @@ def bulk_write_to_db( orm_dags = dag_op.add_dags(session=session) dag_op.update_dags(orm_dags, processor_subdir=processor_subdir, session=session) - DagCode.bulk_sync_to_db((dag.fileloc for dag in dags), session=session) asset_op = AssetModelOperation.collect(dag_op.dags) @@ -2069,6 +2069,9 @@ class DagModel(Base): NUM_DAGS_PER_DAGRUN_QUERY = airflow_conf.getint( "scheduler", "max_dagruns_to_create_per_loop", fallback=10 ) + dag_versions = relationship( + "DagVersion", back_populates="dag_model", cascade="all, delete, delete-orphan" + ) def __init__(self, **kwargs): super().__init__(**kwargs) @@ -2275,9 +2278,10 @@ def dag_ready(dag_id: str, cond: BaseAsset, statuses: dict) -> bool | None: dag_statuses = {} for dag_id, records in by_dag.items(): dag_statuses[dag_id] = {x.asset.uri: True for x in records} - ser_dags = session.scalars( - select(SerializedDagModel).where(SerializedDagModel.dag_id.in_(dag_statuses.keys())) - ).all() + ser_dags = SerializedDagModel.get_latest_serialized_dags( + dag_ids=list(dag_statuses.keys()), session=session + ) + for ser_dag in ser_dags: dag_id = ser_dag.dag_id statuses = dag_statuses[dag_id] @@ -2452,6 +2456,7 @@ def _get_or_create_dagrun( if dr: session.delete(dr) session.commit() + dag_version = DagVersion.get_latest_version(dag.dag_id, session=session) dr = dag.create_dagrun( state=DagRunState.RUNNING, execution_date=execution_date, @@ -2461,6 +2466,7 @@ def _get_or_create_dagrun( conf=conf, data_interval=data_interval, triggered_by=triggered_by, + dag_version=dag_version, ) log.info("created dagrun %s", dr) return dr diff --git a/airflow/models/dag_version.py b/airflow/models/dag_version.py new file mode 100644 index 0000000000000..92511f93f5f6f --- /dev/null +++ b/airflow/models/dag_version.py @@ -0,0 +1,167 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +import logging +from typing import TYPE_CHECKING + +import uuid6 +from sqlalchemy import Column, ForeignKey, Integer, UniqueConstraint, select +from sqlalchemy.orm import relationship +from sqlalchemy_utils import UUIDType + +from airflow.models.base import Base, StringID +from airflow.utils import timezone +from airflow.utils.session import NEW_SESSION, provide_session +from airflow.utils.sqlalchemy import UtcDateTime, with_row_locks + +if TYPE_CHECKING: + from sqlalchemy.orm import Session + from sqlalchemy.sql import Select + +log = logging.getLogger(__name__) + + +class DagVersion(Base): + """Model to track the versions of DAGs in the database.""" + + __tablename__ = "dag_version" + id = Column(UUIDType(binary=False), primary_key=True, default=uuid6.uuid7) + version_number = Column(Integer, nullable=False, default=1) + version_name = Column(StringID()) + dag_id = Column(StringID(), ForeignKey("dag.dag_id", ondelete="CASCADE"), nullable=False) + dag_model = relationship("DagModel", back_populates="dag_versions") + dag_code = relationship( + "DagCode", + back_populates="dag_version", + uselist=False, + cascade="all, delete, delete-orphan", + cascade_backrefs=False, + ) + serialized_dag = relationship( + "SerializedDagModel", + back_populates="dag_version", + uselist=False, + cascade="all, delete, delete-orphan", + cascade_backrefs=False, + ) + dag_runs = relationship("DagRun", back_populates="dag_version", cascade="all, delete, delete-orphan") + task_instances = relationship("TaskInstance", back_populates="dag_version") + created_at = Column(UtcDateTime, default=timezone.utcnow) + + __table_args__ = ( + UniqueConstraint("dag_id", "version_number", name="dag_id_v_name_v_number_unique_constraint"), + ) + + def __repr__(self): + """Represent the object as a string.""" + return f"<DagVersion {self.dag_id} {self.version}>" + + @classmethod + @provide_session + def write_dag( + cls, + *, + dag_id: str, + version_name: str | None = None, + version_number: int = 1, + session: Session = NEW_SESSION, + ) -> DagVersion: + """ + Write a new DagVersion into database. + + Checks if a version of the DAG exists and increments the version number if it does. + + :param dag_id: The DAG ID. + :param version_name: The version name. + :param version_number: The version number. + :param session: The database session. + :return: The DagVersion object. + """ + existing_dag_version = session.scalar( + with_row_locks(cls._latest_version_select(dag_id), of=DagVersion, session=session, nowait=True) + ) + if existing_dag_version: + version_number = existing_dag_version.version_number + 1 + + dag_version = DagVersion( + dag_id=dag_id, + version_number=version_number, + version_name=version_name, + ) + log.debug("Writing DagVersion %s to the DB", dag_version) + session.add(dag_version) + # Flush is necessary here due to the unique constraint and other linked tables + session.flush() + log.debug("DagVersion %s written to the DB", dag_version) + return dag_version + + @classmethod + def _latest_version_select(cls, dag_id: str) -> Select: + """ + Get the select object to get the latest version of the DAG. + + :param dag_id: The DAG ID. + :return: The select object. + """ + return select(cls).where(cls.dag_id == dag_id).order_by(cls.created_at.desc()).limit(1) + + @classmethod + @provide_session + def get_latest_version(cls, dag_id: str, *, session: Session = NEW_SESSION) -> DagVersion | None: + """ + Get the latest version of the DAG. + + :param dag_id: The DAG ID. + :param session: The database session. + :return: The latest version of the DAG or None if not found. + """ + return session.scalar(cls._latest_version_select(dag_id)) + + @classmethod + @provide_session + def get_version( + cls, + dag_id: str, + version_number: int = 1, + *, + session: Session = NEW_SESSION, + ) -> DagVersion | None: + """ + Get the version of the DAG. + + :param dag_id: The DAG ID. + :param version_number: The version number. + :param session: The database session. + :return: The version of the DAG or None if not found. + """ + version_select_obj = ( + select(cls) + .where(cls.dag_id == dag_id, cls.version_number == version_number) + .order_by(cls.version_number.desc()) + .limit(1) + ) + return session.scalar(version_select_obj) + + @property + def version(self) -> str: + """A human-friendly representation of the version.""" + name = f"{self.version_number}" + if self.version_name: + name = f"{self.version_name}-{self.version_number}" + return name diff --git a/airflow/models/dagbag.py b/airflow/models/dagbag.py index c9ad8edaa4018..5b57c7983ea14 100644 --- a/airflow/models/dagbag.py +++ b/airflow/models/dagbag.py @@ -650,13 +650,12 @@ def _serialize_dag_capturing_errors(dag, session, processor_subdir): ) log.debug("Calling the DAG.bulk_sync_to_db method") try: + DAG.bulk_write_to_db(dags.values(), processor_subdir=processor_subdir, session=session) # Write Serialized DAGs to DB, capturing errors for dag in dags.values(): serialize_errors.extend( _serialize_dag_capturing_errors(dag, session, processor_subdir) ) - - DAG.bulk_write_to_db(dags.values(), processor_subdir=processor_subdir, session=session) except OperationalError: session.rollback() raise diff --git a/airflow/models/dagcode.py b/airflow/models/dagcode.py index 321f819999bf6..c78f6cafaa6fa 100644 --- a/airflow/models/dagcode.py +++ b/airflow/models/dagcode.py @@ -17,26 +17,30 @@ from __future__ import annotations import logging -import os import struct -from datetime import datetime -from typing import TYPE_CHECKING, Collection, Iterable +from typing import TYPE_CHECKING, Collection -from sqlalchemy import BigInteger, Column, String, Text, delete, select +import uuid6 +from sqlalchemy import BigInteger, Column, ForeignKey, String, Text, delete, select from sqlalchemy.dialects.mysql import MEDIUMTEXT +from sqlalchemy.orm import relationship from sqlalchemy.sql.expression import literal +from sqlalchemy_utils import UUIDType from airflow.api_internal.internal_api_call import internal_api_call -from airflow.exceptions import AirflowException, DagCodeNotFound +from airflow.configuration import conf +from airflow.exceptions import DagCodeNotFound from airflow.models.base import Base from airflow.utils import timezone -from airflow.utils.file import correct_maybe_zipped, open_maybe_zipped +from airflow.utils.file import open_maybe_zipped from airflow.utils.session import NEW_SESSION, provide_session from airflow.utils.sqlalchemy import UtcDateTime if TYPE_CHECKING: from sqlalchemy.orm import Session + from airflow.models.dag_version import DagVersion + log = logging.getLogger(__name__) @@ -50,84 +54,38 @@ class DagCode(Base): """ __tablename__ = "dag_code" - - fileloc_hash = Column(BigInteger, nullable=False, primary_key=True, autoincrement=False) + id = Column(UUIDType(binary=False), primary_key=True, default=uuid6.uuid7) + fileloc_hash = Column(BigInteger, nullable=False) fileloc = Column(String(2000), nullable=False) # The max length of fileloc exceeds the limit of indexing. last_updated = Column(UtcDateTime, nullable=False) source_code = Column(Text().with_variant(MEDIUMTEXT(), "mysql"), nullable=False) + dag_version_id = Column( + UUIDType(binary=False), ForeignKey("dag_version.id", ondelete="CASCADE"), nullable=False, unique=True + ) + dag_version = relationship("DagVersion", back_populates="dag_code", uselist=False) - def __init__(self, full_filepath: str, source_code: str | None = None): + def __init__(self, dag_version, full_filepath: str, source_code: str | None = None): + self.dag_version = dag_version self.fileloc = full_filepath self.fileloc_hash = DagCode.dag_fileloc_hash(self.fileloc) self.last_updated = timezone.utcnow() self.source_code = source_code or DagCode.code(self.fileloc) - @provide_session - def sync_to_db(self, session: Session = NEW_SESSION) -> None: - """ - Write code into database. - - :param session: ORM Session - """ - self.bulk_sync_to_db([self.fileloc], session) - @classmethod @provide_session - def bulk_sync_to_db(cls, filelocs: Iterable[str], session: Session = NEW_SESSION) -> None: + def write_dag(cls, dag_version: DagVersion, fileloc: str, session: Session = NEW_SESSION) -> DagCode: """ - Write code in bulk into database. + Write code into database. - :param filelocs: file paths of DAGs to sync + :param fileloc: file path of DAG to sync :param session: ORM Session """ - filelocs = set(filelocs) - filelocs_to_hashes = {fileloc: DagCode.dag_fileloc_hash(fileloc) for fileloc in filelocs} - existing_orm_dag_codes = session.scalars( - select(DagCode) - .filter(DagCode.fileloc_hash.in_(filelocs_to_hashes.values())) - .with_for_update(of=DagCode) - ).all() - - if existing_orm_dag_codes: - existing_orm_dag_codes_map = { - orm_dag_code.fileloc: orm_dag_code for orm_dag_code in existing_orm_dag_codes - } - else: - existing_orm_dag_codes_map = {} - - existing_orm_dag_codes_by_fileloc_hashes = {orm.fileloc_hash: orm for orm in existing_orm_dag_codes} - existing_orm_filelocs = {orm.fileloc for orm in existing_orm_dag_codes_by_fileloc_hashes.values()} - if not existing_orm_filelocs.issubset(filelocs): - conflicting_filelocs = existing_orm_filelocs.difference(filelocs) - hashes_to_filelocs = {DagCode.dag_fileloc_hash(fileloc): fileloc for fileloc in filelocs} - message = "" - for fileloc in conflicting_filelocs: - filename = hashes_to_filelocs[DagCode.dag_fileloc_hash(fileloc)] - message += ( - f"Filename '{filename}' causes a hash collision in the " - f"database with '{fileloc}'. Please rename the file." - ) - raise AirflowException(message) - - existing_filelocs = {dag_code.fileloc for dag_code in existing_orm_dag_codes} - missing_filelocs = filelocs.difference(existing_filelocs) - - for fileloc in missing_filelocs: - orm_dag_code = DagCode(fileloc, cls._get_code_from_file(fileloc)) - session.add(orm_dag_code) - - for fileloc in existing_filelocs: - current_version = existing_orm_dag_codes_by_fileloc_hashes[filelocs_to_hashes[fileloc]] - file_mod_time = datetime.fromtimestamp( - os.path.getmtime(correct_maybe_zipped(fileloc)), tz=timezone.utc - ) - - if file_mod_time > current_version.last_updated: - orm_dag_code = existing_orm_dag_codes_map[fileloc] - orm_dag_code.last_updated = file_mod_time - orm_dag_code.source_code = cls._get_code_from_file(orm_dag_code.fileloc) - session.merge(orm_dag_code) + log.debug("Writing DAG file %s into DagCode table", fileloc) + dag_code = DagCode(dag_version, fileloc, cls._get_code_from_file(fileloc)) + session.add(dag_code) + log.debug("DAG file %s written into DagCode table", fileloc) + return dag_code @classmethod @internal_api_call @@ -170,7 +128,9 @@ def has_dag(cls, fileloc: str, session: Session = NEW_SESSION) -> bool: """ fileloc_hash = cls.dag_fileloc_hash(fileloc) return ( - session.scalars(select(literal(True)).where(cls.fileloc_hash == fileloc_hash)).one_or_none() + session.scalars( + select(literal(True)).where(cls.fileloc_hash == fileloc_hash).limit(1) + ).one_or_none() is not None ) @@ -196,14 +156,25 @@ def code(cls, fileloc, session: Session = NEW_SESSION) -> str: @staticmethod def _get_code_from_file(fileloc): - with open_maybe_zipped(fileloc, "r") as f: - code = f.read() - return code + try: + with open_maybe_zipped(fileloc, "r") as f: + code = f.read() + return code + except FileNotFoundError: + test_mode = conf.get("core", "unit_test_mode") + if test_mode: + return "source_code" + raise @classmethod @provide_session def _get_code_from_db(cls, fileloc, session: Session = NEW_SESSION) -> str: - dag_code = session.scalar(select(cls).where(cls.fileloc_hash == cls.dag_fileloc_hash(fileloc))) + dag_code = session.scalar( + select(cls) + .where(cls.fileloc_hash == cls.dag_fileloc_hash(fileloc)) + .order_by(cls.last_updated.desc()) + .limit(1) + ) if not dag_code: raise DagCodeNotFound() else: diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 5de0466a6be0e..635cd73ccd8d7 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -48,6 +48,7 @@ from sqlalchemy.orm import declared_attr, joinedload, relationship, synonym, validates from sqlalchemy.sql.expression import case, false, select, true from sqlalchemy.sql.functions import coalesce +from sqlalchemy_utils import UUIDType from airflow import settings from airflow.api_internal.internal_api_call import internal_api_call @@ -59,6 +60,7 @@ from airflow.models.abstractoperator import NotMapped from airflow.models.backfill import Backfill from airflow.models.base import Base, StringID +from airflow.models.dag_version import DagVersion from airflow.models.expandinput import NotFullyPopulated from airflow.models.taskinstance import TaskInstance as TI from airflow.models.tasklog import LogTemplate @@ -144,7 +146,6 @@ class DagRun(Base, LoggingMixin): data_interval_end = Column(UtcDateTime) # When a scheduler last attempted to schedule TIs for this DagRun last_scheduling_decision = Column(UtcDateTime) - dag_hash = Column(String(32)) # Foreign key to LogTemplate. DagRun rows created prior to this column's # existence have this set to NULL. Later rows automatically populate this on # insert to point to the latest LogTemplate entry. @@ -164,6 +165,8 @@ class DagRun(Base, LoggingMixin): It's possible this could change if e.g. the dag run is cleared to be rerun, or perhaps re-backfilled. """ + dag_version_id = Column(UUIDType(binary=False), ForeignKey("dag_version.id", ondelete="CASCADE")) + dag_version = relationship("DagVersion", back_populates="dag_runs") # Remove this `if` after upgrading Sphinx-AutoAPI if not TYPE_CHECKING and "BUILDING_AIRFLOW_DOCS" in os.environ: @@ -231,11 +234,11 @@ def __init__( conf: Any | None = None, state: DagRunState | None = None, run_type: str | None = None, - dag_hash: str | None = None, creating_job_id: int | None = None, data_interval: tuple[datetime, datetime] | None = None, triggered_by: DagRunTriggeredByType | None = None, backfill_id: int | None = None, + dag_version: DagVersion | None = None, ): if data_interval is None: # Legacy: Only happen for runs created prior to Airflow 2.2. @@ -256,11 +259,11 @@ def __init__( else: self.queued_at = queued_at self.run_type = run_type - self.dag_hash = dag_hash self.creating_job_id = creating_job_id self.backfill_id = backfill_id self.clear_number = 0 self.triggered_by = triggered_by + self.dag_version = dag_version super().__init__() def __repr__(self): @@ -994,8 +997,9 @@ def recalculate(self) -> _UnfinishedStates: "DagRun Finished: dag_id=%s, execution_date=%s, run_id=%s, " "run_start_date=%s, run_end_date=%s, run_duration=%s, " "state=%s, external_trigger=%s, run_type=%s, " - "data_interval_start=%s, data_interval_end=%s, dag_hash=%s" + "data_interval_start=%s, data_interval_end=%s, dag_version_name=%s" ) + dagv = session.scalar(select(DagVersion).where(DagVersion.id == self.dag_version_id)) self.log.info( msg, self.dag_id, @@ -1013,7 +1017,7 @@ def recalculate(self) -> _UnfinishedStates: self.run_type, self.data_interval_start, self.data_interval_end, - self.dag_hash, + dagv.version if dagv else None, ) with Trace.start_span_from_dagrun(dagrun=self) as span: @@ -1037,7 +1041,7 @@ def recalculate(self) -> _UnfinishedStates: "run_type": str(self.run_type), "data_interval_start": str(self.data_interval_start), "data_interval_end": str(self.data_interval_end), - "dag_hash": str(self.dag_hash), + "dag_version": str(dagv.version if dagv else None), "conf": str(self.conf), } if span.is_recording(): @@ -1454,7 +1458,9 @@ def _get_task_creator( def create_ti_mapping(task: Operator, indexes: Iterable[int]) -> Iterator[dict[str, Any]]: created_counts[task.task_type] += 1 for map_index in indexes: - yield TI.insert_mapping(self.run_id, task, map_index=map_index) + yield TI.insert_mapping( + self.run_id, task, map_index=map_index, dag_version_id=self.dag_version_id + ) creator = create_ti_mapping @@ -1462,7 +1468,7 @@ def create_ti_mapping(task: Operator, indexes: Iterable[int]) -> Iterator[dict[s def create_ti(task: Operator, indexes: Iterable[int]) -> Iterator[TI]: for map_index in indexes: - ti = TI(task, run_id=self.run_id, map_index=map_index) + ti = TI(task, run_id=self.run_id, map_index=map_index, dag_version_id=self.dag_version_id) ti_mutation_hook(ti) created_counts[ti.operator] += 1 yield ti diff --git a/airflow/models/serialized_dag.py b/airflow/models/serialized_dag.py index 32be31d721e34..0d5667cd48fc9 100644 --- a/airflow/models/serialized_dag.py +++ b/airflow/models/serialized_dag.py @@ -25,14 +25,17 @@ from typing import TYPE_CHECKING, Any, Collection import sqlalchemy_jsonfield -from sqlalchemy import BigInteger, Column, Index, LargeBinary, String, and_, exc, or_, select +import uuid6 +from sqlalchemy import Column, ForeignKey, LargeBinary, String, exc, or_, select from sqlalchemy.orm import backref, foreign, relationship from sqlalchemy.sql.expression import func, literal +from sqlalchemy_utils import UUIDType from airflow.api_internal.internal_api_call import internal_api_call from airflow.exceptions import TaskNotFound from airflow.models.base import ID_LEN, Base from airflow.models.dag import DagModel +from airflow.models.dag_version import DagVersion from airflow.models.dagcode import DagCode from airflow.models.dagrun import DagRun from airflow.serialization.dag_dependency import DagDependency @@ -76,19 +79,14 @@ class SerializedDagModel(Base): """ __tablename__ = "serialized_dag" - - dag_id = Column(String(ID_LEN), primary_key=True) - fileloc = Column(String(2000), nullable=False) - # The max length of fileloc exceeds the limit of indexing. - fileloc_hash = Column(BigInteger(), nullable=False) + id = Column(UUIDType(binary=False), primary_key=True, default=uuid6.uuid7) + dag_id = Column(String(ID_LEN), nullable=False) _data = Column("data", sqlalchemy_jsonfield.JSONField(json=json), nullable=True) _data_compressed = Column("data_compressed", LargeBinary, nullable=True) last_updated = Column(UtcDateTime, nullable=False) dag_hash = Column(String(32), nullable=False) processor_subdir = Column(String(2000), nullable=True) - __table_args__ = (Index("idx_fileloc_hash", fileloc_hash, unique=False),) - dag_runs = relationship( DagRun, primaryjoin=dag_id == foreign(DagRun.dag_id), # type: ignore @@ -103,13 +101,15 @@ class SerializedDagModel(Base): innerjoin=True, backref=backref("serialized_dag", uselist=False, innerjoin=True), ) + dag_version_id = Column( + UUIDType(binary=False), ForeignKey("dag_version.id", ondelete="CASCADE"), nullable=False, unique=True + ) + dag_version = relationship("DagVersion", back_populates="serialized_dag") load_op_links = True def __init__(self, dag: DAG, processor_subdir: str | None = None) -> None: self.dag_id = dag.dag_id - self.fileloc = dag.fileloc - self.fileloc_hash = DagCode.dag_fileloc_hash(self.fileloc) self.last_updated = timezone.utcnow() self.processor_subdir = processor_subdir @@ -194,7 +194,9 @@ def write_dag( log.debug("Checking if DAG (%s) changed", dag.dag_id) new_serialized_dag = cls(dag, processor_subdir) serialized_dag_db = session.execute( - select(cls.dag_hash, cls.processor_subdir).where(cls.dag_id == dag.dag_id) + select(cls.dag_hash, cls.processor_subdir) + .where(cls.dag_id == dag.dag_id) + .order_by(cls.last_updated.desc()) ).first() if ( @@ -204,12 +206,52 @@ def write_dag( ): log.debug("Serialized DAG (%s) is unchanged. Skipping writing to DB", dag.dag_id) return False - + dagv = DagVersion.write_dag( + version_name=dag.version_name, + dag_id=dag.dag_id, + session=session, + ) log.debug("Writing Serialized DAG: %s to the DB", dag.dag_id) - session.merge(new_serialized_dag) + new_serialized_dag.dag_version = dagv + session.add(new_serialized_dag) log.debug("DAG: %s written to the DB", dag.dag_id) + + DagCode.write_dag(dagv, dag.fileloc, session=session) return True + @classmethod + def latest_item_select_object(cls, dag_id): + return select(cls).where(cls.dag_id == dag_id).order_by(cls.last_updated.desc()).limit(1) + + @classmethod + @provide_session + def get_latest_serialized_dags( + cls, *, dag_ids: list[str], session: Session = NEW_SESSION + ) -> list[SerializedDagModel]: + """ + Get the latest serialized dags of given DAGs. + + :param dag_ids: The list of DAG IDs. + :param session: The database session. + :return: The latest serialized dag of the DAGs. + """ + # Subquery to get the latest serdag per dag_id + latest_serdag_subquery = ( + session.query(cls.dag_id, func.max(cls.last_updated).label("last_updated")) + .filter(cls.dag_id.in_(dag_ids)) + .group_by(cls.dag_id) + .subquery() + ) + latest_serdags = session.scalars( + select(cls) + .join( + latest_serdag_subquery, + cls.last_updated == latest_serdag_subquery.c.last_updated, + ) + .where(cls.dag_id.in_(dag_ids)) + ).all() + return latest_serdags or [] + @classmethod @provide_session def read_all_dags(cls, session: Session = NEW_SESSION) -> dict[str, SerializedDAG]: @@ -219,7 +261,18 @@ def read_all_dags(cls, session: Session = NEW_SESSION) -> dict[str, SerializedDA :param session: ORM Session :returns: a dict of DAGs read from database """ - serialized_dags = session.scalars(select(cls)) + latest_serialized_dag_subquery = ( + session.query(cls.dag_id, func.max(cls.last_updated).label("max_updated")) + .group_by(cls.dag_id) + .subquery() + ) + serialized_dags = session.scalars( + select(cls).join( + latest_serialized_dag_subquery, + (cls.dag_id == latest_serialized_dag_subquery.c.dag_id) + and (cls.last_updated == latest_serialized_dag_subquery.c.max_updated), + ) + ) dags = {} for row in serialized_dags: @@ -287,22 +340,17 @@ def remove_deleted_dags( :param processor_subdir: dag processor subdir :param session: ORM Session """ - alive_fileloc_hashes = [DagCode.dag_fileloc_hash(fileloc) for fileloc in alive_dag_filelocs] - log.debug( "Deleting Serialized DAGs (for which DAG files are deleted) from %s table ", cls.__tablename__ ) - + # Deleting the DagModel cascade deletes the serialized Dag through the dag version relationship session.execute( - cls.__table__.delete().where( - and_( - cls.fileloc_hash.notin_(alive_fileloc_hashes), - cls.fileloc.notin_(alive_dag_filelocs), - or_( - cls.processor_subdir.is_(None), - cls.processor_subdir == processor_subdir, - ), - ) + DagModel.__table__.delete().where( + DagModel.fileloc.notin_(alive_dag_filelocs), + or_( + DagModel.processor_subdir.is_(None), + DagModel.processor_subdir == processor_subdir, + ), ) ) @@ -334,11 +382,7 @@ def get(cls, dag_id: str, session: Session = NEW_SESSION) -> SerializedDagModel :param dag_id: the DAG to fetch :param session: ORM Session """ - row = session.scalar(select(cls).where(cls.dag_id == dag_id)) - if row: - return row - - return session.scalar(select(cls).where(cls.dag_id == dag_id)) + return session.scalar(cls.latest_item_select_object(dag_id)) @staticmethod @provide_session @@ -373,7 +417,9 @@ def get_last_updated_datetime(cls, dag_id: str, session: Session = NEW_SESSION) :param dag_id: DAG ID :param session: ORM Session """ - return session.scalar(select(cls.last_updated).where(cls.dag_id == dag_id)) + return session.scalar( + select(cls.last_updated).where(cls.dag_id == dag_id).order_by(cls.last_updated.desc()).limit(1) + ) @classmethod @provide_session @@ -395,7 +441,9 @@ def get_latest_version_hash(cls, dag_id: str, session: Session = NEW_SESSION) -> :param session: ORM Session :return: DAG Hash, or None if the DAG is not found """ - return session.scalar(select(cls.dag_hash).where(cls.dag_id == dag_id)) + return session.scalar( + select(cls.dag_hash).where(cls.dag_id == dag_id).order_by(cls.last_updated.desc()).limit(1) + ) @classmethod def get_latest_version_hash_and_updated_datetime( @@ -413,7 +461,10 @@ def get_latest_version_hash_and_updated_datetime( :return: A tuple of DAG Hash and last updated datetime, or None if the DAG is not found """ return session.execute( - select(cls.dag_hash, cls.last_updated).where(cls.dag_id == dag_id) + select(cls.dag_hash, cls.last_updated) + .where(cls.dag_id == dag_id) + .order_by(cls.last_updated.desc()) + .limit(1) ).one_or_none() @classmethod @@ -424,14 +475,27 @@ def get_dag_dependencies(cls, session: Session = NEW_SESSION) -> dict[str, list[ :param session: ORM Session """ + latest_sdag_subquery = ( + session.query(cls.dag_id, func.max(cls.last_updated).label("max_updated")) + .group_by(cls.dag_id) + .subquery() + ) if session.bind.dialect.name in ["sqlite", "mysql"]: query = session.execute( - select(cls.dag_id, func.json_extract(cls._data, "$.dag.dag_dependencies")) + select(cls.dag_id, func.json_extract(cls._data, "$.dag.dag_dependencies")).join( + latest_sdag_subquery, + (cls.dag_id == latest_sdag_subquery.c.dag_id) + and (cls.last_updated == latest_sdag_subquery.c.max_updated), + ) ) iterator = ((dag_id, json.loads(deps_data) if deps_data else []) for dag_id, deps_data in query) else: iterator = session.execute( - select(cls.dag_id, func.json_extract_path(cls._data, "dag", "dag_dependencies")) + select(cls.dag_id, func.json_extract_path(cls._data, "dag", "dag_dependencies")).join( + latest_sdag_subquery, + (cls.dag_id == latest_sdag_subquery.c.dag_id) + and (cls.last_updated == latest_sdag_subquery.c.max_updated), + ) ) return {dag_id: [DagDependency(**d) for d in (deps_data or [])] for dag_id, deps_data in iterator} @@ -439,10 +503,9 @@ def get_dag_dependencies(cls, session: Session = NEW_SESSION) -> dict[str, list[ @internal_api_call @provide_session def get_serialized_dag(dag_id: str, task_id: str, session: Session = NEW_SESSION) -> Operator | None: - from airflow.models.serialized_dag import SerializedDagModel - try: - model = session.get(SerializedDagModel, dag_id) + # get the latest version of the DAG + model = session.scalar(SerializedDagModel.latest_item_select_object(dag_id)) if model: return model.dag.get_task(task_id) except (exc.NoResultFound, TaskNotFound): diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index c525a40a14ab5..a8a96a25f6298 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -45,6 +45,7 @@ Column, DateTime, Float, + ForeignKey, ForeignKeyConstraint, Index, Integer, @@ -69,6 +70,7 @@ from sqlalchemy.orm import lazyload, reconstructor, relationship from sqlalchemy.orm.attributes import NO_VALUE, set_committed_value from sqlalchemy.sql.expression import case, select +from sqlalchemy_utils import UUIDType from airflow import settings from airflow.api_internal.internal_api_call import InternalApiConfig, internal_api_call @@ -821,6 +823,7 @@ def _set_ti_attrs(target, source, include_dag_run=False): target.trigger_id = source.trigger_id target.next_method = source.next_method target.next_kwargs = source.next_kwargs + target.dag_version_id = source.dag_version_id if include_dag_run: target.execution_date = source.execution_date @@ -839,7 +842,7 @@ def _set_ti_attrs(target, source, include_dag_run=False): target.dag_run.data_interval_start = source.dag_run.data_interval_start target.dag_run.data_interval_end = source.dag_run.data_interval_end target.dag_run.last_scheduling_decision = source.dag_run.last_scheduling_decision - target.dag_run.dag_hash = source.dag_run.dag_hash + target.dag_run.dag_version_id = source.dag_run.dag_version_id target.dag_run.updated_at = source.dag_run.updated_at target.dag_run.log_template_id = source.dag_run.log_template_id @@ -1876,8 +1879,10 @@ class TaskInstance(Base, LoggingMixin): next_kwargs = Column(MutableDict.as_mutable(ExtendedJSON)) _task_display_property_value = Column("task_display_name", String(2000), nullable=True) + dag_version_id = Column(UUIDType(binary=False), ForeignKey("dag_version.id", ondelete="CASCADE")) + dag_version = relationship("DagVersion", back_populates="task_instances") # If adding new fields here then remember to add them to - # refresh_from_db() or they won't display in the UI correctly + # _set_ti_attrs() or they won't display in the UI correctly __table_args__ = ( Index("ti_dag_state", dag_id, state), @@ -1942,11 +1947,13 @@ def __init__( run_id: str | None = None, state: str | None = None, map_index: int = -1, + dag_version_id: UUIDType | None = None, ): super().__init__() self.dag_id = task.dag_id self.task_id = task.task_id self.map_index = map_index + self.dag_version_id = dag_version_id self.refresh_from_task(task) if TYPE_CHECKING: assert self.task @@ -1978,7 +1985,7 @@ def stats_tags(self) -> dict[str, str]: return _stats_tags(task_instance=self) @staticmethod - def insert_mapping(run_id: str, task: Operator, map_index: int) -> dict[str, Any]: + def insert_mapping(run_id: str, task: Operator, map_index: int, dag_version_id: int) -> dict[str, Any]: """ Insert mapping. @@ -2007,6 +2014,7 @@ def insert_mapping(run_id: str, task: Operator, map_index: int) -> dict[str, Any "custom_operator_name": getattr(task, "custom_operator_name", None), "map_index": map_index, "_task_display_property_value": task.task_display_name, + "dag_version_id": dag_version_id, } @reconstructor diff --git a/airflow/models/taskinstancehistory.py b/airflow/models/taskinstancehistory.py index 8c77daf925793..e587cf083e3b5 100644 --- a/airflow/models/taskinstancehistory.py +++ b/airflow/models/taskinstancehistory.py @@ -33,6 +33,7 @@ text, ) from sqlalchemy.ext.mutable import MutableDict +from sqlalchemy_utils import UUIDType from airflow.models.base import Base, StringID from airflow.utils import timezone @@ -91,6 +92,7 @@ class TaskInstanceHistory(Base): next_kwargs = Column(MutableDict.as_mutable(ExtendedJSON)) task_display_name = Column("task_display_name", String(2000), nullable=True) + dag_version_id = Column(UUIDType(binary=False)) def __init__( self, diff --git a/airflow/serialization/pydantic/dag_run.py b/airflow/serialization/pydantic/dag_run.py index fd12ca12c0184..a0175e3749d9c 100644 --- a/airflow/serialization/pydantic/dag_run.py +++ b/airflow/serialization/pydantic/dag_run.py @@ -52,7 +52,7 @@ class DagRunPydantic(BaseModelPydantic): data_interval_start: Optional[datetime] data_interval_end: Optional[datetime] last_scheduling_decision: Optional[datetime] - dag_hash: Optional[str] + dag_version_id: Optional[int] updated_at: Optional[datetime] dag: Optional[PydanticDag] consumed_asset_events: List[AssetEventPydantic] # noqa: UP006 diff --git a/airflow/serialization/schema.json b/airflow/serialization/schema.json index 32ccd3dfff9c1..b26b59339816c 100644 --- a/airflow/serialization/schema.json +++ b/airflow/serialization/schema.json @@ -158,6 +158,7 @@ }, "dag_display_name": { "type" : "string"}, "description": { "type" : "string"}, + "version_name": {"type": "string"}, "_concurrency": { "type" : "number"}, "max_active_tasks": { "type" : "number"}, "max_active_runs": { "type" : "number"}, diff --git a/airflow/utils/db.py b/airflow/utils/db.py index d23f54068b59e..d5218b6050e6d 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -97,7 +97,7 @@ class MappedClassProtocol(Protocol): "2.9.2": "686269002441", "2.10.0": "22ed7efa9da2", "2.10.3": "5f2621c13b39", - "3.0.0": "d03e4a635aa3", + "3.0.0": "2b47dc6bc8df", } diff --git a/airflow/www/views.py b/airflow/www/views.py index e287c027a894e..cce2ec8b88c8b 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -105,6 +105,7 @@ from airflow.models import Connection, DagModel, DagTag, Log, Trigger, XCom from airflow.models.asset import AssetDagRunQueue, AssetEvent, AssetModel, DagScheduleAssetReference from airflow.models.dag import get_asset_triggered_next_run_info +from airflow.models.dag_version import DagVersion from airflow.models.dagrun import RUN_ID_REGEX, DagRun, DagRunType from airflow.models.errors import ParseImportError from airflow.models.serialized_dag import SerializedDagModel @@ -2201,6 +2202,7 @@ def trigger(self, dag_id: str, session: Session = NEW_SESSION): ) try: + dag_version = DagVersion.get_latest_version(dag.dag_id) dag_run = dag.create_dagrun( run_type=DagRunType.MANUAL, execution_date=execution_date, @@ -2208,7 +2210,7 @@ def trigger(self, dag_id: str, session: Session = NEW_SESSION): state=DagRunState.QUEUED, conf=run_conf, external_trigger=True, - dag_hash=get_airflow_app().dag_bag.dags_hash.get(dag_id), + dag_version=dag_version, run_id=run_id, triggered_by=DagRunTriggeredByType.UI, ) diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index 572ce439c231b..c8963880f842e 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -5ec1019b1b0f43b29fc83638c2a13c0bda90b7e4f0ff542aeab401bbfa9a83e4 \ No newline at end of file +f997746cdee45147831f81bcd2d43ec3ca45d7429afa691e385104987ed51d88 \ No newline at end of file diff --git a/docs/apache-airflow/img/airflow_erd.svg b/docs/apache-airflow/img/airflow_erd.svg index ba935dd6c4be4..b3caf10cba3e1 100644 --- a/docs/apache-airflow/img/airflow_erd.svg +++ b/docs/apache-airflow/img/airflow_erd.svg @@ -4,2271 +4,2353 @@ <!-- Generated by graphviz version 2.43.0 (0) --> <!-- Title: %3 Pages: 1 --> -<svg width="1682pt" height="5347pt" - viewBox="0.00 0.00 1682.00 5347.00" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink"> -<g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 5343)"> +<svg width="2035pt" height="4768pt" + viewBox="0.00 0.00 2035.00 4768.00" xmlns="http://www.w3.org/2000/svg" xmlns:xlink="http://www.w3.org/1999/xlink"> +<g id="graph0" class="graph" transform="scale(1 1) rotate(0) translate(4 4764)"> <title>%3</title> -<polygon fill="white" stroke="transparent" points="-4,4 -4,-5343 1678,-5343 1678,4 -4,4"/> +<polygon fill="white" stroke="transparent" points="-4,4 -4,-4764 2031,-4764 2031,4 -4,4"/> <!-- log --> <g id="node1" class="node"> <title>log</title> -<polygon fill="none" stroke="black" points="41.5,-304 41.5,-332 322.5,-332 322.5,-304 41.5,-304"/> -<text text-anchor="start" x="168" y="-315.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">log</text> -<polygon fill="none" stroke="black" points="41.5,-279 41.5,-304 322.5,-304 322.5,-279 41.5,-279"/> -<text text-anchor="start" x="46.5" y="-288.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="59.5" y="-288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="64.5" y="-288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="141.5" y="-288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="41.5,-254 41.5,-279 322.5,-279 322.5,-254 41.5,-254"/> -<text text-anchor="start" x="46.5" y="-263.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> -<text text-anchor="start" x="92.5" y="-263.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="97.5" y="-263.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="41.5,-229 41.5,-254 322.5,-254 322.5,-229 41.5,-229"/> -<text text-anchor="start" x="46.5" y="-238.8" font-family="Helvetica,sans-Serif" font-size="14.00">dttm</text> -<text text-anchor="start" x="80.5" y="-238.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="85.5" y="-238.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="41.5,-204 41.5,-229 322.5,-229 322.5,-204 41.5,-204"/> -<text text-anchor="start" x="46.5" y="-213.8" font-family="Helvetica,sans-Serif" font-size="14.00">event</text> -<text text-anchor="start" x="86.5" y="-213.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="91.5" y="-213.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(60)]</text> -<polygon fill="none" stroke="black" points="41.5,-179 41.5,-204 322.5,-204 322.5,-179 41.5,-179"/> -<text text-anchor="start" x="46.5" y="-188.8" font-family="Helvetica,sans-Serif" font-size="14.00">execution_date</text> -<text text-anchor="start" x="151.5" y="-188.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="156.5" y="-188.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="41.5,-154 41.5,-179 322.5,-179 322.5,-154 41.5,-154"/> -<text text-anchor="start" x="46.5" y="-163.8" font-family="Helvetica,sans-Serif" font-size="14.00">extra</text> -<text text-anchor="start" x="83.5" y="-163.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="88.5" y="-163.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<polygon fill="none" stroke="black" points="41.5,-129 41.5,-154 322.5,-154 322.5,-129 41.5,-129"/> -<text text-anchor="start" x="46.5" y="-138.8" font-family="Helvetica,sans-Serif" font-size="14.00">map_index</text> -<text text-anchor="start" x="122.5" y="-138.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="127.5" y="-138.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="41.5,-104 41.5,-129 322.5,-129 322.5,-104 41.5,-104"/> -<text text-anchor="start" x="46.5" y="-113.8" font-family="Helvetica,sans-Serif" font-size="14.00">owner</text> -<text text-anchor="start" x="89.5" y="-113.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="94.5" y="-113.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> -<polygon fill="none" stroke="black" points="41.5,-79 41.5,-104 322.5,-104 322.5,-79 41.5,-79"/> -<text text-anchor="start" x="46.5" y="-88.8" font-family="Helvetica,sans-Serif" font-size="14.00">owner_display_name</text> -<text text-anchor="start" x="191.5" y="-88.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="196.5" y="-88.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> -<polygon fill="none" stroke="black" points="41.5,-54 41.5,-79 322.5,-79 322.5,-54 41.5,-54"/> -<text text-anchor="start" x="46.5" y="-63.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> -<text text-anchor="start" x="90.5" y="-63.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="95.5" y="-63.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="41.5,-29 41.5,-54 322.5,-54 322.5,-29 41.5,-29"/> -<text text-anchor="start" x="46.5" y="-38.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_id</text> -<text text-anchor="start" x="95.5" y="-38.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="100.5" y="-38.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="41.5,-4 41.5,-29 322.5,-29 322.5,-4 41.5,-4"/> -<text text-anchor="start" x="46.5" y="-13.8" font-family="Helvetica,sans-Serif" font-size="14.00">try_number</text> -<text text-anchor="start" x="128.5" y="-13.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="133.5" y="-13.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="70,-304 70,-332 351,-332 351,-304 70,-304"/> +<text text-anchor="start" x="196.5" y="-315.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">log</text> +<polygon fill="none" stroke="black" points="70,-279 70,-304 351,-304 351,-279 70,-279"/> +<text text-anchor="start" x="75" y="-288.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="88" y="-288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="93" y="-288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="170" y="-288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="70,-254 70,-279 351,-279 351,-254 70,-254"/> +<text text-anchor="start" x="75" y="-263.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> +<text text-anchor="start" x="121" y="-263.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="126" y="-263.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="70,-229 70,-254 351,-254 351,-229 70,-229"/> +<text text-anchor="start" x="75" y="-238.8" font-family="Helvetica,sans-Serif" font-size="14.00">dttm</text> +<text text-anchor="start" x="109" y="-238.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="114" y="-238.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="70,-204 70,-229 351,-229 351,-204 70,-204"/> +<text text-anchor="start" x="75" y="-213.8" font-family="Helvetica,sans-Serif" font-size="14.00">event</text> +<text text-anchor="start" x="115" y="-213.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="120" y="-213.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(60)]</text> +<polygon fill="none" stroke="black" points="70,-179 70,-204 351,-204 351,-179 70,-179"/> +<text text-anchor="start" x="75" y="-188.8" font-family="Helvetica,sans-Serif" font-size="14.00">execution_date</text> +<text text-anchor="start" x="180" y="-188.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="185" y="-188.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="70,-154 70,-179 351,-179 351,-154 70,-154"/> +<text text-anchor="start" x="75" y="-163.8" font-family="Helvetica,sans-Serif" font-size="14.00">extra</text> +<text text-anchor="start" x="112" y="-163.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="117" y="-163.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="70,-129 70,-154 351,-154 351,-129 70,-129"/> +<text text-anchor="start" x="75" y="-138.8" font-family="Helvetica,sans-Serif" font-size="14.00">map_index</text> +<text text-anchor="start" x="151" y="-138.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="156" y="-138.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="70,-104 70,-129 351,-129 351,-104 70,-104"/> +<text text-anchor="start" x="75" y="-113.8" font-family="Helvetica,sans-Serif" font-size="14.00">owner</text> +<text text-anchor="start" x="118" y="-113.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="123" y="-113.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> +<polygon fill="none" stroke="black" points="70,-79 70,-104 351,-104 351,-79 70,-79"/> +<text text-anchor="start" x="75" y="-88.8" font-family="Helvetica,sans-Serif" font-size="14.00">owner_display_name</text> +<text text-anchor="start" x="220" y="-88.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="225" y="-88.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> +<polygon fill="none" stroke="black" points="70,-54 70,-79 351,-79 351,-54 70,-54"/> +<text text-anchor="start" x="75" y="-63.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> +<text text-anchor="start" x="119" y="-63.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="124" y="-63.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="70,-29 70,-54 351,-54 351,-29 70,-29"/> +<text text-anchor="start" x="75" y="-38.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_id</text> +<text text-anchor="start" x="124" y="-38.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="129" y="-38.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="70,-4 70,-29 351,-29 351,-4 70,-4"/> +<text text-anchor="start" x="75" y="-13.8" font-family="Helvetica,sans-Serif" font-size="14.00">try_number</text> +<text text-anchor="start" x="157" y="-13.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="162" y="-13.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> </g> <!-- slot_pool --> <g id="node2" class="node"> <title>slot_pool</title> -<polygon fill="none" stroke="black" points="36.5,-483 36.5,-511 327.5,-511 327.5,-483 36.5,-483"/> -<text text-anchor="start" x="141" y="-494.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">slot_pool</text> -<polygon fill="none" stroke="black" points="36.5,-458 36.5,-483 327.5,-483 327.5,-458 36.5,-458"/> -<text text-anchor="start" x="41.5" y="-467.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="54.5" y="-467.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="59.5" y="-467.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="136.5" y="-467.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="36.5,-433 36.5,-458 327.5,-458 327.5,-433 36.5,-433"/> -<text text-anchor="start" x="41.5" y="-442.8" font-family="Helvetica,sans-Serif" font-size="14.00">description</text> -<text text-anchor="start" x="119.5" y="-442.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="124.5" y="-442.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<polygon fill="none" stroke="black" points="36.5,-408 36.5,-433 327.5,-433 327.5,-408 36.5,-408"/> -<text text-anchor="start" x="41.5" y="-417.8" font-family="Helvetica,sans-Serif" font-size="14.00">include_deferred</text> -<text text-anchor="start" x="157.5" y="-417.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="162.5" y="-417.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<text text-anchor="start" x="246.5" y="-417.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="36.5,-383 36.5,-408 327.5,-408 327.5,-383 36.5,-383"/> -<text text-anchor="start" x="41.5" y="-392.8" font-family="Helvetica,sans-Serif" font-size="14.00">pool</text> -<text text-anchor="start" x="71.5" y="-392.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="76.5" y="-392.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<polygon fill="none" stroke="black" points="36.5,-358 36.5,-383 327.5,-383 327.5,-358 36.5,-358"/> -<text text-anchor="start" x="41.5" y="-367.8" font-family="Helvetica,sans-Serif" font-size="14.00">slots</text> -<text text-anchor="start" x="74.5" y="-367.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="79.5" y="-367.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="65,-483 65,-511 356,-511 356,-483 65,-483"/> +<text text-anchor="start" x="169.5" y="-494.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">slot_pool</text> +<polygon fill="none" stroke="black" points="65,-458 65,-483 356,-483 356,-458 65,-458"/> +<text text-anchor="start" x="70" y="-467.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="83" y="-467.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="88" y="-467.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="165" y="-467.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="65,-433 65,-458 356,-458 356,-433 65,-433"/> +<text text-anchor="start" x="70" y="-442.8" font-family="Helvetica,sans-Serif" font-size="14.00">description</text> +<text text-anchor="start" x="148" y="-442.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="153" y="-442.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="65,-408 65,-433 356,-433 356,-408 65,-408"/> +<text text-anchor="start" x="70" y="-417.8" font-family="Helvetica,sans-Serif" font-size="14.00">include_deferred</text> +<text text-anchor="start" x="186" y="-417.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="191" y="-417.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<text text-anchor="start" x="275" y="-417.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="65,-383 65,-408 356,-408 356,-383 65,-383"/> +<text text-anchor="start" x="70" y="-392.8" font-family="Helvetica,sans-Serif" font-size="14.00">pool</text> +<text text-anchor="start" x="100" y="-392.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="105" y="-392.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<polygon fill="none" stroke="black" points="65,-358 65,-383 356,-383 356,-358 65,-358"/> +<text text-anchor="start" x="70" y="-367.8" font-family="Helvetica,sans-Serif" font-size="14.00">slots</text> +<text text-anchor="start" x="103" y="-367.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="108" y="-367.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> </g> <!-- callback_request --> <g id="node3" class="node"> <title>callback_request</title> -<polygon fill="none" stroke="black" points="33.5,-688 33.5,-716 330.5,-716 330.5,-688 33.5,-688"/> -<text text-anchor="start" x="106.5" y="-699.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">callback_request</text> -<polygon fill="none" stroke="black" points="33.5,-663 33.5,-688 330.5,-688 330.5,-663 33.5,-663"/> -<text text-anchor="start" x="38.5" y="-672.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="51.5" y="-672.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="56.5" y="-672.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="133.5" y="-672.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="33.5,-638 33.5,-663 330.5,-663 330.5,-638 33.5,-638"/> -<text text-anchor="start" x="38.5" y="-647.8" font-family="Helvetica,sans-Serif" font-size="14.00">callback_data</text> -<text text-anchor="start" x="132.5" y="-647.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="137.5" y="-647.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<text text-anchor="start" x="188.5" y="-647.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="33.5,-613 33.5,-638 330.5,-638 330.5,-613 33.5,-613"/> -<text text-anchor="start" x="38.5" y="-622.8" font-family="Helvetica,sans-Serif" font-size="14.00">callback_type</text> -<text text-anchor="start" x="132.5" y="-622.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="137.5" y="-622.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(20)]</text> -<text text-anchor="start" x="249.5" y="-622.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="33.5,-588 33.5,-613 330.5,-613 330.5,-588 33.5,-588"/> -<text text-anchor="start" x="38.5" y="-597.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> -<text text-anchor="start" x="111.5" y="-597.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="116.5" y="-597.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="212.5" y="-597.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="33.5,-563 33.5,-588 330.5,-588 330.5,-563 33.5,-563"/> -<text text-anchor="start" x="38.5" y="-572.8" font-family="Helvetica,sans-Serif" font-size="14.00">priority_weight</text> -<text text-anchor="start" x="142.5" y="-572.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="147.5" y="-572.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="224.5" y="-572.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="33.5,-538 33.5,-563 330.5,-563 330.5,-538 33.5,-538"/> -<text text-anchor="start" x="38.5" y="-547.8" font-family="Helvetica,sans-Serif" font-size="14.00">processor_subdir</text> -<text text-anchor="start" x="157.5" y="-547.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="162.5" y="-547.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<polygon fill="none" stroke="black" points="62,-688 62,-716 359,-716 359,-688 62,-688"/> +<text text-anchor="start" x="135" y="-699.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">callback_request</text> +<polygon fill="none" stroke="black" points="62,-663 62,-688 359,-688 359,-663 62,-663"/> +<text text-anchor="start" x="67" y="-672.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="80" y="-672.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="85" y="-672.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="162" y="-672.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="62,-638 62,-663 359,-663 359,-638 62,-638"/> +<text text-anchor="start" x="67" y="-647.8" font-family="Helvetica,sans-Serif" font-size="14.00">callback_data</text> +<text text-anchor="start" x="161" y="-647.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="166" y="-647.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<text text-anchor="start" x="217" y="-647.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="62,-613 62,-638 359,-638 359,-613 62,-613"/> +<text text-anchor="start" x="67" y="-622.8" font-family="Helvetica,sans-Serif" font-size="14.00">callback_type</text> +<text text-anchor="start" x="161" y="-622.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="166" y="-622.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(20)]</text> +<text text-anchor="start" x="278" y="-622.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="62,-588 62,-613 359,-613 359,-588 62,-588"/> +<text text-anchor="start" x="67" y="-597.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="140" y="-597.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="145" y="-597.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="241" y="-597.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="62,-563 62,-588 359,-588 359,-563 62,-563"/> +<text text-anchor="start" x="67" y="-572.8" font-family="Helvetica,sans-Serif" font-size="14.00">priority_weight</text> +<text text-anchor="start" x="171" y="-572.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="176" y="-572.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="253" y="-572.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="62,-538 62,-563 359,-563 359,-538 62,-538"/> +<text text-anchor="start" x="67" y="-547.8" font-family="Helvetica,sans-Serif" font-size="14.00">processor_subdir</text> +<text text-anchor="start" x="186" y="-547.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="191" y="-547.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> </g> <!-- dag_priority_parsing_request --> <g id="node4" class="node"> <title>dag_priority_parsing_request</title> -<polygon fill="none" stroke="black" points="45.5,-792 45.5,-820 317.5,-820 317.5,-792 45.5,-792"/> -<text text-anchor="start" x="50.5" y="-803.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_priority_parsing_request</text> -<polygon fill="none" stroke="black" points="45.5,-767 45.5,-792 317.5,-792 317.5,-767 45.5,-767"/> -<text text-anchor="start" x="50.5" y="-776.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="63.5" y="-776.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="68.5" y="-776.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(32)]</text> -<text text-anchor="start" x="180.5" y="-776.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="45.5,-742 45.5,-767 317.5,-767 317.5,-742 45.5,-742"/> -<text text-anchor="start" x="50.5" y="-751.8" font-family="Helvetica,sans-Serif" font-size="14.00">fileloc</text> -<text text-anchor="start" x="91.5" y="-751.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="96.5" y="-751.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> -<text text-anchor="start" x="226.5" y="-751.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -</g> -<!-- dag_code --> -<g id="node5" class="node"> -<title>dag_code</title> -<polygon fill="none" stroke="black" points="43.5,-946 43.5,-974 320.5,-974 320.5,-946 43.5,-946"/> -<text text-anchor="start" x="139.5" y="-957.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_code</text> -<polygon fill="none" stroke="black" points="43.5,-921 43.5,-946 320.5,-946 320.5,-921 43.5,-921"/> -<text text-anchor="start" x="48.5" y="-930.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">fileloc_hash</text> -<text text-anchor="start" x="130.5" y="-930.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="135.5" y="-930.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BIGINT]</text> -<text text-anchor="start" x="199.5" y="-930.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="43.5,-896 43.5,-921 320.5,-921 320.5,-896 43.5,-896"/> -<text text-anchor="start" x="48.5" y="-905.8" font-family="Helvetica,sans-Serif" font-size="14.00">fileloc</text> -<text text-anchor="start" x="89.5" y="-905.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="94.5" y="-905.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> -<text text-anchor="start" x="224.5" y="-905.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="43.5,-871 43.5,-896 320.5,-896 320.5,-871 43.5,-871"/> -<text text-anchor="start" x="48.5" y="-880.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_updated</text> -<text text-anchor="start" x="138.5" y="-880.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="143.5" y="-880.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="239.5" y="-880.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="43.5,-846 43.5,-871 320.5,-871 320.5,-846 43.5,-846"/> -<text text-anchor="start" x="48.5" y="-855.8" font-family="Helvetica,sans-Serif" font-size="14.00">source_code</text> -<text text-anchor="start" x="134.5" y="-855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="139.5" y="-855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<text text-anchor="start" x="189.5" y="-855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="74,-792 74,-820 346,-820 346,-792 74,-792"/> +<text text-anchor="start" x="79" y="-803.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_priority_parsing_request</text> +<polygon fill="none" stroke="black" points="74,-767 74,-792 346,-792 346,-767 74,-767"/> +<text text-anchor="start" x="79" y="-776.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="92" y="-776.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="97" y="-776.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(32)]</text> +<text text-anchor="start" x="209" y="-776.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="74,-742 74,-767 346,-767 346,-742 74,-742"/> +<text text-anchor="start" x="79" y="-751.8" font-family="Helvetica,sans-Serif" font-size="14.00">fileloc</text> +<text text-anchor="start" x="120" y="-751.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="125" y="-751.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<text text-anchor="start" x="255" y="-751.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- connection --> -<g id="node6" class="node"> +<g id="node5" class="node"> <title>connection</title> -<polygon fill="none" stroke="black" points="39.5,-1300 39.5,-1328 323.5,-1328 323.5,-1300 39.5,-1300"/> -<text text-anchor="start" x="131.5" y="-1311.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">connection</text> -<polygon fill="none" stroke="black" points="39.5,-1275 39.5,-1300 323.5,-1300 323.5,-1275 39.5,-1275"/> -<text text-anchor="start" x="44.5" y="-1284.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="57.5" y="-1284.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="62.5" y="-1284.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="139.5" y="-1284.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="39.5,-1250 39.5,-1275 323.5,-1275 323.5,-1250 39.5,-1250"/> -<text text-anchor="start" x="44.5" y="-1259.8" font-family="Helvetica,sans-Serif" font-size="14.00">conn_id</text> -<text text-anchor="start" x="98.5" y="-1259.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="103.5" y="-1259.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="224.5" y="-1259.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="39.5,-1225 39.5,-1250 323.5,-1250 323.5,-1225 39.5,-1225"/> -<text text-anchor="start" x="44.5" y="-1234.8" font-family="Helvetica,sans-Serif" font-size="14.00">conn_type</text> -<text text-anchor="start" x="116.5" y="-1234.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="121.5" y="-1234.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> -<text text-anchor="start" x="242.5" y="-1234.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="39.5,-1200 39.5,-1225 323.5,-1225 323.5,-1200 39.5,-1200"/> -<text text-anchor="start" x="44.5" y="-1209.8" font-family="Helvetica,sans-Serif" font-size="14.00">description</text> -<text text-anchor="start" x="122.5" y="-1209.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="127.5" y="-1209.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<polygon fill="none" stroke="black" points="39.5,-1175 39.5,-1200 323.5,-1200 323.5,-1175 39.5,-1175"/> -<text text-anchor="start" x="44.5" y="-1184.8" font-family="Helvetica,sans-Serif" font-size="14.00">extra</text> -<text text-anchor="start" x="81.5" y="-1184.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="86.5" y="-1184.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<polygon fill="none" stroke="black" points="39.5,-1150 39.5,-1175 323.5,-1175 323.5,-1150 39.5,-1150"/> -<text text-anchor="start" x="44.5" y="-1159.8" font-family="Helvetica,sans-Serif" font-size="14.00">host</text> -<text text-anchor="start" x="75.5" y="-1159.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="80.5" y="-1159.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> -<polygon fill="none" stroke="black" points="39.5,-1125 39.5,-1150 323.5,-1150 323.5,-1125 39.5,-1125"/> -<text text-anchor="start" x="44.5" y="-1134.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_encrypted</text> -<text text-anchor="start" x="133.5" y="-1134.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="138.5" y="-1134.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<polygon fill="none" stroke="black" points="39.5,-1100 39.5,-1125 323.5,-1125 323.5,-1100 39.5,-1100"/> -<text text-anchor="start" x="44.5" y="-1109.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_extra_encrypted</text> -<text text-anchor="start" x="176.5" y="-1109.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="181.5" y="-1109.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<polygon fill="none" stroke="black" points="39.5,-1075 39.5,-1100 323.5,-1100 323.5,-1075 39.5,-1075"/> -<text text-anchor="start" x="44.5" y="-1084.8" font-family="Helvetica,sans-Serif" font-size="14.00">login</text> -<text text-anchor="start" x="78.5" y="-1084.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="83.5" y="-1084.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<polygon fill="none" stroke="black" points="39.5,-1050 39.5,-1075 323.5,-1075 323.5,-1050 39.5,-1050"/> -<text text-anchor="start" x="44.5" y="-1059.8" font-family="Helvetica,sans-Serif" font-size="14.00">password</text> -<text text-anchor="start" x="111.5" y="-1059.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="116.5" y="-1059.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<polygon fill="none" stroke="black" points="39.5,-1025 39.5,-1050 323.5,-1050 323.5,-1025 39.5,-1025"/> -<text text-anchor="start" x="44.5" y="-1034.8" font-family="Helvetica,sans-Serif" font-size="14.00">port</text> -<text text-anchor="start" x="73.5" y="-1034.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="78.5" y="-1034.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="39.5,-1000 39.5,-1025 323.5,-1025 323.5,-1000 39.5,-1000"/> -<text text-anchor="start" x="44.5" y="-1009.8" font-family="Helvetica,sans-Serif" font-size="14.00">schema</text> -<text text-anchor="start" x="99.5" y="-1009.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="104.5" y="-1009.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> +<polygon fill="none" stroke="black" points="68,-1146 68,-1174 352,-1174 352,-1146 68,-1146"/> +<text text-anchor="start" x="160" y="-1157.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">connection</text> +<polygon fill="none" stroke="black" points="68,-1121 68,-1146 352,-1146 352,-1121 68,-1121"/> +<text text-anchor="start" x="73" y="-1130.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="86" y="-1130.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="91" y="-1130.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="168" y="-1130.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="68,-1096 68,-1121 352,-1121 352,-1096 68,-1096"/> +<text text-anchor="start" x="73" y="-1105.8" font-family="Helvetica,sans-Serif" font-size="14.00">conn_id</text> +<text text-anchor="start" x="127" y="-1105.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="132" y="-1105.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="253" y="-1105.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="68,-1071 68,-1096 352,-1096 352,-1071 68,-1071"/> +<text text-anchor="start" x="73" y="-1080.8" font-family="Helvetica,sans-Serif" font-size="14.00">conn_type</text> +<text text-anchor="start" x="145" y="-1080.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="150" y="-1080.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> +<text text-anchor="start" x="271" y="-1080.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="68,-1046 68,-1071 352,-1071 352,-1046 68,-1046"/> +<text text-anchor="start" x="73" y="-1055.8" font-family="Helvetica,sans-Serif" font-size="14.00">description</text> +<text text-anchor="start" x="151" y="-1055.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="156" y="-1055.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="68,-1021 68,-1046 352,-1046 352,-1021 68,-1021"/> +<text text-anchor="start" x="73" y="-1030.8" font-family="Helvetica,sans-Serif" font-size="14.00">extra</text> +<text text-anchor="start" x="110" y="-1030.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="115" y="-1030.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="68,-996 68,-1021 352,-1021 352,-996 68,-996"/> +<text text-anchor="start" x="73" y="-1005.8" font-family="Helvetica,sans-Serif" font-size="14.00">host</text> +<text text-anchor="start" x="104" y="-1005.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="109" y="-1005.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> +<polygon fill="none" stroke="black" points="68,-971 68,-996 352,-996 352,-971 68,-971"/> +<text text-anchor="start" x="73" y="-980.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_encrypted</text> +<text text-anchor="start" x="162" y="-980.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="167" y="-980.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<polygon fill="none" stroke="black" points="68,-946 68,-971 352,-971 352,-946 68,-946"/> +<text text-anchor="start" x="73" y="-955.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_extra_encrypted</text> +<text text-anchor="start" x="205" y="-955.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="210" y="-955.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<polygon fill="none" stroke="black" points="68,-921 68,-946 352,-946 352,-921 68,-921"/> +<text text-anchor="start" x="73" y="-930.8" font-family="Helvetica,sans-Serif" font-size="14.00">login</text> +<text text-anchor="start" x="107" y="-930.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="112" y="-930.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="68,-896 68,-921 352,-921 352,-896 68,-896"/> +<text text-anchor="start" x="73" y="-905.8" font-family="Helvetica,sans-Serif" font-size="14.00">password</text> +<text text-anchor="start" x="140" y="-905.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="145" y="-905.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="68,-871 68,-896 352,-896 352,-871 68,-871"/> +<text text-anchor="start" x="73" y="-880.8" font-family="Helvetica,sans-Serif" font-size="14.00">port</text> +<text text-anchor="start" x="102" y="-880.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="107" y="-880.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="68,-846 68,-871 352,-871 352,-846 68,-846"/> +<text text-anchor="start" x="73" y="-855.8" font-family="Helvetica,sans-Serif" font-size="14.00">schema</text> +<text text-anchor="start" x="128" y="-855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="133" y="-855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> </g> <!-- variable --> -<g id="node7" class="node"> +<g id="node6" class="node"> <title>variable</title> -<polygon fill="none" stroke="black" points="87.5,-1479 87.5,-1507 275.5,-1507 275.5,-1479 87.5,-1479"/> -<text text-anchor="start" x="145" y="-1490.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">variable</text> -<polygon fill="none" stroke="black" points="87.5,-1454 87.5,-1479 275.5,-1479 275.5,-1454 87.5,-1454"/> -<text text-anchor="start" x="92.5" y="-1463.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="105.5" y="-1463.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="110.5" y="-1463.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="187.5" y="-1463.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="87.5,-1429 87.5,-1454 275.5,-1454 275.5,-1429 87.5,-1429"/> -<text text-anchor="start" x="92.5" y="-1438.8" font-family="Helvetica,sans-Serif" font-size="14.00">description</text> -<text text-anchor="start" x="170.5" y="-1438.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="175.5" y="-1438.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<polygon fill="none" stroke="black" points="87.5,-1404 87.5,-1429 275.5,-1429 275.5,-1404 87.5,-1404"/> -<text text-anchor="start" x="92.5" y="-1413.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_encrypted</text> -<text text-anchor="start" x="181.5" y="-1413.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="186.5" y="-1413.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<polygon fill="none" stroke="black" points="87.5,-1379 87.5,-1404 275.5,-1404 275.5,-1379 87.5,-1379"/> -<text text-anchor="start" x="92.5" y="-1388.8" font-family="Helvetica,sans-Serif" font-size="14.00">key</text> -<text text-anchor="start" x="117.5" y="-1388.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="122.5" y="-1388.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="87.5,-1354 87.5,-1379 275.5,-1379 275.5,-1354 87.5,-1354"/> -<text text-anchor="start" x="92.5" y="-1363.8" font-family="Helvetica,sans-Serif" font-size="14.00">val</text> -<text text-anchor="start" x="113.5" y="-1363.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="118.5" y="-1363.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="116,-1325 116,-1353 304,-1353 304,-1325 116,-1325"/> +<text text-anchor="start" x="173.5" y="-1336.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">variable</text> +<polygon fill="none" stroke="black" points="116,-1300 116,-1325 304,-1325 304,-1300 116,-1300"/> +<text text-anchor="start" x="121" y="-1309.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="134" y="-1309.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="139" y="-1309.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="216" y="-1309.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="116,-1275 116,-1300 304,-1300 304,-1275 116,-1275"/> +<text text-anchor="start" x="121" y="-1284.8" font-family="Helvetica,sans-Serif" font-size="14.00">description</text> +<text text-anchor="start" x="199" y="-1284.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="204" y="-1284.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="116,-1250 116,-1275 304,-1275 304,-1250 116,-1250"/> +<text text-anchor="start" x="121" y="-1259.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_encrypted</text> +<text text-anchor="start" x="210" y="-1259.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="215" y="-1259.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<polygon fill="none" stroke="black" points="116,-1225 116,-1250 304,-1250 304,-1225 116,-1225"/> +<text text-anchor="start" x="121" y="-1234.8" font-family="Helvetica,sans-Serif" font-size="14.00">key</text> +<text text-anchor="start" x="146" y="-1234.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="151" y="-1234.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="116,-1200 116,-1225 304,-1225 304,-1200 116,-1200"/> +<text text-anchor="start" x="121" y="-1209.8" font-family="Helvetica,sans-Serif" font-size="14.00">val</text> +<text text-anchor="start" x="142" y="-1209.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="147" y="-1209.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> </g> <!-- import_error --> -<g id="node8" class="node"> +<g id="node7" class="node"> <title>import_error</title> -<polygon fill="none" stroke="black" points="49.5,-1658 49.5,-1686 313.5,-1686 313.5,-1658 49.5,-1658"/> -<text text-anchor="start" x="124" y="-1669.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">import_error</text> -<polygon fill="none" stroke="black" points="49.5,-1633 49.5,-1658 313.5,-1658 313.5,-1633 49.5,-1633"/> -<text text-anchor="start" x="54.5" y="-1642.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="67.5" y="-1642.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="72.5" y="-1642.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="149.5" y="-1642.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="49.5,-1608 49.5,-1633 313.5,-1633 313.5,-1608 49.5,-1608"/> -<text text-anchor="start" x="54.5" y="-1617.8" font-family="Helvetica,sans-Serif" font-size="14.00">filename</text> -<text text-anchor="start" x="115.5" y="-1617.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="120.5" y="-1617.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1024)]</text> -<polygon fill="none" stroke="black" points="49.5,-1583 49.5,-1608 313.5,-1608 313.5,-1583 49.5,-1583"/> -<text text-anchor="start" x="54.5" y="-1592.8" font-family="Helvetica,sans-Serif" font-size="14.00">processor_subdir</text> -<text text-anchor="start" x="173.5" y="-1592.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="178.5" y="-1592.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> -<polygon fill="none" stroke="black" points="49.5,-1558 49.5,-1583 313.5,-1583 313.5,-1558 49.5,-1558"/> -<text text-anchor="start" x="54.5" y="-1567.8" font-family="Helvetica,sans-Serif" font-size="14.00">stacktrace</text> -<text text-anchor="start" x="127.5" y="-1567.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="132.5" y="-1567.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<polygon fill="none" stroke="black" points="49.5,-1533 49.5,-1558 313.5,-1558 313.5,-1533 49.5,-1533"/> -<text text-anchor="start" x="54.5" y="-1542.8" font-family="Helvetica,sans-Serif" font-size="14.00">timestamp</text> -<text text-anchor="start" x="129.5" y="-1542.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="134.5" y="-1542.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="78,-1504 78,-1532 342,-1532 342,-1504 78,-1504"/> +<text text-anchor="start" x="152.5" y="-1515.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">import_error</text> +<polygon fill="none" stroke="black" points="78,-1479 78,-1504 342,-1504 342,-1479 78,-1479"/> +<text text-anchor="start" x="83" y="-1488.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="96" y="-1488.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="101" y="-1488.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="178" y="-1488.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="78,-1454 78,-1479 342,-1479 342,-1454 78,-1454"/> +<text text-anchor="start" x="83" y="-1463.8" font-family="Helvetica,sans-Serif" font-size="14.00">filename</text> +<text text-anchor="start" x="144" y="-1463.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="149" y="-1463.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1024)]</text> +<polygon fill="none" stroke="black" points="78,-1429 78,-1454 342,-1454 342,-1429 78,-1429"/> +<text text-anchor="start" x="83" y="-1438.8" font-family="Helvetica,sans-Serif" font-size="14.00">processor_subdir</text> +<text text-anchor="start" x="202" y="-1438.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="207" y="-1438.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<polygon fill="none" stroke="black" points="78,-1404 78,-1429 342,-1429 342,-1404 78,-1404"/> +<text text-anchor="start" x="83" y="-1413.8" font-family="Helvetica,sans-Serif" font-size="14.00">stacktrace</text> +<text text-anchor="start" x="156" y="-1413.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="161" y="-1413.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="78,-1379 78,-1404 342,-1404 342,-1379 78,-1379"/> +<text text-anchor="start" x="83" y="-1388.8" font-family="Helvetica,sans-Serif" font-size="14.00">timestamp</text> +<text text-anchor="start" x="158" y="-1388.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="163" y="-1388.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> </g> <!-- job --> -<g id="node9" class="node"> +<g id="node8" class="node"> <title>job</title> -<polygon fill="none" stroke="black" points="62.5,-1963 62.5,-1991 300.5,-1991 300.5,-1963 62.5,-1963"/> -<text text-anchor="start" x="167.5" y="-1974.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">job</text> -<polygon fill="none" stroke="black" points="62.5,-1938 62.5,-1963 300.5,-1963 300.5,-1938 62.5,-1938"/> -<text text-anchor="start" x="67.5" y="-1947.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="80.5" y="-1947.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="85.5" y="-1947.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="162.5" y="-1947.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="62.5,-1913 62.5,-1938 300.5,-1938 300.5,-1913 62.5,-1913"/> -<text text-anchor="start" x="67.5" y="-1922.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> -<text text-anchor="start" x="113.5" y="-1922.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="118.5" y="-1922.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="62.5,-1888 62.5,-1913 300.5,-1913 300.5,-1888 62.5,-1888"/> -<text text-anchor="start" x="67.5" y="-1897.8" font-family="Helvetica,sans-Serif" font-size="14.00">end_date</text> -<text text-anchor="start" x="131.5" y="-1897.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="136.5" y="-1897.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="62.5,-1863 62.5,-1888 300.5,-1888 300.5,-1863 62.5,-1863"/> -<text text-anchor="start" x="67.5" y="-1872.8" font-family="Helvetica,sans-Serif" font-size="14.00">executor_class</text> -<text text-anchor="start" x="169.5" y="-1872.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="174.5" y="-1872.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> -<polygon fill="none" stroke="black" points="62.5,-1838 62.5,-1863 300.5,-1863 300.5,-1838 62.5,-1838"/> -<text text-anchor="start" x="67.5" y="-1847.8" font-family="Helvetica,sans-Serif" font-size="14.00">hostname</text> -<text text-anchor="start" x="137.5" y="-1847.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="142.5" y="-1847.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> -<polygon fill="none" stroke="black" points="62.5,-1813 62.5,-1838 300.5,-1838 300.5,-1813 62.5,-1813"/> -<text text-anchor="start" x="67.5" y="-1822.8" font-family="Helvetica,sans-Serif" font-size="14.00">job_type</text> -<text text-anchor="start" x="126.5" y="-1822.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="131.5" y="-1822.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(30)]</text> -<polygon fill="none" stroke="black" points="62.5,-1788 62.5,-1813 300.5,-1813 300.5,-1788 62.5,-1788"/> -<text text-anchor="start" x="67.5" y="-1797.8" font-family="Helvetica,sans-Serif" font-size="14.00">latest_heartbeat</text> -<text text-anchor="start" x="180.5" y="-1797.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="185.5" y="-1797.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="62.5,-1763 62.5,-1788 300.5,-1788 300.5,-1763 62.5,-1763"/> -<text text-anchor="start" x="67.5" y="-1772.8" font-family="Helvetica,sans-Serif" font-size="14.00">start_date</text> -<text text-anchor="start" x="137.5" y="-1772.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="142.5" y="-1772.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="62.5,-1738 62.5,-1763 300.5,-1763 300.5,-1738 62.5,-1738"/> -<text text-anchor="start" x="67.5" y="-1747.8" font-family="Helvetica,sans-Serif" font-size="14.00">state</text> -<text text-anchor="start" x="102.5" y="-1747.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="107.5" y="-1747.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(20)]</text> -<polygon fill="none" stroke="black" points="62.5,-1713 62.5,-1738 300.5,-1738 300.5,-1713 62.5,-1713"/> -<text text-anchor="start" x="67.5" y="-1722.8" font-family="Helvetica,sans-Serif" font-size="14.00">unixname</text> -<text text-anchor="start" x="137.5" y="-1722.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="142.5" y="-1722.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -</g> -<!-- serialized_dag --> -<g id="node10" class="node"> -<title>serialized_dag</title> -<polygon fill="none" stroke="black" points="43.5,-2217 43.5,-2245 320.5,-2245 320.5,-2217 43.5,-2217"/> -<text text-anchor="start" x="118.5" y="-2228.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">serialized_dag</text> -<polygon fill="none" stroke="black" points="43.5,-2192 43.5,-2217 320.5,-2217 320.5,-2192 43.5,-2192"/> -<text text-anchor="start" x="48.5" y="-2201.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="94.5" y="-2201.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="99.5" y="-2201.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="220.5" y="-2201.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="43.5,-2167 43.5,-2192 320.5,-2192 320.5,-2167 43.5,-2167"/> -<text text-anchor="start" x="48.5" y="-2176.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_hash</text> -<text text-anchor="start" x="115.5" y="-2176.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="120.5" y="-2176.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(32)]</text> -<text text-anchor="start" x="232.5" y="-2176.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="43.5,-2142 43.5,-2167 320.5,-2167 320.5,-2142 43.5,-2142"/> -<text text-anchor="start" x="48.5" y="-2151.8" font-family="Helvetica,sans-Serif" font-size="14.00">data</text> -<text text-anchor="start" x="79.5" y="-2151.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="84.5" y="-2151.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<polygon fill="none" stroke="black" points="43.5,-2117 43.5,-2142 320.5,-2142 320.5,-2117 43.5,-2117"/> -<text text-anchor="start" x="48.5" y="-2126.8" font-family="Helvetica,sans-Serif" font-size="14.00">data_compressed</text> -<text text-anchor="start" x="170.5" y="-2126.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="175.5" y="-2126.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> -<polygon fill="none" stroke="black" points="43.5,-2092 43.5,-2117 320.5,-2117 320.5,-2092 43.5,-2092"/> -<text text-anchor="start" x="48.5" y="-2101.8" font-family="Helvetica,sans-Serif" font-size="14.00">fileloc</text> -<text text-anchor="start" x="89.5" y="-2101.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="94.5" y="-2101.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> -<text text-anchor="start" x="224.5" y="-2101.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="43.5,-2067 43.5,-2092 320.5,-2092 320.5,-2067 43.5,-2067"/> -<text text-anchor="start" x="48.5" y="-2076.8" font-family="Helvetica,sans-Serif" font-size="14.00">fileloc_hash</text> -<text text-anchor="start" x="130.5" y="-2076.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="135.5" y="-2076.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BIGINT]</text> -<text text-anchor="start" x="199.5" y="-2076.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="43.5,-2042 43.5,-2067 320.5,-2067 320.5,-2042 43.5,-2042"/> -<text text-anchor="start" x="48.5" y="-2051.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_updated</text> -<text text-anchor="start" x="138.5" y="-2051.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="143.5" y="-2051.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="239.5" y="-2051.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="43.5,-2017 43.5,-2042 320.5,-2042 320.5,-2017 43.5,-2017"/> -<text text-anchor="start" x="48.5" y="-2026.8" font-family="Helvetica,sans-Serif" font-size="14.00">processor_subdir</text> -<text text-anchor="start" x="167.5" y="-2026.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="172.5" y="-2026.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<polygon fill="none" stroke="black" points="91,-1809 91,-1837 329,-1837 329,-1809 91,-1809"/> +<text text-anchor="start" x="196" y="-1820.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">job</text> +<polygon fill="none" stroke="black" points="91,-1784 91,-1809 329,-1809 329,-1784 91,-1784"/> +<text text-anchor="start" x="96" y="-1793.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="109" y="-1793.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="114" y="-1793.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="191" y="-1793.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="91,-1759 91,-1784 329,-1784 329,-1759 91,-1759"/> +<text text-anchor="start" x="96" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> +<text text-anchor="start" x="142" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="147" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="91,-1734 91,-1759 329,-1759 329,-1734 91,-1734"/> +<text text-anchor="start" x="96" y="-1743.8" font-family="Helvetica,sans-Serif" font-size="14.00">end_date</text> +<text text-anchor="start" x="160" y="-1743.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="165" y="-1743.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="91,-1709 91,-1734 329,-1734 329,-1709 91,-1709"/> +<text text-anchor="start" x="96" y="-1718.8" font-family="Helvetica,sans-Serif" font-size="14.00">executor_class</text> +<text text-anchor="start" x="198" y="-1718.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="203" y="-1718.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> +<polygon fill="none" stroke="black" points="91,-1684 91,-1709 329,-1709 329,-1684 91,-1684"/> +<text text-anchor="start" x="96" y="-1693.8" font-family="Helvetica,sans-Serif" font-size="14.00">hostname</text> +<text text-anchor="start" x="166" y="-1693.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="171" y="-1693.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> +<polygon fill="none" stroke="black" points="91,-1659 91,-1684 329,-1684 329,-1659 91,-1659"/> +<text text-anchor="start" x="96" y="-1668.8" font-family="Helvetica,sans-Serif" font-size="14.00">job_type</text> +<text text-anchor="start" x="155" y="-1668.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="160" y="-1668.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(30)]</text> +<polygon fill="none" stroke="black" points="91,-1634 91,-1659 329,-1659 329,-1634 91,-1634"/> +<text text-anchor="start" x="96" y="-1643.8" font-family="Helvetica,sans-Serif" font-size="14.00">latest_heartbeat</text> +<text text-anchor="start" x="209" y="-1643.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="214" y="-1643.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="91,-1609 91,-1634 329,-1634 329,-1609 91,-1609"/> +<text text-anchor="start" x="96" y="-1618.8" font-family="Helvetica,sans-Serif" font-size="14.00">start_date</text> +<text text-anchor="start" x="166" y="-1618.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="171" y="-1618.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="91,-1584 91,-1609 329,-1609 329,-1584 91,-1584"/> +<text text-anchor="start" x="96" y="-1593.8" font-family="Helvetica,sans-Serif" font-size="14.00">state</text> +<text text-anchor="start" x="131" y="-1593.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="136" y="-1593.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(20)]</text> +<polygon fill="none" stroke="black" points="91,-1559 91,-1584 329,-1584 329,-1559 91,-1559"/> +<text text-anchor="start" x="96" y="-1568.8" font-family="Helvetica,sans-Serif" font-size="14.00">unixname</text> +<text text-anchor="start" x="166" y="-1568.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="171" y="-1568.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> </g> <!-- asset_alias --> -<g id="node11" class="node"> +<g id="node9" class="node"> <title>asset_alias</title> -<polygon fill="none" stroke="black" points="515,-4331 515,-4359 777,-4359 777,-4331 515,-4331"/> -<text text-anchor="start" x="597" y="-4342.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_alias</text> -<polygon fill="none" stroke="black" points="515,-4306 515,-4331 777,-4331 777,-4306 515,-4306"/> -<text text-anchor="start" x="520" y="-4315.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="533" y="-4315.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="538" y="-4315.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="615" y="-4315.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="515,-4281 515,-4306 777,-4306 777,-4281 515,-4281"/> -<text text-anchor="start" x="520" y="-4290.8" font-family="Helvetica,sans-Serif" font-size="14.00">group</text> -<text text-anchor="start" x="561" y="-4290.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="566" y="-4290.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> -<text text-anchor="start" x="696" y="-4290.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="515,-4256 515,-4281 777,-4281 777,-4256 515,-4256"/> -<text text-anchor="start" x="520" y="-4265.8" font-family="Helvetica,sans-Serif" font-size="14.00">name</text> -<text text-anchor="start" x="560" y="-4265.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="565" y="-4265.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> -<text text-anchor="start" x="695" y="-4265.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="79,-3212 79,-3240 341,-3240 341,-3212 79,-3212"/> +<text text-anchor="start" x="161" y="-3223.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_alias</text> +<polygon fill="none" stroke="black" points="79,-3187 79,-3212 341,-3212 341,-3187 79,-3187"/> +<text text-anchor="start" x="84" y="-3196.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="97" y="-3196.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="102" y="-3196.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="179" y="-3196.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="79,-3162 79,-3187 341,-3187 341,-3162 79,-3162"/> +<text text-anchor="start" x="84" y="-3171.8" font-family="Helvetica,sans-Serif" font-size="14.00">group</text> +<text text-anchor="start" x="125" y="-3171.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="130" y="-3171.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> +<text text-anchor="start" x="260" y="-3171.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="79,-3137 79,-3162 341,-3162 341,-3137 79,-3137"/> +<text text-anchor="start" x="84" y="-3146.8" font-family="Helvetica,sans-Serif" font-size="14.00">name</text> +<text text-anchor="start" x="124" y="-3146.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="129" y="-3146.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> +<text text-anchor="start" x="259" y="-3146.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset_alias_asset --> -<g id="node12" class="node"> +<g id="node10" class="node"> <title>asset_alias_asset</title> -<polygon fill="none" stroke="black" points="991,-4676 991,-4704 1216,-4704 1216,-4676 991,-4676"/> -<text text-anchor="start" x="1026.5" y="-4687.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_alias_asset</text> -<polygon fill="none" stroke="black" points="991,-4651 991,-4676 1216,-4676 1216,-4651 991,-4651"/> -<text text-anchor="start" x="996" y="-4660.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">alias_id</text> -<text text-anchor="start" x="1048" y="-4660.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1053" y="-4660.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1130" y="-4660.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="991,-4626 991,-4651 1216,-4651 1216,-4626 991,-4626"/> -<text text-anchor="start" x="996" y="-4635.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">asset_id</text> -<text text-anchor="start" x="1053" y="-4635.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1058" y="-4635.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1135" y="-4635.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="562.5,-3793 562.5,-3821 787.5,-3821 787.5,-3793 562.5,-3793"/> +<text text-anchor="start" x="598" y="-3804.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_alias_asset</text> +<polygon fill="none" stroke="black" points="562.5,-3768 562.5,-3793 787.5,-3793 787.5,-3768 562.5,-3768"/> +<text text-anchor="start" x="567.5" y="-3777.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">alias_id</text> +<text text-anchor="start" x="619.5" y="-3777.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="624.5" y="-3777.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="701.5" y="-3777.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="562.5,-3743 562.5,-3768 787.5,-3768 787.5,-3743 562.5,-3743"/> +<text text-anchor="start" x="567.5" y="-3752.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">asset_id</text> +<text text-anchor="start" x="624.5" y="-3752.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="629.5" y="-3752.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="706.5" y="-3752.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset_alias--asset_alias_asset --> <g id="edge1" class="edge"> <title>asset_alias--asset_alias_asset</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M785.19,-4351.55C811.32,-4364.77 836.53,-4381.65 856,-4403 922.58,-4476.01 857.66,-4544.63 929,-4613 943.94,-4627.32 962.68,-4637.71 982.26,-4645.25"/> -<text text-anchor="start" x="951.26" y="-4634.05" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="785.19" y="-4340.35" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M239.94,-3244.75C311.34,-3382.29 490.38,-3726.85 493,-3729 510.54,-3743.41 531.86,-3753.9 553.62,-3761.53"/> +<text text-anchor="start" x="522.62" y="-3750.33" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="229.94" y="-3248.55" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- asset_alias_asset_event --> -<g id="node13" class="node"> +<g id="node11" class="node"> <title>asset_alias_asset_event</title> -<polygon fill="none" stroke="black" points="990,-3463 990,-3491 1217,-3491 1217,-3463 990,-3463"/> -<text text-anchor="start" x="997" y="-3474.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_alias_asset_event</text> -<polygon fill="none" stroke="black" points="990,-3438 990,-3463 1217,-3463 1217,-3438 990,-3438"/> -<text text-anchor="start" x="995" y="-3447.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">alias_id</text> -<text text-anchor="start" x="1047" y="-3447.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1052" y="-3447.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1129" y="-3447.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="990,-3413 990,-3438 1217,-3438 1217,-3413 990,-3413"/> -<text text-anchor="start" x="995" y="-3422.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">event_id</text> -<text text-anchor="start" x="1054" y="-3422.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1059" y="-3422.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1136" y="-3422.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="561.5,-3071 561.5,-3099 788.5,-3099 788.5,-3071 561.5,-3071"/> +<text text-anchor="start" x="568.5" y="-3082.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_alias_asset_event</text> +<polygon fill="none" stroke="black" points="561.5,-3046 561.5,-3071 788.5,-3071 788.5,-3046 561.5,-3046"/> +<text text-anchor="start" x="566.5" y="-3055.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">alias_id</text> +<text text-anchor="start" x="618.5" y="-3055.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="623.5" y="-3055.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="700.5" y="-3055.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="561.5,-3021 561.5,-3046 788.5,-3046 788.5,-3021 561.5,-3021"/> +<text text-anchor="start" x="566.5" y="-3030.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">event_id</text> +<text text-anchor="start" x="625.5" y="-3030.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="630.5" y="-3030.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="707.5" y="-3030.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset_alias--asset_alias_asset_event --> <g id="edge2" class="edge"> <title>asset_alias--asset_alias_asset_event</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M785.15,-4289.79C812.26,-4280.01 837.87,-4265.24 856,-4243 908.13,-4179.04 876.57,-3567.71 929,-3504 942.55,-3487.53 961.11,-3476.15 981.06,-3468.3"/> -<text text-anchor="start" x="950.06" y="-3457.1" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="785.15" y="-4278.59" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M349.14,-3150.47C413.95,-3132.39 490.48,-3111.05 552.92,-3093.63"/> +<text text-anchor="start" x="521.92" y="-3082.43" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="349.14" y="-3139.27" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag_schedule_asset_alias_reference --> -<g id="node14" class="node"> +<g id="node12" class="node"> <title>dag_schedule_asset_alias_reference</title> -<polygon fill="none" stroke="black" points="937,-4314 937,-4342 1269,-4342 1269,-4314 937,-4314"/> -<text text-anchor="start" x="942" y="-4325.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_schedule_asset_alias_reference</text> -<polygon fill="none" stroke="black" points="937,-4289 937,-4314 1269,-4314 1269,-4289 937,-4289"/> -<text text-anchor="start" x="942" y="-4298.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">alias_id</text> -<text text-anchor="start" x="994" y="-4298.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="999" y="-4298.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1076" y="-4298.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="937,-4264 937,-4289 1269,-4289 1269,-4264 937,-4264"/> -<text text-anchor="start" x="942" y="-4273.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="988" y="-4273.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="993" y="-4273.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1114" y="-4273.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="937,-4239 937,-4264 1269,-4264 1269,-4239 937,-4239"/> -<text text-anchor="start" x="942" y="-4248.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> -<text text-anchor="start" x="1015" y="-4248.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1020" y="-4248.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1116" y="-4248.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="937,-4214 937,-4239 1269,-4239 1269,-4214 937,-4214"/> -<text text-anchor="start" x="942" y="-4223.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> -<text text-anchor="start" x="1021" y="-4223.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1026" y="-4223.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1122" y="-4223.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="508.5,-3225 508.5,-3253 840.5,-3253 840.5,-3225 508.5,-3225"/> +<text text-anchor="start" x="513.5" y="-3236.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_schedule_asset_alias_reference</text> +<polygon fill="none" stroke="black" points="508.5,-3200 508.5,-3225 840.5,-3225 840.5,-3200 508.5,-3200"/> +<text text-anchor="start" x="513.5" y="-3209.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">alias_id</text> +<text text-anchor="start" x="565.5" y="-3209.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="570.5" y="-3209.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="647.5" y="-3209.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="508.5,-3175 508.5,-3200 840.5,-3200 840.5,-3175 508.5,-3175"/> +<text text-anchor="start" x="513.5" y="-3184.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> +<text text-anchor="start" x="559.5" y="-3184.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="564.5" y="-3184.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="685.5" y="-3184.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="508.5,-3150 508.5,-3175 840.5,-3175 840.5,-3150 508.5,-3150"/> +<text text-anchor="start" x="513.5" y="-3159.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="586.5" y="-3159.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="591.5" y="-3159.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="687.5" y="-3159.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="508.5,-3125 508.5,-3150 840.5,-3150 840.5,-3125 508.5,-3125"/> +<text text-anchor="start" x="513.5" y="-3134.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> +<text text-anchor="start" x="592.5" y="-3134.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="597.5" y="-3134.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="693.5" y="-3134.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset_alias--dag_schedule_asset_alias_reference --> <g id="edge3" class="edge"> <title>asset_alias--dag_schedule_asset_alias_reference</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M785.06,-4298.9C830.3,-4295.91 881.15,-4292.56 928.66,-4289.43"/> -<text text-anchor="start" x="897.66" y="-4278.23" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="785.06" y="-4302.7" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M349.14,-3189C396.65,-3189 450.46,-3189 500.4,-3189"/> +<text text-anchor="start" x="469.4" y="-3177.8" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="349.14" y="-3177.8" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- asset --> -<g id="node15" class="node"> +<g id="node13" class="node"> <title>asset</title> -<polygon fill="none" stroke="black" points="513,-4591 513,-4619 779,-4619 779,-4591 513,-4591"/> -<text text-anchor="start" x="621.5" y="-4602.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset</text> -<polygon fill="none" stroke="black" points="513,-4566 513,-4591 779,-4591 779,-4566 513,-4566"/> -<text text-anchor="start" x="518" y="-4575.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="531" y="-4575.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="536" y="-4575.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="613" y="-4575.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="513,-4541 513,-4566 779,-4566 779,-4541 513,-4541"/> -<text text-anchor="start" x="518" y="-4550.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> -<text text-anchor="start" x="591" y="-4550.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="596" y="-4550.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="692" y="-4550.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="513,-4516 513,-4541 779,-4541 779,-4516 513,-4516"/> -<text text-anchor="start" x="518" y="-4525.8" font-family="Helvetica,sans-Serif" font-size="14.00">extra</text> -<text text-anchor="start" x="555" y="-4525.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="560" y="-4525.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<text text-anchor="start" x="611" y="-4525.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="513,-4491 513,-4516 779,-4516 779,-4491 513,-4491"/> -<text text-anchor="start" x="518" y="-4500.8" font-family="Helvetica,sans-Serif" font-size="14.00">group</text> -<text text-anchor="start" x="559" y="-4500.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="564" y="-4500.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> -<text text-anchor="start" x="694" y="-4500.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="513,-4466 513,-4491 779,-4491 779,-4466 513,-4466"/> -<text text-anchor="start" x="518" y="-4475.8" font-family="Helvetica,sans-Serif" font-size="14.00">name</text> -<text text-anchor="start" x="558" y="-4475.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="563" y="-4475.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> -<text text-anchor="start" x="693" y="-4475.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="513,-4441 513,-4466 779,-4466 779,-4441 513,-4441"/> -<text text-anchor="start" x="518" y="-4450.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> -<text text-anchor="start" x="597" y="-4450.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="602" y="-4450.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="698" y="-4450.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="513,-4416 513,-4441 779,-4441 779,-4416 513,-4416"/> -<text text-anchor="start" x="518" y="-4425.8" font-family="Helvetica,sans-Serif" font-size="14.00">uri</text> -<text text-anchor="start" x="537" y="-4425.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="542" y="-4425.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> -<text text-anchor="start" x="672" y="-4425.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="77,-3780 77,-3808 343,-3808 343,-3780 77,-3780"/> +<text text-anchor="start" x="185.5" y="-3791.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset</text> +<polygon fill="none" stroke="black" points="77,-3755 77,-3780 343,-3780 343,-3755 77,-3755"/> +<text text-anchor="start" x="82" y="-3764.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="95" y="-3764.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="100" y="-3764.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="177" y="-3764.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="77,-3730 77,-3755 343,-3755 343,-3730 77,-3730"/> +<text text-anchor="start" x="82" y="-3739.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="155" y="-3739.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="160" y="-3739.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="256" y="-3739.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="77,-3705 77,-3730 343,-3730 343,-3705 77,-3705"/> +<text text-anchor="start" x="82" y="-3714.8" font-family="Helvetica,sans-Serif" font-size="14.00">extra</text> +<text text-anchor="start" x="119" y="-3714.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="124" y="-3714.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<text text-anchor="start" x="175" y="-3714.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="77,-3680 77,-3705 343,-3705 343,-3680 77,-3680"/> +<text text-anchor="start" x="82" y="-3689.8" font-family="Helvetica,sans-Serif" font-size="14.00">group</text> +<text text-anchor="start" x="123" y="-3689.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="128" y="-3689.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> +<text text-anchor="start" x="258" y="-3689.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="77,-3655 77,-3680 343,-3680 343,-3655 77,-3655"/> +<text text-anchor="start" x="82" y="-3664.8" font-family="Helvetica,sans-Serif" font-size="14.00">name</text> +<text text-anchor="start" x="122" y="-3664.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="127" y="-3664.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> +<text text-anchor="start" x="257" y="-3664.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="77,-3630 77,-3655 343,-3655 343,-3630 77,-3630"/> +<text text-anchor="start" x="82" y="-3639.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> +<text text-anchor="start" x="161" y="-3639.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="166" y="-3639.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="262" y="-3639.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="77,-3605 77,-3630 343,-3630 343,-3605 77,-3605"/> +<text text-anchor="start" x="82" y="-3614.8" font-family="Helvetica,sans-Serif" font-size="14.00">uri</text> +<text text-anchor="start" x="101" y="-3614.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="106" y="-3614.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> +<text text-anchor="start" x="236" y="-3614.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset--asset_alias_asset --> <g id="edge4" class="edge"> <title>asset--asset_alias_asset</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M787.29,-4587.31C821,-4602.02 857.3,-4616.31 892,-4627 920.77,-4635.86 952.43,-4642.93 982.25,-4648.44"/> -<text text-anchor="start" x="951.25" y="-4652.24" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="787.29" y="-4576.11" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M351.02,-3729.71C415.76,-3740.2 491.86,-3752.54 553.87,-3762.6"/> +<text text-anchor="start" x="522.87" y="-3766.4" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="351.02" y="-3718.51" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- asset_active --> -<g id="node16" class="node"> +<g id="node14" class="node"> <title>asset_active</title> -<polygon fill="none" stroke="black" points="973,-4572 973,-4600 1234,-4600 1234,-4572 973,-4572"/> -<text text-anchor="start" x="1048" y="-4583.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_active</text> -<polygon fill="none" stroke="black" points="973,-4547 973,-4572 1234,-4572 1234,-4547 973,-4547"/> -<text text-anchor="start" x="978" y="-4556.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">name</text> -<text text-anchor="start" x="1018" y="-4556.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1023" y="-4556.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> -<text text-anchor="start" x="1153" y="-4556.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="973,-4522 973,-4547 1234,-4547 1234,-4522 973,-4522"/> -<text text-anchor="start" x="978" y="-4531.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">uri</text> -<text text-anchor="start" x="997" y="-4531.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1002" y="-4531.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> -<text text-anchor="start" x="1132" y="-4531.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="544.5,-3897 544.5,-3925 805.5,-3925 805.5,-3897 544.5,-3897"/> +<text text-anchor="start" x="619.5" y="-3908.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_active</text> +<polygon fill="none" stroke="black" points="544.5,-3872 544.5,-3897 805.5,-3897 805.5,-3872 544.5,-3872"/> +<text text-anchor="start" x="549.5" y="-3881.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">name</text> +<text text-anchor="start" x="589.5" y="-3881.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="594.5" y="-3881.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> +<text text-anchor="start" x="724.5" y="-3881.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="544.5,-3847 544.5,-3872 805.5,-3872 805.5,-3847 544.5,-3847"/> +<text text-anchor="start" x="549.5" y="-3856.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">uri</text> +<text text-anchor="start" x="568.5" y="-3856.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="573.5" y="-3856.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1500)]</text> +<text text-anchor="start" x="703.5" y="-3856.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset--asset_active --> <g id="edge5" class="edge"> <title>asset--asset_active</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M787.23,-4524.96C843.5,-4529.66 908.16,-4535.78 964.19,-4541.72"/> -<text text-anchor="start" x="954.19" y="-4530.52" font-family="Times,serif" font-size="14.00">1</text> -<text text-anchor="start" x="787.23" y="-4513.76" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M351.02,-3766.24C395.82,-3786.21 446,-3807.75 493,-3825 510.14,-3831.29 528.42,-3837.32 546.52,-3842.99"/> +<text text-anchor="start" x="536.52" y="-3831.79" font-family="Times,serif" font-size="14.00">1</text> +<text text-anchor="start" x="351.02" y="-3755.04" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- asset--asset_active --> <g id="edge6" class="edge"> <title>asset--asset_active</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M787.23,-4537.54C843.5,-4543.48 908.16,-4549.59 964.19,-4554.24"/> -<text text-anchor="start" x="954.19" y="-4558.04" font-family="Times,serif" font-size="14.00">1</text> -<text text-anchor="start" x="787.23" y="-4541.34" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M351.02,-3783.37C395.82,-3804.21 446,-3825.75 493,-3843 506.78,-3848.06 521.29,-3852.94 535.85,-3857.51"/> +<text text-anchor="start" x="525.85" y="-3846.31" font-family="Times,serif" font-size="14.00">1</text> +<text text-anchor="start" x="351.02" y="-3772.17" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag_schedule_asset_reference --> -<g id="node17" class="node"> +<g id="node15" class="node"> <title>dag_schedule_asset_reference</title> -<polygon fill="none" stroke="black" points="962,-4468 962,-4496 1244,-4496 1244,-4468 962,-4468"/> -<text text-anchor="start" x="967" y="-4479.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_schedule_asset_reference</text> -<polygon fill="none" stroke="black" points="962,-4443 962,-4468 1244,-4468 1244,-4443 962,-4443"/> -<text text-anchor="start" x="967" y="-4452.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">asset_id</text> -<text text-anchor="start" x="1024" y="-4452.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1029" y="-4452.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1106" y="-4452.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="962,-4418 962,-4443 1244,-4443 1244,-4418 962,-4418"/> -<text text-anchor="start" x="967" y="-4427.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1013" y="-4427.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1018" y="-4427.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1139" y="-4427.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="962,-4393 962,-4418 1244,-4418 1244,-4393 962,-4393"/> -<text text-anchor="start" x="967" y="-4402.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> -<text text-anchor="start" x="1040" y="-4402.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1045" y="-4402.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1141" y="-4402.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="962,-4368 962,-4393 1244,-4393 1244,-4368 962,-4368"/> -<text text-anchor="start" x="967" y="-4377.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> -<text text-anchor="start" x="1046" y="-4377.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1051" y="-4377.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1147" y="-4377.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="533.5,-3509 533.5,-3537 815.5,-3537 815.5,-3509 533.5,-3509"/> +<text text-anchor="start" x="538.5" y="-3520.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_schedule_asset_reference</text> +<polygon fill="none" stroke="black" points="533.5,-3484 533.5,-3509 815.5,-3509 815.5,-3484 533.5,-3484"/> +<text text-anchor="start" x="538.5" y="-3493.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">asset_id</text> +<text text-anchor="start" x="595.5" y="-3493.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="600.5" y="-3493.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="677.5" y="-3493.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="533.5,-3459 533.5,-3484 815.5,-3484 815.5,-3459 533.5,-3459"/> +<text text-anchor="start" x="538.5" y="-3468.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> +<text text-anchor="start" x="584.5" y="-3468.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="589.5" y="-3468.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="710.5" y="-3468.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="533.5,-3434 533.5,-3459 815.5,-3459 815.5,-3434 533.5,-3434"/> +<text text-anchor="start" x="538.5" y="-3443.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="611.5" y="-3443.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="616.5" y="-3443.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="712.5" y="-3443.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="533.5,-3409 533.5,-3434 815.5,-3434 815.5,-3409 533.5,-3409"/> +<text text-anchor="start" x="538.5" y="-3418.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> +<text text-anchor="start" x="617.5" y="-3418.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="622.5" y="-3418.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="718.5" y="-3418.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset--dag_schedule_asset_reference --> <g id="edge7" class="edge"> <title>asset--dag_schedule_asset_reference</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M787.23,-4491.5C840.1,-4481.5 900.39,-4470.11 953.96,-4459.98"/> -<text text-anchor="start" x="922.96" y="-4448.78" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="787.23" y="-4480.3" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M351.12,-3625.2C395.92,-3600.06 446.08,-3572.99 493,-3550 503.4,-3544.9 514.22,-3539.8 525.17,-3534.79"/> +<text text-anchor="start" x="494.17" y="-3523.59" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="351.12" y="-3614" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_outlet_asset_reference --> -<g id="node18" class="node"> +<g id="node16" class="node"> <title>task_outlet_asset_reference</title> -<polygon fill="none" stroke="black" points="970,-4159 970,-4187 1236,-4187 1236,-4159 970,-4159"/> -<text text-anchor="start" x="977.5" y="-4170.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_outlet_asset_reference</text> -<polygon fill="none" stroke="black" points="970,-4134 970,-4159 1236,-4159 1236,-4134 970,-4134"/> -<text text-anchor="start" x="975" y="-4143.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">asset_id</text> -<text text-anchor="start" x="1032" y="-4143.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1037" y="-4143.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1114" y="-4143.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="970,-4109 970,-4134 1236,-4134 1236,-4109 970,-4109"/> -<text text-anchor="start" x="975" y="-4118.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1021" y="-4118.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1026" y="-4118.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1147" y="-4118.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="970,-4084 970,-4109 1236,-4109 1236,-4084 970,-4084"/> -<text text-anchor="start" x="975" y="-4093.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">task_id</text> -<text text-anchor="start" x="1024" y="-4093.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1029" y="-4093.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1150" y="-4093.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="970,-4059 970,-4084 1236,-4084 1236,-4059 970,-4059"/> -<text text-anchor="start" x="975" y="-4068.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> -<text text-anchor="start" x="1048" y="-4068.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1053" y="-4068.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1149" y="-4068.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="970,-4034 970,-4059 1236,-4059 1236,-4034 970,-4034"/> -<text text-anchor="start" x="975" y="-4043.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> -<text text-anchor="start" x="1054" y="-4043.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1059" y="-4043.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1155" y="-4043.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="541.5,-3688 541.5,-3716 807.5,-3716 807.5,-3688 541.5,-3688"/> +<text text-anchor="start" x="549" y="-3699.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_outlet_asset_reference</text> +<polygon fill="none" stroke="black" points="541.5,-3663 541.5,-3688 807.5,-3688 807.5,-3663 541.5,-3663"/> +<text text-anchor="start" x="546.5" y="-3672.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">asset_id</text> +<text text-anchor="start" x="603.5" y="-3672.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="608.5" y="-3672.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="685.5" y="-3672.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="541.5,-3638 541.5,-3663 807.5,-3663 807.5,-3638 541.5,-3638"/> +<text text-anchor="start" x="546.5" y="-3647.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> +<text text-anchor="start" x="592.5" y="-3647.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="597.5" y="-3647.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="718.5" y="-3647.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="541.5,-3613 541.5,-3638 807.5,-3638 807.5,-3613 541.5,-3613"/> +<text text-anchor="start" x="546.5" y="-3622.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">task_id</text> +<text text-anchor="start" x="595.5" y="-3622.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="600.5" y="-3622.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="721.5" y="-3622.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="541.5,-3588 541.5,-3613 807.5,-3613 807.5,-3588 541.5,-3588"/> +<text text-anchor="start" x="546.5" y="-3597.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="619.5" y="-3597.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="624.5" y="-3597.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="720.5" y="-3597.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="541.5,-3563 541.5,-3588 807.5,-3588 807.5,-3563 541.5,-3563"/> +<text text-anchor="start" x="546.5" y="-3572.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> +<text text-anchor="start" x="625.5" y="-3572.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="630.5" y="-3572.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="726.5" y="-3572.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset--task_outlet_asset_reference --> <g id="edge8" class="edge"> <title>asset--task_outlet_asset_reference</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M787.27,-4436.45C812.33,-4417.55 836.6,-4395.88 856,-4372 908.11,-4307.86 872.84,-4261.62 929,-4201 938.66,-4190.57 949.81,-4181.14 961.7,-4172.66"/> -<text text-anchor="start" x="930.7" y="-4161.46" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="787.27" y="-4425.25" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M351.02,-3686.72C408.71,-3678.36 475.43,-3668.69 533.16,-3660.33"/> +<text text-anchor="start" x="502.16" y="-3649.13" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="351.02" y="-3675.52" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- asset_dag_run_queue --> -<g id="node19" class="node"> +<g id="node17" class="node"> <title>asset_dag_run_queue</title> -<polygon fill="none" stroke="black" points="950,-3980 950,-4008 1256,-4008 1256,-3980 950,-3980"/> -<text text-anchor="start" x="1007" y="-3991.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_dag_run_queue</text> -<polygon fill="none" stroke="black" points="950,-3955 950,-3980 1256,-3980 1256,-3955 950,-3955"/> -<text text-anchor="start" x="955" y="-3964.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">asset_id</text> -<text text-anchor="start" x="1012" y="-3964.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1017" y="-3964.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1094" y="-3964.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="950,-3930 950,-3955 1256,-3955 1256,-3930 950,-3930"/> -<text text-anchor="start" x="955" y="-3939.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">target_dag_id</text> -<text text-anchor="start" x="1049" y="-3939.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1054" y="-3939.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1175" y="-3939.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="950,-3905 950,-3930 1256,-3930 1256,-3905 950,-3905"/> -<text text-anchor="start" x="955" y="-3914.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> -<text text-anchor="start" x="1028" y="-3914.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1033" y="-3914.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1129" y="-3914.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="521.5,-3354 521.5,-3382 827.5,-3382 827.5,-3354 521.5,-3354"/> +<text text-anchor="start" x="578.5" y="-3365.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_dag_run_queue</text> +<polygon fill="none" stroke="black" points="521.5,-3329 521.5,-3354 827.5,-3354 827.5,-3329 521.5,-3329"/> +<text text-anchor="start" x="526.5" y="-3338.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">asset_id</text> +<text text-anchor="start" x="583.5" y="-3338.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="588.5" y="-3338.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="665.5" y="-3338.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="521.5,-3304 521.5,-3329 827.5,-3329 827.5,-3304 521.5,-3304"/> +<text text-anchor="start" x="526.5" y="-3313.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">target_dag_id</text> +<text text-anchor="start" x="620.5" y="-3313.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="625.5" y="-3313.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="746.5" y="-3313.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="521.5,-3279 521.5,-3304 827.5,-3304 827.5,-3279 521.5,-3279"/> +<text text-anchor="start" x="526.5" y="-3288.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="599.5" y="-3288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="604.5" y="-3288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="700.5" y="-3288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset--asset_dag_run_queue --> <g id="edge9" class="edge"> <title>asset--asset_dag_run_queue</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M787.17,-4442.4C813.4,-4422.6 838.15,-4399.07 856,-4372 943.73,-4238.99 825.05,-4141.76 929,-4021 932.97,-4016.39 937.3,-4012.11 941.92,-4008.14"/> -<text text-anchor="start" x="910.92" y="-3996.94" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="787.17" y="-4446.2" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M285.36,-3601.3C337.01,-3534.54 411.09,-3450.84 493,-3396 499.5,-3391.65 506.33,-3387.54 513.38,-3383.68"/> +<text text-anchor="start" x="482.38" y="-3372.48" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="275.36" y="-3590.1" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- asset_event --> -<g id="node20" class="node"> +<g id="node18" class="node"> <title>asset_event</title> -<polygon fill="none" stroke="black" points="515,-3501 515,-3529 777,-3529 777,-3501 515,-3501"/> -<text text-anchor="start" x="592.5" y="-3512.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_event</text> -<polygon fill="none" stroke="black" points="515,-3476 515,-3501 777,-3501 777,-3476 515,-3476"/> -<text text-anchor="start" x="520" y="-3485.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="533" y="-3485.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="538" y="-3485.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="615" y="-3485.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="515,-3451 515,-3476 777,-3476 777,-3451 515,-3451"/> -<text text-anchor="start" x="520" y="-3460.8" font-family="Helvetica,sans-Serif" font-size="14.00">asset_id</text> -<text text-anchor="start" x="577" y="-3460.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="582" y="-3460.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="659" y="-3460.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="515,-3426 515,-3451 777,-3451 777,-3426 515,-3426"/> -<text text-anchor="start" x="520" y="-3435.8" font-family="Helvetica,sans-Serif" font-size="14.00">extra</text> -<text text-anchor="start" x="557" y="-3435.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="562" y="-3435.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<text text-anchor="start" x="613" y="-3435.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="515,-3401 515,-3426 777,-3426 777,-3401 515,-3401"/> -<text text-anchor="start" x="520" y="-3410.8" font-family="Helvetica,sans-Serif" font-size="14.00">source_dag_id</text> -<text text-anchor="start" x="619" y="-3410.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="624" y="-3410.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="515,-3376 515,-3401 777,-3401 777,-3376 515,-3376"/> -<text text-anchor="start" x="520" y="-3385.8" font-family="Helvetica,sans-Serif" font-size="14.00">source_map_index</text> -<text text-anchor="start" x="649" y="-3385.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="654" y="-3385.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="515,-3351 515,-3376 777,-3376 777,-3351 515,-3351"/> -<text text-anchor="start" x="520" y="-3360.8" font-family="Helvetica,sans-Serif" font-size="14.00">source_run_id</text> -<text text-anchor="start" x="617" y="-3360.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="622" y="-3360.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="515,-3326 515,-3351 777,-3351 777,-3326 515,-3326"/> -<text text-anchor="start" x="520" y="-3335.8" font-family="Helvetica,sans-Serif" font-size="14.00">source_task_id</text> -<text text-anchor="start" x="622" y="-3335.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="627" y="-3335.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="515,-3301 515,-3326 777,-3326 777,-3301 515,-3301"/> -<text text-anchor="start" x="520" y="-3310.8" font-family="Helvetica,sans-Serif" font-size="14.00">timestamp</text> -<text text-anchor="start" x="595" y="-3310.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="600" y="-3310.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="696" y="-3310.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="79,-2063 79,-2091 341,-2091 341,-2063 79,-2063"/> +<text text-anchor="start" x="156.5" y="-2074.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">asset_event</text> +<polygon fill="none" stroke="black" points="79,-2038 79,-2063 341,-2063 341,-2038 79,-2038"/> +<text text-anchor="start" x="84" y="-2047.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="97" y="-2047.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="102" y="-2047.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="179" y="-2047.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="79,-2013 79,-2038 341,-2038 341,-2013 79,-2013"/> +<text text-anchor="start" x="84" y="-2022.8" font-family="Helvetica,sans-Serif" font-size="14.00">asset_id</text> +<text text-anchor="start" x="141" y="-2022.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="146" y="-2022.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="223" y="-2022.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="79,-1988 79,-2013 341,-2013 341,-1988 79,-1988"/> +<text text-anchor="start" x="84" y="-1997.8" font-family="Helvetica,sans-Serif" font-size="14.00">extra</text> +<text text-anchor="start" x="121" y="-1997.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="126" y="-1997.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<text text-anchor="start" x="177" y="-1997.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="79,-1963 79,-1988 341,-1988 341,-1963 79,-1963"/> +<text text-anchor="start" x="84" y="-1972.8" font-family="Helvetica,sans-Serif" font-size="14.00">source_dag_id</text> +<text text-anchor="start" x="183" y="-1972.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="188" y="-1972.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="79,-1938 79,-1963 341,-1963 341,-1938 79,-1938"/> +<text text-anchor="start" x="84" y="-1947.8" font-family="Helvetica,sans-Serif" font-size="14.00">source_map_index</text> +<text text-anchor="start" x="213" y="-1947.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="218" y="-1947.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="79,-1913 79,-1938 341,-1938 341,-1913 79,-1913"/> +<text text-anchor="start" x="84" y="-1922.8" font-family="Helvetica,sans-Serif" font-size="14.00">source_run_id</text> +<text text-anchor="start" x="181" y="-1922.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="186" y="-1922.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="79,-1888 79,-1913 341,-1913 341,-1888 79,-1888"/> +<text text-anchor="start" x="84" y="-1897.8" font-family="Helvetica,sans-Serif" font-size="14.00">source_task_id</text> +<text text-anchor="start" x="186" y="-1897.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="191" y="-1897.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="79,-1863 79,-1888 341,-1888 341,-1863 79,-1863"/> +<text text-anchor="start" x="84" y="-1872.8" font-family="Helvetica,sans-Serif" font-size="14.00">timestamp</text> +<text text-anchor="start" x="159" y="-1872.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="164" y="-1872.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="260" y="-1872.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset_event--asset_alias_asset_event --> <g id="edge10" class="edge"> <title>asset_event--asset_alias_asset_event</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M785.06,-3426.23C847.69,-3431.32 921.08,-3437.29 981.45,-3442.2"/> -<text text-anchor="start" x="950.45" y="-3431" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="785.06" y="-3415.03" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M305.67,-2095.33C347.92,-2154.56 393.95,-2229.42 420,-2304 471.8,-2452.27 392.5,-2886.3 493,-3007 508.46,-3025.57 530.07,-3037.84 552.98,-3045.89"/> +<text text-anchor="start" x="521.98" y="-3034.69" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="295.67" y="-2099.13" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dagrun_asset_event --> -<g id="node21" class="node"> +<g id="node19" class="node"> <title>dagrun_asset_event</title> -<polygon fill="none" stroke="black" points="981,-3359 981,-3387 1226,-3387 1226,-3359 981,-3359"/> -<text text-anchor="start" x="1014" y="-3370.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dagrun_asset_event</text> -<polygon fill="none" stroke="black" points="981,-3334 981,-3359 1226,-3359 1226,-3334 981,-3334"/> -<text text-anchor="start" x="986" y="-3343.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_run_id</text> -<text text-anchor="start" x="1063" y="-3343.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1068" y="-3343.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1145" y="-3343.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="981,-3309 981,-3334 1226,-3334 1226,-3309 981,-3309"/> -<text text-anchor="start" x="986" y="-3318.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">event_id</text> -<text text-anchor="start" x="1045" y="-3318.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1050" y="-3318.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1127" y="-3318.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1346,-1788 1346,-1816 1591,-1816 1591,-1788 1346,-1788"/> +<text text-anchor="start" x="1379" y="-1799.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dagrun_asset_event</text> +<polygon fill="none" stroke="black" points="1346,-1763 1346,-1788 1591,-1788 1591,-1763 1346,-1763"/> +<text text-anchor="start" x="1351" y="-1772.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_run_id</text> +<text text-anchor="start" x="1428" y="-1772.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1433" y="-1772.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1510" y="-1772.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1346,-1738 1346,-1763 1591,-1763 1591,-1738 1346,-1738"/> +<text text-anchor="start" x="1351" y="-1747.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">event_id</text> +<text text-anchor="start" x="1410" y="-1747.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1415" y="-1747.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1492" y="-1747.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- asset_event--dagrun_asset_event --> <g id="edge11" class="edge"> <title>asset_event--dagrun_asset_event</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M785.06,-3394.67C844.55,-3385.91 913.75,-3375.72 972.28,-3367.1"/> -<text text-anchor="start" x="941.28" y="-3355.9" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="785.06" y="-3383.47" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M349.22,-1903.04C460.18,-1846.44 607.89,-1778 673.5,-1778 673.5,-1778 673.5,-1778 1082,-1778 1167.05,-1778 1262.68,-1777.75 1337.35,-1777.5"/> +<text text-anchor="start" x="1306.35" y="-1766.3" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="349.22" y="-1891.84" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag --> -<g id="node22" class="node"> +<g id="node20" class="node"> <title>dag</title> -<polygon fill="none" stroke="black" points="444,-4202 444,-4230 848,-4230 848,-4202 444,-4202"/> -<text text-anchor="start" x="629" y="-4213.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag</text> -<polygon fill="none" stroke="black" points="444,-4177 444,-4202 848,-4202 848,-4177 444,-4177"/> -<text text-anchor="start" x="449" y="-4186.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="495" y="-4186.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="500" y="-4186.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="621" y="-4186.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="444,-4152 444,-4177 848,-4177 848,-4152 444,-4152"/> -<text text-anchor="start" x="449" y="-4161.8" font-family="Helvetica,sans-Serif" font-size="14.00">asset_expression</text> -<text text-anchor="start" x="568" y="-4161.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="573" y="-4161.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<polygon fill="none" stroke="black" points="444,-4127 444,-4152 848,-4152 848,-4127 444,-4127"/> -<text text-anchor="start" x="449" y="-4136.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_display_name</text> -<text text-anchor="start" x="578" y="-4136.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="583" y="-4136.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> -<polygon fill="none" stroke="black" points="444,-4102 444,-4127 848,-4127 848,-4102 444,-4102"/> -<text text-anchor="start" x="449" y="-4111.8" font-family="Helvetica,sans-Serif" font-size="14.00">default_view</text> -<text text-anchor="start" x="537" y="-4111.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="542" y="-4111.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(25)]</text> -<polygon fill="none" stroke="black" points="444,-4077 444,-4102 848,-4102 848,-4077 444,-4077"/> -<text text-anchor="start" x="449" y="-4086.8" font-family="Helvetica,sans-Serif" font-size="14.00">description</text> -<text text-anchor="start" x="527" y="-4086.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="532" y="-4086.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<polygon fill="none" stroke="black" points="444,-4052 444,-4077 848,-4077 848,-4052 444,-4052"/> -<text text-anchor="start" x="449" y="-4061.8" font-family="Helvetica,sans-Serif" font-size="14.00">fileloc</text> -<text text-anchor="start" x="490" y="-4061.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="495" y="-4061.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> -<polygon fill="none" stroke="black" points="444,-4027 444,-4052 848,-4052 848,-4027 444,-4027"/> -<text text-anchor="start" x="449" y="-4036.8" font-family="Helvetica,sans-Serif" font-size="14.00">has_import_errors</text> -<text text-anchor="start" x="574" y="-4036.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="579" y="-4036.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<polygon fill="none" stroke="black" points="444,-4002 444,-4027 848,-4027 848,-4002 444,-4002"/> -<text text-anchor="start" x="449" y="-4011.8" font-family="Helvetica,sans-Serif" font-size="14.00">has_task_concurrency_limits</text> -<text text-anchor="start" x="646" y="-4011.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="651" y="-4011.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<text text-anchor="start" x="735" y="-4011.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="444,-3977 444,-4002 848,-4002 848,-3977 444,-3977"/> -<text text-anchor="start" x="449" y="-3986.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_active</text> -<text text-anchor="start" x="509" y="-3986.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="514" y="-3986.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<polygon fill="none" stroke="black" points="444,-3952 444,-3977 848,-3977 848,-3952 444,-3952"/> -<text text-anchor="start" x="449" y="-3961.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_paused</text> -<text text-anchor="start" x="519" y="-3961.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="524" y="-3961.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<polygon fill="none" stroke="black" points="444,-3927 444,-3952 848,-3952 848,-3927 444,-3927"/> -<text text-anchor="start" x="449" y="-3936.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_expired</text> -<text text-anchor="start" x="533" y="-3936.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="538" y="-3936.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="444,-3902 444,-3927 848,-3927 848,-3902 444,-3902"/> -<text text-anchor="start" x="449" y="-3911.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_parsed_time</text> -<text text-anchor="start" x="567" y="-3911.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="572" y="-3911.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="444,-3877 444,-3902 848,-3902 848,-3877 444,-3877"/> -<text text-anchor="start" x="449" y="-3886.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_active_runs</text> -<text text-anchor="start" x="566" y="-3886.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="571" y="-3886.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="444,-3852 444,-3877 848,-3877 848,-3852 444,-3852"/> -<text text-anchor="start" x="449" y="-3861.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_active_tasks</text> -<text text-anchor="start" x="571" y="-3861.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="576" y="-3861.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="653" y="-3861.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="444,-3827 444,-3852 848,-3852 848,-3827 444,-3827"/> -<text text-anchor="start" x="449" y="-3836.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_consecutive_failed_dag_runs</text> -<text text-anchor="start" x="685" y="-3836.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="690" y="-3836.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="767" y="-3836.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="444,-3802 444,-3827 848,-3827 848,-3802 444,-3802"/> -<text text-anchor="start" x="449" y="-3811.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_dagrun</text> -<text text-anchor="start" x="537" y="-3811.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="542" y="-3811.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="444,-3777 444,-3802 848,-3802 848,-3777 444,-3777"/> -<text text-anchor="start" x="449" y="-3786.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_dagrun_create_after</text> -<text text-anchor="start" x="626" y="-3786.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="631" y="-3786.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="444,-3752 444,-3777 848,-3777 848,-3752 444,-3752"/> -<text text-anchor="start" x="449" y="-3761.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_dagrun_data_interval_end</text> -<text text-anchor="start" x="667" y="-3761.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="672" y="-3761.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="444,-3727 444,-3752 848,-3752 848,-3727 444,-3727"/> -<text text-anchor="start" x="449" y="-3736.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_dagrun_data_interval_start</text> -<text text-anchor="start" x="673" y="-3736.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="678" y="-3736.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="444,-3702 444,-3727 848,-3727 848,-3702 444,-3702"/> -<text text-anchor="start" x="449" y="-3711.8" font-family="Helvetica,sans-Serif" font-size="14.00">owners</text> -<text text-anchor="start" x="500" y="-3711.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="505" y="-3711.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> -<polygon fill="none" stroke="black" points="444,-3677 444,-3702 848,-3702 848,-3677 444,-3677"/> -<text text-anchor="start" x="449" y="-3686.8" font-family="Helvetica,sans-Serif" font-size="14.00">processor_subdir</text> -<text text-anchor="start" x="568" y="-3686.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="573" y="-3686.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> -<polygon fill="none" stroke="black" points="444,-3652 444,-3677 848,-3677 848,-3652 444,-3652"/> -<text text-anchor="start" x="449" y="-3661.8" font-family="Helvetica,sans-Serif" font-size="14.00">timetable_description</text> -<text text-anchor="start" x="599" y="-3661.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="604" y="-3661.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="444,-3627 444,-3652 848,-3652 848,-3627 444,-3627"/> -<text text-anchor="start" x="449" y="-3636.8" font-family="Helvetica,sans-Serif" font-size="14.00">timetable_summary</text> -<text text-anchor="start" x="588" y="-3636.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="593" y="-3636.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="8,-2892 8,-2920 412,-2920 412,-2892 8,-2892"/> +<text text-anchor="start" x="193" y="-2903.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag</text> +<polygon fill="none" stroke="black" points="8,-2867 8,-2892 412,-2892 412,-2867 8,-2867"/> +<text text-anchor="start" x="13" y="-2876.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> +<text text-anchor="start" x="59" y="-2876.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="64" y="-2876.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="185" y="-2876.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="8,-2842 8,-2867 412,-2867 412,-2842 8,-2842"/> +<text text-anchor="start" x="13" y="-2851.8" font-family="Helvetica,sans-Serif" font-size="14.00">asset_expression</text> +<text text-anchor="start" x="132" y="-2851.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="137" y="-2851.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<polygon fill="none" stroke="black" points="8,-2817 8,-2842 412,-2842 412,-2817 8,-2817"/> +<text text-anchor="start" x="13" y="-2826.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_display_name</text> +<text text-anchor="start" x="142" y="-2826.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="147" y="-2826.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<polygon fill="none" stroke="black" points="8,-2792 8,-2817 412,-2817 412,-2792 8,-2792"/> +<text text-anchor="start" x="13" y="-2801.8" font-family="Helvetica,sans-Serif" font-size="14.00">default_view</text> +<text text-anchor="start" x="101" y="-2801.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="106" y="-2801.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(25)]</text> +<polygon fill="none" stroke="black" points="8,-2767 8,-2792 412,-2792 412,-2767 8,-2767"/> +<text text-anchor="start" x="13" y="-2776.8" font-family="Helvetica,sans-Serif" font-size="14.00">description</text> +<text text-anchor="start" x="91" y="-2776.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="96" y="-2776.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<polygon fill="none" stroke="black" points="8,-2742 8,-2767 412,-2767 412,-2742 8,-2742"/> +<text text-anchor="start" x="13" y="-2751.8" font-family="Helvetica,sans-Serif" font-size="14.00">fileloc</text> +<text text-anchor="start" x="54" y="-2751.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="59" y="-2751.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<polygon fill="none" stroke="black" points="8,-2717 8,-2742 412,-2742 412,-2717 8,-2717"/> +<text text-anchor="start" x="13" y="-2726.8" font-family="Helvetica,sans-Serif" font-size="14.00">has_import_errors</text> +<text text-anchor="start" x="138" y="-2726.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="143" y="-2726.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<polygon fill="none" stroke="black" points="8,-2692 8,-2717 412,-2717 412,-2692 8,-2692"/> +<text text-anchor="start" x="13" y="-2701.8" font-family="Helvetica,sans-Serif" font-size="14.00">has_task_concurrency_limits</text> +<text text-anchor="start" x="210" y="-2701.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="215" y="-2701.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<text text-anchor="start" x="299" y="-2701.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="8,-2667 8,-2692 412,-2692 412,-2667 8,-2667"/> +<text text-anchor="start" x="13" y="-2676.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_active</text> +<text text-anchor="start" x="73" y="-2676.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="78" y="-2676.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<polygon fill="none" stroke="black" points="8,-2642 8,-2667 412,-2667 412,-2642 8,-2642"/> +<text text-anchor="start" x="13" y="-2651.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_paused</text> +<text text-anchor="start" x="83" y="-2651.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="88" y="-2651.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<polygon fill="none" stroke="black" points="8,-2617 8,-2642 412,-2642 412,-2617 8,-2617"/> +<text text-anchor="start" x="13" y="-2626.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_expired</text> +<text text-anchor="start" x="97" y="-2626.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="102" y="-2626.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="8,-2592 8,-2617 412,-2617 412,-2592 8,-2592"/> +<text text-anchor="start" x="13" y="-2601.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_parsed_time</text> +<text text-anchor="start" x="131" y="-2601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="136" y="-2601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="8,-2567 8,-2592 412,-2592 412,-2567 8,-2567"/> +<text text-anchor="start" x="13" y="-2576.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_active_runs</text> +<text text-anchor="start" x="130" y="-2576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="135" y="-2576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="8,-2542 8,-2567 412,-2567 412,-2542 8,-2542"/> +<text text-anchor="start" x="13" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_active_tasks</text> +<text text-anchor="start" x="135" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="140" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="217" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="8,-2517 8,-2542 412,-2542 412,-2517 8,-2517"/> +<text text-anchor="start" x="13" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_consecutive_failed_dag_runs</text> +<text text-anchor="start" x="249" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="254" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="331" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="8,-2492 8,-2517 412,-2517 412,-2492 8,-2492"/> +<text text-anchor="start" x="13" y="-2501.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_dagrun</text> +<text text-anchor="start" x="101" y="-2501.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="106" y="-2501.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="8,-2467 8,-2492 412,-2492 412,-2467 8,-2467"/> +<text text-anchor="start" x="13" y="-2476.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_dagrun_create_after</text> +<text text-anchor="start" x="190" y="-2476.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="195" y="-2476.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="8,-2442 8,-2467 412,-2467 412,-2442 8,-2442"/> +<text text-anchor="start" x="13" y="-2451.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_dagrun_data_interval_end</text> +<text text-anchor="start" x="231" y="-2451.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="236" y="-2451.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="8,-2417 8,-2442 412,-2442 412,-2417 8,-2417"/> +<text text-anchor="start" x="13" y="-2426.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_dagrun_data_interval_start</text> +<text text-anchor="start" x="237" y="-2426.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="242" y="-2426.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="8,-2392 8,-2417 412,-2417 412,-2392 8,-2392"/> +<text text-anchor="start" x="13" y="-2401.8" font-family="Helvetica,sans-Serif" font-size="14.00">owners</text> +<text text-anchor="start" x="64" y="-2401.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="69" y="-2401.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<polygon fill="none" stroke="black" points="8,-2367 8,-2392 412,-2392 412,-2367 8,-2367"/> +<text text-anchor="start" x="13" y="-2376.8" font-family="Helvetica,sans-Serif" font-size="14.00">processor_subdir</text> +<text text-anchor="start" x="132" y="-2376.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="137" y="-2376.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<polygon fill="none" stroke="black" points="8,-2342 8,-2367 412,-2367 412,-2342 8,-2342"/> +<text text-anchor="start" x="13" y="-2351.8" font-family="Helvetica,sans-Serif" font-size="14.00">timetable_description</text> +<text text-anchor="start" x="163" y="-2351.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="168" y="-2351.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="8,-2317 8,-2342 412,-2342 412,-2317 8,-2317"/> +<text text-anchor="start" x="13" y="-2326.8" font-family="Helvetica,sans-Serif" font-size="14.00">timetable_summary</text> +<text text-anchor="start" x="152" y="-2326.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="157" y="-2326.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> </g> <!-- dag--dag_schedule_asset_alias_reference --> <g id="edge12" class="edge"> <title>dag--dag_schedule_asset_alias_reference</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M856.36,-4144.65C879.96,-4164.49 904.39,-4183.36 929,-4200 934.05,-4203.41 939.27,-4206.74 944.62,-4209.98"/> -<text text-anchor="start" x="913.62" y="-4198.78" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="856.36" y="-4133.45" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M344.8,-2924.58C385.06,-2992.6 434.44,-3059.89 493,-3112 496.44,-3115.06 500.01,-3118.01 503.7,-3120.85"/> +<text text-anchor="start" x="472.7" y="-3109.65" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="334.8" y="-2928.38" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag--dag_schedule_asset_reference --> <g id="edge13" class="edge"> <title>dag--dag_schedule_asset_reference</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M850.57,-4234.76C852.39,-4237.52 854.2,-4240.27 856,-4243 888.68,-4292.62 883.78,-4316.45 929,-4355 936.76,-4361.61 945.17,-4367.75 953.97,-4373.44"/> -<text text-anchor="start" x="922.97" y="-4362.24" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="840.57" y="-4238.56" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M344.93,-2924.7C371.4,-2989.89 397.82,-3058.76 420,-3124 460.16,-3242.1 411.41,-3300.65 493,-3395 502.27,-3405.72 513.27,-3415.01 525.21,-3423.05"/> +<text text-anchor="start" x="494.21" y="-3411.85" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="344.93" y="-2928.5" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag--task_outlet_asset_reference --> <g id="edge14" class="edge"> <title>dag--task_outlet_asset_reference</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M856.19,-4012.68C892.02,-4027.01 928.46,-4041.58 961.92,-4054.97"/> -<text text-anchor="start" x="930.92" y="-4043.77" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="856.19" y="-4001.48" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M351.07,-2924.7C376.91,-2989.51 401.53,-3058.23 420,-3124 471.95,-3308.94 377.14,-3396.78 493,-3550 503.98,-3564.52 517.95,-3576.86 533.24,-3587.3"/> +<text text-anchor="start" x="502.24" y="-3576.1" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="351.07" y="-2928.5" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag--asset_dag_run_queue --> <g id="edge15" class="edge"> <title>dag--asset_dag_run_queue</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M856.19,-3941.87C885.02,-3943.65 914.24,-3945.45 941.99,-3947.15"/> -<text text-anchor="start" x="910.99" y="-3935.95" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="856.19" y="-3930.67" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M337.94,-2924.56C405.84,-3085.44 478.45,-3252.38 493,-3266 499.2,-3271.81 505.95,-3277.1 513.09,-3281.93"/> +<text text-anchor="start" x="482.09" y="-3270.73" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="327.94" y="-2928.36" font-family="Times,serif" font-size="14.00">1</text> +</g> +<!-- dag_version --> +<g id="node21" class="node"> +<title>dag_version</title> +<polygon fill="none" stroke="black" points="534.5,-2249 534.5,-2277 815.5,-2277 815.5,-2249 534.5,-2249"/> +<text text-anchor="start" x="620.5" y="-2260.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_version</text> +<polygon fill="none" stroke="black" points="534.5,-2224 534.5,-2249 815.5,-2249 815.5,-2224 534.5,-2224"/> +<text text-anchor="start" x="539.5" y="-2233.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="552.5" y="-2233.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="557.5" y="-2233.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [UUID]</text> +<text text-anchor="start" x="609.5" y="-2233.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="534.5,-2199 534.5,-2224 815.5,-2224 815.5,-2199 534.5,-2199"/> +<text text-anchor="start" x="539.5" y="-2208.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="612.5" y="-2208.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="617.5" y="-2208.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="534.5,-2174 534.5,-2199 815.5,-2199 815.5,-2174 534.5,-2174"/> +<text text-anchor="start" x="539.5" y="-2183.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> +<text text-anchor="start" x="585.5" y="-2183.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="590.5" y="-2183.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="711.5" y="-2183.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="534.5,-2149 534.5,-2174 815.5,-2174 815.5,-2149 534.5,-2149"/> +<text text-anchor="start" x="539.5" y="-2158.8" font-family="Helvetica,sans-Serif" font-size="14.00">version_name</text> +<text text-anchor="start" x="636.5" y="-2158.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="641.5" y="-2158.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="534.5,-2124 534.5,-2149 815.5,-2149 815.5,-2124 534.5,-2124"/> +<text text-anchor="start" x="539.5" y="-2133.8" font-family="Helvetica,sans-Serif" font-size="14.00">version_number</text> +<text text-anchor="start" x="652.5" y="-2133.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="657.5" y="-2133.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="734.5" y="-2133.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +</g> +<!-- dag--dag_version --> +<g id="edge16" class="edge"> +<title>dag--dag_version</title> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M420.4,-2354.28C443.68,-2331.25 468.04,-2309.38 493,-2290 503.21,-2282.07 514.24,-2274.58 525.64,-2267.56"/> +<text text-anchor="start" x="494.64" y="-2256.36" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="420.4" y="-2343.08" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag_tag --> -<g id="node23" class="node"> +<g id="node22" class="node"> <title>dag_tag</title> -<polygon fill="none" stroke="black" points="974,-3851 974,-3879 1232,-3879 1232,-3851 974,-3851"/> -<text text-anchor="start" x="1067.5" y="-3862.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_tag</text> -<polygon fill="none" stroke="black" points="974,-3826 974,-3851 1232,-3851 1232,-3826 974,-3826"/> -<text text-anchor="start" x="979" y="-3835.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1025" y="-3835.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1030" y="-3835.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1151" y="-3835.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="974,-3801 974,-3826 1232,-3826 1232,-3801 974,-3801"/> -<text text-anchor="start" x="979" y="-3810.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">name</text> -<text text-anchor="start" x="1019" y="-3810.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1024" y="-3810.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(100)]</text> -<text text-anchor="start" x="1145" y="-3810.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="545.5,-2483 545.5,-2511 803.5,-2511 803.5,-2483 545.5,-2483"/> +<text text-anchor="start" x="639" y="-2494.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_tag</text> +<polygon fill="none" stroke="black" points="545.5,-2458 545.5,-2483 803.5,-2483 803.5,-2458 545.5,-2458"/> +<text text-anchor="start" x="550.5" y="-2467.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> +<text text-anchor="start" x="596.5" y="-2467.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="601.5" y="-2467.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="722.5" y="-2467.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="545.5,-2433 545.5,-2458 803.5,-2458 803.5,-2433 545.5,-2433"/> +<text text-anchor="start" x="550.5" y="-2442.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">name</text> +<text text-anchor="start" x="590.5" y="-2442.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="595.5" y="-2442.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(100)]</text> +<text text-anchor="start" x="716.5" y="-2442.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- dag--dag_tag --> -<g id="edge16" class="edge"> +<g id="edge17" class="edge"> <title>dag--dag_tag</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M856.19,-3888.08C893.42,-3880.8 931.31,-3873.39 965.83,-3866.63"/> -<text text-anchor="start" x="934.83" y="-3855.43" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="856.19" y="-3876.88" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M420.1,-2547.41C444.62,-2539.35 469.33,-2531.39 493,-2524 507.32,-2519.53 522.32,-2514.98 537.3,-2510.54"/> +<text text-anchor="start" x="506.3" y="-2499.34" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="420.1" y="-2536.21" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag_owner_attributes --> -<g id="node24" class="node"> +<g id="node23" class="node"> <title>dag_owner_attributes</title> -<polygon fill="none" stroke="black" points="974,-3746 974,-3774 1232,-3774 1232,-3746 974,-3746"/> -<text text-anchor="start" x="1005" y="-3757.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_owner_attributes</text> -<polygon fill="none" stroke="black" points="974,-3721 974,-3746 1232,-3746 1232,-3721 974,-3721"/> -<text text-anchor="start" x="979" y="-3730.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1025" y="-3730.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1030" y="-3730.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1151" y="-3730.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="974,-3696 974,-3721 1232,-3721 1232,-3696 974,-3696"/> -<text text-anchor="start" x="979" y="-3705.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">owner</text> -<text text-anchor="start" x="1022" y="-3705.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1027" y="-3705.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> -<text text-anchor="start" x="1148" y="-3705.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="974,-3671 974,-3696 1232,-3696 1232,-3671 974,-3671"/> -<text text-anchor="start" x="979" y="-3680.8" font-family="Helvetica,sans-Serif" font-size="14.00">link</text> -<text text-anchor="start" x="1004" y="-3680.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1009" y="-3680.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> -<text text-anchor="start" x="1130" y="-3680.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="545.5,-2378 545.5,-2406 803.5,-2406 803.5,-2378 545.5,-2378"/> +<text text-anchor="start" x="576.5" y="-2389.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_owner_attributes</text> +<polygon fill="none" stroke="black" points="545.5,-2353 545.5,-2378 803.5,-2378 803.5,-2353 545.5,-2353"/> +<text text-anchor="start" x="550.5" y="-2362.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> +<text text-anchor="start" x="596.5" y="-2362.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="601.5" y="-2362.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="722.5" y="-2362.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="545.5,-2328 545.5,-2353 803.5,-2353 803.5,-2328 545.5,-2328"/> +<text text-anchor="start" x="550.5" y="-2337.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">owner</text> +<text text-anchor="start" x="593.5" y="-2337.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="598.5" y="-2337.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> +<text text-anchor="start" x="719.5" y="-2337.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="545.5,-2303 545.5,-2328 803.5,-2328 803.5,-2303 545.5,-2303"/> +<text text-anchor="start" x="550.5" y="-2312.8" font-family="Helvetica,sans-Serif" font-size="14.00">link</text> +<text text-anchor="start" x="575.5" y="-2312.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="580.5" y="-2312.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(500)]</text> +<text text-anchor="start" x="701.5" y="-2312.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- dag--dag_owner_attributes --> -<g id="edge17" class="edge"> +<g id="edge18" class="edge"> <title>dag--dag_owner_attributes</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M856.06,-3821.46C880.48,-3809.8 905.17,-3798.4 929,-3788 940.91,-3782.8 953.38,-3777.62 965.94,-3772.58"/> -<text text-anchor="start" x="934.94" y="-3761.38" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="856.06" y="-3810.26" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M420.39,-2462.17C444.32,-2446.99 468.82,-2432.59 493,-2420 507.03,-2412.69 522.07,-2405.86 537.26,-2399.58"/> +<text text-anchor="start" x="506.26" y="-2388.38" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="420.39" y="-2450.97" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag_warning --> -<g id="node25" class="node"> +<g id="node24" class="node"> <title>dag_warning</title> -<polygon fill="none" stroke="black" points="955,-3617 955,-3645 1252,-3645 1252,-3617 955,-3617"/> -<text text-anchor="start" x="1046" y="-3628.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_warning</text> -<polygon fill="none" stroke="black" points="955,-3592 955,-3617 1252,-3617 1252,-3592 955,-3592"/> -<text text-anchor="start" x="960" y="-3601.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1006" y="-3601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1011" y="-3601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1132" y="-3601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="955,-3567 955,-3592 1252,-3592 1252,-3567 955,-3567"/> -<text text-anchor="start" x="960" y="-3576.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">warning_type</text> -<text text-anchor="start" x="1054" y="-3576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1059" y="-3576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(50)]</text> -<text text-anchor="start" x="1171" y="-3576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="955,-3542 955,-3567 1252,-3567 1252,-3542 955,-3542"/> -<text text-anchor="start" x="960" y="-3551.8" font-family="Helvetica,sans-Serif" font-size="14.00">message</text> -<text text-anchor="start" x="1023" y="-3551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1028" y="-3551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<text text-anchor="start" x="1078" y="-3551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="955,-3517 955,-3542 1252,-3542 1252,-3517 955,-3517"/> -<text text-anchor="start" x="960" y="-3526.8" font-family="Helvetica,sans-Serif" font-size="14.00">timestamp</text> -<text text-anchor="start" x="1035" y="-3526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1040" y="-3526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1136" y="-3526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="526.5,-2070 526.5,-2098 823.5,-2098 823.5,-2070 526.5,-2070"/> +<text text-anchor="start" x="617.5" y="-2081.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_warning</text> +<polygon fill="none" stroke="black" points="526.5,-2045 526.5,-2070 823.5,-2070 823.5,-2045 526.5,-2045"/> +<text text-anchor="start" x="531.5" y="-2054.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> +<text text-anchor="start" x="577.5" y="-2054.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="582.5" y="-2054.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="703.5" y="-2054.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="526.5,-2020 526.5,-2045 823.5,-2045 823.5,-2020 526.5,-2020"/> +<text text-anchor="start" x="531.5" y="-2029.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">warning_type</text> +<text text-anchor="start" x="625.5" y="-2029.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="630.5" y="-2029.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(50)]</text> +<text text-anchor="start" x="742.5" y="-2029.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="526.5,-1995 526.5,-2020 823.5,-2020 823.5,-1995 526.5,-1995"/> +<text text-anchor="start" x="531.5" y="-2004.8" font-family="Helvetica,sans-Serif" font-size="14.00">message</text> +<text text-anchor="start" x="594.5" y="-2004.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="599.5" y="-2004.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<text text-anchor="start" x="649.5" y="-2004.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="526.5,-1970 526.5,-1995 823.5,-1995 823.5,-1970 526.5,-1970"/> +<text text-anchor="start" x="531.5" y="-1979.8" font-family="Helvetica,sans-Serif" font-size="14.00">timestamp</text> +<text text-anchor="start" x="606.5" y="-1979.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="611.5" y="-1979.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="707.5" y="-1979.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- dag--dag_warning --> -<g id="edge18" class="edge"> +<g id="edge19" class="edge"> <title>dag--dag_warning</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M856.27,-3713.23C879.89,-3693.41 904.35,-3674.58 929,-3658 934.59,-3654.24 940.4,-3650.59 946.35,-3647.05"/> -<text text-anchor="start" x="915.35" y="-3635.85" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="856.27" y="-3702.03" font-family="Times,serif" font-size="14.00">1</text> -</g> -<!-- log_template --> -<g id="node26" class="node"> -<title>log_template</title> -<polygon fill="none" stroke="black" points="51.5,-2837 51.5,-2865 311.5,-2865 311.5,-2837 51.5,-2837"/> -<text text-anchor="start" x="123.5" y="-2848.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">log_template</text> -<polygon fill="none" stroke="black" points="51.5,-2812 51.5,-2837 311.5,-2837 311.5,-2812 51.5,-2812"/> -<text text-anchor="start" x="56.5" y="-2821.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="69.5" y="-2821.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="74.5" y="-2821.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="151.5" y="-2821.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="51.5,-2787 51.5,-2812 311.5,-2812 311.5,-2787 51.5,-2787"/> -<text text-anchor="start" x="56.5" y="-2796.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> -<text text-anchor="start" x="129.5" y="-2796.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="134.5" y="-2796.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="230.5" y="-2796.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="51.5,-2762 51.5,-2787 311.5,-2787 311.5,-2762 51.5,-2762"/> -<text text-anchor="start" x="56.5" y="-2771.8" font-family="Helvetica,sans-Serif" font-size="14.00">elasticsearch_id</text> -<text text-anchor="start" x="166.5" y="-2771.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="171.5" y="-2771.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<text text-anchor="start" x="221.5" y="-2771.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="51.5,-2737 51.5,-2762 311.5,-2762 311.5,-2737 51.5,-2737"/> -<text text-anchor="start" x="56.5" y="-2746.8" font-family="Helvetica,sans-Serif" font-size="14.00">filename</text> -<text text-anchor="start" x="117.5" y="-2746.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="122.5" y="-2746.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<text text-anchor="start" x="172.5" y="-2746.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M338.09,-2313.38C379.31,-2240.17 430.81,-2167.06 493,-2111 500.59,-2104.16 508.88,-2097.86 517.6,-2092.08"/> +<text text-anchor="start" x="486.6" y="-2080.88" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="328.09" y="-2302.18" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag_run --> -<g id="node27" class="node"> +<g id="node25" class="node"> <title>dag_run</title> -<polygon fill="none" stroke="black" points="505,-3049 505,-3077 788,-3077 788,-3049 505,-3049"/> -<text text-anchor="start" x="610" y="-3060.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_run</text> -<polygon fill="none" stroke="black" points="505,-3024 505,-3049 788,-3049 788,-3024 505,-3024"/> -<text text-anchor="start" x="510" y="-3033.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="523" y="-3033.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="528" y="-3033.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="605" y="-3033.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="505,-2999 505,-3024 788,-3024 788,-2999 505,-2999"/> -<text text-anchor="start" x="510" y="-3008.8" font-family="Helvetica,sans-Serif" font-size="14.00">backfill_id</text> -<text text-anchor="start" x="580" y="-3008.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="585" y="-3008.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="505,-2974 505,-2999 788,-2999 788,-2974 505,-2974"/> -<text text-anchor="start" x="510" y="-2983.8" font-family="Helvetica,sans-Serif" font-size="14.00">clear_number</text> -<text text-anchor="start" x="606" y="-2983.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="611" y="-2983.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="688" y="-2983.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="505,-2949 505,-2974 788,-2974 788,-2949 505,-2949"/> -<text text-anchor="start" x="510" y="-2958.8" font-family="Helvetica,sans-Serif" font-size="14.00">conf</text> -<text text-anchor="start" x="541" y="-2958.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="546" y="-2958.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> -<polygon fill="none" stroke="black" points="505,-2924 505,-2949 788,-2949 788,-2924 505,-2924"/> -<text text-anchor="start" x="510" y="-2933.8" font-family="Helvetica,sans-Serif" font-size="14.00">creating_job_id</text> -<text text-anchor="start" x="614" y="-2933.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="619" y="-2933.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="505,-2899 505,-2924 788,-2924 788,-2899 505,-2899"/> -<text text-anchor="start" x="510" y="-2908.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_hash</text> -<text text-anchor="start" x="577" y="-2908.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="582" y="-2908.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(32)]</text> -<polygon fill="none" stroke="black" points="505,-2874 505,-2899 788,-2899 788,-2874 505,-2874"/> -<text text-anchor="start" x="510" y="-2883.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> -<text text-anchor="start" x="556" y="-2883.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="561" y="-2883.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="682" y="-2883.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="505,-2849 505,-2874 788,-2874 788,-2849 505,-2849"/> -<text text-anchor="start" x="510" y="-2858.8" font-family="Helvetica,sans-Serif" font-size="14.00">data_interval_end</text> -<text text-anchor="start" x="634" y="-2858.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="639" y="-2858.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="505,-2824 505,-2849 788,-2849 788,-2824 505,-2824"/> -<text text-anchor="start" x="510" y="-2833.8" font-family="Helvetica,sans-Serif" font-size="14.00">data_interval_start</text> -<text text-anchor="start" x="640" y="-2833.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="645" y="-2833.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="505,-2799 505,-2824 788,-2824 788,-2799 505,-2799"/> -<text text-anchor="start" x="510" y="-2808.8" font-family="Helvetica,sans-Serif" font-size="14.00">end_date</text> -<text text-anchor="start" x="574" y="-2808.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="579" y="-2808.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="505,-2774 505,-2799 788,-2799 788,-2774 505,-2774"/> -<text text-anchor="start" x="510" y="-2783.8" font-family="Helvetica,sans-Serif" font-size="14.00">external_trigger</text> -<text text-anchor="start" x="621" y="-2783.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="626" y="-2783.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<polygon fill="none" stroke="black" points="505,-2749 505,-2774 788,-2774 788,-2749 505,-2749"/> -<text text-anchor="start" x="510" y="-2758.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_scheduling_decision</text> -<text text-anchor="start" x="682" y="-2758.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="687" y="-2758.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="505,-2724 505,-2749 788,-2749 788,-2724 505,-2724"/> -<text text-anchor="start" x="510" y="-2733.8" font-family="Helvetica,sans-Serif" font-size="14.00">log_template_id</text> -<text text-anchor="start" x="619" y="-2733.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="624" y="-2733.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="505,-2699 505,-2724 788,-2724 788,-2699 505,-2699"/> -<text text-anchor="start" x="510" y="-2708.8" font-family="Helvetica,sans-Serif" font-size="14.00">logical_date</text> -<text text-anchor="start" x="592" y="-2708.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="597" y="-2708.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="693" y="-2708.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="505,-2674 505,-2699 788,-2699 788,-2674 505,-2674"/> -<text text-anchor="start" x="510" y="-2683.8" font-family="Helvetica,sans-Serif" font-size="14.00">queued_at</text> -<text text-anchor="start" x="583" y="-2683.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="588" y="-2683.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="505,-2649 505,-2674 788,-2674 788,-2649 505,-2649"/> -<text text-anchor="start" x="510" y="-2658.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> -<text text-anchor="start" x="554" y="-2658.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="559" y="-2658.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="680" y="-2658.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="505,-2624 505,-2649 788,-2649 788,-2624 505,-2624"/> -<text text-anchor="start" x="510" y="-2633.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_type</text> -<text text-anchor="start" x="572" y="-2633.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="577" y="-2633.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(50)]</text> -<text text-anchor="start" x="689" y="-2633.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="505,-2599 505,-2624 788,-2624 788,-2599 505,-2599"/> -<text text-anchor="start" x="510" y="-2608.8" font-family="Helvetica,sans-Serif" font-size="14.00">start_date</text> -<text text-anchor="start" x="580" y="-2608.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="585" y="-2608.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="505,-2574 505,-2599 788,-2599 788,-2574 505,-2574"/> -<text text-anchor="start" x="510" y="-2583.8" font-family="Helvetica,sans-Serif" font-size="14.00">state</text> -<text text-anchor="start" x="545" y="-2583.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="550" y="-2583.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(50)]</text> -<polygon fill="none" stroke="black" points="505,-2549 505,-2574 788,-2574 788,-2549 505,-2549"/> -<text text-anchor="start" x="510" y="-2558.8" font-family="Helvetica,sans-Serif" font-size="14.00">triggered_by</text> -<text text-anchor="start" x="598" y="-2558.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="603" y="-2558.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(50)]</text> -<polygon fill="none" stroke="black" points="505,-2524 505,-2549 788,-2549 788,-2524 505,-2524"/> -<text text-anchor="start" x="510" y="-2533.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> -<text text-anchor="start" x="589" y="-2533.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="594" y="-2533.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="940,-3010 940,-3038 1223,-3038 1223,-3010 940,-3010"/> +<text text-anchor="start" x="1045" y="-3021.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_run</text> +<polygon fill="none" stroke="black" points="940,-2985 940,-3010 1223,-3010 1223,-2985 940,-2985"/> +<text text-anchor="start" x="945" y="-2994.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="958" y="-2994.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="963" y="-2994.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1040" y="-2994.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="940,-2960 940,-2985 1223,-2985 1223,-2960 940,-2960"/> +<text text-anchor="start" x="945" y="-2969.8" font-family="Helvetica,sans-Serif" font-size="14.00">backfill_id</text> +<text text-anchor="start" x="1015" y="-2969.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1020" y="-2969.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="940,-2935 940,-2960 1223,-2960 1223,-2935 940,-2935"/> +<text text-anchor="start" x="945" y="-2944.8" font-family="Helvetica,sans-Serif" font-size="14.00">clear_number</text> +<text text-anchor="start" x="1041" y="-2944.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1046" y="-2944.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1123" y="-2944.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="940,-2910 940,-2935 1223,-2935 1223,-2910 940,-2910"/> +<text text-anchor="start" x="945" y="-2919.8" font-family="Helvetica,sans-Serif" font-size="14.00">conf</text> +<text text-anchor="start" x="976" y="-2919.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="981" y="-2919.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> +<polygon fill="none" stroke="black" points="940,-2885 940,-2910 1223,-2910 1223,-2885 940,-2885"/> +<text text-anchor="start" x="945" y="-2894.8" font-family="Helvetica,sans-Serif" font-size="14.00">creating_job_id</text> +<text text-anchor="start" x="1049" y="-2894.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1054" y="-2894.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="940,-2860 940,-2885 1223,-2885 1223,-2860 940,-2860"/> +<text text-anchor="start" x="945" y="-2869.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> +<text text-anchor="start" x="991" y="-2869.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="996" y="-2869.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1117" y="-2869.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="940,-2835 940,-2860 1223,-2860 1223,-2835 940,-2835"/> +<text text-anchor="start" x="945" y="-2844.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_version_id</text> +<text text-anchor="start" x="1049" y="-2844.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1054" y="-2844.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [UUID]</text> +<polygon fill="none" stroke="black" points="940,-2810 940,-2835 1223,-2835 1223,-2810 940,-2810"/> +<text text-anchor="start" x="945" y="-2819.8" font-family="Helvetica,sans-Serif" font-size="14.00">data_interval_end</text> +<text text-anchor="start" x="1069" y="-2819.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1074" y="-2819.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="940,-2785 940,-2810 1223,-2810 1223,-2785 940,-2785"/> +<text text-anchor="start" x="945" y="-2794.8" font-family="Helvetica,sans-Serif" font-size="14.00">data_interval_start</text> +<text text-anchor="start" x="1075" y="-2794.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1080" y="-2794.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="940,-2760 940,-2785 1223,-2785 1223,-2760 940,-2760"/> +<text text-anchor="start" x="945" y="-2769.8" font-family="Helvetica,sans-Serif" font-size="14.00">end_date</text> +<text text-anchor="start" x="1009" y="-2769.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1014" y="-2769.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="940,-2735 940,-2760 1223,-2760 1223,-2735 940,-2735"/> +<text text-anchor="start" x="945" y="-2744.8" font-family="Helvetica,sans-Serif" font-size="14.00">external_trigger</text> +<text text-anchor="start" x="1056" y="-2744.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1061" y="-2744.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<polygon fill="none" stroke="black" points="940,-2710 940,-2735 1223,-2735 1223,-2710 940,-2710"/> +<text text-anchor="start" x="945" y="-2719.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_scheduling_decision</text> +<text text-anchor="start" x="1117" y="-2719.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1122" y="-2719.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="940,-2685 940,-2710 1223,-2710 1223,-2685 940,-2685"/> +<text text-anchor="start" x="945" y="-2694.8" font-family="Helvetica,sans-Serif" font-size="14.00">log_template_id</text> +<text text-anchor="start" x="1054" y="-2694.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1059" y="-2694.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="940,-2660 940,-2685 1223,-2685 1223,-2660 940,-2660"/> +<text text-anchor="start" x="945" y="-2669.8" font-family="Helvetica,sans-Serif" font-size="14.00">logical_date</text> +<text text-anchor="start" x="1027" y="-2669.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1032" y="-2669.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1128" y="-2669.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="940,-2635 940,-2660 1223,-2660 1223,-2635 940,-2635"/> +<text text-anchor="start" x="945" y="-2644.8" font-family="Helvetica,sans-Serif" font-size="14.00">queued_at</text> +<text text-anchor="start" x="1018" y="-2644.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1023" y="-2644.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="940,-2610 940,-2635 1223,-2635 1223,-2610 940,-2610"/> +<text text-anchor="start" x="945" y="-2619.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> +<text text-anchor="start" x="989" y="-2619.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="994" y="-2619.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1115" y="-2619.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="940,-2585 940,-2610 1223,-2610 1223,-2585 940,-2585"/> +<text text-anchor="start" x="945" y="-2594.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_type</text> +<text text-anchor="start" x="1007" y="-2594.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1012" y="-2594.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(50)]</text> +<text text-anchor="start" x="1124" y="-2594.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="940,-2560 940,-2585 1223,-2585 1223,-2560 940,-2560"/> +<text text-anchor="start" x="945" y="-2569.8" font-family="Helvetica,sans-Serif" font-size="14.00">start_date</text> +<text text-anchor="start" x="1015" y="-2569.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1020" y="-2569.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="940,-2535 940,-2560 1223,-2560 1223,-2535 940,-2535"/> +<text text-anchor="start" x="945" y="-2544.8" font-family="Helvetica,sans-Serif" font-size="14.00">state</text> +<text text-anchor="start" x="980" y="-2544.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="985" y="-2544.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(50)]</text> +<polygon fill="none" stroke="black" points="940,-2510 940,-2535 1223,-2535 1223,-2510 940,-2510"/> +<text text-anchor="start" x="945" y="-2519.8" font-family="Helvetica,sans-Serif" font-size="14.00">triggered_by</text> +<text text-anchor="start" x="1033" y="-2519.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1038" y="-2519.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(50)]</text> +<polygon fill="none" stroke="black" points="940,-2485 940,-2510 1223,-2510 1223,-2485 940,-2485"/> +<text text-anchor="start" x="945" y="-2494.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> +<text text-anchor="start" x="1024" y="-2494.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1029" y="-2494.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +</g> +<!-- dag_version--dag_run --> +<g id="edge20" class="edge"> +<title>dag_version--dag_run</title> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M823.44,-2261.95C835.29,-2270.29 846.38,-2279.62 856,-2290 915.22,-2353.94 892.61,-2392.81 929,-2472 930.42,-2475.09 931.86,-2478.2 933.31,-2481.32"/> +<text text-anchor="start" x="902.31" y="-2470.12" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="823.44" y="-2250.75" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> -<!-- log_template--dag_run --> -<g id="edge19" class="edge"> -<title>log_template--dag_run</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M319.7,-2801C375.23,-2801 439.48,-2801 496.14,-2801"/> -<text text-anchor="start" x="465.14" y="-2789.8" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="319.7" y="-2789.8" font-family="Times,serif" font-size="14.00">{0,1}</text> +<!-- dag_code --> +<g id="node26" class="node"> +<title>dag_code</title> +<polygon fill="none" stroke="black" points="943,-1960 943,-1988 1220,-1988 1220,-1960 943,-1960"/> +<text text-anchor="start" x="1039" y="-1971.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_code</text> +<polygon fill="none" stroke="black" points="943,-1935 943,-1960 1220,-1960 1220,-1935 943,-1935"/> +<text text-anchor="start" x="948" y="-1944.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="961" y="-1944.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="966" y="-1944.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [UUID]</text> +<text text-anchor="start" x="1018" y="-1944.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="943,-1910 943,-1935 1220,-1935 1220,-1910 943,-1910"/> +<text text-anchor="start" x="948" y="-1919.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_version_id</text> +<text text-anchor="start" x="1052" y="-1919.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1057" y="-1919.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [UUID]</text> +<text text-anchor="start" x="1109" y="-1919.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="943,-1885 943,-1910 1220,-1910 1220,-1885 943,-1885"/> +<text text-anchor="start" x="948" y="-1894.8" font-family="Helvetica,sans-Serif" font-size="14.00">fileloc</text> +<text text-anchor="start" x="989" y="-1894.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="994" y="-1894.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<text text-anchor="start" x="1124" y="-1894.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="943,-1860 943,-1885 1220,-1885 1220,-1860 943,-1860"/> +<text text-anchor="start" x="948" y="-1869.8" font-family="Helvetica,sans-Serif" font-size="14.00">fileloc_hash</text> +<text text-anchor="start" x="1030" y="-1869.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1035" y="-1869.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BIGINT]</text> +<text text-anchor="start" x="1099" y="-1869.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="943,-1835 943,-1860 1220,-1860 1220,-1835 943,-1835"/> +<text text-anchor="start" x="948" y="-1844.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_updated</text> +<text text-anchor="start" x="1038" y="-1844.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1043" y="-1844.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1139" y="-1844.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="943,-1810 943,-1835 1220,-1835 1220,-1810 943,-1810"/> +<text text-anchor="start" x="948" y="-1819.8" font-family="Helvetica,sans-Serif" font-size="14.00">source_code</text> +<text text-anchor="start" x="1034" y="-1819.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1039" y="-1819.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<text text-anchor="start" x="1089" y="-1819.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +</g> +<!-- dag_version--dag_code --> +<g id="edge21" class="edge"> +<title>dag_version--dag_code</title> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M823.1,-2135.83C834.75,-2128.2 845.9,-2119.93 856,-2111 899.97,-2072.15 887.68,-2042.66 929,-2001 932.01,-1997.97 935.12,-1994.97 938.31,-1992.02"/> +<text text-anchor="start" x="907.31" y="-1995.82" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="823.1" y="-2124.63" font-family="Times,serif" font-size="14.00">1</text> </g> -<!-- dag_run--dagrun_asset_event --> -<g id="edge20" class="edge"> -<title>dag_run--dagrun_asset_event</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M737.98,-3081.58C775,-3158.83 825.22,-3236.32 892,-3291 915.02,-3309.85 943.75,-3322.49 972.49,-3330.97"/> -<text text-anchor="start" x="941.49" y="-3319.77" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="727.98" y="-3085.38" font-family="Times,serif" font-size="14.00">1</text> +<!-- serialized_dag --> +<g id="node27" class="node"> +<title>serialized_dag</title> +<polygon fill="none" stroke="black" points="943,-2214 943,-2242 1220,-2242 1220,-2214 943,-2214"/> +<text text-anchor="start" x="1018" y="-2225.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">serialized_dag</text> +<polygon fill="none" stroke="black" points="943,-2189 943,-2214 1220,-2214 1220,-2189 943,-2189"/> +<text text-anchor="start" x="948" y="-2198.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="961" y="-2198.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="966" y="-2198.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [UUID]</text> +<text text-anchor="start" x="1018" y="-2198.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="943,-2164 943,-2189 1220,-2189 1220,-2164 943,-2164"/> +<text text-anchor="start" x="948" y="-2173.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_hash</text> +<text text-anchor="start" x="1015" y="-2173.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1020" y="-2173.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(32)]</text> +<text text-anchor="start" x="1132" y="-2173.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="943,-2139 943,-2164 1220,-2164 1220,-2139 943,-2139"/> +<text text-anchor="start" x="948" y="-2148.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> +<text text-anchor="start" x="994" y="-2148.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="999" y="-2148.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1120" y="-2148.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="943,-2114 943,-2139 1220,-2139 1220,-2114 943,-2114"/> +<text text-anchor="start" x="948" y="-2123.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_version_id</text> +<text text-anchor="start" x="1052" y="-2123.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1057" y="-2123.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [UUID]</text> +<text text-anchor="start" x="1109" y="-2123.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="943,-2089 943,-2114 1220,-2114 1220,-2089 943,-2089"/> +<text text-anchor="start" x="948" y="-2098.8" font-family="Helvetica,sans-Serif" font-size="14.00">data</text> +<text text-anchor="start" x="979" y="-2098.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="984" y="-2098.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<polygon fill="none" stroke="black" points="943,-2064 943,-2089 1220,-2089 1220,-2064 943,-2064"/> +<text text-anchor="start" x="948" y="-2073.8" font-family="Helvetica,sans-Serif" font-size="14.00">data_compressed</text> +<text text-anchor="start" x="1070" y="-2073.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1075" y="-2073.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> +<polygon fill="none" stroke="black" points="943,-2039 943,-2064 1220,-2064 1220,-2039 943,-2039"/> +<text text-anchor="start" x="948" y="-2048.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_updated</text> +<text text-anchor="start" x="1038" y="-2048.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1043" y="-2048.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1139" y="-2048.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="943,-2014 943,-2039 1220,-2039 1220,-2014 943,-2014"/> +<text text-anchor="start" x="948" y="-2023.8" font-family="Helvetica,sans-Serif" font-size="14.00">processor_subdir</text> +<text text-anchor="start" x="1067" y="-2023.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1072" y="-2023.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +</g> +<!-- dag_version--serialized_dag --> +<g id="edge22" class="edge"> +<title>dag_version--serialized_dag</title> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M823.25,-2174.34C859.47,-2167.8 898.22,-2160.81 934.34,-2154.29"/> +<text text-anchor="start" x="903.34" y="-2143.09" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="823.25" y="-2163.14" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance --> <g id="node28" class="node"> <title>task_instance</title> -<polygon fill="none" stroke="black" points="949,-3199 949,-3227 1257,-3227 1257,-3199 949,-3199"/> -<text text-anchor="start" x="1041.5" y="-3210.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_instance</text> -<polygon fill="none" stroke="black" points="949,-3174 949,-3199 1257,-3199 1257,-3174 949,-3174"/> -<text text-anchor="start" x="954" y="-3183.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="967" y="-3183.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="972" y="-3183.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [UUID]</text> -<text text-anchor="start" x="1024" y="-3183.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="949,-3149 949,-3174 1257,-3174 1257,-3149 949,-3149"/> -<text text-anchor="start" x="954" y="-3158.8" font-family="Helvetica,sans-Serif" font-size="14.00">custom_operator_name</text> -<text text-anchor="start" x="1117" y="-3158.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1122" y="-3158.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="949,-3124 949,-3149 1257,-3149 1257,-3124 949,-3124"/> -<text text-anchor="start" x="954" y="-3133.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1000" y="-3133.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1005" y="-3133.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1126" y="-3133.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="949,-3099 949,-3124 1257,-3124 1257,-3099 949,-3099"/> -<text text-anchor="start" x="954" y="-3108.8" font-family="Helvetica,sans-Serif" font-size="14.00">duration</text> -<text text-anchor="start" x="1013" y="-3108.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1018" y="-3108.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [DOUBLE_PRECISION]</text> -<polygon fill="none" stroke="black" points="949,-3074 949,-3099 1257,-3099 1257,-3074 949,-3074"/> -<text text-anchor="start" x="954" y="-3083.8" font-family="Helvetica,sans-Serif" font-size="14.00">end_date</text> -<text text-anchor="start" x="1018" y="-3083.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1023" y="-3083.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="949,-3049 949,-3074 1257,-3074 1257,-3049 949,-3049"/> -<text text-anchor="start" x="954" y="-3058.8" font-family="Helvetica,sans-Serif" font-size="14.00">executor</text> -<text text-anchor="start" x="1015" y="-3058.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1020" y="-3058.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="949,-3024 949,-3049 1257,-3049 1257,-3024 949,-3024"/> -<text text-anchor="start" x="954" y="-3033.8" font-family="Helvetica,sans-Serif" font-size="14.00">executor_config</text> -<text text-anchor="start" x="1064" y="-3033.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1069" y="-3033.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> -<polygon fill="none" stroke="black" points="949,-2999 949,-3024 1257,-3024 1257,-2999 949,-2999"/> -<text text-anchor="start" x="954" y="-3008.8" font-family="Helvetica,sans-Serif" font-size="14.00">external_executor_id</text> -<text text-anchor="start" x="1097" y="-3008.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1102" y="-3008.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="949,-2974 949,-2999 1257,-2999 1257,-2974 949,-2974"/> -<text text-anchor="start" x="954" y="-2983.8" font-family="Helvetica,sans-Serif" font-size="14.00">hostname</text> -<text text-anchor="start" x="1024" y="-2983.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1029" y="-2983.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="949,-2949 949,-2974 1257,-2974 1257,-2949 949,-2949"/> -<text text-anchor="start" x="954" y="-2958.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_heartbeat_at</text> -<text text-anchor="start" x="1074" y="-2958.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1079" y="-2958.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="949,-2924 949,-2949 1257,-2949 1257,-2924 949,-2924"/> -<text text-anchor="start" x="954" y="-2933.8" font-family="Helvetica,sans-Serif" font-size="14.00">map_index</text> -<text text-anchor="start" x="1030" y="-2933.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1035" y="-2933.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1112" y="-2933.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="949,-2899 949,-2924 1257,-2924 1257,-2899 949,-2899"/> -<text text-anchor="start" x="954" y="-2908.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_tries</text> -<text text-anchor="start" x="1022" y="-2908.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1027" y="-2908.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="949,-2874 949,-2899 1257,-2899 1257,-2874 949,-2874"/> -<text text-anchor="start" x="954" y="-2883.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_kwargs</text> -<text text-anchor="start" x="1042" y="-2883.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1047" y="-2883.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<polygon fill="none" stroke="black" points="949,-2849 949,-2874 1257,-2874 1257,-2849 949,-2849"/> -<text text-anchor="start" x="954" y="-2858.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_method</text> -<text text-anchor="start" x="1045" y="-2858.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1050" y="-2858.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="949,-2824 949,-2849 1257,-2849 1257,-2824 949,-2824"/> -<text text-anchor="start" x="954" y="-2833.8" font-family="Helvetica,sans-Serif" font-size="14.00">operator</text> -<text text-anchor="start" x="1014" y="-2833.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1019" y="-2833.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="949,-2799 949,-2824 1257,-2824 1257,-2799 949,-2799"/> -<text text-anchor="start" x="954" y="-2808.8" font-family="Helvetica,sans-Serif" font-size="14.00">pid</text> -<text text-anchor="start" x="976" y="-2808.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="981" y="-2808.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="949,-2774 949,-2799 1257,-2799 1257,-2774 949,-2774"/> -<text text-anchor="start" x="954" y="-2783.8" font-family="Helvetica,sans-Serif" font-size="14.00">pool</text> -<text text-anchor="start" x="984" y="-2783.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="989" y="-2783.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<text text-anchor="start" x="1110" y="-2783.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="949,-2749 949,-2774 1257,-2774 1257,-2749 949,-2749"/> -<text text-anchor="start" x="954" y="-2758.8" font-family="Helvetica,sans-Serif" font-size="14.00">pool_slots</text> -<text text-anchor="start" x="1023" y="-2758.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1028" y="-2758.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1105" y="-2758.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="949,-2724 949,-2749 1257,-2749 1257,-2724 949,-2724"/> -<text text-anchor="start" x="954" y="-2733.8" font-family="Helvetica,sans-Serif" font-size="14.00">priority_weight</text> -<text text-anchor="start" x="1058" y="-2733.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1063" y="-2733.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="949,-2699 949,-2724 1257,-2724 1257,-2699 949,-2699"/> -<text text-anchor="start" x="954" y="-2708.8" font-family="Helvetica,sans-Serif" font-size="14.00">queue</text> -<text text-anchor="start" x="998" y="-2708.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1003" y="-2708.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<polygon fill="none" stroke="black" points="949,-2674 949,-2699 1257,-2699 1257,-2674 949,-2674"/> -<text text-anchor="start" x="954" y="-2683.8" font-family="Helvetica,sans-Serif" font-size="14.00">queued_by_job_id</text> -<text text-anchor="start" x="1078" y="-2683.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1083" y="-2683.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="949,-2649 949,-2674 1257,-2674 1257,-2649 949,-2649"/> -<text text-anchor="start" x="954" y="-2658.8" font-family="Helvetica,sans-Serif" font-size="14.00">queued_dttm</text> -<text text-anchor="start" x="1047" y="-2658.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1052" y="-2658.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="949,-2624 949,-2649 1257,-2649 1257,-2624 949,-2624"/> -<text text-anchor="start" x="954" y="-2633.8" font-family="Helvetica,sans-Serif" font-size="14.00">rendered_map_index</text> -<text text-anchor="start" x="1099" y="-2633.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1104" y="-2633.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="949,-2599 949,-2624 1257,-2624 1257,-2599 949,-2599"/> -<text text-anchor="start" x="954" y="-2608.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> -<text text-anchor="start" x="998" y="-2608.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1003" y="-2608.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1124" y="-2608.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="949,-2574 949,-2599 1257,-2599 1257,-2574 949,-2574"/> -<text text-anchor="start" x="954" y="-2583.8" font-family="Helvetica,sans-Serif" font-size="14.00">start_date</text> -<text text-anchor="start" x="1024" y="-2583.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1029" y="-2583.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="949,-2549 949,-2574 1257,-2574 1257,-2549 949,-2549"/> -<text text-anchor="start" x="954" y="-2558.8" font-family="Helvetica,sans-Serif" font-size="14.00">state</text> -<text text-anchor="start" x="989" y="-2558.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="994" y="-2558.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(20)]</text> -<polygon fill="none" stroke="black" points="949,-2524 949,-2549 1257,-2549 1257,-2524 949,-2524"/> -<text text-anchor="start" x="954" y="-2533.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_display_name</text> -<text text-anchor="start" x="1086" y="-2533.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1091" y="-2533.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> -<polygon fill="none" stroke="black" points="949,-2499 949,-2524 1257,-2524 1257,-2499 949,-2499"/> -<text text-anchor="start" x="954" y="-2508.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_id</text> -<text text-anchor="start" x="1003" y="-2508.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1008" y="-2508.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1129" y="-2508.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="949,-2474 949,-2499 1257,-2499 1257,-2474 949,-2474"/> -<text text-anchor="start" x="954" y="-2483.8" font-family="Helvetica,sans-Serif" font-size="14.00">trigger_id</text> -<text text-anchor="start" x="1021" y="-2483.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1026" y="-2483.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="949,-2449 949,-2474 1257,-2474 1257,-2449 949,-2449"/> -<text text-anchor="start" x="954" y="-2458.8" font-family="Helvetica,sans-Serif" font-size="14.00">trigger_timeout</text> -<text text-anchor="start" x="1062" y="-2458.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1067" y="-2458.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="949,-2424 949,-2449 1257,-2449 1257,-2424 949,-2424"/> -<text text-anchor="start" x="954" y="-2433.8" font-family="Helvetica,sans-Serif" font-size="14.00">try_number</text> -<text text-anchor="start" x="1036" y="-2433.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1041" y="-2433.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="949,-2399 949,-2424 1257,-2424 1257,-2399 949,-2399"/> -<text text-anchor="start" x="954" y="-2408.8" font-family="Helvetica,sans-Serif" font-size="14.00">unixname</text> -<text text-anchor="start" x="1024" y="-2408.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1029" y="-2408.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="949,-2374 949,-2399 1257,-2399 1257,-2374 949,-2374"/> -<text text-anchor="start" x="954" y="-2383.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> -<text text-anchor="start" x="1033" y="-2383.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1038" y="-2383.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1314,-2692 1314,-2720 1622,-2720 1622,-2692 1314,-2692"/> +<text text-anchor="start" x="1406.5" y="-2703.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_instance</text> +<polygon fill="none" stroke="black" points="1314,-2667 1314,-2692 1622,-2692 1622,-2667 1314,-2667"/> +<text text-anchor="start" x="1319" y="-2676.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="1332" y="-2676.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1337" y="-2676.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [UUID]</text> +<text text-anchor="start" x="1389" y="-2676.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1314,-2642 1314,-2667 1622,-2667 1622,-2642 1314,-2642"/> +<text text-anchor="start" x="1319" y="-2651.8" font-family="Helvetica,sans-Serif" font-size="14.00">custom_operator_name</text> +<text text-anchor="start" x="1482" y="-2651.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1487" y="-2651.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1314,-2617 1314,-2642 1622,-2642 1622,-2617 1314,-2617"/> +<text text-anchor="start" x="1319" y="-2626.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> +<text text-anchor="start" x="1365" y="-2626.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1370" y="-2626.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1491" y="-2626.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1314,-2592 1314,-2617 1622,-2617 1622,-2592 1314,-2592"/> +<text text-anchor="start" x="1319" y="-2601.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_version_id</text> +<text text-anchor="start" x="1423" y="-2601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1428" y="-2601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [UUID]</text> +<polygon fill="none" stroke="black" points="1314,-2567 1314,-2592 1622,-2592 1622,-2567 1314,-2567"/> +<text text-anchor="start" x="1319" y="-2576.8" font-family="Helvetica,sans-Serif" font-size="14.00">duration</text> +<text text-anchor="start" x="1378" y="-2576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1383" y="-2576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [DOUBLE_PRECISION]</text> +<polygon fill="none" stroke="black" points="1314,-2542 1314,-2567 1622,-2567 1622,-2542 1314,-2542"/> +<text text-anchor="start" x="1319" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00">end_date</text> +<text text-anchor="start" x="1383" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1388" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1314,-2517 1314,-2542 1622,-2542 1622,-2517 1314,-2517"/> +<text text-anchor="start" x="1319" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00">executor</text> +<text text-anchor="start" x="1380" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1385" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1314,-2492 1314,-2517 1622,-2517 1622,-2492 1314,-2492"/> +<text text-anchor="start" x="1319" y="-2501.8" font-family="Helvetica,sans-Serif" font-size="14.00">executor_config</text> +<text text-anchor="start" x="1429" y="-2501.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1434" y="-2501.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> +<polygon fill="none" stroke="black" points="1314,-2467 1314,-2492 1622,-2492 1622,-2467 1314,-2467"/> +<text text-anchor="start" x="1319" y="-2476.8" font-family="Helvetica,sans-Serif" font-size="14.00">external_executor_id</text> +<text text-anchor="start" x="1462" y="-2476.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1467" y="-2476.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="1314,-2442 1314,-2467 1622,-2467 1622,-2442 1314,-2442"/> +<text text-anchor="start" x="1319" y="-2451.8" font-family="Helvetica,sans-Serif" font-size="14.00">hostname</text> +<text text-anchor="start" x="1389" y="-2451.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1394" y="-2451.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1314,-2417 1314,-2442 1622,-2442 1622,-2417 1314,-2417"/> +<text text-anchor="start" x="1319" y="-2426.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_heartbeat_at</text> +<text text-anchor="start" x="1439" y="-2426.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1444" y="-2426.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1314,-2392 1314,-2417 1622,-2417 1622,-2392 1314,-2392"/> +<text text-anchor="start" x="1319" y="-2401.8" font-family="Helvetica,sans-Serif" font-size="14.00">map_index</text> +<text text-anchor="start" x="1395" y="-2401.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1400" y="-2401.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1477" y="-2401.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1314,-2367 1314,-2392 1622,-2392 1622,-2367 1314,-2367"/> +<text text-anchor="start" x="1319" y="-2376.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_tries</text> +<text text-anchor="start" x="1387" y="-2376.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1392" y="-2376.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1314,-2342 1314,-2367 1622,-2367 1622,-2342 1314,-2342"/> +<text text-anchor="start" x="1319" y="-2351.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_kwargs</text> +<text text-anchor="start" x="1407" y="-2351.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1412" y="-2351.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<polygon fill="none" stroke="black" points="1314,-2317 1314,-2342 1622,-2342 1622,-2317 1314,-2317"/> +<text text-anchor="start" x="1319" y="-2326.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_method</text> +<text text-anchor="start" x="1410" y="-2326.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1415" y="-2326.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1314,-2292 1314,-2317 1622,-2317 1622,-2292 1314,-2292"/> +<text text-anchor="start" x="1319" y="-2301.8" font-family="Helvetica,sans-Serif" font-size="14.00">operator</text> +<text text-anchor="start" x="1379" y="-2301.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1384" y="-2301.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1314,-2267 1314,-2292 1622,-2292 1622,-2267 1314,-2267"/> +<text text-anchor="start" x="1319" y="-2276.8" font-family="Helvetica,sans-Serif" font-size="14.00">pid</text> +<text text-anchor="start" x="1341" y="-2276.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1346" y="-2276.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1314,-2242 1314,-2267 1622,-2267 1622,-2242 1314,-2242"/> +<text text-anchor="start" x="1319" y="-2251.8" font-family="Helvetica,sans-Serif" font-size="14.00">pool</text> +<text text-anchor="start" x="1349" y="-2251.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1354" y="-2251.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<text text-anchor="start" x="1475" y="-2251.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1314,-2217 1314,-2242 1622,-2242 1622,-2217 1314,-2217"/> +<text text-anchor="start" x="1319" y="-2226.8" font-family="Helvetica,sans-Serif" font-size="14.00">pool_slots</text> +<text text-anchor="start" x="1388" y="-2226.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1393" y="-2226.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1470" y="-2226.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1314,-2192 1314,-2217 1622,-2217 1622,-2192 1314,-2192"/> +<text text-anchor="start" x="1319" y="-2201.8" font-family="Helvetica,sans-Serif" font-size="14.00">priority_weight</text> +<text text-anchor="start" x="1423" y="-2201.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1428" y="-2201.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1314,-2167 1314,-2192 1622,-2192 1622,-2167 1314,-2167"/> +<text text-anchor="start" x="1319" y="-2176.8" font-family="Helvetica,sans-Serif" font-size="14.00">queue</text> +<text text-anchor="start" x="1363" y="-2176.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1368" y="-2176.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<polygon fill="none" stroke="black" points="1314,-2142 1314,-2167 1622,-2167 1622,-2142 1314,-2142"/> +<text text-anchor="start" x="1319" y="-2151.8" font-family="Helvetica,sans-Serif" font-size="14.00">queued_by_job_id</text> +<text text-anchor="start" x="1443" y="-2151.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1448" y="-2151.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1314,-2117 1314,-2142 1622,-2142 1622,-2117 1314,-2117"/> +<text text-anchor="start" x="1319" y="-2126.8" font-family="Helvetica,sans-Serif" font-size="14.00">queued_dttm</text> +<text text-anchor="start" x="1412" y="-2126.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1417" y="-2126.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1314,-2092 1314,-2117 1622,-2117 1622,-2092 1314,-2092"/> +<text text-anchor="start" x="1319" y="-2101.8" font-family="Helvetica,sans-Serif" font-size="14.00">rendered_map_index</text> +<text text-anchor="start" x="1464" y="-2101.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1469" y="-2101.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="1314,-2067 1314,-2092 1622,-2092 1622,-2067 1314,-2067"/> +<text text-anchor="start" x="1319" y="-2076.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> +<text text-anchor="start" x="1363" y="-2076.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1368" y="-2076.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1489" y="-2076.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1314,-2042 1314,-2067 1622,-2067 1622,-2042 1314,-2042"/> +<text text-anchor="start" x="1319" y="-2051.8" font-family="Helvetica,sans-Serif" font-size="14.00">start_date</text> +<text text-anchor="start" x="1389" y="-2051.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1394" y="-2051.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1314,-2017 1314,-2042 1622,-2042 1622,-2017 1314,-2017"/> +<text text-anchor="start" x="1319" y="-2026.8" font-family="Helvetica,sans-Serif" font-size="14.00">state</text> +<text text-anchor="start" x="1354" y="-2026.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1359" y="-2026.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(20)]</text> +<polygon fill="none" stroke="black" points="1314,-1992 1314,-2017 1622,-2017 1622,-1992 1314,-1992"/> +<text text-anchor="start" x="1319" y="-2001.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_display_name</text> +<text text-anchor="start" x="1451" y="-2001.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1456" y="-2001.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<polygon fill="none" stroke="black" points="1314,-1967 1314,-1992 1622,-1992 1622,-1967 1314,-1967"/> +<text text-anchor="start" x="1319" y="-1976.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_id</text> +<text text-anchor="start" x="1368" y="-1976.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1373" y="-1976.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1494" y="-1976.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1314,-1942 1314,-1967 1622,-1967 1622,-1942 1314,-1942"/> +<text text-anchor="start" x="1319" y="-1951.8" font-family="Helvetica,sans-Serif" font-size="14.00">trigger_id</text> +<text text-anchor="start" x="1386" y="-1951.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1391" y="-1951.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1314,-1917 1314,-1942 1622,-1942 1622,-1917 1314,-1917"/> +<text text-anchor="start" x="1319" y="-1926.8" font-family="Helvetica,sans-Serif" font-size="14.00">trigger_timeout</text> +<text text-anchor="start" x="1427" y="-1926.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1432" y="-1926.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1314,-1892 1314,-1917 1622,-1917 1622,-1892 1314,-1892"/> +<text text-anchor="start" x="1319" y="-1901.8" font-family="Helvetica,sans-Serif" font-size="14.00">try_number</text> +<text text-anchor="start" x="1401" y="-1901.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1406" y="-1901.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1314,-1867 1314,-1892 1622,-1892 1622,-1867 1314,-1867"/> +<text text-anchor="start" x="1319" y="-1876.8" font-family="Helvetica,sans-Serif" font-size="14.00">unixname</text> +<text text-anchor="start" x="1389" y="-1876.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1394" y="-1876.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1314,-1842 1314,-1867 1622,-1867 1622,-1842 1314,-1842"/> +<text text-anchor="start" x="1319" y="-1851.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> +<text text-anchor="start" x="1398" y="-1851.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1403" y="-1851.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +</g> +<!-- dag_version--task_instance --> +<g id="edge23" class="edge"> +<title>dag_version--task_instance</title> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M823.26,-2236.67C857.78,-2243.83 894.53,-2250.53 929,-2255 1054.89,-2271.34 1198.54,-2277.66 1305.73,-2280"/> +<text text-anchor="start" x="1274.73" y="-2268.8" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="823.26" y="-2225.47" font-family="Times,serif" font-size="14.00">{0,1}</text> +</g> +<!-- dag_run--dagrun_asset_event --> +<g id="edge25" class="edge"> +<title>dag_run--dagrun_asset_event</title> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1230.04,-2481.38C1231.06,-2478.25 1232.05,-2475.12 1233,-2472 1275.02,-2334.47 1214.94,-1940.3 1306,-1829 1314.65,-1818.43 1325.47,-1809.98 1337.38,-1803.24"/> +<text text-anchor="start" x="1306.38" y="-1792.04" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1220.04" y="-2470.18" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag_run--task_instance --> -<g id="edge21" class="edge"> +<g id="edge26" class="edge"> <title>dag_run--task_instance</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M795.67,-2794.62C842.08,-2794.17 893.48,-2794.13 940.71,-2794.51"/> -<text text-anchor="start" x="909.71" y="-2783.31" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="795.67" y="-2783.42" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1230.58,-2569.79C1255.24,-2538.79 1280.92,-2506.68 1305.88,-2475.63"/> +<text text-anchor="start" x="1274.88" y="-2464.43" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1230.58" y="-2558.59" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag_run--task_instance --> -<g id="edge22" class="edge"> +<g id="edge27" class="edge"> <title>dag_run--task_instance</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M795.67,-2807.38C842.08,-2807.83 893.48,-2807.87 940.71,-2807.49"/> -<text text-anchor="start" x="909.71" y="-2811.29" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="795.67" y="-2811.18" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1230.58,-2582.97C1255.24,-2552.33 1280.92,-2520.28 1305.88,-2488.97"/> +<text text-anchor="start" x="1274.88" y="-2492.77" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1230.58" y="-2586.77" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- backfill_dag_run --> -<g id="node29" class="node"> +<g id="node30" class="node"> <title>backfill_dag_run</title> -<polygon fill="none" stroke="black" points="969,-2321 969,-2349 1238,-2349 1238,-2321 969,-2321"/> -<text text-anchor="start" x="1031" y="-2332.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">backfill_dag_run</text> -<polygon fill="none" stroke="black" points="969,-2296 969,-2321 1238,-2321 1238,-2296 969,-2296"/> -<text text-anchor="start" x="974" y="-2305.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="987" y="-2305.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="992" y="-2305.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1069" y="-2305.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="969,-2271 969,-2296 1238,-2296 1238,-2271 969,-2271"/> -<text text-anchor="start" x="974" y="-2280.8" font-family="Helvetica,sans-Serif" font-size="14.00">backfill_id</text> -<text text-anchor="start" x="1044" y="-2280.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1049" y="-2280.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1126" y="-2280.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="969,-2246 969,-2271 1238,-2271 1238,-2246 969,-2246"/> -<text text-anchor="start" x="974" y="-2255.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_run_id</text> -<text text-anchor="start" x="1051" y="-2255.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1056" y="-2255.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="969,-2221 969,-2246 1238,-2246 1238,-2221 969,-2221"/> -<text text-anchor="start" x="974" y="-2230.8" font-family="Helvetica,sans-Serif" font-size="14.00">exception_reason</text> -<text text-anchor="start" x="1095" y="-2230.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1100" y="-2230.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="969,-2196 969,-2221 1238,-2221 1238,-2196 969,-2196"/> -<text text-anchor="start" x="974" y="-2205.8" font-family="Helvetica,sans-Serif" font-size="14.00">logical_date</text> -<text text-anchor="start" x="1056" y="-2205.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1061" y="-2205.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1157" y="-2205.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="969,-2171 969,-2196 1238,-2196 1238,-2171 969,-2171"/> -<text text-anchor="start" x="974" y="-2180.8" font-family="Helvetica,sans-Serif" font-size="14.00">sort_ordinal</text> -<text text-anchor="start" x="1056" y="-2180.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1061" y="-2180.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1138" y="-2180.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1334,-3132 1334,-3160 1603,-3160 1603,-3132 1334,-3132"/> +<text text-anchor="start" x="1396" y="-3143.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">backfill_dag_run</text> +<polygon fill="none" stroke="black" points="1334,-3107 1334,-3132 1603,-3132 1603,-3107 1334,-3107"/> +<text text-anchor="start" x="1339" y="-3116.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="1352" y="-3116.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1357" y="-3116.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1434" y="-3116.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1334,-3082 1334,-3107 1603,-3107 1603,-3082 1334,-3082"/> +<text text-anchor="start" x="1339" y="-3091.8" font-family="Helvetica,sans-Serif" font-size="14.00">backfill_id</text> +<text text-anchor="start" x="1409" y="-3091.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1414" y="-3091.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1491" y="-3091.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1334,-3057 1334,-3082 1603,-3082 1603,-3057 1334,-3057"/> +<text text-anchor="start" x="1339" y="-3066.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_run_id</text> +<text text-anchor="start" x="1416" y="-3066.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1421" y="-3066.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1334,-3032 1334,-3057 1603,-3057 1603,-3032 1334,-3032"/> +<text text-anchor="start" x="1339" y="-3041.8" font-family="Helvetica,sans-Serif" font-size="14.00">exception_reason</text> +<text text-anchor="start" x="1460" y="-3041.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1465" y="-3041.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="1334,-3007 1334,-3032 1603,-3032 1603,-3007 1334,-3007"/> +<text text-anchor="start" x="1339" y="-3016.8" font-family="Helvetica,sans-Serif" font-size="14.00">logical_date</text> +<text text-anchor="start" x="1421" y="-3016.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1426" y="-3016.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1522" y="-3016.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1334,-2982 1334,-3007 1603,-3007 1603,-2982 1334,-2982"/> +<text text-anchor="start" x="1339" y="-2991.8" font-family="Helvetica,sans-Serif" font-size="14.00">sort_ordinal</text> +<text text-anchor="start" x="1421" y="-2991.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1426" y="-2991.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1503" y="-2991.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- dag_run--backfill_dag_run --> -<g id="edge23" class="edge"> +<g id="edge28" class="edge"> <title>dag_run--backfill_dag_run</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M795.54,-2604.36C816.71,-2573.68 837.53,-2541.9 856,-2511 894.04,-2447.36 878.59,-2415.37 929,-2361 938.34,-2350.93 948.92,-2341.56 960.13,-2332.91"/> -<text text-anchor="start" x="929.13" y="-2321.71" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="795.54" y="-2593.16" font-family="Times,serif" font-size="14.00">{0,1}</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1230.74,-2931.54C1243.25,-2943.01 1256.07,-2953.97 1269,-2964 1286.44,-2977.53 1305.83,-2990.3 1325.46,-3001.99"/> +<text text-anchor="start" x="1294.46" y="-2990.79" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1230.74" y="-2920.34" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- dag_run_note --> -<g id="node30" class="node"> +<g id="node31" class="node"> <title>dag_run_note</title> -<polygon fill="none" stroke="black" points="970,-2116 970,-2144 1236,-2144 1236,-2116 970,-2116"/> -<text text-anchor="start" x="1042.5" y="-2127.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_run_note</text> -<polygon fill="none" stroke="black" points="970,-2091 970,-2116 1236,-2116 1236,-2091 970,-2091"/> -<text text-anchor="start" x="975" y="-2100.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_run_id</text> -<text text-anchor="start" x="1052" y="-2100.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1057" y="-2100.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1134" y="-2100.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="970,-2066 970,-2091 1236,-2091 1236,-2066 970,-2066"/> -<text text-anchor="start" x="975" y="-2075.8" font-family="Helvetica,sans-Serif" font-size="14.00">content</text> -<text text-anchor="start" x="1028" y="-2075.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1033" y="-2075.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="970,-2041 970,-2066 1236,-2066 1236,-2041 970,-2041"/> -<text text-anchor="start" x="975" y="-2050.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> -<text text-anchor="start" x="1048" y="-2050.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1053" y="-2050.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1149" y="-2050.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="970,-2016 970,-2041 1236,-2041 1236,-2016 970,-2016"/> -<text text-anchor="start" x="975" y="-2025.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> -<text text-anchor="start" x="1054" y="-2025.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1059" y="-2025.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1155" y="-2025.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="970,-1991 970,-2016 1236,-2016 1236,-1991 970,-1991"/> -<text text-anchor="start" x="975" y="-2000.8" font-family="Helvetica,sans-Serif" font-size="14.00">user_id</text> -<text text-anchor="start" x="1026" y="-2000.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1031" y="-2000.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(128)]</text> +<polygon fill="none" stroke="black" points="1335,-2871 1335,-2899 1601,-2899 1601,-2871 1335,-2871"/> +<text text-anchor="start" x="1407.5" y="-2882.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">dag_run_note</text> +<polygon fill="none" stroke="black" points="1335,-2846 1335,-2871 1601,-2871 1601,-2846 1335,-2846"/> +<text text-anchor="start" x="1340" y="-2855.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_run_id</text> +<text text-anchor="start" x="1417" y="-2855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1422" y="-2855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1499" y="-2855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1335,-2821 1335,-2846 1601,-2846 1601,-2821 1335,-2821"/> +<text text-anchor="start" x="1340" y="-2830.8" font-family="Helvetica,sans-Serif" font-size="14.00">content</text> +<text text-anchor="start" x="1393" y="-2830.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1398" y="-2830.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1335,-2796 1335,-2821 1601,-2821 1601,-2796 1335,-2796"/> +<text text-anchor="start" x="1340" y="-2805.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="1413" y="-2805.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1418" y="-2805.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1514" y="-2805.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1335,-2771 1335,-2796 1601,-2796 1601,-2771 1335,-2771"/> +<text text-anchor="start" x="1340" y="-2780.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> +<text text-anchor="start" x="1419" y="-2780.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1424" y="-2780.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1520" y="-2780.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1335,-2746 1335,-2771 1601,-2771 1601,-2746 1335,-2746"/> +<text text-anchor="start" x="1340" y="-2755.8" font-family="Helvetica,sans-Serif" font-size="14.00">user_id</text> +<text text-anchor="start" x="1391" y="-2755.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1396" y="-2755.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(128)]</text> </g> <!-- dag_run--dag_run_note --> -<g id="edge24" class="edge"> +<g id="edge29" class="edge"> <title>dag_run--dag_run_note</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M795.57,-2618.86C818.47,-2584.52 839.83,-2547.79 856,-2511 920.48,-2364.34 832.36,-2285.78 929,-2158 938.19,-2145.85 949.49,-2135.19 961.89,-2125.87"/> -<text text-anchor="start" x="951.89" y="-2114.67" font-family="Times,serif" font-size="14.00">1</text> -<text text-anchor="start" x="795.57" y="-2622.66" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1230.58,-2785.54C1262.23,-2790.56 1295.55,-2795.84 1326.87,-2800.8"/> +<text text-anchor="start" x="1316.87" y="-2789.6" font-family="Times,serif" font-size="14.00">1</text> +<text text-anchor="start" x="1230.58" y="-2774.34" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_reschedule --> -<g id="node31" class="node"> +<g id="node32" class="node"> <title>task_reschedule</title> -<polygon fill="none" stroke="black" points="1362,-3379 1362,-3407 1663,-3407 1663,-3379 1362,-3379"/> -<text text-anchor="start" x="1440" y="-3390.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_reschedule</text> -<polygon fill="none" stroke="black" points="1362,-3354 1362,-3379 1663,-3379 1663,-3354 1362,-3354"/> -<text text-anchor="start" x="1367" y="-3363.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="1380" y="-3363.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1385" y="-3363.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1462" y="-3363.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1362,-3329 1362,-3354 1663,-3354 1663,-3329 1362,-3329"/> -<text text-anchor="start" x="1367" y="-3338.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1413" y="-3338.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1418" y="-3338.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1539" y="-3338.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1362,-3304 1362,-3329 1663,-3329 1663,-3304 1362,-3304"/> -<text text-anchor="start" x="1367" y="-3313.8" font-family="Helvetica,sans-Serif" font-size="14.00">duration</text> -<text text-anchor="start" x="1426" y="-3313.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1431" y="-3313.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1508" y="-3313.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1362,-3279 1362,-3304 1663,-3304 1663,-3279 1362,-3279"/> -<text text-anchor="start" x="1367" y="-3288.8" font-family="Helvetica,sans-Serif" font-size="14.00">end_date</text> -<text text-anchor="start" x="1431" y="-3288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1436" y="-3288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1532" y="-3288.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1362,-3254 1362,-3279 1663,-3279 1663,-3254 1362,-3254"/> -<text text-anchor="start" x="1367" y="-3263.8" font-family="Helvetica,sans-Serif" font-size="14.00">map_index</text> -<text text-anchor="start" x="1443" y="-3263.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1448" y="-3263.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1525" y="-3263.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1362,-3229 1362,-3254 1663,-3254 1663,-3229 1362,-3229"/> -<text text-anchor="start" x="1367" y="-3238.8" font-family="Helvetica,sans-Serif" font-size="14.00">reschedule_date</text> -<text text-anchor="start" x="1481" y="-3238.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1486" y="-3238.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1582" y="-3238.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1362,-3204 1362,-3229 1663,-3229 1663,-3204 1362,-3204"/> -<text text-anchor="start" x="1367" y="-3213.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> -<text text-anchor="start" x="1411" y="-3213.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1416" y="-3213.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1537" y="-3213.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1362,-3179 1362,-3204 1663,-3204 1663,-3179 1362,-3179"/> -<text text-anchor="start" x="1367" y="-3188.8" font-family="Helvetica,sans-Serif" font-size="14.00">start_date</text> -<text text-anchor="start" x="1437" y="-3188.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1442" y="-3188.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1538" y="-3188.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1362,-3154 1362,-3179 1663,-3179 1663,-3154 1362,-3154"/> -<text text-anchor="start" x="1367" y="-3163.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_id</text> -<text text-anchor="start" x="1416" y="-3163.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1421" y="-3163.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1542" y="-3163.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1362,-3129 1362,-3154 1663,-3154 1663,-3129 1362,-3129"/> -<text text-anchor="start" x="1367" y="-3138.8" font-family="Helvetica,sans-Serif" font-size="14.00">try_number</text> -<text text-anchor="start" x="1449" y="-3138.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1454" y="-3138.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1531" y="-3138.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1715,-3525 1715,-3553 2016,-3553 2016,-3525 1715,-3525"/> +<text text-anchor="start" x="1793" y="-3536.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_reschedule</text> +<polygon fill="none" stroke="black" points="1715,-3500 1715,-3525 2016,-3525 2016,-3500 1715,-3500"/> +<text text-anchor="start" x="1720" y="-3509.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="1733" y="-3509.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1738" y="-3509.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1815" y="-3509.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1715,-3475 1715,-3500 2016,-3500 2016,-3475 1715,-3475"/> +<text text-anchor="start" x="1720" y="-3484.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> +<text text-anchor="start" x="1766" y="-3484.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1771" y="-3484.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1892" y="-3484.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1715,-3450 1715,-3475 2016,-3475 2016,-3450 1715,-3450"/> +<text text-anchor="start" x="1720" y="-3459.8" font-family="Helvetica,sans-Serif" font-size="14.00">duration</text> +<text text-anchor="start" x="1779" y="-3459.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1784" y="-3459.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1861" y="-3459.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1715,-3425 1715,-3450 2016,-3450 2016,-3425 1715,-3425"/> +<text text-anchor="start" x="1720" y="-3434.8" font-family="Helvetica,sans-Serif" font-size="14.00">end_date</text> +<text text-anchor="start" x="1784" y="-3434.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1789" y="-3434.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1885" y="-3434.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1715,-3400 1715,-3425 2016,-3425 2016,-3400 1715,-3400"/> +<text text-anchor="start" x="1720" y="-3409.8" font-family="Helvetica,sans-Serif" font-size="14.00">map_index</text> +<text text-anchor="start" x="1796" y="-3409.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1801" y="-3409.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1878" y="-3409.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1715,-3375 1715,-3400 2016,-3400 2016,-3375 1715,-3375"/> +<text text-anchor="start" x="1720" y="-3384.8" font-family="Helvetica,sans-Serif" font-size="14.00">reschedule_date</text> +<text text-anchor="start" x="1834" y="-3384.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1839" y="-3384.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1935" y="-3384.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1715,-3350 1715,-3375 2016,-3375 2016,-3350 1715,-3350"/> +<text text-anchor="start" x="1720" y="-3359.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> +<text text-anchor="start" x="1764" y="-3359.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1769" y="-3359.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1890" y="-3359.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1715,-3325 1715,-3350 2016,-3350 2016,-3325 1715,-3325"/> +<text text-anchor="start" x="1720" y="-3334.8" font-family="Helvetica,sans-Serif" font-size="14.00">start_date</text> +<text text-anchor="start" x="1790" y="-3334.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1795" y="-3334.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1891" y="-3334.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1715,-3300 1715,-3325 2016,-3325 2016,-3300 1715,-3300"/> +<text text-anchor="start" x="1720" y="-3309.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_id</text> +<text text-anchor="start" x="1769" y="-3309.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1774" y="-3309.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1895" y="-3309.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1715,-3275 1715,-3300 2016,-3300 2016,-3275 1715,-3275"/> +<text text-anchor="start" x="1720" y="-3284.8" font-family="Helvetica,sans-Serif" font-size="14.00">try_number</text> +<text text-anchor="start" x="1802" y="-3284.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1807" y="-3284.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1884" y="-3284.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- dag_run--task_reschedule --> -<g id="edge25" class="edge"> +<g id="edge30" class="edge"> <title>dag_run--task_reschedule</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M777.11,-3081.71C817.26,-3140.56 867.53,-3195.48 929,-3231 998.58,-3271.2 1205.95,-3272.29 1353.29,-3269.48"/> -<text text-anchor="start" x="1322.29" y="-3258.28" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="767.11" y="-3085.51" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1230.58,-2864.77C1254.89,-2879.3 1280.49,-2892.75 1306,-2903 1441.67,-2957.52 1523.02,-2860.34 1630,-2960 1727.88,-3051.18 1635.61,-3136.44 1703,-3252 1706.69,-3258.33 1710.72,-3264.56 1715.02,-3270.68"/> +<text text-anchor="start" x="1684.02" y="-3259.48" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1230.58" y="-2853.57" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- dag_run--task_reschedule --> -<g id="edge26" class="edge"> +<g id="edge31" class="edge"> <title>dag_run--task_reschedule</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M765.67,-3081.76C807.59,-3147.69 861.55,-3210.03 929,-3249 998.58,-3289.2 1205.95,-3290.29 1353.29,-3283.18"/> -<text text-anchor="start" x="1322.29" y="-3286.98" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="755.67" y="-3085.56" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1230.58,-2882.56C1254.89,-2897.3 1280.49,-2910.75 1306,-2921 1441.67,-2975.52 1523.02,-2878.34 1630,-2978 1727.88,-3069.18 1635.61,-3154.44 1703,-3270 1704.07,-3271.83 1705.17,-3273.66 1706.29,-3275.47"/> +<text text-anchor="start" x="1675.29" y="-3279.27" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1230.58" y="-2871.36" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_reschedule --> -<g id="edge38" class="edge"> +<g id="edge43" class="edge"> <title>task_instance--task_reschedule</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2989.76C1292.62,-3023.44 1321.5,-3057.7 1350,-3089 1360.74,-3100.8 1372.23,-3112.76 1383.94,-3124.65"/> -<text text-anchor="start" x="1352.94" y="-3113.45" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2978.56" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.13,-2706.47C1652.36,-2786.9 1654.41,-2808.27 1667,-2891 1690.06,-3042.54 1635.26,-3096.5 1703,-3234 1709.3,-3246.8 1717.15,-3259.09 1725.96,-3270.84"/> +<text text-anchor="start" x="1725.96" y="-3259.64" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.13" y="-2695.27" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_reschedule --> -<g id="edge39" class="edge"> +<g id="edge44" class="edge"> <title>task_instance--task_reschedule</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-3007.51C1292.62,-3041.44 1321.5,-3075.7 1350,-3107 1355.43,-3112.97 1361.06,-3118.98 1366.81,-3125"/> -<text text-anchor="start" x="1335.81" y="-3128.8" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2996.31" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.13,-2724.47C1652.36,-2804.9 1654.41,-2826.27 1667,-2909 1690.06,-3060.54 1635.26,-3114.5 1703,-3252 1706.15,-3258.4 1709.69,-3264.67 1713.54,-3270.81"/> +<text text-anchor="start" x="1713.54" y="-3259.61" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.13" y="-2713.27" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_reschedule --> -<g id="edge40" class="edge"> +<g id="edge45" class="edge"> <title>task_instance--task_reschedule</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-3025.27C1292.62,-3059.44 1321.5,-3093.7 1350,-3125 1351.04,-3126.14 1352.08,-3127.28 1353.13,-3128.42"/> -<text text-anchor="start" x="1322.13" y="-3117.22" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-3014.07" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1624.97,-2724.14C1626.69,-2730.12 1628.36,-2736.08 1630,-2742 1652.35,-2822.82 1654.39,-2844.1 1667,-2927 1690.06,-3078.54 1635.26,-3132.5 1703,-3270 1704.11,-3272.25 1705.26,-3274.48 1706.46,-3276.7"/> +<text text-anchor="start" x="1675.46" y="-3265.5" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1614.97" y="-2727.94" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_reschedule --> -<g id="edge41" class="edge"> +<g id="edge46" class="edge"> <title>task_instance--task_reschedule</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-3043.03C1292.62,-3077.44 1321.5,-3111.7 1350,-3143 1351.04,-3144.14 1352.08,-3145.28 1353.13,-3146.42"/> -<text text-anchor="start" x="1322.13" y="-3150.22" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-3031.83" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1619.73,-2724.11C1623.29,-2736.18 1626.73,-2748.16 1630,-2760 1652.35,-2840.82 1654.39,-2862.1 1667,-2945 1690.06,-3096.54 1635.26,-3150.5 1703,-3288 1704.11,-3290.25 1705.26,-3292.48 1706.46,-3294.7"/> +<text text-anchor="start" x="1675.46" y="-3298.5" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1609.73" y="-2727.91" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- rendered_task_instance_fields --> -<g id="node34" class="node"> +<g id="node35" class="node"> <title>rendered_task_instance_fields</title> -<polygon fill="none" stroke="black" points="1372,-3075 1372,-3103 1652,-3103 1652,-3075 1372,-3075"/> -<text text-anchor="start" x="1377" y="-3086.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">rendered_task_instance_fields</text> -<polygon fill="none" stroke="black" points="1372,-3050 1372,-3075 1652,-3075 1652,-3050 1372,-3050"/> -<text text-anchor="start" x="1377" y="-3059.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1423" y="-3059.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1428" y="-3059.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1549" y="-3059.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1372,-3025 1372,-3050 1652,-3050 1652,-3025 1372,-3025"/> -<text text-anchor="start" x="1377" y="-3034.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">map_index</text> -<text text-anchor="start" x="1453" y="-3034.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1458" y="-3034.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1535" y="-3034.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1372,-3000 1372,-3025 1652,-3025 1652,-3000 1372,-3000"/> -<text text-anchor="start" x="1377" y="-3009.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">run_id</text> -<text text-anchor="start" x="1421" y="-3009.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1426" y="-3009.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1547" y="-3009.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1372,-2975 1372,-3000 1652,-3000 1652,-2975 1372,-2975"/> -<text text-anchor="start" x="1377" y="-2984.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">task_id</text> -<text text-anchor="start" x="1426" y="-2984.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1431" y="-2984.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1552" y="-2984.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1372,-2950 1372,-2975 1652,-2975 1652,-2950 1372,-2950"/> -<text text-anchor="start" x="1377" y="-2959.8" font-family="Helvetica,sans-Serif" font-size="14.00">k8s_pod_yaml</text> -<text text-anchor="start" x="1476" y="-2959.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1481" y="-2959.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<polygon fill="none" stroke="black" points="1372,-2925 1372,-2950 1652,-2950 1652,-2925 1372,-2925"/> -<text text-anchor="start" x="1377" y="-2934.8" font-family="Helvetica,sans-Serif" font-size="14.00">rendered_fields</text> -<text text-anchor="start" x="1484" y="-2934.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1489" y="-2934.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<text text-anchor="start" x="1540" y="-2934.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1725,-2342 1725,-2370 2005,-2370 2005,-2342 1725,-2342"/> +<text text-anchor="start" x="1730" y="-2353.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">rendered_task_instance_fields</text> +<polygon fill="none" stroke="black" points="1725,-2317 1725,-2342 2005,-2342 2005,-2317 1725,-2317"/> +<text text-anchor="start" x="1730" y="-2326.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> +<text text-anchor="start" x="1776" y="-2326.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1781" y="-2326.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1902" y="-2326.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1725,-2292 1725,-2317 2005,-2317 2005,-2292 1725,-2292"/> +<text text-anchor="start" x="1730" y="-2301.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">map_index</text> +<text text-anchor="start" x="1806" y="-2301.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1811" y="-2301.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1888" y="-2301.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1725,-2267 1725,-2292 2005,-2292 2005,-2267 1725,-2267"/> +<text text-anchor="start" x="1730" y="-2276.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">run_id</text> +<text text-anchor="start" x="1774" y="-2276.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1779" y="-2276.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1900" y="-2276.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1725,-2242 1725,-2267 2005,-2267 2005,-2242 1725,-2242"/> +<text text-anchor="start" x="1730" y="-2251.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">task_id</text> +<text text-anchor="start" x="1779" y="-2251.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1784" y="-2251.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1905" y="-2251.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1725,-2217 1725,-2242 2005,-2242 2005,-2217 1725,-2217"/> +<text text-anchor="start" x="1730" y="-2226.8" font-family="Helvetica,sans-Serif" font-size="14.00">k8s_pod_yaml</text> +<text text-anchor="start" x="1829" y="-2226.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1834" y="-2226.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<polygon fill="none" stroke="black" points="1725,-2192 1725,-2217 2005,-2217 2005,-2192 1725,-2192"/> +<text text-anchor="start" x="1730" y="-2201.8" font-family="Helvetica,sans-Serif" font-size="14.00">rendered_fields</text> +<text text-anchor="start" x="1837" y="-2201.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1842" y="-2201.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<text text-anchor="start" x="1893" y="-2201.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- task_instance--rendered_task_instance_fields --> -<g id="edge30" class="edge"> +<g id="edge35" class="edge"> <title>task_instance--rendered_task_instance_fields</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2865.48C1299.9,-2883.01 1336.38,-2902.26 1370.28,-2920.99"/> -<text text-anchor="start" x="1339.28" y="-2909.79" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2854.28" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2261.06C1658.92,-2260.57 1688.67,-2260.66 1716.94,-2261.35"/> +<text text-anchor="start" x="1685.94" y="-2250.15" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2249.86" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--rendered_task_instance_fields --> -<g id="edge31" class="edge"> +<g id="edge36" class="edge"> <title>task_instance--rendered_task_instance_fields</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2878.72C1297.68,-2895.54 1331.73,-2913.39 1363.75,-2930.42"/> -<text text-anchor="start" x="1332.75" y="-2934.22" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2882.52" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2274.35C1658.92,-2274.19 1688.67,-2274.22 1716.94,-2274.45"/> +<text text-anchor="start" x="1685.94" y="-2278.25" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2278.15" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--rendered_task_instance_fields --> -<g id="edge32" class="edge"> +<g id="edge37" class="edge"> <title>task_instance--rendered_task_instance_fields</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2891.95C1297.68,-2909.19 1331.73,-2926.97 1363.75,-2943.45"/> -<text text-anchor="start" x="1332.75" y="-2947.25" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2895.75" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2287.65C1658.92,-2287.81 1688.67,-2287.78 1716.94,-2287.55"/> +<text text-anchor="start" x="1685.94" y="-2291.35" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2291.45" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--rendered_task_instance_fields --> -<g id="edge33" class="edge"> +<g id="edge38" class="edge"> <title>task_instance--rendered_task_instance_fields</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2905.19C1297.68,-2922.83 1331.73,-2940.55 1363.75,-2956.49"/> -<text text-anchor="start" x="1332.75" y="-2960.29" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2908.99" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2300.94C1658.92,-2301.43 1688.67,-2301.34 1716.94,-2300.65"/> +<text text-anchor="start" x="1685.94" y="-2304.45" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2304.74" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_map --> -<g id="node35" class="node"> +<g id="node36" class="node"> <title>task_map</title> -<polygon fill="none" stroke="black" points="1382,-2871 1382,-2899 1643,-2899 1643,-2871 1382,-2871"/> -<text text-anchor="start" x="1470" y="-2882.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_map</text> -<polygon fill="none" stroke="black" points="1382,-2846 1382,-2871 1643,-2871 1643,-2846 1382,-2846"/> -<text text-anchor="start" x="1387" y="-2855.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1433" y="-2855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1438" y="-2855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1559" y="-2855.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1382,-2821 1382,-2846 1643,-2846 1643,-2821 1382,-2821"/> -<text text-anchor="start" x="1387" y="-2830.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">map_index</text> -<text text-anchor="start" x="1463" y="-2830.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1468" y="-2830.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1545" y="-2830.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1382,-2796 1382,-2821 1643,-2821 1643,-2796 1382,-2796"/> -<text text-anchor="start" x="1387" y="-2805.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">run_id</text> -<text text-anchor="start" x="1431" y="-2805.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1436" y="-2805.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1557" y="-2805.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1382,-2771 1382,-2796 1643,-2796 1643,-2771 1382,-2771"/> -<text text-anchor="start" x="1387" y="-2780.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">task_id</text> -<text text-anchor="start" x="1436" y="-2780.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1441" y="-2780.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1562" y="-2780.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1382,-2746 1382,-2771 1643,-2771 1643,-2746 1382,-2746"/> -<text text-anchor="start" x="1387" y="-2755.8" font-family="Helvetica,sans-Serif" font-size="14.00">keys</text> -<text text-anchor="start" x="1419" y="-2755.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1424" y="-2755.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<polygon fill="none" stroke="black" points="1382,-2721 1382,-2746 1643,-2746 1643,-2721 1382,-2721"/> -<text text-anchor="start" x="1387" y="-2730.8" font-family="Helvetica,sans-Serif" font-size="14.00">length</text> -<text text-anchor="start" x="1432" y="-2730.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1437" y="-2730.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1514" y="-2730.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1735,-2138 1735,-2166 1996,-2166 1996,-2138 1735,-2138"/> +<text text-anchor="start" x="1823" y="-2149.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_map</text> +<polygon fill="none" stroke="black" points="1735,-2113 1735,-2138 1996,-2138 1996,-2113 1735,-2113"/> +<text text-anchor="start" x="1740" y="-2122.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> +<text text-anchor="start" x="1786" y="-2122.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1791" y="-2122.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1912" y="-2122.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1735,-2088 1735,-2113 1996,-2113 1996,-2088 1735,-2088"/> +<text text-anchor="start" x="1740" y="-2097.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">map_index</text> +<text text-anchor="start" x="1816" y="-2097.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1821" y="-2097.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1898" y="-2097.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1735,-2063 1735,-2088 1996,-2088 1996,-2063 1735,-2063"/> +<text text-anchor="start" x="1740" y="-2072.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">run_id</text> +<text text-anchor="start" x="1784" y="-2072.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1789" y="-2072.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1910" y="-2072.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1735,-2038 1735,-2063 1996,-2063 1996,-2038 1735,-2038"/> +<text text-anchor="start" x="1740" y="-2047.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">task_id</text> +<text text-anchor="start" x="1789" y="-2047.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1794" y="-2047.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1915" y="-2047.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1735,-2013 1735,-2038 1996,-2038 1996,-2013 1735,-2013"/> +<text text-anchor="start" x="1740" y="-2022.8" font-family="Helvetica,sans-Serif" font-size="14.00">keys</text> +<text text-anchor="start" x="1772" y="-2022.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1777" y="-2022.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<polygon fill="none" stroke="black" points="1735,-1988 1735,-2013 1996,-2013 1996,-1988 1735,-1988"/> +<text text-anchor="start" x="1740" y="-1997.8" font-family="Helvetica,sans-Serif" font-size="14.00">length</text> +<text text-anchor="start" x="1785" y="-1997.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1790" y="-1997.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1867" y="-1997.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- task_instance--task_map --> -<g id="edge34" class="edge"> +<g id="edge39" class="edge"> <title>task_instance--task_map</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2784.71C1300.9,-2784.82 1338.48,-2785.85 1373.2,-2787.67"/> -<text text-anchor="start" x="1342.2" y="-2776.47" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2773.51" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2177.79C1662.12,-2160.77 1695.32,-2143.81 1726.34,-2128.69"/> +<text text-anchor="start" x="1695.34" y="-2117.49" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2166.59" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_map --> -<g id="edge35" class="edge"> +<g id="edge40" class="edge"> <title>task_instance--task_map</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2797.95C1300.9,-2798.51 1338.48,-2799.41 1373.2,-2800.53"/> -<text text-anchor="start" x="1342.2" y="-2804.33" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2801.75" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2191.08C1662.12,-2174.43 1695.32,-2157.35 1726.34,-2141.63"/> +<text text-anchor="start" x="1695.34" y="-2145.43" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2194.88" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_map --> -<g id="edge36" class="edge"> +<g id="edge41" class="edge"> <title>task_instance--task_map</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2811.18C1300.9,-2812.2 1338.48,-2812.96 1373.2,-2813.38"/> -<text text-anchor="start" x="1342.2" y="-2817.18" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2814.98" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2204.38C1662.12,-2188.09 1695.32,-2170.88 1726.34,-2154.56"/> +<text text-anchor="start" x="1695.34" y="-2158.36" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2208.18" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_map --> -<g id="edge37" class="edge"> +<g id="edge42" class="edge"> <title>task_instance--task_map</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2824.42C1300.9,-2825.88 1338.48,-2826.52 1373.2,-2826.23"/> -<text text-anchor="start" x="1342.2" y="-2830.03" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2828.22" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2217.67C1662.12,-2201.75 1695.32,-2184.42 1726.34,-2167.49"/> +<text text-anchor="start" x="1695.34" y="-2171.29" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2221.47" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- xcom --> -<g id="node36" class="node"> +<g id="node37" class="node"> <title>xcom</title> -<polygon fill="none" stroke="black" points="1381,-2667 1381,-2695 1643,-2695 1643,-2667 1381,-2667"/> -<text text-anchor="start" x="1487.5" y="-2678.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">xcom</text> -<polygon fill="none" stroke="black" points="1381,-2642 1381,-2667 1643,-2667 1643,-2642 1381,-2642"/> -<text text-anchor="start" x="1386" y="-2651.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_run_id</text> -<text text-anchor="start" x="1463" y="-2651.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1468" y="-2651.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1545" y="-2651.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1381,-2617 1381,-2642 1643,-2642 1643,-2617 1381,-2617"/> -<text text-anchor="start" x="1386" y="-2626.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">key</text> -<text text-anchor="start" x="1411" y="-2626.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1416" y="-2626.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(512)]</text> -<text text-anchor="start" x="1537" y="-2626.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1381,-2592 1381,-2617 1643,-2617 1643,-2592 1381,-2592"/> -<text text-anchor="start" x="1386" y="-2601.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">map_index</text> -<text text-anchor="start" x="1462" y="-2601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1467" y="-2601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1544" y="-2601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1381,-2567 1381,-2592 1643,-2592 1643,-2567 1381,-2567"/> -<text text-anchor="start" x="1386" y="-2576.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">task_id</text> -<text text-anchor="start" x="1435" y="-2576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1440" y="-2576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1561" y="-2576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1381,-2542 1381,-2567 1643,-2567 1643,-2542 1381,-2542"/> -<text text-anchor="start" x="1386" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1432" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1437" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1558" y="-2551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1381,-2517 1381,-2542 1643,-2542 1643,-2517 1381,-2517"/> -<text text-anchor="start" x="1386" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> -<text text-anchor="start" x="1430" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1435" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1556" y="-2526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1381,-2492 1381,-2517 1643,-2517 1643,-2492 1381,-2492"/> -<text text-anchor="start" x="1386" y="-2501.8" font-family="Helvetica,sans-Serif" font-size="14.00">timestamp</text> -<text text-anchor="start" x="1461" y="-2501.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1466" y="-2501.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1562" y="-2501.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1381,-2467 1381,-2492 1643,-2492 1643,-2467 1381,-2467"/> -<text text-anchor="start" x="1386" y="-2476.8" font-family="Helvetica,sans-Serif" font-size="14.00">value</text> -<text text-anchor="start" x="1424" y="-2476.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1429" y="-2476.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> +<polygon fill="none" stroke="black" points="1734,-1934 1734,-1962 1996,-1962 1996,-1934 1734,-1934"/> +<text text-anchor="start" x="1840.5" y="-1945.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">xcom</text> +<polygon fill="none" stroke="black" points="1734,-1909 1734,-1934 1996,-1934 1996,-1909 1734,-1909"/> +<text text-anchor="start" x="1739" y="-1918.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_run_id</text> +<text text-anchor="start" x="1816" y="-1918.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1821" y="-1918.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1898" y="-1918.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1734,-1884 1734,-1909 1996,-1909 1996,-1884 1734,-1884"/> +<text text-anchor="start" x="1739" y="-1893.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">key</text> +<text text-anchor="start" x="1764" y="-1893.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1769" y="-1893.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(512)]</text> +<text text-anchor="start" x="1890" y="-1893.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1734,-1859 1734,-1884 1996,-1884 1996,-1859 1734,-1859"/> +<text text-anchor="start" x="1739" y="-1868.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">map_index</text> +<text text-anchor="start" x="1815" y="-1868.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1820" y="-1868.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1897" y="-1868.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1734,-1834 1734,-1859 1996,-1859 1996,-1834 1734,-1834"/> +<text text-anchor="start" x="1739" y="-1843.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">task_id</text> +<text text-anchor="start" x="1788" y="-1843.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1793" y="-1843.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1914" y="-1843.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1734,-1809 1734,-1834 1996,-1834 1996,-1809 1734,-1809"/> +<text text-anchor="start" x="1739" y="-1818.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> +<text text-anchor="start" x="1785" y="-1818.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1790" y="-1818.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1911" y="-1818.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1734,-1784 1734,-1809 1996,-1809 1996,-1784 1734,-1784"/> +<text text-anchor="start" x="1739" y="-1793.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> +<text text-anchor="start" x="1783" y="-1793.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1788" y="-1793.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1909" y="-1793.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1734,-1759 1734,-1784 1996,-1784 1996,-1759 1734,-1759"/> +<text text-anchor="start" x="1739" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00">timestamp</text> +<text text-anchor="start" x="1814" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1819" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1915" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1734,-1734 1734,-1759 1996,-1759 1996,-1734 1734,-1734"/> +<text text-anchor="start" x="1739" y="-1743.8" font-family="Helvetica,sans-Serif" font-size="14.00">value</text> +<text text-anchor="start" x="1777" y="-1743.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1782" y="-1743.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> </g> <!-- task_instance--xcom --> -<g id="edge42" class="edge"> +<g id="edge47" class="edge"> <title>task_instance--xcom</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2694.04C1300.79,-2674.1 1338.25,-2654.04 1372.88,-2636.37"/> -<text text-anchor="start" x="1341.88" y="-2625.17" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2682.84" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.14,-2029.79C1653.52,-2001.16 1678.13,-1973.25 1703,-1948 1710.25,-1940.64 1717.92,-1933.33 1725.82,-1926.17"/> +<text text-anchor="start" x="1694.82" y="-1914.97" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.14" y="-2018.59" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--xcom --> -<g id="edge43" class="edge"> +<g id="edge48" class="edge"> <title>task_instance--xcom</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2707.27C1300.79,-2687.78 1338.25,-2667.6 1372.88,-2649.23"/> -<text text-anchor="start" x="1341.88" y="-2653.03" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2711.07" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.14,-2047.63C1653.52,-2019.16 1678.13,-1991.25 1703,-1966 1710.25,-1958.64 1717.92,-1951.33 1725.82,-1944.14"/> +<text text-anchor="start" x="1694.82" y="-1932.94" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.14" y="-2036.43" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--xcom --> -<g id="edge44" class="edge"> +<g id="edge49" class="edge"> <title>task_instance--xcom</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2720.51C1300.79,-2701.47 1338.25,-2681.16 1372.88,-2662.09"/> -<text text-anchor="start" x="1341.88" y="-2665.89" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2724.31" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.14,-2065.47C1653.52,-2037.16 1678.13,-2009.25 1703,-1984 1710.25,-1976.64 1717.92,-1969.33 1725.82,-1962.12"/> +<text text-anchor="start" x="1694.82" y="-1950.92" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.14" y="-2054.27" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--xcom --> -<g id="edge45" class="edge"> +<g id="edge50" class="edge"> <title>task_instance--xcom</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.15,-2733.75C1300.79,-2715.16 1338.25,-2694.72 1372.88,-2674.95"/> -<text text-anchor="start" x="1341.88" y="-2678.75" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.15" y="-2737.55" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.14,-2083.31C1653.52,-2055.16 1678.13,-2027.25 1703,-2002 1714.87,-1989.95 1727.87,-1978 1741.19,-1966.36"/> +<text text-anchor="start" x="1710.19" y="-1970.16" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.14" y="-2072.11" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance_note --> -<g id="node37" class="node"> +<g id="node38" class="node"> <title>task_instance_note</title> -<polygon fill="none" stroke="black" points="1379,-2413 1379,-2441 1645,-2441 1645,-2413 1379,-2413"/> -<text text-anchor="start" x="1426" y="-2424.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_instance_note</text> -<polygon fill="none" stroke="black" points="1379,-2388 1379,-2413 1645,-2413 1645,-2388 1379,-2388"/> -<text text-anchor="start" x="1384" y="-2397.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1430" y="-2397.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1435" y="-2397.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1556" y="-2397.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1379,-2363 1379,-2388 1645,-2388 1645,-2363 1379,-2363"/> -<text text-anchor="start" x="1384" y="-2372.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">map_index</text> -<text text-anchor="start" x="1460" y="-2372.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1465" y="-2372.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1542" y="-2372.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1379,-2338 1379,-2363 1645,-2363 1645,-2338 1379,-2338"/> -<text text-anchor="start" x="1384" y="-2347.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">run_id</text> -<text text-anchor="start" x="1428" y="-2347.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1433" y="-2347.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1554" y="-2347.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1379,-2313 1379,-2338 1645,-2338 1645,-2313 1379,-2313"/> -<text text-anchor="start" x="1384" y="-2322.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">task_id</text> -<text text-anchor="start" x="1433" y="-2322.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1438" y="-2322.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1559" y="-2322.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1379,-2288 1379,-2313 1645,-2313 1645,-2288 1379,-2288"/> -<text text-anchor="start" x="1384" y="-2297.8" font-family="Helvetica,sans-Serif" font-size="14.00">content</text> -<text text-anchor="start" x="1437" y="-2297.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1442" y="-2297.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="1379,-2263 1379,-2288 1645,-2288 1645,-2263 1379,-2263"/> -<text text-anchor="start" x="1384" y="-2272.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> -<text text-anchor="start" x="1457" y="-2272.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1462" y="-2272.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1558" y="-2272.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1379,-2238 1379,-2263 1645,-2263 1645,-2238 1379,-2238"/> -<text text-anchor="start" x="1384" y="-2247.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> -<text text-anchor="start" x="1463" y="-2247.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1468" y="-2247.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="1564" y="-2247.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1379,-2213 1379,-2238 1645,-2238 1645,-2213 1379,-2213"/> -<text text-anchor="start" x="1384" y="-2222.8" font-family="Helvetica,sans-Serif" font-size="14.00">user_id</text> -<text text-anchor="start" x="1435" y="-2222.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1440" y="-2222.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(128)]</text> +<polygon fill="none" stroke="black" points="1732,-1680 1732,-1708 1998,-1708 1998,-1680 1732,-1680"/> +<text text-anchor="start" x="1779" y="-1691.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_instance_note</text> +<polygon fill="none" stroke="black" points="1732,-1655 1732,-1680 1998,-1680 1998,-1655 1732,-1655"/> +<text text-anchor="start" x="1737" y="-1664.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">dag_id</text> +<text text-anchor="start" x="1783" y="-1664.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1788" y="-1664.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1909" y="-1664.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1732,-1630 1732,-1655 1998,-1655 1998,-1630 1732,-1630"/> +<text text-anchor="start" x="1737" y="-1639.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">map_index</text> +<text text-anchor="start" x="1813" y="-1639.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1818" y="-1639.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1895" y="-1639.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1732,-1605 1732,-1630 1998,-1630 1998,-1605 1732,-1605"/> +<text text-anchor="start" x="1737" y="-1614.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">run_id</text> +<text text-anchor="start" x="1781" y="-1614.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1786" y="-1614.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1907" y="-1614.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1732,-1580 1732,-1605 1998,-1605 1998,-1580 1732,-1580"/> +<text text-anchor="start" x="1737" y="-1589.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">task_id</text> +<text text-anchor="start" x="1786" y="-1589.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1791" y="-1589.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1912" y="-1589.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1732,-1555 1732,-1580 1998,-1580 1998,-1555 1732,-1555"/> +<text text-anchor="start" x="1737" y="-1564.8" font-family="Helvetica,sans-Serif" font-size="14.00">content</text> +<text text-anchor="start" x="1790" y="-1564.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1795" y="-1564.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1732,-1530 1732,-1555 1998,-1555 1998,-1530 1732,-1530"/> +<text text-anchor="start" x="1737" y="-1539.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="1810" y="-1539.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1815" y="-1539.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1911" y="-1539.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1732,-1505 1732,-1530 1998,-1530 1998,-1505 1732,-1505"/> +<text text-anchor="start" x="1737" y="-1514.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> +<text text-anchor="start" x="1816" y="-1514.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1821" y="-1514.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1917" y="-1514.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1732,-1480 1732,-1505 1998,-1505 1998,-1480 1732,-1480"/> +<text text-anchor="start" x="1737" y="-1489.8" font-family="Helvetica,sans-Serif" font-size="14.00">user_id</text> +<text text-anchor="start" x="1788" y="-1489.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1793" y="-1489.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(128)]</text> </g> <!-- task_instance--task_instance_note --> -<g id="edge46" class="edge"> +<g id="edge51" class="edge"> <title>task_instance--task_instance_note</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.24,-2527.94C1291.99,-2492.54 1320.63,-2457.92 1350,-2427 1356.62,-2420.03 1363.64,-2413.12 1370.89,-2406.36"/> -<text text-anchor="start" x="1339.89" y="-2395.16" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.24" y="-2516.74" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1614.66,-1837.66C1640.4,-1786.89 1669.79,-1737.74 1703,-1694 1709.35,-1685.64 1716.38,-1677.51 1723.83,-1669.73"/> +<text text-anchor="start" x="1692.83" y="-1658.53" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1604.66" y="-1826.46" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_instance_note --> -<g id="edge47" class="edge"> +<g id="edge52" class="edge"> <title>task_instance--task_instance_note</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.24,-2545.74C1291.99,-2510.54 1320.63,-2475.92 1350,-2445 1356.62,-2438.03 1363.64,-2431.12 1370.89,-2424.34"/> -<text text-anchor="start" x="1339.89" y="-2413.14" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.24" y="-2534.54" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1623.88,-1837.73C1647.36,-1793.42 1673.69,-1750.6 1703,-1712 1709.35,-1703.64 1716.38,-1695.51 1723.83,-1687.7"/> +<text text-anchor="start" x="1692.83" y="-1676.5" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1623.88" y="-1826.53" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_instance_note --> -<g id="edge48" class="edge"> +<g id="edge53" class="edge"> <title>task_instance--task_instance_note</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.24,-2563.53C1291.99,-2528.54 1320.63,-2493.92 1350,-2463 1356.62,-2456.03 1363.64,-2449.12 1370.89,-2442.32"/> -<text text-anchor="start" x="1339.89" y="-2431.12" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.24" y="-2552.33" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.07,-1844.13C1651.96,-1804 1676.24,-1765.25 1703,-1730 1709.35,-1721.64 1716.38,-1713.51 1723.83,-1705.67"/> +<text text-anchor="start" x="1692.83" y="-1694.47" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.07" y="-1847.93" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_instance_note --> -<g id="edge49" class="edge"> +<g id="edge54" class="edge"> <title>task_instance--task_instance_note</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265.24,-2581.32C1291.99,-2546.54 1320.63,-2511.92 1350,-2481 1361.52,-2468.87 1374.24,-2456.91 1387.33,-2445.31"/> -<text text-anchor="start" x="1356.33" y="-2449.11" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265.24" y="-2570.12" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.07,-1862.05C1651.96,-1822 1676.24,-1783.25 1703,-1748 1712.47,-1735.52 1723.44,-1723.58 1735.12,-1712.23"/> +<text text-anchor="start" x="1704.12" y="-1716.03" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.07" y="-1865.85" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance_history --> -<g id="node38" class="node"> +<g id="node39" class="node"> <title>task_instance_history</title> -<polygon fill="none" stroke="black" points="1358,-2159 1358,-2187 1666,-2187 1666,-2159 1358,-2159"/> -<text text-anchor="start" x="1414.5" y="-2170.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_instance_history</text> -<polygon fill="none" stroke="black" points="1358,-2134 1358,-2159 1666,-2159 1666,-2134 1358,-2134"/> -<text text-anchor="start" x="1363" y="-2143.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="1376" y="-2143.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1381" y="-2143.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1458" y="-2143.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1358,-2109 1358,-2134 1666,-2134 1666,-2109 1358,-2109"/> -<text text-anchor="start" x="1363" y="-2118.8" font-family="Helvetica,sans-Serif" font-size="14.00">custom_operator_name</text> -<text text-anchor="start" x="1526" y="-2118.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1531" y="-2118.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="1358,-2084 1358,-2109 1666,-2109 1666,-2084 1358,-2084"/> -<text text-anchor="start" x="1363" y="-2093.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> -<text text-anchor="start" x="1409" y="-2093.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1414" y="-2093.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1535" y="-2093.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1358,-2059 1358,-2084 1666,-2084 1666,-2059 1358,-2059"/> -<text text-anchor="start" x="1363" y="-2068.8" font-family="Helvetica,sans-Serif" font-size="14.00">duration</text> -<text text-anchor="start" x="1422" y="-2068.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1427" y="-2068.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [DOUBLE_PRECISION]</text> -<polygon fill="none" stroke="black" points="1358,-2034 1358,-2059 1666,-2059 1666,-2034 1358,-2034"/> -<text text-anchor="start" x="1363" y="-2043.8" font-family="Helvetica,sans-Serif" font-size="14.00">end_date</text> -<text text-anchor="start" x="1427" y="-2043.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1432" y="-2043.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="1358,-2009 1358,-2034 1666,-2034 1666,-2009 1358,-2009"/> -<text text-anchor="start" x="1363" y="-2018.8" font-family="Helvetica,sans-Serif" font-size="14.00">executor</text> -<text text-anchor="start" x="1424" y="-2018.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1429" y="-2018.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="1358,-1984 1358,-2009 1666,-2009 1666,-1984 1358,-1984"/> -<text text-anchor="start" x="1363" y="-1993.8" font-family="Helvetica,sans-Serif" font-size="14.00">executor_config</text> -<text text-anchor="start" x="1473" y="-1993.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1478" y="-1993.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> -<polygon fill="none" stroke="black" points="1358,-1959 1358,-1984 1666,-1984 1666,-1959 1358,-1959"/> -<text text-anchor="start" x="1363" y="-1968.8" font-family="Helvetica,sans-Serif" font-size="14.00">external_executor_id</text> -<text text-anchor="start" x="1506" y="-1968.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1511" y="-1968.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="1358,-1934 1358,-1959 1666,-1959 1666,-1934 1358,-1934"/> -<text text-anchor="start" x="1363" y="-1943.8" font-family="Helvetica,sans-Serif" font-size="14.00">hostname</text> -<text text-anchor="start" x="1433" y="-1943.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1438" y="-1943.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="1358,-1909 1358,-1934 1666,-1934 1666,-1909 1358,-1909"/> -<text text-anchor="start" x="1363" y="-1918.8" font-family="Helvetica,sans-Serif" font-size="14.00">map_index</text> -<text text-anchor="start" x="1439" y="-1918.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1444" y="-1918.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1521" y="-1918.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1358,-1884 1358,-1909 1666,-1909 1666,-1884 1358,-1884"/> -<text text-anchor="start" x="1363" y="-1893.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_tries</text> -<text text-anchor="start" x="1431" y="-1893.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1436" y="-1893.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="1358,-1859 1358,-1884 1666,-1884 1666,-1859 1358,-1859"/> -<text text-anchor="start" x="1363" y="-1868.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_kwargs</text> -<text text-anchor="start" x="1451" y="-1868.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1456" y="-1868.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<polygon fill="none" stroke="black" points="1358,-1834 1358,-1859 1666,-1859 1666,-1834 1358,-1834"/> -<text text-anchor="start" x="1363" y="-1843.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_method</text> -<text text-anchor="start" x="1454" y="-1843.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1459" y="-1843.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="1358,-1809 1358,-1834 1666,-1834 1666,-1809 1358,-1809"/> -<text text-anchor="start" x="1363" y="-1818.8" font-family="Helvetica,sans-Serif" font-size="14.00">operator</text> -<text text-anchor="start" x="1423" y="-1818.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1428" y="-1818.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="1358,-1784 1358,-1809 1666,-1809 1666,-1784 1358,-1784"/> -<text text-anchor="start" x="1363" y="-1793.8" font-family="Helvetica,sans-Serif" font-size="14.00">pid</text> -<text text-anchor="start" x="1385" y="-1793.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1390" y="-1793.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="1358,-1759 1358,-1784 1666,-1784 1666,-1759 1358,-1759"/> -<text text-anchor="start" x="1363" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00">pool</text> -<text text-anchor="start" x="1393" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1398" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<text text-anchor="start" x="1519" y="-1768.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1358,-1734 1358,-1759 1666,-1759 1666,-1734 1358,-1734"/> -<text text-anchor="start" x="1363" y="-1743.8" font-family="Helvetica,sans-Serif" font-size="14.00">pool_slots</text> -<text text-anchor="start" x="1432" y="-1743.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1437" y="-1743.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1514" y="-1743.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1358,-1709 1358,-1734 1666,-1734 1666,-1709 1358,-1709"/> -<text text-anchor="start" x="1363" y="-1718.8" font-family="Helvetica,sans-Serif" font-size="14.00">priority_weight</text> -<text text-anchor="start" x="1467" y="-1718.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1472" y="-1718.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="1358,-1684 1358,-1709 1666,-1709 1666,-1684 1358,-1684"/> -<text text-anchor="start" x="1363" y="-1693.8" font-family="Helvetica,sans-Serif" font-size="14.00">queue</text> -<text text-anchor="start" x="1407" y="-1693.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1412" y="-1693.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<polygon fill="none" stroke="black" points="1358,-1659 1358,-1684 1666,-1684 1666,-1659 1358,-1659"/> -<text text-anchor="start" x="1363" y="-1668.8" font-family="Helvetica,sans-Serif" font-size="14.00">queued_by_job_id</text> -<text text-anchor="start" x="1487" y="-1668.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1492" y="-1668.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="1358,-1634 1358,-1659 1666,-1659 1666,-1634 1358,-1634"/> -<text text-anchor="start" x="1363" y="-1643.8" font-family="Helvetica,sans-Serif" font-size="14.00">queued_dttm</text> -<text text-anchor="start" x="1456" y="-1643.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1461" y="-1643.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="1358,-1609 1358,-1634 1666,-1634 1666,-1609 1358,-1609"/> -<text text-anchor="start" x="1363" y="-1618.8" font-family="Helvetica,sans-Serif" font-size="14.00">rendered_map_index</text> -<text text-anchor="start" x="1508" y="-1618.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1513" y="-1618.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<polygon fill="none" stroke="black" points="1358,-1584 1358,-1609 1666,-1609 1666,-1584 1358,-1584"/> -<text text-anchor="start" x="1363" y="-1593.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> -<text text-anchor="start" x="1407" y="-1593.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1412" y="-1593.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1533" y="-1593.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1358,-1559 1358,-1584 1666,-1584 1666,-1559 1358,-1559"/> -<text text-anchor="start" x="1363" y="-1568.8" font-family="Helvetica,sans-Serif" font-size="14.00">start_date</text> -<text text-anchor="start" x="1433" y="-1568.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1438" y="-1568.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="1358,-1534 1358,-1559 1666,-1559 1666,-1534 1358,-1534"/> -<text text-anchor="start" x="1363" y="-1543.8" font-family="Helvetica,sans-Serif" font-size="14.00">state</text> -<text text-anchor="start" x="1398" y="-1543.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1403" y="-1543.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(20)]</text> -<polygon fill="none" stroke="black" points="1358,-1509 1358,-1534 1666,-1534 1666,-1509 1358,-1509"/> -<text text-anchor="start" x="1363" y="-1518.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_display_name</text> -<text text-anchor="start" x="1495" y="-1518.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1500" y="-1518.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> -<polygon fill="none" stroke="black" points="1358,-1484 1358,-1509 1666,-1509 1666,-1484 1358,-1484"/> -<text text-anchor="start" x="1363" y="-1493.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_id</text> -<text text-anchor="start" x="1412" y="-1493.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1417" y="-1493.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="1538" y="-1493.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1358,-1459 1358,-1484 1666,-1484 1666,-1459 1358,-1459"/> -<text text-anchor="start" x="1363" y="-1468.8" font-family="Helvetica,sans-Serif" font-size="14.00">trigger_id</text> -<text text-anchor="start" x="1430" y="-1468.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1435" y="-1468.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="1358,-1434 1358,-1459 1666,-1459 1666,-1434 1358,-1434"/> -<text text-anchor="start" x="1363" y="-1443.8" font-family="Helvetica,sans-Serif" font-size="14.00">trigger_timeout</text> -<text text-anchor="start" x="1471" y="-1443.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1476" y="-1443.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="1358,-1409 1358,-1434 1666,-1434 1666,-1409 1358,-1409"/> -<text text-anchor="start" x="1363" y="-1418.8" font-family="Helvetica,sans-Serif" font-size="14.00">try_number</text> -<text text-anchor="start" x="1445" y="-1418.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1450" y="-1418.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1527" y="-1418.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1358,-1384 1358,-1409 1666,-1409 1666,-1384 1358,-1384"/> -<text text-anchor="start" x="1363" y="-1393.8" font-family="Helvetica,sans-Serif" font-size="14.00">unixname</text> -<text text-anchor="start" x="1433" y="-1393.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1438" y="-1393.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<polygon fill="none" stroke="black" points="1358,-1359 1358,-1384 1666,-1384 1666,-1359 1358,-1359"/> -<text text-anchor="start" x="1363" y="-1368.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> -<text text-anchor="start" x="1442" y="-1368.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1447" y="-1368.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1711,-3220 1711,-3248 2019,-3248 2019,-3220 1711,-3220"/> +<text text-anchor="start" x="1767.5" y="-3231.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">task_instance_history</text> +<polygon fill="none" stroke="black" points="1711,-3195 1711,-3220 2019,-3220 2019,-3195 1711,-3195"/> +<text text-anchor="start" x="1716" y="-3204.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="1729" y="-3204.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1734" y="-3204.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1811" y="-3204.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1711,-3170 1711,-3195 2019,-3195 2019,-3170 1711,-3170"/> +<text text-anchor="start" x="1716" y="-3179.8" font-family="Helvetica,sans-Serif" font-size="14.00">custom_operator_name</text> +<text text-anchor="start" x="1879" y="-3179.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1884" y="-3179.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1711,-3145 1711,-3170 2019,-3170 2019,-3145 1711,-3145"/> +<text text-anchor="start" x="1716" y="-3154.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> +<text text-anchor="start" x="1762" y="-3154.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1767" y="-3154.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1888" y="-3154.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1711,-3120 1711,-3145 2019,-3145 2019,-3120 1711,-3120"/> +<text text-anchor="start" x="1716" y="-3129.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_version_id</text> +<text text-anchor="start" x="1820" y="-3129.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1825" y="-3129.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [UUID]</text> +<polygon fill="none" stroke="black" points="1711,-3095 1711,-3120 2019,-3120 2019,-3095 1711,-3095"/> +<text text-anchor="start" x="1716" y="-3104.8" font-family="Helvetica,sans-Serif" font-size="14.00">duration</text> +<text text-anchor="start" x="1775" y="-3104.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1780" y="-3104.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [DOUBLE_PRECISION]</text> +<polygon fill="none" stroke="black" points="1711,-3070 1711,-3095 2019,-3095 2019,-3070 1711,-3070"/> +<text text-anchor="start" x="1716" y="-3079.8" font-family="Helvetica,sans-Serif" font-size="14.00">end_date</text> +<text text-anchor="start" x="1780" y="-3079.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1785" y="-3079.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1711,-3045 1711,-3070 2019,-3070 2019,-3045 1711,-3045"/> +<text text-anchor="start" x="1716" y="-3054.8" font-family="Helvetica,sans-Serif" font-size="14.00">executor</text> +<text text-anchor="start" x="1777" y="-3054.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1782" y="-3054.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1711,-3020 1711,-3045 2019,-3045 2019,-3020 1711,-3020"/> +<text text-anchor="start" x="1716" y="-3029.8" font-family="Helvetica,sans-Serif" font-size="14.00">executor_config</text> +<text text-anchor="start" x="1826" y="-3029.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1831" y="-3029.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> +<polygon fill="none" stroke="black" points="1711,-2995 1711,-3020 2019,-3020 2019,-2995 1711,-2995"/> +<text text-anchor="start" x="1716" y="-3004.8" font-family="Helvetica,sans-Serif" font-size="14.00">external_executor_id</text> +<text text-anchor="start" x="1859" y="-3004.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1864" y="-3004.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="1711,-2970 1711,-2995 2019,-2995 2019,-2970 1711,-2970"/> +<text text-anchor="start" x="1716" y="-2979.8" font-family="Helvetica,sans-Serif" font-size="14.00">hostname</text> +<text text-anchor="start" x="1786" y="-2979.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1791" y="-2979.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1711,-2945 1711,-2970 2019,-2970 2019,-2945 1711,-2945"/> +<text text-anchor="start" x="1716" y="-2954.8" font-family="Helvetica,sans-Serif" font-size="14.00">map_index</text> +<text text-anchor="start" x="1792" y="-2954.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1797" y="-2954.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1874" y="-2954.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1711,-2920 1711,-2945 2019,-2945 2019,-2920 1711,-2920"/> +<text text-anchor="start" x="1716" y="-2929.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_tries</text> +<text text-anchor="start" x="1784" y="-2929.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1789" y="-2929.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1711,-2895 1711,-2920 2019,-2920 2019,-2895 1711,-2895"/> +<text text-anchor="start" x="1716" y="-2904.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_kwargs</text> +<text text-anchor="start" x="1804" y="-2904.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1809" y="-2904.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<polygon fill="none" stroke="black" points="1711,-2870 1711,-2895 2019,-2895 2019,-2870 1711,-2870"/> +<text text-anchor="start" x="1716" y="-2879.8" font-family="Helvetica,sans-Serif" font-size="14.00">next_method</text> +<text text-anchor="start" x="1807" y="-2879.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1812" y="-2879.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1711,-2845 1711,-2870 2019,-2870 2019,-2845 1711,-2845"/> +<text text-anchor="start" x="1716" y="-2854.8" font-family="Helvetica,sans-Serif" font-size="14.00">operator</text> +<text text-anchor="start" x="1776" y="-2854.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1781" y="-2854.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1711,-2820 1711,-2845 2019,-2845 2019,-2820 1711,-2820"/> +<text text-anchor="start" x="1716" y="-2829.8" font-family="Helvetica,sans-Serif" font-size="14.00">pid</text> +<text text-anchor="start" x="1738" y="-2829.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1743" y="-2829.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1711,-2795 1711,-2820 2019,-2820 2019,-2795 1711,-2795"/> +<text text-anchor="start" x="1716" y="-2804.8" font-family="Helvetica,sans-Serif" font-size="14.00">pool</text> +<text text-anchor="start" x="1746" y="-2804.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1751" y="-2804.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<text text-anchor="start" x="1872" y="-2804.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1711,-2770 1711,-2795 2019,-2795 2019,-2770 1711,-2770"/> +<text text-anchor="start" x="1716" y="-2779.8" font-family="Helvetica,sans-Serif" font-size="14.00">pool_slots</text> +<text text-anchor="start" x="1785" y="-2779.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1790" y="-2779.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1867" y="-2779.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1711,-2745 1711,-2770 2019,-2770 2019,-2745 1711,-2745"/> +<text text-anchor="start" x="1716" y="-2754.8" font-family="Helvetica,sans-Serif" font-size="14.00">priority_weight</text> +<text text-anchor="start" x="1820" y="-2754.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1825" y="-2754.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1711,-2720 1711,-2745 2019,-2745 2019,-2720 1711,-2720"/> +<text text-anchor="start" x="1716" y="-2729.8" font-family="Helvetica,sans-Serif" font-size="14.00">queue</text> +<text text-anchor="start" x="1760" y="-2729.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1765" y="-2729.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<polygon fill="none" stroke="black" points="1711,-2695 1711,-2720 2019,-2720 2019,-2695 1711,-2695"/> +<text text-anchor="start" x="1716" y="-2704.8" font-family="Helvetica,sans-Serif" font-size="14.00">queued_by_job_id</text> +<text text-anchor="start" x="1840" y="-2704.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1845" y="-2704.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1711,-2670 1711,-2695 2019,-2695 2019,-2670 1711,-2670"/> +<text text-anchor="start" x="1716" y="-2679.8" font-family="Helvetica,sans-Serif" font-size="14.00">queued_dttm</text> +<text text-anchor="start" x="1809" y="-2679.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1814" y="-2679.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1711,-2645 1711,-2670 2019,-2670 2019,-2645 1711,-2645"/> +<text text-anchor="start" x="1716" y="-2654.8" font-family="Helvetica,sans-Serif" font-size="14.00">rendered_map_index</text> +<text text-anchor="start" x="1861" y="-2654.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1866" y="-2654.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<polygon fill="none" stroke="black" points="1711,-2620 1711,-2645 2019,-2645 2019,-2620 1711,-2620"/> +<text text-anchor="start" x="1716" y="-2629.8" font-family="Helvetica,sans-Serif" font-size="14.00">run_id</text> +<text text-anchor="start" x="1760" y="-2629.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1765" y="-2629.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1886" y="-2629.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1711,-2595 1711,-2620 2019,-2620 2019,-2595 1711,-2595"/> +<text text-anchor="start" x="1716" y="-2604.8" font-family="Helvetica,sans-Serif" font-size="14.00">start_date</text> +<text text-anchor="start" x="1786" y="-2604.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1791" y="-2604.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1711,-2570 1711,-2595 2019,-2595 2019,-2570 1711,-2570"/> +<text text-anchor="start" x="1716" y="-2579.8" font-family="Helvetica,sans-Serif" font-size="14.00">state</text> +<text text-anchor="start" x="1751" y="-2579.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1756" y="-2579.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(20)]</text> +<polygon fill="none" stroke="black" points="1711,-2545 1711,-2570 2019,-2570 2019,-2545 1711,-2545"/> +<text text-anchor="start" x="1716" y="-2554.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_display_name</text> +<text text-anchor="start" x="1848" y="-2554.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1853" y="-2554.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(2000)]</text> +<polygon fill="none" stroke="black" points="1711,-2520 1711,-2545 2019,-2545 2019,-2520 1711,-2520"/> +<text text-anchor="start" x="1716" y="-2529.8" font-family="Helvetica,sans-Serif" font-size="14.00">task_id</text> +<text text-anchor="start" x="1765" y="-2529.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1770" y="-2529.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="1891" y="-2529.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1711,-2495 1711,-2520 2019,-2520 2019,-2495 1711,-2495"/> +<text text-anchor="start" x="1716" y="-2504.8" font-family="Helvetica,sans-Serif" font-size="14.00">trigger_id</text> +<text text-anchor="start" x="1783" y="-2504.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1788" y="-2504.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="1711,-2470 1711,-2495 2019,-2495 2019,-2470 1711,-2470"/> +<text text-anchor="start" x="1716" y="-2479.8" font-family="Helvetica,sans-Serif" font-size="14.00">trigger_timeout</text> +<text text-anchor="start" x="1824" y="-2479.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1829" y="-2479.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="1711,-2445 1711,-2470 2019,-2470 2019,-2445 1711,-2445"/> +<text text-anchor="start" x="1716" y="-2454.8" font-family="Helvetica,sans-Serif" font-size="14.00">try_number</text> +<text text-anchor="start" x="1798" y="-2454.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1803" y="-2454.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1880" y="-2454.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="1711,-2420 1711,-2445 2019,-2445 2019,-2420 1711,-2420"/> +<text text-anchor="start" x="1716" y="-2429.8" font-family="Helvetica,sans-Serif" font-size="14.00">unixname</text> +<text text-anchor="start" x="1786" y="-2429.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1791" y="-2429.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<polygon fill="none" stroke="black" points="1711,-2395 1711,-2420 2019,-2420 2019,-2395 1711,-2395"/> +<text text-anchor="start" x="1716" y="-2404.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> +<text text-anchor="start" x="1795" y="-2404.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1800" y="-2404.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> </g> <!-- task_instance--task_instance_history --> -<g id="edge50" class="edge"> +<g id="edge55" class="edge"> <title>task_instance--task_instance_history</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265,-2374.49C1292.94,-2303.4 1321.99,-2230.03 1349.92,-2160"/> -<text text-anchor="start" x="1318.92" y="-2148.8" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265" y="-2363.29" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2481.88C1654.23,-2514.34 1678.97,-2548.23 1702.96,-2581.5"/> +<text text-anchor="start" x="1671.96" y="-2570.3" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2470.68" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_instance_history --> -<g id="edge51" class="edge"> +<g id="edge56" class="edge"> <title>task_instance--task_instance_history</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265,-2387.72C1292.94,-2316.99 1321.99,-2243.61 1349.92,-2173.23"/> -<text text-anchor="start" x="1318.92" y="-2177.03" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265" y="-2391.52" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2495.18C1654.23,-2527.91 1678.97,-2561.79 1702.96,-2594.8"/> +<text text-anchor="start" x="1671.96" y="-2598.6" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2498.98" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_instance_history --> -<g id="edge52" class="edge"> +<g id="edge57" class="edge"> <title>task_instance--task_instance_history</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265,-2400.96C1292.94,-2330.58 1321.99,-2257.2 1349.92,-2186.47"/> -<text text-anchor="start" x="1318.92" y="-2190.27" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265" y="-2404.76" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2508.47C1654.23,-2541.48 1678.97,-2575.36 1702.96,-2608.09"/> +<text text-anchor="start" x="1671.96" y="-2611.89" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2512.27" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- task_instance--task_instance_history --> -<g id="edge53" class="edge"> +<g id="edge58" class="edge"> <title>task_instance--task_instance_history</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1265,-2414.19C1294.05,-2341.38 1324.3,-2264.95 1353.25,-2191.24"/> -<text text-anchor="start" x="1353.25" y="-2195.04" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="1265" y="-2417.99" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1630.23,-2521.77C1654.23,-2555.05 1678.97,-2588.93 1702.96,-2621.38"/> +<text text-anchor="start" x="1671.96" y="-2625.18" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1630.23" y="-2525.57" font-family="Times,serif" font-size="14.00">1</text> +</g> +<!-- log_template --> +<g id="node29" class="node"> +<title>log_template</title> +<polygon fill="none" stroke="black" points="544.5,-2637 544.5,-2665 804.5,-2665 804.5,-2637 544.5,-2637"/> +<text text-anchor="start" x="616.5" y="-2648.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">log_template</text> +<polygon fill="none" stroke="black" points="544.5,-2612 544.5,-2637 804.5,-2637 804.5,-2612 544.5,-2612"/> +<text text-anchor="start" x="549.5" y="-2621.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="562.5" y="-2621.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="567.5" y="-2621.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="644.5" y="-2621.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="544.5,-2587 544.5,-2612 804.5,-2612 804.5,-2587 544.5,-2587"/> +<text text-anchor="start" x="549.5" y="-2596.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="622.5" y="-2596.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="627.5" y="-2596.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="723.5" y="-2596.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="544.5,-2562 544.5,-2587 804.5,-2587 804.5,-2562 544.5,-2562"/> +<text text-anchor="start" x="549.5" y="-2571.8" font-family="Helvetica,sans-Serif" font-size="14.00">elasticsearch_id</text> +<text text-anchor="start" x="659.5" y="-2571.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="664.5" y="-2571.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<text text-anchor="start" x="714.5" y="-2571.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="544.5,-2537 544.5,-2562 804.5,-2562 804.5,-2537 544.5,-2537"/> +<text text-anchor="start" x="549.5" y="-2546.8" font-family="Helvetica,sans-Serif" font-size="14.00">filename</text> +<text text-anchor="start" x="610.5" y="-2546.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="615.5" y="-2546.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<text text-anchor="start" x="665.5" y="-2546.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +</g> +<!-- log_template--dag_run --> +<g id="edge24" class="edge"> +<title>log_template--dag_run</title> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M812.73,-2655.62C850.88,-2670.8 892.48,-2687.36 931.25,-2702.79"/> +<text text-anchor="start" x="900.25" y="-2691.59" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="812.73" y="-2644.42" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- backfill --> -<g id="node32" class="node"> +<g id="node33" class="node"> <title>backfill</title> -<polygon fill="none" stroke="black" points="8.5,-2546 8.5,-2574 355.5,-2574 355.5,-2546 8.5,-2546"/> -<text text-anchor="start" x="149.5" y="-2557.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">backfill</text> -<polygon fill="none" stroke="black" points="8.5,-2521 8.5,-2546 355.5,-2546 355.5,-2521 8.5,-2521"/> -<text text-anchor="start" x="13.5" y="-2530.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="26.5" y="-2530.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="31.5" y="-2530.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="108.5" y="-2530.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="8.5,-2496 8.5,-2521 355.5,-2521 355.5,-2496 8.5,-2496"/> -<text text-anchor="start" x="13.5" y="-2505.8" font-family="Helvetica,sans-Serif" font-size="14.00">completed_at</text> -<text text-anchor="start" x="107.5" y="-2505.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="112.5" y="-2505.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="8.5,-2471 8.5,-2496 355.5,-2496 355.5,-2471 8.5,-2471"/> -<text text-anchor="start" x="13.5" y="-2480.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> -<text text-anchor="start" x="86.5" y="-2480.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="91.5" y="-2480.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="187.5" y="-2480.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="8.5,-2446 8.5,-2471 355.5,-2471 355.5,-2446 8.5,-2446"/> -<text text-anchor="start" x="13.5" y="-2455.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> -<text text-anchor="start" x="59.5" y="-2455.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="64.5" y="-2455.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="185.5" y="-2455.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="8.5,-2421 8.5,-2446 355.5,-2446 355.5,-2421 8.5,-2421"/> -<text text-anchor="start" x="13.5" y="-2430.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_run_conf</text> -<text text-anchor="start" x="107.5" y="-2430.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="112.5" y="-2430.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> -<text text-anchor="start" x="163.5" y="-2430.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="8.5,-2396 8.5,-2421 355.5,-2421 355.5,-2396 8.5,-2396"/> -<text text-anchor="start" x="13.5" y="-2405.8" font-family="Helvetica,sans-Serif" font-size="14.00">from_date</text> -<text text-anchor="start" x="83.5" y="-2405.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="88.5" y="-2405.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="184.5" y="-2405.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="8.5,-2371 8.5,-2396 355.5,-2396 355.5,-2371 8.5,-2371"/> -<text text-anchor="start" x="13.5" y="-2380.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_paused</text> -<text text-anchor="start" x="83.5" y="-2380.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="88.5" y="-2380.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<polygon fill="none" stroke="black" points="8.5,-2346 8.5,-2371 355.5,-2371 355.5,-2346 8.5,-2346"/> -<text text-anchor="start" x="13.5" y="-2355.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_active_runs</text> -<text text-anchor="start" x="130.5" y="-2355.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="135.5" y="-2355.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="212.5" y="-2355.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="8.5,-2321 8.5,-2346 355.5,-2346 355.5,-2321 8.5,-2321"/> -<text text-anchor="start" x="13.5" y="-2330.8" font-family="Helvetica,sans-Serif" font-size="14.00">reprocess_behavior</text> -<text text-anchor="start" x="148.5" y="-2330.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="153.5" y="-2330.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="274.5" y="-2330.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="8.5,-2296 8.5,-2321 355.5,-2321 355.5,-2296 8.5,-2296"/> -<text text-anchor="start" x="13.5" y="-2305.8" font-family="Helvetica,sans-Serif" font-size="14.00">to_date</text> -<text text-anchor="start" x="65.5" y="-2305.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="70.5" y="-2305.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="166.5" y="-2305.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="8.5,-2271 8.5,-2296 355.5,-2296 355.5,-2271 8.5,-2271"/> -<text text-anchor="start" x="13.5" y="-2280.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> -<text text-anchor="start" x="92.5" y="-2280.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="97.5" y="-2280.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="193.5" y="-2280.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="501.5,-2966 501.5,-2994 848.5,-2994 848.5,-2966 501.5,-2966"/> +<text text-anchor="start" x="642.5" y="-2977.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">backfill</text> +<polygon fill="none" stroke="black" points="501.5,-2941 501.5,-2966 848.5,-2966 848.5,-2941 501.5,-2941"/> +<text text-anchor="start" x="506.5" y="-2950.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="519.5" y="-2950.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="524.5" y="-2950.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="601.5" y="-2950.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="501.5,-2916 501.5,-2941 848.5,-2941 848.5,-2916 501.5,-2916"/> +<text text-anchor="start" x="506.5" y="-2925.8" font-family="Helvetica,sans-Serif" font-size="14.00">completed_at</text> +<text text-anchor="start" x="600.5" y="-2925.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="605.5" y="-2925.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="501.5,-2891 501.5,-2916 848.5,-2916 848.5,-2891 501.5,-2891"/> +<text text-anchor="start" x="506.5" y="-2900.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_at</text> +<text text-anchor="start" x="579.5" y="-2900.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="584.5" y="-2900.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="680.5" y="-2900.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="501.5,-2866 501.5,-2891 848.5,-2891 848.5,-2866 501.5,-2866"/> +<text text-anchor="start" x="506.5" y="-2875.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_id</text> +<text text-anchor="start" x="552.5" y="-2875.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="557.5" y="-2875.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="678.5" y="-2875.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="501.5,-2841 501.5,-2866 848.5,-2866 848.5,-2841 501.5,-2841"/> +<text text-anchor="start" x="506.5" y="-2850.8" font-family="Helvetica,sans-Serif" font-size="14.00">dag_run_conf</text> +<text text-anchor="start" x="600.5" y="-2850.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="605.5" y="-2850.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [JSON]</text> +<text text-anchor="start" x="656.5" y="-2850.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="501.5,-2816 501.5,-2841 848.5,-2841 848.5,-2816 501.5,-2816"/> +<text text-anchor="start" x="506.5" y="-2825.8" font-family="Helvetica,sans-Serif" font-size="14.00">from_date</text> +<text text-anchor="start" x="576.5" y="-2825.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="581.5" y="-2825.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="677.5" y="-2825.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="501.5,-2791 501.5,-2816 848.5,-2816 848.5,-2791 501.5,-2791"/> +<text text-anchor="start" x="506.5" y="-2800.8" font-family="Helvetica,sans-Serif" font-size="14.00">is_paused</text> +<text text-anchor="start" x="576.5" y="-2800.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="581.5" y="-2800.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<polygon fill="none" stroke="black" points="501.5,-2766 501.5,-2791 848.5,-2791 848.5,-2766 501.5,-2766"/> +<text text-anchor="start" x="506.5" y="-2775.8" font-family="Helvetica,sans-Serif" font-size="14.00">max_active_runs</text> +<text text-anchor="start" x="623.5" y="-2775.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="628.5" y="-2775.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="705.5" y="-2775.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="501.5,-2741 501.5,-2766 848.5,-2766 848.5,-2741 501.5,-2741"/> +<text text-anchor="start" x="506.5" y="-2750.8" font-family="Helvetica,sans-Serif" font-size="14.00">reprocess_behavior</text> +<text text-anchor="start" x="641.5" y="-2750.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="646.5" y="-2750.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="767.5" y="-2750.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="501.5,-2716 501.5,-2741 848.5,-2741 848.5,-2716 501.5,-2716"/> +<text text-anchor="start" x="506.5" y="-2725.8" font-family="Helvetica,sans-Serif" font-size="14.00">to_date</text> +<text text-anchor="start" x="558.5" y="-2725.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="563.5" y="-2725.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="659.5" y="-2725.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="501.5,-2691 501.5,-2716 848.5,-2716 848.5,-2691 501.5,-2691"/> +<text text-anchor="start" x="506.5" y="-2700.8" font-family="Helvetica,sans-Serif" font-size="14.00">updated_at</text> +<text text-anchor="start" x="585.5" y="-2700.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="590.5" y="-2700.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="686.5" y="-2700.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- backfill--dag_run --> -<g id="edge27" class="edge"> +<g id="edge32" class="edge"> <title>backfill--dag_run</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M363.3,-2570.77C407.39,-2606.8 454.04,-2644.93 496.43,-2679.57"/> -<text text-anchor="start" x="465.43" y="-2668.37" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="363.3" y="-2559.57" font-family="Times,serif" font-size="14.00">{0,1}</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M856.16,-2806.82C881.32,-2801.79 906.92,-2796.66 931.4,-2791.76"/> +<text text-anchor="start" x="900.4" y="-2780.56" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="856.16" y="-2795.62" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- backfill--backfill_dag_run --> -<g id="edge28" class="edge"> +<g id="edge33" class="edge"> <title>backfill--backfill_dag_run</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M363.04,-2352.35C387.3,-2344.58 412.08,-2337.51 436,-2332 614.08,-2291.02 824.45,-2273.22 960.39,-2265.59"/> -<text text-anchor="start" x="929.39" y="-2254.39" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="363.04" y="-2341.15" font-family="Times,serif" font-size="14.00">1</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M849.13,-2998.62C880.61,-3023.56 909.57,-3043.64 929,-3051 1056.72,-3099.39 1213.92,-3099.05 1325.34,-3089.85"/> +<text text-anchor="start" x="1294.34" y="-3078.65" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="839.13" y="-3002.42" font-family="Times,serif" font-size="14.00">1</text> </g> <!-- trigger --> -<g id="node33" class="node"> +<g id="node34" class="node"> <title>trigger</title> -<polygon fill="none" stroke="black" points="502,-2470 502,-2498 790,-2498 790,-2470 502,-2470"/> -<text text-anchor="start" x="614.5" y="-2481.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">trigger</text> -<polygon fill="none" stroke="black" points="502,-2445 502,-2470 790,-2470 790,-2445 502,-2445"/> -<text text-anchor="start" x="507" y="-2454.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="520" y="-2454.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="525" y="-2454.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="602" y="-2454.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="502,-2420 502,-2445 790,-2445 790,-2420 502,-2420"/> -<text text-anchor="start" x="507" y="-2429.8" font-family="Helvetica,sans-Serif" font-size="14.00">classpath</text> -<text text-anchor="start" x="574" y="-2429.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="579" y="-2429.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> -<text text-anchor="start" x="709" y="-2429.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="502,-2395 502,-2420 790,-2420 790,-2395 502,-2395"/> -<text text-anchor="start" x="507" y="-2404.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_date</text> -<text text-anchor="start" x="597" y="-2404.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="602" y="-2404.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<text text-anchor="start" x="698" y="-2404.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="502,-2370 502,-2395 790,-2395 790,-2370 502,-2370"/> -<text text-anchor="start" x="507" y="-2379.8" font-family="Helvetica,sans-Serif" font-size="14.00">kwargs</text> -<text text-anchor="start" x="557" y="-2379.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="562" y="-2379.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> -<text text-anchor="start" x="612" y="-2379.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="502,-2345 502,-2370 790,-2370 790,-2345 502,-2345"/> -<text text-anchor="start" x="507" y="-2354.8" font-family="Helvetica,sans-Serif" font-size="14.00">triggerer_id</text> -<text text-anchor="start" x="588" y="-2354.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="593" y="-2354.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="937,-2431 937,-2459 1225,-2459 1225,-2431 937,-2431"/> +<text text-anchor="start" x="1049.5" y="-2442.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">trigger</text> +<polygon fill="none" stroke="black" points="937,-2406 937,-2431 1225,-2431 1225,-2406 937,-2406"/> +<text text-anchor="start" x="942" y="-2415.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="955" y="-2415.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="960" y="-2415.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1037" y="-2415.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="937,-2381 937,-2406 1225,-2406 1225,-2381 937,-2381"/> +<text text-anchor="start" x="942" y="-2390.8" font-family="Helvetica,sans-Serif" font-size="14.00">classpath</text> +<text text-anchor="start" x="1009" y="-2390.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1014" y="-2390.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(1000)]</text> +<text text-anchor="start" x="1144" y="-2390.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="937,-2356 937,-2381 1225,-2381 1225,-2356 937,-2356"/> +<text text-anchor="start" x="942" y="-2365.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_date</text> +<text text-anchor="start" x="1032" y="-2365.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1037" y="-2365.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<text text-anchor="start" x="1133" y="-2365.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="937,-2331 937,-2356 1225,-2356 1225,-2331 937,-2331"/> +<text text-anchor="start" x="942" y="-2340.8" font-family="Helvetica,sans-Serif" font-size="14.00">kwargs</text> +<text text-anchor="start" x="992" y="-2340.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="997" y="-2340.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TEXT]</text> +<text text-anchor="start" x="1047" y="-2340.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="937,-2306 937,-2331 1225,-2331 1225,-2306 937,-2306"/> +<text text-anchor="start" x="942" y="-2315.8" font-family="Helvetica,sans-Serif" font-size="14.00">triggerer_id</text> +<text text-anchor="start" x="1023" y="-2315.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1028" y="-2315.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> </g> <!-- trigger--task_instance --> -<g id="edge29" class="edge"> +<g id="edge34" class="edge"> <title>trigger--task_instance</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M798.09,-2476.24C818.4,-2486.38 838.3,-2497.96 856,-2511 885.77,-2532.93 914.28,-2559.14 940.67,-2586.71"/> -<text text-anchor="start" x="909.67" y="-2575.51" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="798.09" y="-2465.04" font-family="Times,serif" font-size="14.00">{0,1}</text> -</g> -<!-- alembic_version --> -<g id="node39" class="node"> -<title>alembic_version</title> -<polygon fill="none" stroke="black" points="35.5,-3350 35.5,-3378 328.5,-3378 328.5,-3350 35.5,-3350"/> -<text text-anchor="start" x="109.5" y="-3361.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">alembic_version</text> -<polygon fill="none" stroke="black" points="35.5,-3325 35.5,-3350 328.5,-3350 328.5,-3325 35.5,-3325"/> -<text text-anchor="start" x="40.5" y="-3334.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">version_num</text> -<text text-anchor="start" x="130.5" y="-3334.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="135.5" y="-3334.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(32)]</text> -<text text-anchor="start" x="247.5" y="-3334.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M1233.06,-2342.98C1256.97,-2336.64 1281.77,-2330.07 1305.9,-2323.68"/> +<text text-anchor="start" x="1274.9" y="-2312.48" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="1233.06" y="-2331.78" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- session --> <g id="node40" class="node"> <title>session</title> -<polygon fill="none" stroke="black" points="77.5,-4231 77.5,-4259 285.5,-4259 285.5,-4231 77.5,-4231"/> -<text text-anchor="start" x="147.5" y="-4242.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">session</text> -<polygon fill="none" stroke="black" points="77.5,-4206 77.5,-4231 285.5,-4231 285.5,-4206 77.5,-4206"/> -<text text-anchor="start" x="82.5" y="-4215.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="95.5" y="-4215.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="100.5" y="-4215.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="177.5" y="-4215.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="77.5,-4181 77.5,-4206 285.5,-4206 285.5,-4181 77.5,-4181"/> -<text text-anchor="start" x="82.5" y="-4190.8" font-family="Helvetica,sans-Serif" font-size="14.00">data</text> -<text text-anchor="start" x="113.5" y="-4190.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="118.5" y="-4190.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> -<polygon fill="none" stroke="black" points="77.5,-4156 77.5,-4181 285.5,-4181 285.5,-4156 77.5,-4156"/> -<text text-anchor="start" x="82.5" y="-4165.8" font-family="Helvetica,sans-Serif" font-size="14.00">expiry</text> -<text text-anchor="start" x="126.5" y="-4165.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="131.5" y="-4165.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="77.5,-4131 77.5,-4156 285.5,-4156 285.5,-4131 77.5,-4131"/> -<text text-anchor="start" x="82.5" y="-4140.8" font-family="Helvetica,sans-Serif" font-size="14.00">session_id</text> -<text text-anchor="start" x="154.5" y="-4140.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="159.5" y="-4140.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(255)]</text> +<polygon fill="none" stroke="black" points="106,-3935 106,-3963 314,-3963 314,-3935 106,-3935"/> +<text text-anchor="start" x="176" y="-3946.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">session</text> +<polygon fill="none" stroke="black" points="106,-3910 106,-3935 314,-3935 314,-3910 106,-3910"/> +<text text-anchor="start" x="111" y="-3919.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="124" y="-3919.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="129" y="-3919.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="206" y="-3919.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="106,-3885 106,-3910 314,-3910 314,-3885 106,-3885"/> +<text text-anchor="start" x="111" y="-3894.8" font-family="Helvetica,sans-Serif" font-size="14.00">data</text> +<text text-anchor="start" x="142" y="-3894.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="147" y="-3894.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BYTEA]</text> +<polygon fill="none" stroke="black" points="106,-3860 106,-3885 314,-3885 314,-3860 106,-3860"/> +<text text-anchor="start" x="111" y="-3869.8" font-family="Helvetica,sans-Serif" font-size="14.00">expiry</text> +<text text-anchor="start" x="155" y="-3869.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="160" y="-3869.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="106,-3835 106,-3860 314,-3860 314,-3835 106,-3835"/> +<text text-anchor="start" x="111" y="-3844.8" font-family="Helvetica,sans-Serif" font-size="14.00">session_id</text> +<text text-anchor="start" x="183" y="-3844.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="188" y="-3844.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(255)]</text> </g> -<!-- ab_user --> +<!-- alembic_version --> <g id="node41" class="node"> +<title>alembic_version</title> +<polygon fill="none" stroke="black" points="64,-4014 64,-4042 357,-4042 357,-4014 64,-4014"/> +<text text-anchor="start" x="138" y="-4025.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">alembic_version</text> +<polygon fill="none" stroke="black" points="64,-3989 64,-4014 357,-4014 357,-3989 64,-3989"/> +<text text-anchor="start" x="69" y="-3998.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">version_num</text> +<text text-anchor="start" x="159" y="-3998.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="164" y="-3998.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(32)]</text> +<text text-anchor="start" x="276" y="-3998.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +</g> +<!-- ab_user --> +<g id="node42" class="node"> <title>ab_user</title> -<polygon fill="none" stroke="black" points="503,-5271 503,-5299 789,-5299 789,-5271 503,-5271"/> -<text text-anchor="start" x="611" y="-5282.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_user</text> -<polygon fill="none" stroke="black" points="503,-5246 503,-5271 789,-5271 789,-5246 503,-5246"/> -<text text-anchor="start" x="508" y="-5255.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="521" y="-5255.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="526" y="-5255.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="603" y="-5255.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="503,-5221 503,-5246 789,-5246 789,-5221 503,-5221"/> -<text text-anchor="start" x="508" y="-5230.8" font-family="Helvetica,sans-Serif" font-size="14.00">active</text> -<text text-anchor="start" x="550" y="-5230.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="555" y="-5230.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> -<polygon fill="none" stroke="black" points="503,-5196 503,-5221 789,-5221 789,-5196 503,-5196"/> -<text text-anchor="start" x="508" y="-5205.8" font-family="Helvetica,sans-Serif" font-size="14.00">changed_by_fk</text> -<text text-anchor="start" x="613" y="-5205.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="618" y="-5205.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="503,-5171 503,-5196 789,-5196 789,-5171 503,-5171"/> -<text text-anchor="start" x="508" y="-5180.8" font-family="Helvetica,sans-Serif" font-size="14.00">changed_on</text> -<text text-anchor="start" x="593" y="-5180.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="598" y="-5180.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="503,-5146 503,-5171 789,-5171 789,-5146 503,-5146"/> -<text text-anchor="start" x="508" y="-5155.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_by_fk</text> -<text text-anchor="start" x="605" y="-5155.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="610" y="-5155.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="503,-5121 503,-5146 789,-5146 789,-5121 503,-5121"/> -<text text-anchor="start" x="508" y="-5130.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_on</text> -<text text-anchor="start" x="584" y="-5130.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="589" y="-5130.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="503,-5096 503,-5121 789,-5121 789,-5096 503,-5096"/> -<text text-anchor="start" x="508" y="-5105.8" font-family="Helvetica,sans-Serif" font-size="14.00">email</text> -<text text-anchor="start" x="546" y="-5105.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="551" y="-5105.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(512)]</text> -<text text-anchor="start" x="672" y="-5105.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="503,-5071 503,-5096 789,-5096 789,-5071 503,-5071"/> -<text text-anchor="start" x="508" y="-5080.8" font-family="Helvetica,sans-Serif" font-size="14.00">fail_login_count</text> -<text text-anchor="start" x="616" y="-5080.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="621" y="-5080.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="503,-5046 503,-5071 789,-5071 789,-5046 503,-5046"/> -<text text-anchor="start" x="508" y="-5055.8" font-family="Helvetica,sans-Serif" font-size="14.00">first_name</text> -<text text-anchor="start" x="582" y="-5055.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="587" y="-5055.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<text text-anchor="start" x="708" y="-5055.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="503,-5021 503,-5046 789,-5046 789,-5021 503,-5021"/> -<text text-anchor="start" x="508" y="-5030.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_login</text> -<text text-anchor="start" x="574" y="-5030.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="579" y="-5030.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="503,-4996 503,-5021 789,-5021 789,-4996 503,-4996"/> -<text text-anchor="start" x="508" y="-5005.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_name</text> -<text text-anchor="start" x="579" y="-5005.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="584" y="-5005.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<text text-anchor="start" x="705" y="-5005.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="503,-4971 503,-4996 789,-4996 789,-4971 503,-4971"/> -<text text-anchor="start" x="508" y="-4980.8" font-family="Helvetica,sans-Serif" font-size="14.00">login_count</text> -<text text-anchor="start" x="588" y="-4980.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="593" y="-4980.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="503,-4946 503,-4971 789,-4971 789,-4946 503,-4946"/> -<text text-anchor="start" x="508" y="-4955.8" font-family="Helvetica,sans-Serif" font-size="14.00">password</text> -<text text-anchor="start" x="575" y="-4955.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="580" y="-4955.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<polygon fill="none" stroke="black" points="503,-4921 503,-4946 789,-4946 789,-4921 503,-4921"/> -<text text-anchor="start" x="508" y="-4930.8" font-family="Helvetica,sans-Serif" font-size="14.00">username</text> -<text text-anchor="start" x="578" y="-4930.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="583" y="-4930.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(512)]</text> -<text text-anchor="start" x="704" y="-4930.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="531.5,-4692 531.5,-4720 817.5,-4720 817.5,-4692 531.5,-4692"/> +<text text-anchor="start" x="639.5" y="-4703.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_user</text> +<polygon fill="none" stroke="black" points="531.5,-4667 531.5,-4692 817.5,-4692 817.5,-4667 531.5,-4667"/> +<text text-anchor="start" x="536.5" y="-4676.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="549.5" y="-4676.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="554.5" y="-4676.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="631.5" y="-4676.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="531.5,-4642 531.5,-4667 817.5,-4667 817.5,-4642 531.5,-4642"/> +<text text-anchor="start" x="536.5" y="-4651.8" font-family="Helvetica,sans-Serif" font-size="14.00">active</text> +<text text-anchor="start" x="578.5" y="-4651.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="583.5" y="-4651.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [BOOLEAN]</text> +<polygon fill="none" stroke="black" points="531.5,-4617 531.5,-4642 817.5,-4642 817.5,-4617 531.5,-4617"/> +<text text-anchor="start" x="536.5" y="-4626.8" font-family="Helvetica,sans-Serif" font-size="14.00">changed_by_fk</text> +<text text-anchor="start" x="641.5" y="-4626.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="646.5" y="-4626.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="531.5,-4592 531.5,-4617 817.5,-4617 817.5,-4592 531.5,-4592"/> +<text text-anchor="start" x="536.5" y="-4601.8" font-family="Helvetica,sans-Serif" font-size="14.00">changed_on</text> +<text text-anchor="start" x="621.5" y="-4601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="626.5" y="-4601.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="531.5,-4567 531.5,-4592 817.5,-4592 817.5,-4567 531.5,-4567"/> +<text text-anchor="start" x="536.5" y="-4576.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_by_fk</text> +<text text-anchor="start" x="633.5" y="-4576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="638.5" y="-4576.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="531.5,-4542 531.5,-4567 817.5,-4567 817.5,-4542 531.5,-4542"/> +<text text-anchor="start" x="536.5" y="-4551.8" font-family="Helvetica,sans-Serif" font-size="14.00">created_on</text> +<text text-anchor="start" x="612.5" y="-4551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="617.5" y="-4551.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="531.5,-4517 531.5,-4542 817.5,-4542 817.5,-4517 531.5,-4517"/> +<text text-anchor="start" x="536.5" y="-4526.8" font-family="Helvetica,sans-Serif" font-size="14.00">email</text> +<text text-anchor="start" x="574.5" y="-4526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="579.5" y="-4526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(512)]</text> +<text text-anchor="start" x="700.5" y="-4526.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="531.5,-4492 531.5,-4517 817.5,-4517 817.5,-4492 531.5,-4492"/> +<text text-anchor="start" x="536.5" y="-4501.8" font-family="Helvetica,sans-Serif" font-size="14.00">fail_login_count</text> +<text text-anchor="start" x="644.5" y="-4501.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="649.5" y="-4501.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="531.5,-4467 531.5,-4492 817.5,-4492 817.5,-4467 531.5,-4467"/> +<text text-anchor="start" x="536.5" y="-4476.8" font-family="Helvetica,sans-Serif" font-size="14.00">first_name</text> +<text text-anchor="start" x="610.5" y="-4476.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="615.5" y="-4476.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<text text-anchor="start" x="736.5" y="-4476.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="531.5,-4442 531.5,-4467 817.5,-4467 817.5,-4442 531.5,-4442"/> +<text text-anchor="start" x="536.5" y="-4451.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_login</text> +<text text-anchor="start" x="602.5" y="-4451.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="607.5" y="-4451.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="531.5,-4417 531.5,-4442 817.5,-4442 817.5,-4417 531.5,-4417"/> +<text text-anchor="start" x="536.5" y="-4426.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_name</text> +<text text-anchor="start" x="607.5" y="-4426.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="612.5" y="-4426.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<text text-anchor="start" x="733.5" y="-4426.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="531.5,-4392 531.5,-4417 817.5,-4417 817.5,-4392 531.5,-4392"/> +<text text-anchor="start" x="536.5" y="-4401.8" font-family="Helvetica,sans-Serif" font-size="14.00">login_count</text> +<text text-anchor="start" x="616.5" y="-4401.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="621.5" y="-4401.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="531.5,-4367 531.5,-4392 817.5,-4392 817.5,-4367 531.5,-4367"/> +<text text-anchor="start" x="536.5" y="-4376.8" font-family="Helvetica,sans-Serif" font-size="14.00">password</text> +<text text-anchor="start" x="603.5" y="-4376.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="608.5" y="-4376.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<polygon fill="none" stroke="black" points="531.5,-4342 531.5,-4367 817.5,-4367 817.5,-4342 531.5,-4342"/> +<text text-anchor="start" x="536.5" y="-4351.8" font-family="Helvetica,sans-Serif" font-size="14.00">username</text> +<text text-anchor="start" x="606.5" y="-4351.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="611.5" y="-4351.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(512)]</text> +<text text-anchor="start" x="732.5" y="-4351.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- ab_user--ab_user --> -<g id="edge54" class="edge"> +<g id="edge59" class="edge"> <title>ab_user--ab_user</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M630.23,-5303.02C634.02,-5314.27 639.27,-5321 646,-5321 652.73,-5321 657.98,-5314.27 661.77,-5303.02"/> -<text text-anchor="start" x="661.77" y="-5306.82" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="630.23" y="-5306.82" font-family="Times,serif" font-size="14.00">{0,1}</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M660.53,-4724.02C663.88,-4735.27 668.54,-4742 674.5,-4742 680.46,-4742 685.12,-4735.27 688.47,-4724.02"/> +<text text-anchor="start" x="688.47" y="-4727.82" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="660.53" y="-4727.82" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- ab_user--ab_user --> -<g id="edge55" class="edge"> +<g id="edge60" class="edge"> <title>ab_user--ab_user</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M607.05,-5303.38C614.4,-5325.01 627.38,-5339 646,-5339 664.62,-5339 677.6,-5325.01 684.95,-5303.38"/> -<text text-anchor="start" x="684.95" y="-5307.18" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="566.05" y="-5307.18" font-family="Times,serif" font-size="14.00">{0,1}</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M639.99,-4724.38C646.5,-4746.01 658,-4760 674.5,-4760 691,-4760 702.5,-4746.01 709.01,-4724.38"/> +<text text-anchor="start" x="709.01" y="-4728.18" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="598.99" y="-4728.18" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- ab_user_role --> -<g id="node42" class="node"> +<g id="node43" class="node"> <title>ab_user_role</title> -<polygon fill="none" stroke="black" points="1013,-5034 1013,-5062 1194,-5062 1194,-5034 1013,-5034"/> -<text text-anchor="start" x="1046.5" y="-5045.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_user_role</text> -<polygon fill="none" stroke="black" points="1013,-5009 1013,-5034 1194,-5034 1194,-5009 1013,-5009"/> -<text text-anchor="start" x="1018" y="-5018.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="1031" y="-5018.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1036" y="-5018.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1113" y="-5018.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="1013,-4984 1013,-5009 1194,-5009 1194,-4984 1013,-4984"/> -<text text-anchor="start" x="1018" y="-4993.8" font-family="Helvetica,sans-Serif" font-size="14.00">role_id</text> -<text text-anchor="start" x="1064" y="-4993.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1069" y="-4993.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="1013,-4959 1013,-4984 1194,-4984 1194,-4959 1013,-4959"/> -<text text-anchor="start" x="1018" y="-4968.8" font-family="Helvetica,sans-Serif" font-size="14.00">user_id</text> -<text text-anchor="start" x="1069" y="-4968.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1074" y="-4968.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="991,-4408 991,-4436 1172,-4436 1172,-4408 991,-4408"/> +<text text-anchor="start" x="1024.5" y="-4419.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_user_role</text> +<polygon fill="none" stroke="black" points="991,-4383 991,-4408 1172,-4408 1172,-4383 991,-4383"/> +<text text-anchor="start" x="996" y="-4392.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="1009" y="-4392.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1014" y="-4392.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1091" y="-4392.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="991,-4358 991,-4383 1172,-4383 1172,-4358 991,-4358"/> +<text text-anchor="start" x="996" y="-4367.8" font-family="Helvetica,sans-Serif" font-size="14.00">role_id</text> +<text text-anchor="start" x="1042" y="-4367.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1047" y="-4367.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="991,-4333 991,-4358 1172,-4358 1172,-4333 991,-4333"/> +<text text-anchor="start" x="996" y="-4342.8" font-family="Helvetica,sans-Serif" font-size="14.00">user_id</text> +<text text-anchor="start" x="1047" y="-4342.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1052" y="-4342.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> </g> <!-- ab_user--ab_user_role --> -<g id="edge56" class="edge"> +<g id="edge61" class="edge"> <title>ab_user--ab_user_role</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M797.25,-5077.31C865.32,-5062.5 943.87,-5045.41 1004.17,-5032.29"/> -<text text-anchor="start" x="973.17" y="-5021.09" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="797.25" y="-5066.11" font-family="Times,serif" font-size="14.00">{0,1}</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M825.55,-4476.84C877.93,-4457.93 935.34,-4437.21 982.34,-4420.25"/> +<text text-anchor="start" x="951.34" y="-4409.05" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="825.55" y="-4465.64" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- ab_register_user --> -<g id="node43" class="node"> +<g id="node44" class="node"> <title>ab_register_user</title> -<polygon fill="none" stroke="black" points="38.5,-5056 38.5,-5084 324.5,-5084 324.5,-5056 38.5,-5056"/> -<text text-anchor="start" x="106.5" y="-5067.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_register_user</text> -<polygon fill="none" stroke="black" points="38.5,-5031 38.5,-5056 324.5,-5056 324.5,-5031 38.5,-5031"/> -<text text-anchor="start" x="43.5" y="-5040.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="56.5" y="-5040.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="61.5" y="-5040.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="138.5" y="-5040.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="38.5,-5006 38.5,-5031 324.5,-5031 324.5,-5006 38.5,-5006"/> -<text text-anchor="start" x="43.5" y="-5015.8" font-family="Helvetica,sans-Serif" font-size="14.00">email</text> -<text text-anchor="start" x="81.5" y="-5015.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="86.5" y="-5015.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(512)]</text> -<text text-anchor="start" x="207.5" y="-5015.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="38.5,-4981 38.5,-5006 324.5,-5006 324.5,-4981 38.5,-4981"/> -<text text-anchor="start" x="43.5" y="-4990.8" font-family="Helvetica,sans-Serif" font-size="14.00">first_name</text> -<text text-anchor="start" x="117.5" y="-4990.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="122.5" y="-4990.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<text text-anchor="start" x="243.5" y="-4990.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="38.5,-4956 38.5,-4981 324.5,-4981 324.5,-4956 38.5,-4956"/> -<text text-anchor="start" x="43.5" y="-4965.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_name</text> -<text text-anchor="start" x="114.5" y="-4965.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="119.5" y="-4965.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<text text-anchor="start" x="240.5" y="-4965.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="38.5,-4931 38.5,-4956 324.5,-4956 324.5,-4931 38.5,-4931"/> -<text text-anchor="start" x="43.5" y="-4940.8" font-family="Helvetica,sans-Serif" font-size="14.00">password</text> -<text text-anchor="start" x="110.5" y="-4940.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="115.5" y="-4940.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<polygon fill="none" stroke="black" points="38.5,-4906 38.5,-4931 324.5,-4931 324.5,-4906 38.5,-4906"/> -<text text-anchor="start" x="43.5" y="-4915.8" font-family="Helvetica,sans-Serif" font-size="14.00">registration_date</text> -<text text-anchor="start" x="161.5" y="-4915.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="166.5" y="-4915.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> -<polygon fill="none" stroke="black" points="38.5,-4881 38.5,-4906 324.5,-4906 324.5,-4881 38.5,-4881"/> -<text text-anchor="start" x="43.5" y="-4890.8" font-family="Helvetica,sans-Serif" font-size="14.00">registration_hash</text> -<text text-anchor="start" x="164.5" y="-4890.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="169.5" y="-4890.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> -<polygon fill="none" stroke="black" points="38.5,-4856 38.5,-4881 324.5,-4881 324.5,-4856 38.5,-4856"/> -<text text-anchor="start" x="43.5" y="-4865.8" font-family="Helvetica,sans-Serif" font-size="14.00">username</text> -<text text-anchor="start" x="113.5" y="-4865.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="118.5" y="-4865.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(512)]</text> -<text text-anchor="start" x="239.5" y="-4865.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="67,-4477 67,-4505 353,-4505 353,-4477 67,-4477"/> +<text text-anchor="start" x="135" y="-4488.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_register_user</text> +<polygon fill="none" stroke="black" points="67,-4452 67,-4477 353,-4477 353,-4452 67,-4452"/> +<text text-anchor="start" x="72" y="-4461.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="85" y="-4461.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="90" y="-4461.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="167" y="-4461.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="67,-4427 67,-4452 353,-4452 353,-4427 67,-4427"/> +<text text-anchor="start" x="72" y="-4436.8" font-family="Helvetica,sans-Serif" font-size="14.00">email</text> +<text text-anchor="start" x="110" y="-4436.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="115" y="-4436.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(512)]</text> +<text text-anchor="start" x="236" y="-4436.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="67,-4402 67,-4427 353,-4427 353,-4402 67,-4402"/> +<text text-anchor="start" x="72" y="-4411.8" font-family="Helvetica,sans-Serif" font-size="14.00">first_name</text> +<text text-anchor="start" x="146" y="-4411.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="151" y="-4411.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<text text-anchor="start" x="272" y="-4411.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="67,-4377 67,-4402 353,-4402 353,-4377 67,-4377"/> +<text text-anchor="start" x="72" y="-4386.8" font-family="Helvetica,sans-Serif" font-size="14.00">last_name</text> +<text text-anchor="start" x="143" y="-4386.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="148" y="-4386.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<text text-anchor="start" x="269" y="-4386.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="67,-4352 67,-4377 353,-4377 353,-4352 67,-4352"/> +<text text-anchor="start" x="72" y="-4361.8" font-family="Helvetica,sans-Serif" font-size="14.00">password</text> +<text text-anchor="start" x="139" y="-4361.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="144" y="-4361.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<polygon fill="none" stroke="black" points="67,-4327 67,-4352 353,-4352 353,-4327 67,-4327"/> +<text text-anchor="start" x="72" y="-4336.8" font-family="Helvetica,sans-Serif" font-size="14.00">registration_date</text> +<text text-anchor="start" x="190" y="-4336.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="195" y="-4336.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [TIMESTAMP]</text> +<polygon fill="none" stroke="black" points="67,-4302 67,-4327 353,-4327 353,-4302 67,-4302"/> +<text text-anchor="start" x="72" y="-4311.8" font-family="Helvetica,sans-Serif" font-size="14.00">registration_hash</text> +<text text-anchor="start" x="193" y="-4311.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="198" y="-4311.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(256)]</text> +<polygon fill="none" stroke="black" points="67,-4277 67,-4302 353,-4302 353,-4277 67,-4277"/> +<text text-anchor="start" x="72" y="-4286.8" font-family="Helvetica,sans-Serif" font-size="14.00">username</text> +<text text-anchor="start" x="142" y="-4286.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="147" y="-4286.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(512)]</text> +<text text-anchor="start" x="268" y="-4286.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- ab_permission --> -<g id="node44" class="node"> +<g id="node45" class="node"> <title>ab_permission</title> -<polygon fill="none" stroke="black" points="55.5,-4802 55.5,-4830 307.5,-4830 307.5,-4802 55.5,-4802"/> -<text text-anchor="start" x="116.5" y="-4813.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_permission</text> -<polygon fill="none" stroke="black" points="55.5,-4777 55.5,-4802 307.5,-4802 307.5,-4777 55.5,-4777"/> -<text text-anchor="start" x="60.5" y="-4786.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="73.5" y="-4786.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="78.5" y="-4786.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="155.5" y="-4786.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="55.5,-4752 55.5,-4777 307.5,-4777 307.5,-4752 55.5,-4752"/> -<text text-anchor="start" x="60.5" y="-4761.8" font-family="Helvetica,sans-Serif" font-size="14.00">name</text> -<text text-anchor="start" x="100.5" y="-4761.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="105.5" y="-4761.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(100)]</text> -<text text-anchor="start" x="226.5" y="-4761.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="84,-4223 84,-4251 336,-4251 336,-4223 84,-4223"/> +<text text-anchor="start" x="145" y="-4234.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_permission</text> +<polygon fill="none" stroke="black" points="84,-4198 84,-4223 336,-4223 336,-4198 84,-4198"/> +<text text-anchor="start" x="89" y="-4207.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="102" y="-4207.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="107" y="-4207.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="184" y="-4207.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="84,-4173 84,-4198 336,-4198 336,-4173 84,-4173"/> +<text text-anchor="start" x="89" y="-4182.8" font-family="Helvetica,sans-Serif" font-size="14.00">name</text> +<text text-anchor="start" x="129" y="-4182.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="134" y="-4182.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(100)]</text> +<text text-anchor="start" x="255" y="-4182.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- ab_permission_view --> -<g id="node45" class="node"> +<g id="node46" class="node"> <title>ab_permission_view</title> -<polygon fill="none" stroke="black" points="551,-4762 551,-4790 741,-4790 741,-4762 551,-4762"/> -<text text-anchor="start" x="556.5" y="-4773.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_permission_view</text> -<polygon fill="none" stroke="black" points="551,-4737 551,-4762 741,-4762 741,-4737 551,-4737"/> -<text text-anchor="start" x="556" y="-4746.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="569" y="-4746.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="574" y="-4746.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="651" y="-4746.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="551,-4712 551,-4737 741,-4737 741,-4712 551,-4712"/> -<text text-anchor="start" x="556" y="-4721.8" font-family="Helvetica,sans-Serif" font-size="14.00">permission_id</text> -<text text-anchor="start" x="652" y="-4721.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="657" y="-4721.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="551,-4687 551,-4712 741,-4712 741,-4687 551,-4687"/> -<text text-anchor="start" x="556" y="-4696.8" font-family="Helvetica,sans-Serif" font-size="14.00">view_menu_id</text> -<text text-anchor="start" x="654" y="-4696.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="659" y="-4696.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="579.5,-4183 579.5,-4211 769.5,-4211 769.5,-4183 579.5,-4183"/> +<text text-anchor="start" x="585" y="-4194.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_permission_view</text> +<polygon fill="none" stroke="black" points="579.5,-4158 579.5,-4183 769.5,-4183 769.5,-4158 579.5,-4158"/> +<text text-anchor="start" x="584.5" y="-4167.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="597.5" y="-4167.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="602.5" y="-4167.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="679.5" y="-4167.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="579.5,-4133 579.5,-4158 769.5,-4158 769.5,-4133 579.5,-4133"/> +<text text-anchor="start" x="584.5" y="-4142.8" font-family="Helvetica,sans-Serif" font-size="14.00">permission_id</text> +<text text-anchor="start" x="680.5" y="-4142.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="685.5" y="-4142.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="579.5,-4108 579.5,-4133 769.5,-4133 769.5,-4108 579.5,-4108"/> +<text text-anchor="start" x="584.5" y="-4117.8" font-family="Helvetica,sans-Serif" font-size="14.00">view_menu_id</text> +<text text-anchor="start" x="682.5" y="-4117.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="687.5" y="-4117.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> </g> <!-- ab_permission--ab_permission_view --> -<g id="edge57" class="edge"> +<g id="edge62" class="edge"> <title>ab_permission--ab_permission_view</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M315.65,-4776.03C387.97,-4767.9 476,-4758 542.85,-4750.48"/> -<text text-anchor="start" x="511.85" y="-4739.28" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="315.65" y="-4764.83" font-family="Times,serif" font-size="14.00">{0,1}</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M344.15,-4197.03C416.47,-4188.9 504.5,-4179 571.35,-4171.48"/> +<text text-anchor="start" x="540.35" y="-4160.28" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="344.15" y="-4185.83" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- ab_permission_view_role --> -<g id="node46" class="node"> +<g id="node47" class="node"> <title>ab_permission_view_role</title> -<polygon fill="none" stroke="black" points="987,-4824 987,-4852 1220,-4852 1220,-4824 987,-4824"/> -<text text-anchor="start" x="992" y="-4835.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_permission_view_role</text> -<polygon fill="none" stroke="black" points="987,-4799 987,-4824 1220,-4824 1220,-4799 987,-4799"/> -<text text-anchor="start" x="992" y="-4808.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="1005" y="-4808.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1010" y="-4808.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="1087" y="-4808.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="987,-4774 987,-4799 1220,-4799 1220,-4774 987,-4774"/> -<text text-anchor="start" x="992" y="-4783.8" font-family="Helvetica,sans-Serif" font-size="14.00">permission_view_id</text> -<text text-anchor="start" x="1126" y="-4783.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1131" y="-4783.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<polygon fill="none" stroke="black" points="987,-4749 987,-4774 1220,-4774 1220,-4749 987,-4749"/> -<text text-anchor="start" x="992" y="-4758.8" font-family="Helvetica,sans-Serif" font-size="14.00">role_id</text> -<text text-anchor="start" x="1038" y="-4758.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="1043" y="-4758.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="965,-4249 965,-4277 1198,-4277 1198,-4249 965,-4249"/> +<text text-anchor="start" x="970" y="-4260.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_permission_view_role</text> +<polygon fill="none" stroke="black" points="965,-4224 965,-4249 1198,-4249 1198,-4224 965,-4224"/> +<text text-anchor="start" x="970" y="-4233.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="983" y="-4233.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="988" y="-4233.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="1065" y="-4233.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="965,-4199 965,-4224 1198,-4224 1198,-4199 965,-4199"/> +<text text-anchor="start" x="970" y="-4208.8" font-family="Helvetica,sans-Serif" font-size="14.00">permission_view_id</text> +<text text-anchor="start" x="1104" y="-4208.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1109" y="-4208.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<polygon fill="none" stroke="black" points="965,-4174 965,-4199 1198,-4199 1198,-4174 965,-4174"/> +<text text-anchor="start" x="970" y="-4183.8" font-family="Helvetica,sans-Serif" font-size="14.00">role_id</text> +<text text-anchor="start" x="1016" y="-4183.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="1021" y="-4183.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> </g> <!-- ab_permission_view--ab_permission_view_role --> -<g id="edge58" class="edge"> +<g id="edge63" class="edge"> <title>ab_permission_view--ab_permission_view_role</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M749.11,-4752.91C816.97,-4762.16 906.48,-4774.36 978.37,-4784.15"/> -<text text-anchor="start" x="947.37" y="-4772.95" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="749.11" y="-4741.71" font-family="Times,serif" font-size="14.00">{0,1}</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M777.57,-4176.65C831.87,-4185.51 899.06,-4196.48 956.3,-4205.82"/> +<text text-anchor="start" x="925.3" y="-4194.62" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="777.57" y="-4165.45" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- ab_view_menu --> -<g id="node47" class="node"> +<g id="node48" class="node"> <title>ab_view_menu</title> -<polygon fill="none" stroke="black" points="55.5,-4698 55.5,-4726 307.5,-4726 307.5,-4698 55.5,-4698"/> -<text text-anchor="start" x="116.5" y="-4709.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_view_menu</text> -<polygon fill="none" stroke="black" points="55.5,-4673 55.5,-4698 307.5,-4698 307.5,-4673 55.5,-4673"/> -<text text-anchor="start" x="60.5" y="-4682.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="73.5" y="-4682.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="78.5" y="-4682.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="155.5" y="-4682.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="55.5,-4648 55.5,-4673 307.5,-4673 307.5,-4648 55.5,-4648"/> -<text text-anchor="start" x="60.5" y="-4657.8" font-family="Helvetica,sans-Serif" font-size="14.00">name</text> -<text text-anchor="start" x="100.5" y="-4657.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="105.5" y="-4657.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> -<text text-anchor="start" x="226.5" y="-4657.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="84,-4119 84,-4147 336,-4147 336,-4119 84,-4119"/> +<text text-anchor="start" x="145" y="-4130.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_view_menu</text> +<polygon fill="none" stroke="black" points="84,-4094 84,-4119 336,-4119 336,-4094 84,-4094"/> +<text text-anchor="start" x="89" y="-4103.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="102" y="-4103.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="107" y="-4103.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="184" y="-4103.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="84,-4069 84,-4094 336,-4094 336,-4069 84,-4069"/> +<text text-anchor="start" x="89" y="-4078.8" font-family="Helvetica,sans-Serif" font-size="14.00">name</text> +<text text-anchor="start" x="129" y="-4078.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="134" y="-4078.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(250)]</text> +<text text-anchor="start" x="255" y="-4078.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- ab_view_menu--ab_permission_view --> -<g id="edge59" class="edge"> +<g id="edge64" class="edge"> <title>ab_view_menu--ab_permission_view</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M315.65,-4701.97C387.97,-4710.1 476,-4720 542.85,-4727.52"/> -<text text-anchor="start" x="511.85" y="-4716.32" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="315.65" y="-4690.77" font-family="Times,serif" font-size="14.00">{0,1}</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M344.15,-4122.97C416.47,-4131.1 504.5,-4141 571.35,-4148.52"/> +<text text-anchor="start" x="540.35" y="-4137.32" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="344.15" y="-4111.77" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- ab_role --> -<g id="node48" class="node"> +<g id="node49" class="node"> <title>ab_role</title> -<polygon fill="none" stroke="black" points="525,-4867 525,-4895 768,-4895 768,-4867 525,-4867"/> -<text text-anchor="start" x="613.5" y="-4878.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_role</text> -<polygon fill="none" stroke="black" points="525,-4842 525,-4867 768,-4867 768,-4842 525,-4842"/> -<text text-anchor="start" x="530" y="-4851.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> -<text text-anchor="start" x="543" y="-4851.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="548" y="-4851.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> -<text text-anchor="start" x="625" y="-4851.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> -<polygon fill="none" stroke="black" points="525,-4817 525,-4842 768,-4842 768,-4817 525,-4817"/> -<text text-anchor="start" x="530" y="-4826.8" font-family="Helvetica,sans-Serif" font-size="14.00">name</text> -<text text-anchor="start" x="570" y="-4826.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="575" y="-4826.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(64)]</text> -<text text-anchor="start" x="687" y="-4826.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="553.5,-4288 553.5,-4316 796.5,-4316 796.5,-4288 553.5,-4288"/> +<text text-anchor="start" x="642" y="-4299.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">ab_role</text> +<polygon fill="none" stroke="black" points="553.5,-4263 553.5,-4288 796.5,-4288 796.5,-4263 553.5,-4263"/> +<text text-anchor="start" x="558.5" y="-4272.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">id</text> +<text text-anchor="start" x="571.5" y="-4272.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="576.5" y="-4272.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [INTEGER]</text> +<text text-anchor="start" x="653.5" y="-4272.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="553.5,-4238 553.5,-4263 796.5,-4263 796.5,-4238 553.5,-4238"/> +<text text-anchor="start" x="558.5" y="-4247.8" font-family="Helvetica,sans-Serif" font-size="14.00">name</text> +<text text-anchor="start" x="598.5" y="-4247.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="603.5" y="-4247.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(64)]</text> +<text text-anchor="start" x="715.5" y="-4247.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> <!-- ab_role--ab_user_role --> -<g id="edge60" class="edge"> +<g id="edge65" class="edge"> <title>ab_role--ab_user_role</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M775.55,-4884.7C802.35,-4891.6 830.27,-4899.47 856,-4908 905.77,-4924.5 959.65,-4946.56 1004.26,-4966.03"/> -<text text-anchor="start" x="973.26" y="-4954.83" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="775.55" y="-4873.5" font-family="Times,serif" font-size="14.00">{0,1}</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M804.04,-4311.32C861.76,-4326.73 928.76,-4344.62 982.28,-4358.91"/> +<text text-anchor="start" x="951.28" y="-4347.71" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="804.04" y="-4300.12" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- ab_role--ab_permission_view_role --> -<g id="edge61" class="edge"> +<g id="edge66" class="edge"> <title>ab_role--ab_permission_view_role</title> -<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M775.56,-4840.46C839.16,-4832.77 915.56,-4823.54 978.46,-4815.93"/> -<text text-anchor="start" x="947.46" y="-4804.73" font-family="Times,serif" font-size="14.00">0..N</text> -<text text-anchor="start" x="775.56" y="-4829.26" font-family="Times,serif" font-size="14.00">{0,1}</text> +<path fill="none" stroke="#7f7f7f" stroke-dasharray="5,2" d="M804.04,-4260.79C852.74,-4254.65 908.04,-4247.68 956.28,-4241.6"/> +<text text-anchor="start" x="925.28" y="-4230.4" font-family="Times,serif" font-size="14.00">0..N</text> +<text text-anchor="start" x="804.04" y="-4249.59" font-family="Times,serif" font-size="14.00">{0,1}</text> </g> <!-- alembic_version_fab --> -<g id="node49" class="node"> +<g id="node50" class="node"> <title>alembic_version_fab</title> -<polygon fill="none" stroke="black" points="35.5,-5135 35.5,-5163 328.5,-5163 328.5,-5135 35.5,-5135"/> -<text text-anchor="start" x="91.5" y="-5146.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">alembic_version_fab</text> -<polygon fill="none" stroke="black" points="35.5,-5110 35.5,-5135 328.5,-5135 328.5,-5110 35.5,-5110"/> -<text text-anchor="start" x="40.5" y="-5119.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">version_num</text> -<text text-anchor="start" x="130.5" y="-5119.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> -<text text-anchor="start" x="135.5" y="-5119.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(32)]</text> -<text text-anchor="start" x="247.5" y="-5119.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> +<polygon fill="none" stroke="black" points="64,-4556 64,-4584 357,-4584 357,-4556 64,-4556"/> +<text text-anchor="start" x="120" y="-4567.2" font-family="Helvetica,sans-Serif" font-weight="bold" font-size="16.00">alembic_version_fab</text> +<polygon fill="none" stroke="black" points="64,-4531 64,-4556 357,-4556 357,-4531 64,-4531"/> +<text text-anchor="start" x="69" y="-4540.8" font-family="Helvetica,sans-Serif" text-decoration="underline" font-size="14.00">version_num</text> +<text text-anchor="start" x="159" y="-4540.8" font-family="Helvetica,sans-Serif" font-size="14.00"> </text> +<text text-anchor="start" x="164" y="-4540.8" font-family="Helvetica,sans-Serif" font-size="14.00"> [VARCHAR(32)]</text> +<text text-anchor="start" x="276" y="-4540.8" font-family="Helvetica,sans-Serif" font-size="14.00"> NOT NULL</text> </g> </g> </svg> diff --git a/docs/apache-airflow/migrations-ref.rst b/docs/apache-airflow/migrations-ref.rst index 61dde39958e21..bc73f387a2d28 100644 --- a/docs/apache-airflow/migrations-ref.rst +++ b/docs/apache-airflow/migrations-ref.rst @@ -39,7 +39,9 @@ Here's the list of all the Database Migrations that are executed via when you ru +-------------------------+------------------+-------------------+--------------------------------------------------------------+ | Revision ID | Revises ID | Airflow Version | Description | +=========================+==================+===================+==============================================================+ -| ``d03e4a635aa3`` (head) | ``d8cd3297971e`` | ``3.0.0`` | Drop DAG pickling. | +| ``2b47dc6bc8df`` (head) | ``d03e4a635aa3`` | ``3.0.0`` | add dag versioning. | ++-------------------------+------------------+-------------------+--------------------------------------------------------------+ +| ``d03e4a635aa3`` | ``d8cd3297971e`` | ``3.0.0`` | Drop DAG pickling. | +-------------------------+------------------+-------------------+--------------------------------------------------------------+ | ``d8cd3297971e`` | ``5f57a45b8433`` | ``3.0.0`` | Add last_heartbeat_at directly to TI. | +-------------------------+------------------+-------------------+--------------------------------------------------------------+ diff --git a/hatch_build.py b/hatch_build.py index 91b9256b4d031..5e95775c62f26 100644 --- a/hatch_build.py +++ b/hatch_build.py @@ -431,6 +431,7 @@ # The issue tracking it is https://github.com/apache/airflow/issues/28723 "sqlalchemy>=1.4.36,<2.0", "sqlalchemy-jsonfield>=1.0", + "sqlalchemy-utils>=0.41.2", "tabulate>=0.7.5", "tenacity>=8.0.0,!=8.2.0", "termcolor>=1.1.0", diff --git a/providers/tests/fab/auth_manager/api_endpoints/test_dag_run_endpoint.py b/providers/tests/fab/auth_manager/api_endpoints/test_dag_run_endpoint.py index 05bac3394ef0f..c1f3e71f306bd 100644 --- a/providers/tests/fab/auth_manager/api_endpoints/test_dag_run_endpoint.py +++ b/providers/tests/fab/auth_manager/api_endpoints/test_dag_run_endpoint.py @@ -138,6 +138,7 @@ def _create_dag(self, dag_id): session.add(dag_instance) dag = DAG(dag_id=dag_id, schedule=None, params={"validated_number": Param(1, minimum=1, maximum=10)}) self.app.dag_bag.bag_dag(dag) + self.app.dag_bag.sync_to_db() return dag_instance def _create_test_dag_run(self, state=DagRunState.RUNNING, extra_dag=False, commit=True, idx_start=1): diff --git a/scripts/ci/pre_commit/check_ti_vs_tis_attributes.py b/scripts/ci/pre_commit/check_ti_vs_tis_attributes.py index 16c1df48a9e8b..d1782e6290744 100755 --- a/scripts/ci/pre_commit/check_ti_vs_tis_attributes.py +++ b/scripts/ci/pre_commit/check_ti_vs_tis_attributes.py @@ -54,6 +54,7 @@ def compare_attributes(path1, path2): "rendered_task_instance_fields", # Storing last heartbeat for historic TIs is not interesting/useful "last_heartbeat_at", + "dag_version", } # exclude attrs not necessary to be in TaskInstanceHistory if not diff: return diff --git a/task_sdk/src/airflow/sdk/definitions/dag.py b/task_sdk/src/airflow/sdk/definitions/dag.py index 479c1ea09b80c..9a124d237ed57 100644 --- a/task_sdk/src/airflow/sdk/definitions/dag.py +++ b/task_sdk/src/airflow/sdk/definitions/dag.py @@ -355,6 +355,7 @@ class DAG: **Warning**: A fail stop dag can only have tasks with the default trigger rule ("all_success"). An exception will be thrown if any task in a fail stop dag has a non default trigger rule. :param dag_display_name: The display name of the DAG which appears on the UI. + :param version_name: The version name of the DAG. This is used to identify the version of the DAG. """ __serialized_fields: ClassVar[frozenset[str] | None] = None @@ -437,6 +438,10 @@ class DAG: has_on_success_callback: bool = attrs.field(init=False) has_on_failure_callback: bool = attrs.field(init=False) + version_name: str | None = attrs.field( + default=None, + validator=attrs.validators.optional(attrs.validators.instance_of(str)), + ) def __attrs_post_init__(self): from airflow.utils import timezone @@ -1063,6 +1068,7 @@ def dag( auto_register: bool = True, fail_stop: bool = False, dag_display_name: str | None = None, + version_name: str | None = None, ) -> Callable[[Callable], Callable[..., DAG]]: """ Python dag decorator which wraps a function into an Airflow DAG. diff --git a/tests/api_connexion/endpoints/test_dag_run_endpoint.py b/tests/api_connexion/endpoints/test_dag_run_endpoint.py index 576b28b153531..6bbd63fa8b49d 100644 --- a/tests/api_connexion/endpoints/test_dag_run_endpoint.py +++ b/tests/api_connexion/endpoints/test_dag_run_endpoint.py @@ -89,6 +89,7 @@ def _create_dag(self, dag_id): session.add(dag_instance) dag = DAG(dag_id=dag_id, schedule=None, params={"validated_number": Param(1, minimum=1, maximum=10)}) self.app.dag_bag.bag_dag(dag) + self.app.dag_bag.sync_to_db() return dag_instance def _create_test_dag_run(self, state=DagRunState.RUNNING, extra_dag=False, commit=True, idx_start=1): @@ -1205,12 +1206,14 @@ def test_raises_validation_error_for_invalid_params(self): assert "Invalid input for param" in response.json["detail"] @mock.patch("airflow.api_connexion.endpoints.dag_run_endpoint.get_airflow_app") - def test_dagrun_creation_exception_is_handled(self, mock_get_app, session): + @mock.patch("airflow.api_connexion.endpoints.dag_run_endpoint.DagVersion") + def test_dagrun_creation_exception_is_handled(self, mock_get_dag_version, mock_get_app, session): self._create_dag("TEST_DAG_ID") error_message = "Encountered Error" mock_get_app.return_value.dag_bag.get_dag.return_value.create_dagrun.side_effect = ValueError( error_message ) + mock_get_dag_version.get_latest_version.return_value = mock.MagicMock() response = self.client.post( "api/v1/dags/TEST_DAG_ID/dagRuns", json={"execution_date": "2020-11-10T08:25:56Z"}, diff --git a/tests/api_connexion/endpoints/test_task_endpoint.py b/tests/api_connexion/endpoints/test_task_endpoint.py index b558f8dbf161e..a6e47f4f6a1bb 100644 --- a/tests/api_connexion/endpoints/test_task_endpoint.py +++ b/tests/api_connexion/endpoints/test_task_endpoint.py @@ -78,7 +78,6 @@ def setup_dag(self, configured_app): with DAG(self.unscheduled_dag_id, start_date=None, schedule=None) as unscheduled_dag: task4 = EmptyOperator(task_id=self.unscheduled_task_id1, params={"is_unscheduled": True}) task5 = EmptyOperator(task_id=self.unscheduled_task_id2, params={"is_unscheduled": True}) - task1 >> task2 task4 >> task5 dag_bag = DagBag(os.devnull, include_examples=False) @@ -87,6 +86,7 @@ def setup_dag(self, configured_app): mapped_dag.dag_id: mapped_dag, unscheduled_dag.dag_id: unscheduled_dag, } + DagBag._sync_to_db(dag_bag.dags) configured_app.dag_bag = dag_bag # type:ignore @staticmethod @@ -246,7 +246,9 @@ def test_unscheduled_task(self): def test_should_respond_200_serialized(self): # Get the dag out of the dagbag before we patch it to an empty one - SerializedDagModel.write_dag(self.app.dag_bag.get_dag(self.dag_id)) + dag = self.app.dag_bag.get_dag(self.dag_id) + dag.sync_to_db() + SerializedDagModel.write_dag(dag) dag_bag = DagBag(os.devnull, include_examples=False, read_dags_from_db=True) patcher = unittest.mock.patch.object(self.app, "dag_bag", dag_bag) diff --git a/tests/cli/commands/test_task_command.py b/tests/cli/commands/test_task_command.py index ed1a2c28754f8..50e3d393f8108 100644 --- a/tests/cli/commands/test_task_command.py +++ b/tests/cli/commands/test_task_command.py @@ -227,7 +227,7 @@ def test_cli_test_different_path(self, session, tmp_path): .one() ) # confirm that the serialized dag location has not been updated - assert ser_dag.fileloc == orig_file_path.as_posix() + assert ser_dag.dag_version.dag_code.fileloc == orig_file_path.as_posix() assert ser_dag.data["dag"]["_processor_dags_folder"] == orig_dags_folder.as_posix() assert ser_dag.data["dag"]["fileloc"] == orig_file_path.as_posix() assert ser_dag.dag._processor_dags_folder == orig_dags_folder.as_posix() diff --git a/tests/dag_processing/test_job_runner.py b/tests/dag_processing/test_job_runner.py index 192a12358e8dd..2bf0bcb6dbb0e 100644 --- a/tests/dag_processing/test_job_runner.py +++ b/tests/dag_processing/test_job_runner.py @@ -53,6 +53,7 @@ from airflow.jobs.dag_processor_job_runner import DagProcessorJobRunner from airflow.jobs.job import Job from airflow.models import DagBag, DagModel, DbCallbackRequest +from airflow.models.dag_version import DagVersion from airflow.models.dagcode import DagCode from airflow.models.serialized_dag import SerializedDagModel from airflow.utils import timezone @@ -664,13 +665,6 @@ def test_scan_stale_dags(self): ) assert active_dag_count == 1 - serialized_dag_count = ( - session.query(func.count(SerializedDagModel.dag_id)) - .filter(SerializedDagModel.fileloc == test_dag_path) - .scalar() - ) - assert serialized_dag_count == 1 - manager.processor._scan_stale_dags() active_dag_count = ( @@ -682,10 +676,12 @@ def test_scan_stale_dags(self): serialized_dag_count = ( session.query(func.count(SerializedDagModel.dag_id)) - .filter(SerializedDagModel.fileloc == test_dag_path) + .filter(SerializedDagModel.dag_id == dag.dag_id) .scalar() ) - assert serialized_dag_count == 0 + # Deactivating the DagModel should not delete the SerializedDagModel + # SerializedDagModel gives history about Dags + assert serialized_dag_count == 1 @pytest.mark.skip_if_database_isolation_mode # Test is broken in db isolation mode @conf_vars( @@ -1088,10 +1084,12 @@ def test_refresh_dags_dir_deactivates_deleted_zipped_dags(self, tmp_path): with mock.patch("airflow.dag_processing.manager.might_contain_dag", return_value=False): manager.processor._refresh_dag_dir() - # Assert dag removed from SDM - assert not SerializedDagModel.has_dag("test_zip_dag") - # assert code deleted - assert not DagCode.has_dag(dag.fileloc) + # Deleting the python file should not delete SDM for versioning sake + assert SerializedDagModel.has_dag("test_zip_dag") + # assert code not deleted for versioning sake + assert DagCode.has_dag(dag.fileloc) + # assert dagversion was not deleted + assert DagVersion.get_latest_version(dag.dag_id) # assert dag deactivated assert not dag.get_is_active() diff --git a/tests/dag_processing/test_processor.py b/tests/dag_processing/test_processor.py index f117b3ffe4581..29aa4e15a3888 100644 --- a/tests/dag_processing/test_processor.py +++ b/tests/dag_processing/test_processor.py @@ -161,6 +161,7 @@ def test_execute_on_failure_callbacks_without_dag(self, mock_ti_handle_failure, with create_session() as session: session.query(TaskInstance).delete() dag = dagbag.get_dag("example_branch_operator") + dag.sync_to_db() triggered_by_kwargs = {"triggered_by": DagRunTriggeredByType.TEST} if AIRFLOW_V_3_0_PLUS else {} dagrun = dag.create_dagrun( state=State.RUNNING, diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index da3ccc201eb4a..d0b147a5c3727 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -54,6 +54,7 @@ from airflow.models.asset import AssetActive, AssetDagRunQueue, AssetEvent, AssetModel from airflow.models.backfill import Backfill, _create_backfill from airflow.models.dag import DAG, DagModel +from airflow.models.dag_version import DagVersion from airflow.models.dagbag import DagBag from airflow.models.dagrun import DagRun from airflow.models.db_callback_request import DbCallbackRequest @@ -141,11 +142,10 @@ def clean_db(): clear_db_runs() clear_db_backfills() clear_db_pools() - clear_db_dags() clear_db_import_errors() clear_db_jobs() clear_db_assets() - # DO NOT try to run clear_db_serialized_dags() here - this will break the tests + # DO NOT try to run clear_db_serialized_dags() or clear_db_dags here - this will break the tests # The tests expect DAGs to be fully loaded here via setUpClass method below @pytest.fixture(autouse=True) @@ -167,9 +167,7 @@ def set_instance_attrs(self, dagbag) -> Generator: # enqueue! self.null_exec: MockExecutor | None = MockExecutor() # Since we don't want to store the code for the DAG defined in this file - with patch("airflow.dag_processing.manager.SerializedDagModel.remove_deleted_dags"), patch( - "airflow.models.dag.DagCode.bulk_sync_to_db" - ): + with patch("airflow.models.serialized_dag.SerializedDagModel.remove_deleted_dags"): yield self.null_exec = None @@ -2875,7 +2873,6 @@ def test_dagrun_root_after_dagrun_unfinished(self, mock_executor): Noted: the DagRun state could be still in running state during CI. """ - clear_db_dags() dag_id = "test_dagrun_states_root_future" dag = self.dagbag.get_dag(dag_id) dag.sync_to_db() @@ -3316,7 +3313,7 @@ def test_verify_integrity_if_dag_not_changed(self, dag_maker): assert tis_count == 1 latest_dag_version = SerializedDagModel.get_latest_version_hash(dr.dag_id, session=session) - assert dr.dag_hash == latest_dag_version + assert dr.dag_version.serialized_dag.dag_hash == latest_dag_version session.rollback() session.close() @@ -3350,7 +3347,7 @@ def test_verify_integrity_if_dag_changed(self, dag_maker): dr = drs[0] dag_version_1 = SerializedDagModel.get_latest_version_hash(dr.dag_id, session=session) - assert dr.dag_hash == dag_version_1 + assert dr.dag_version.serialized_dag.dag_hash == dag_version_1 assert self.job_runner.dagbag.dags == {"test_verify_integrity_if_dag_changed": dag} assert len(self.job_runner.dagbag.dags.get("test_verify_integrity_if_dag_changed").tasks) == 1 @@ -3367,7 +3364,7 @@ def test_verify_integrity_if_dag_changed(self, dag_maker): drs = DagRun.find(dag_id=dag.dag_id, session=session) assert len(drs) == 1 dr = drs[0] - assert dr.dag_hash == dag_version_2 + assert dr.dag_version.serialized_dag.dag_hash == dag_version_2 assert self.job_runner.dagbag.dags == {"test_verify_integrity_if_dag_changed": dag} assert len(self.job_runner.dagbag.dags.get("test_verify_integrity_if_dag_changed").tasks) == 2 @@ -3383,54 +3380,7 @@ def test_verify_integrity_if_dag_changed(self, dag_maker): assert tis_count == 2 latest_dag_version = SerializedDagModel.get_latest_version_hash(dr.dag_id, session=session) - assert dr.dag_hash == latest_dag_version - - session.rollback() - session.close() - - def test_verify_integrity_if_dag_disappeared(self, dag_maker, caplog): - # CleanUp - with create_session() as session: - session.query(SerializedDagModel).filter( - SerializedDagModel.dag_id == "test_verify_integrity_if_dag_disappeared" - ).delete(synchronize_session=False) - - with dag_maker(dag_id="test_verify_integrity_if_dag_disappeared") as dag: - BashOperator(task_id="dummy", bash_command="echo hi") - - scheduler_job = Job() - self.job_runner = SchedulerJobRunner(job=scheduler_job, subdir=os.devnull) - - session = settings.Session() - orm_dag = dag_maker.dag_model - assert orm_dag is not None - - scheduler_job = Job() - self.job_runner = SchedulerJobRunner(job=scheduler_job, subdir=os.devnull) - - self.job_runner.processor_agent = mock.MagicMock() - dag = self.job_runner.dagbag.get_dag("test_verify_integrity_if_dag_disappeared", session=session) - self.job_runner._create_dag_runs([orm_dag], session) - dag_id = dag.dag_id - drs = DagRun.find(dag_id=dag_id, session=session) - assert len(drs) == 1 - dr = drs[0] - - dag_version_1 = SerializedDagModel.get_latest_version_hash(dag_id, session=session) - assert dr.dag_hash == dag_version_1 - assert self.job_runner.dagbag.dags == {"test_verify_integrity_if_dag_disappeared": dag} - assert len(self.job_runner.dagbag.dags.get("test_verify_integrity_if_dag_disappeared").tasks) == 1 - - SerializedDagModel.remove_dag(dag_id=dag_id) - dag = self.job_runner.dagbag.dags[dag_id] - self.job_runner.dagbag.dags = MagicMock() - self.job_runner.dagbag.dags.get.side_effect = [dag, None] - session.flush() - with caplog.at_level(logging.WARNING): - callback = self.job_runner._schedule_dag_run(dr, session) - assert "The DAG disappeared before verifying integrity" in caplog.text - - assert callback is None + assert dr.dag_version.serialized_dag.dag_hash == latest_dag_version session.rollback() session.close() @@ -4015,6 +3965,7 @@ def test_create_dag_runs_assets(self, session, dag_maker): - That the run created is on QUEUED State - That dag_model has next_dagrun """ + clear_db_dags() asset1 = Asset(uri="ds1") asset2 = Asset(uri="ds2") @@ -4386,6 +4337,7 @@ def test_do_schedule_max_active_runs_dag_timed_out(self, dag_maker): session = settings.Session() data_interval = dag.infer_automated_data_interval(DEFAULT_LOGICAL_DATE) triggered_by_kwargs = {"triggered_by": DagRunTriggeredByType.TEST} if AIRFLOW_V_3_0_PLUS else {} + dag_version = DagVersion.get_latest_version(dag.dag_id) run1 = dag.create_dagrun( run_type=DagRunType.SCHEDULED, execution_date=DEFAULT_DATE, @@ -4393,6 +4345,7 @@ def test_do_schedule_max_active_runs_dag_timed_out(self, dag_maker): start_date=timezone.utcnow() - timedelta(seconds=2), session=session, data_interval=data_interval, + dag_version=dag_version, **triggered_by_kwargs, ) @@ -4405,6 +4358,7 @@ def test_do_schedule_max_active_runs_dag_timed_out(self, dag_maker): state=State.QUEUED, session=session, data_interval=data_interval, + dag_version=dag_version, **triggered_by_kwargs, ) @@ -4602,10 +4556,8 @@ def test_do_schedule_max_active_runs_and_manual_trigger(self, dag_maker, mock_ex BashOperator(task_id="dummy3", bash_command="true") session = settings.Session() - dag_run = dag_maker.create_dagrun( - state=State.QUEUED, - session=session, - ) + dag_version = DagVersion.get_latest_version(dag.dag_id) + dag_run = dag_maker.create_dagrun(state=State.QUEUED, session=session, dag_version=dag_version) dag.sync_to_db(session=session) # Update the date fields @@ -4647,23 +4599,31 @@ def test_max_active_runs_in_a_dag_doesnt_stop_running_dag_runs_in_other_dags(sel start_date=DEFAULT_DATE, schedule=timedelta(hours=1), max_active_runs=1, - ): + ) as dag: EmptyOperator(task_id="mytask") - - dr = dag_maker.create_dagrun(run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dag_version = DagVersion.get_latest_version(dag.dag_id) + dr = dag_maker.create_dagrun( + run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) for _ in range(29): - dr = dag_maker.create_dagrun_after(dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun_after( + dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) with dag_maker( "test_dag2", start_date=timezone.datetime(2020, 1, 1), schedule=timedelta(hours=1), - ): + ) as dag2: EmptyOperator(task_id="mytask") - - dr = dag_maker.create_dagrun(run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dag_version = DagVersion.get_latest_version(dag2.dag_id) + dr = dag_maker.create_dagrun( + run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) for _ in range(9): - dr = dag_maker.create_dagrun_after(dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun_after( + dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) scheduler_job = Job() self.job_runner = SchedulerJobRunner(job=scheduler_job, subdir=os.devnull) @@ -4695,20 +4655,29 @@ def test_max_active_runs_in_a_dag_doesnt_prevent_backfill_from_running(self, dag ) as dag: EmptyOperator(task_id="mytask") dag1_dag_id = dag.dag_id - dr = dag_maker.create_dagrun(run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dag_version = DagVersion.get_latest_version(dag1_dag_id) + dr = dag_maker.create_dagrun( + run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) for _ in range(29): - dr = dag_maker.create_dagrun_after(dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun_after( + dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) with dag_maker( "test_dag2", start_date=timezone.datetime(2020, 1, 1), schedule=timedelta(days=1), - ): + ) as dag: EmptyOperator(task_id="mytask") - - dr = dag_maker.create_dagrun(run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dag_version = DagVersion.get_latest_version(dag.dag_id) + dr = dag_maker.create_dagrun( + run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) for _ in range(9): - dr = dag_maker.create_dagrun_after(dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun_after( + dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) scheduler_job = Job() self.job_runner = SchedulerJobRunner(job=scheduler_job, subdir=os.devnull) @@ -4842,19 +4811,30 @@ def _running_counts(): ) dag1_non_b_running, dag1_b_running, total_running = _running_counts() + dag_version = DagVersion.get_latest_version(dag1_dag_id) # now let's create some "normal" dag runs and verify that they can run - dr = dag_maker.create_dagrun(run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun( + run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) for _ in range(29): - dr = dag_maker.create_dagrun_after(dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun_after( + dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) with dag_maker( "test_dag2", start_date=timezone.datetime(2020, 1, 1), schedule=timedelta(days=1), - ): + ) as dag2: EmptyOperator(task_id="mytask") - dr = dag_maker.create_dagrun(run_type=DagRunType.SCHEDULED, state=State.QUEUED) + + dag_version = DagVersion.get_latest_version(dag2.dag_id) + dr = dag_maker.create_dagrun( + run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) for _ in range(9): - dr = dag_maker.create_dagrun_after(dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun_after( + dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) # initial state -- nothing is running assert dag1_non_b_running == 0 @@ -4982,19 +4962,30 @@ def _running_counts(): assert session.scalar(select(func.count()).select_from(DagRun)) == 6 assert session.scalar(select(func.count()).where(DagRun.dag_id == dag1_dag_id)) == 6 + dag_version = DagVersion.get_latest_version(dag1_dag_id) # now let's create some "normal" dag runs and verify that they can run - dr = dag_maker.create_dagrun(run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun( + run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) for _ in range(29): - dr = dag_maker.create_dagrun_after(dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun_after( + dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) with dag_maker( "test_dag2", start_date=timezone.datetime(2020, 1, 1), schedule=timedelta(days=1), - ): + ) as dag2: EmptyOperator(task_id="mytask") - dr = dag_maker.create_dagrun(run_type=DagRunType.SCHEDULED, state=State.QUEUED) + + dag_version = DagVersion.get_latest_version(dag2.dag_id) + dr = dag_maker.create_dagrun( + run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) for _ in range(9): - dr = dag_maker.create_dagrun_after(dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun_after( + dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) # ok at this point, there are new dag runs created, but no new running runs dag1_non_b_running, dag1_b_running, total_running = _running_counts() @@ -5130,9 +5121,14 @@ def test_start_queued_dagruns_do_follow_execution_date_order(self, dag_maker): with dag_maker("test_dag1", max_active_runs=1): EmptyOperator(task_id="mytask") date = DEFAULT_DATE + dag_version = DagVersion.get_latest_version("test_dag1") for i in range(30): dr = dag_maker.create_dagrun( - run_id=f"dagrun_{i}", run_type=DagRunType.SCHEDULED, state=State.QUEUED, execution_date=date + run_id=f"dagrun_{i}", + run_type=DagRunType.SCHEDULED, + state=State.QUEUED, + execution_date=date, + dag_version=dag_version, ) date = dr.execution_date + timedelta(hours=1) scheduler_job = Job() @@ -5175,11 +5171,15 @@ def test_no_dagruns_would_stuck_in_running(self, dag_maker): with dag_maker("test_dagrun_states_are_correct_1", max_active_runs=1, start_date=date) as dag: task1 = EmptyOperator(task_id="dummy_task") - dr1_running = dag_maker.create_dagrun(run_id="dr1_run_1", execution_date=date) + dag_version = DagVersion.get_latest_version(dag.dag_id) + dr1_running = dag_maker.create_dagrun( + run_id="dr1_run_1", execution_date=date, dag_version=dag_version + ) data_interval = dag.infer_automated_data_interval(logical_date) dag_maker.create_dagrun( run_id="dr1_run_2", state=State.QUEUED, + dag_version=dag_version, execution_date=dag.next_dagrun_info( last_automated_dagrun=data_interval, restricted=False ).data_interval.start, @@ -5188,26 +5188,48 @@ def test_no_dagruns_would_stuck_in_running(self, dag_maker): date = timezone.datetime(2020, 1, 1) with dag_maker("test_dagrun_states_are_correct_2", start_date=date) as dag: EmptyOperator(task_id="dummy_task") + dag_version = DagVersion.get_latest_version(dag.dag_id) for i in range(16): - dr = dag_maker.create_dagrun(run_id=f"dr2_run_{i+1}", state=State.RUNNING, execution_date=date) + dr = dag_maker.create_dagrun( + run_id=f"dr2_run_{i+1}", + state=State.RUNNING, + execution_date=date, + dag_version=dag_version, + ) date = dr.execution_date + timedelta(hours=1) dr16 = DagRun.find(run_id="dr2_run_16") date = dr16[0].execution_date + timedelta(hours=1) for i in range(16, 32): - dr = dag_maker.create_dagrun(run_id=f"dr2_run_{i+1}", state=State.QUEUED, execution_date=date) + dr = dag_maker.create_dagrun( + run_id=f"dr2_run_{i+1}", + state=State.QUEUED, + execution_date=date, + dag_version=dag_version, + ) date = dr.execution_date + timedelta(hours=1) # third dag and dagruns date = timezone.datetime(2021, 1, 1) with dag_maker("test_dagrun_states_are_correct_3", start_date=date) as dag: EmptyOperator(task_id="dummy_task") + dag_version = DagVersion.get_latest_version(dag.dag_id) for i in range(16): - dr = dag_maker.create_dagrun(run_id=f"dr3_run_{i+1}", state=State.RUNNING, execution_date=date) + dr = dag_maker.create_dagrun( + run_id=f"dr3_run_{i+1}", + state=State.RUNNING, + execution_date=date, + dag_version=dag_version, + ) date = dr.execution_date + timedelta(hours=1) dr16 = DagRun.find(run_id="dr3_run_16") date = dr16[0].execution_date + timedelta(hours=1) for i in range(16, 32): - dr = dag_maker.create_dagrun(run_id=f"dr2_run_{i+1}", state=State.QUEUED, execution_date=date) + dr = dag_maker.create_dagrun( + run_id=f"dr2_run_{i+1}", + state=State.QUEUED, + execution_date=date, + dag_version=dag_version, + ) date = dr.execution_date + timedelta(hours=1) scheduler_job = Job() @@ -5486,11 +5508,17 @@ def test_runs_respected_after_clear(self, dag_maker): self.job_runner = SchedulerJobRunner(job=scheduler_job, subdir=os.devnull) self.job_runner.processor_agent = mock.MagicMock() - + dag_version = DagVersion.get_latest_version(dag.dag_id) session = settings.Session() - dr = dag_maker.create_dagrun(run_type=DagRunType.SCHEDULED, state=State.QUEUED) - dr = dag_maker.create_dagrun_after(dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED) - dag_maker.create_dagrun_after(dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED) + dr = dag_maker.create_dagrun( + run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) + dr = dag_maker.create_dagrun_after( + dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) + dag_maker.create_dagrun_after( + dr, run_type=DagRunType.SCHEDULED, state=State.QUEUED, dag_version=dag_version + ) dag.clear() assert len(DagRun.find(dag_id=dag.dag_id, state=State.QUEUED, session=session)) == 3 @@ -5816,6 +5844,7 @@ def test_find_zombies_handle_failure_callbacks_are_correctly_passed_to_dag_proce assert expected_failure_callback_requests[0] == callback_requests[0] def test_cleanup_stale_dags(self): + clear_db_dags() dagbag = DagBag(TEST_DAG_FOLDER, read_dags_from_db=False) with create_session() as session: dag = dagbag.get_dag("test_example_bash_operator") diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index e38beb2110ca9..fc7a2b24b836a 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -62,6 +62,7 @@ dag as dag_decorator, get_asset_triggered_next_run_info, ) +from airflow.models.dag_version import DagVersion from airflow.models.dagrun import DagRun from airflow.models.param import DagParam, Param from airflow.models.serialized_dag import SerializedDagModel @@ -141,14 +142,11 @@ def setup_method(self) -> None: clear_db_runs() clear_db_dags() clear_db_assets() - self.patcher_dag_code = mock.patch("airflow.models.dag.DagCode.bulk_sync_to_db") - self.patcher_dag_code.start() def teardown_method(self) -> None: clear_db_runs() clear_db_dags() clear_db_assets() - self.patcher_dag_code.stop() @staticmethod def _clean_up(dag_id: str): @@ -1039,14 +1037,16 @@ def test_existing_dag_is_paused_config(self): assert dag.max_consecutive_failed_dag_runs == 2 def test_existing_dag_is_paused_after_limit(self): - def add_failed_dag_run(id, execution_date): + def add_failed_dag_run(dag, id, execution_date): triggered_by_kwargs = {"triggered_by": DagRunTriggeredByType.TEST} if AIRFLOW_V_3_0_PLUS else {} + dag_v = DagVersion.get_latest_version(dag_id=dag.dag_id) dr = dag.create_dagrun( run_type=DagRunType.MANUAL, run_id="run_id_" + id, execution_date=execution_date, state=State.FAILED, data_interval=(execution_date, execution_date), + dag_version=dag_v, **triggered_by_kwargs, ) ti_op1 = dr.get_task_instance(task_id=op1.task_id, session=session) @@ -1059,14 +1059,16 @@ def add_failed_dag_run(id, execution_date): dag.add_task(op1) session = settings.Session() dag.sync_to_db(session=session) + SerializedDagModel.write_dag(dag) assert not dag.get_is_paused() # dag should be paused after 2 failed dag_runs add_failed_dag_run( + dag, "1", TEST_DATE, ) - add_failed_dag_run("2", TEST_DATE + timedelta(days=1)) + add_failed_dag_run(dag, "2", TEST_DATE + timedelta(days=1)) assert dag.get_is_paused() dag.clear() self._clean_up(dag_id) @@ -1085,8 +1087,7 @@ def test_dag_is_deactivated_upon_dagfile_deletion(self): dag = DAG(dag_id, schedule=None, is_paused_upon_creation=True) dag.fileloc = dag_fileloc session = settings.Session() - with mock.patch("airflow.models.dag.DagCode.bulk_sync_to_db"): - dag.sync_to_db(session=session, processor_subdir="/usr/local/airflow/dags/") + dag.sync_to_db(session=session, processor_subdir="/usr/local/airflow/dags/") orm_dag = session.query(DagModel).filter(DagModel.dag_id == dag_id).one() @@ -2370,9 +2371,8 @@ def test_relative_fileloc_serialized( """ dag = DAG(dag_id="test", schedule=None) dag.fileloc = fileloc - sdm = SerializedDagModel(dag) - session.add(sdm) - session.commit() + dag.sync_to_db() + SerializedDagModel.write_dag(dag) session.expunge_all() sdm = SerializedDagModel.get(dag.dag_id, session) dag = sdm.dag @@ -2383,8 +2383,10 @@ def test__processor_dags_folder(self, session): """Only populated after deserializtion""" dag = DAG(dag_id="test", schedule=None) dag.fileloc = "/abc/test.py" + dag.sync_to_db() assert dag._processor_dags_folder is None - sdm = SerializedDagModel(dag) + SerializedDagModel.write_dag(dag) + sdm = SerializedDagModel.get(dag.dag_id, session) assert sdm.dag._processor_dags_folder == settings.DAGS_FOLDER @pytest.mark.need_serialized_dag diff --git a/tests/models/test_dag_version.py b/tests/models/test_dag_version.py new file mode 100644 index 0000000000000..42a33b4b66f13 --- /dev/null +++ b/tests/models/test_dag_version.py @@ -0,0 +1,113 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import pytest +from sqlalchemy import func, select + +from airflow.models.dag_version import DagVersion +from airflow.models.serialized_dag import SerializedDagModel +from airflow.operators.empty import EmptyOperator + +from tests_common.test_utils.db import clear_db_dags + +pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode] + + +class TestDagVersion: + def setup_method(self): + clear_db_dags() + + def teardown_method(self): + clear_db_dags() + + @pytest.mark.need_serialized_dag + def test_writing_dag_version(self, dag_maker, session): + with dag_maker("test_writing_dag_version") as dag: + pass + + latest_version = DagVersion.get_latest_version(dag.dag_id) + assert latest_version.version_number == 1 + assert not latest_version.version_name + assert latest_version.dag_id == dag.dag_id + + @pytest.mark.need_serialized_dag + def test_writing_dag_version_with_version_name(self, dag_maker, session): + version_name = "my_version" + with dag_maker(version_name=version_name) as dag: + pass + + latest_version = DagVersion.get_latest_version(dag.dag_id) + assert latest_version.version_number == 1 + assert latest_version.version_name == version_name + assert latest_version.dag_id == dag.dag_id + + def test_writing_dag_version_with_changes(self, dag_maker, session): + """This also tested the get_latest_version method""" + version_name = "my_version" + with dag_maker("test1", version_name=version_name) as dag: + EmptyOperator(task_id="task1") + dag.sync_to_db() + SerializedDagModel.write_dag(dag) + # Add extra task to change the dag + with dag_maker("test1", version_name=version_name) as dag2: + EmptyOperator(task_id="task1") + EmptyOperator(task_id="task2") + dag2.sync_to_db() + SerializedDagModel.write_dag(dag2) + + latest_version = DagVersion.get_latest_version(dag.dag_id) + assert latest_version.version_number == 2 + assert latest_version.version_name == version_name + assert 2 == session.scalar(select(func.count()).where(DagVersion.dag_id == dag.dag_id)) + + @pytest.mark.need_serialized_dag + def test_get_version(self, dag_maker, session): + """The two dags have the same version name and number but different dag ids""" + version_name = "my_version" + dag1_id = "test1" + with dag_maker(dag1_id, version_name=version_name): + EmptyOperator(task_id="task1") + + with dag_maker("test2", version_name=version_name): + EmptyOperator(task_id="task1") + + with dag_maker("test3"): + EmptyOperator(task_id="task1") + + version = DagVersion.get_version(dag1_id) + assert version.version_number == 1 + assert version.version_name == version_name + assert version.dag_id == dag1_id + assert version.version == "my_version-1" + + @pytest.mark.need_serialized_dag + def test_version_property(self, dag_maker): + version_name = "my_version" + with dag_maker("test1", version_name=version_name) as dag: + pass + + latest_version = DagVersion.get_latest_version(dag.dag_id) + assert latest_version.version == f"{version_name}-1" + + @pytest.mark.need_serialized_dag + def test_version_property_with_null_version_name(self, dag_maker): + with dag_maker("test1") as dag: + pass + + latest_version = DagVersion.get_latest_version(dag.dag_id) + assert latest_version.version == "1" diff --git a/tests/models/test_dagbag.py b/tests/models/test_dagbag.py index d91f6738822a1..6915e4df0c1fc 100644 --- a/tests/models/test_dagbag.py +++ b/tests/models/test_dagbag.py @@ -64,10 +64,10 @@ def db_clean_up(): class TestDagBag: - def setup_class(self): + def setup_class(cls): db_clean_up() - def teardown_class(self): + def teardown_class(cls): db_clean_up() def test_get_existing_dag(self, tmp_path): @@ -723,6 +723,7 @@ def _sync_to_db(): dagbag.sync_to_db(session=session) dag = dagbag.dags["test_example_bash_operator"] + dag.sync_to_db() _sync_to_db() mock_sync_perm_for_dag.assert_called_once_with(dag, session=session) @@ -820,6 +821,7 @@ def test_get_dag_with_dag_serialization(self): with time_machine.travel((tz.datetime(2020, 1, 5, 0, 0, 0)), tick=False): example_bash_op_dag = DagBag(include_examples=True).dags.get("example_bash_operator") + example_bash_op_dag.sync_to_db() SerializedDagModel.write_dag(dag=example_bash_op_dag) dag_bag = DagBag(read_dags_from_db=True) @@ -837,6 +839,7 @@ def test_get_dag_with_dag_serialization(self): # Make a change in the DAG and write Serialized DAG to the DB with time_machine.travel((tz.datetime(2020, 1, 5, 0, 0, 6)), tick=False): example_bash_op_dag.tags.add("new_tag") + example_bash_op_dag.sync_to_db() SerializedDagModel.write_dag(dag=example_bash_op_dag) # Since min_serialized_dag_fetch_interval is passed verify that calling 'dag_bag.get_dag' @@ -852,15 +855,16 @@ def test_get_dag_with_dag_serialization(self): @pytest.mark.skip_if_database_isolation_mode # Does not work in db isolation mode @patch("airflow.models.dagbag.settings.MIN_SERIALIZED_DAG_UPDATE_INTERVAL", 5) @patch("airflow.models.dagbag.settings.MIN_SERIALIZED_DAG_FETCH_INTERVAL", 5) - def test_get_dag_refresh_race_condition(self): + def test_get_dag_refresh_race_condition(self, session): """ Test that DagBag.get_dag correctly refresh the Serialized DAG even if SerializedDagModel.last_updated is before DagBag.dags_last_fetched. """ - + db_clean_up() # serialize the initial version of the DAG with time_machine.travel((tz.datetime(2020, 1, 5, 0, 0, 0)), tick=False): example_bash_op_dag = DagBag(include_examples=True).dags.get("example_bash_operator") + example_bash_op_dag.sync_to_db() SerializedDagModel.write_dag(dag=example_bash_op_dag) # deserialize the DAG @@ -886,6 +890,7 @@ def test_get_dag_refresh_race_condition(self): # long before the transaction is committed with time_machine.travel((tz.datetime(2020, 1, 5, 1, 0, 0)), tick=False): example_bash_op_dag.tags.add("new_tag") + example_bash_op_dag.sync_to_db() SerializedDagModel.write_dag(dag=example_bash_op_dag) # Since min_serialized_dag_fetch_interval is passed verify that calling 'dag_bag.get_dag' @@ -906,6 +911,7 @@ def test_collect_dags_from_db(self): example_dags = dagbag.dags for dag in example_dags.values(): + dag.sync_to_db() SerializedDagModel.write_dag(dag) new_dagbag = DagBag(read_dags_from_db=True) diff --git a/tests/models/test_dagcode.py b/tests/models/test_dagcode.py index 26b29ea8f9c02..fd7d761f9103f 100644 --- a/tests/models/test_dagcode.py +++ b/tests/models/test_dagcode.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -from datetime import timedelta from unittest.mock import patch import pytest @@ -25,13 +24,16 @@ import airflow.example_dags as example_dags_module from airflow.exceptions import AirflowException from airflow.models import DagBag +from airflow.models.dag import DAG +from airflow.models.dag_version import DagVersion from airflow.models.dagcode import DagCode +from airflow.models.serialized_dag import SerializedDagModel as SDM # To move it to a shared module. from airflow.utils.file import open_maybe_zipped from airflow.utils.session import create_session -from tests_common.test_utils.db import clear_db_dag_code +from tests_common.test_utils.db import clear_db_dag_code, clear_db_dags pytestmark = [pytest.mark.db_test, pytest.mark.skip_if_database_isolation_mode] @@ -39,6 +41,7 @@ def make_example_dags(module): """Loads DAGs from a module for test.""" dagbag = DagBag(module.__path__[0]) + DAG.bulk_write_to_db(dagbag.dags.values()) return dagbag.dags @@ -46,55 +49,35 @@ class TestDagCode: """Unit tests for DagCode.""" def setup_method(self): + clear_db_dags() clear_db_dag_code() def teardown_method(self): + clear_db_dags() clear_db_dag_code() def _write_two_example_dags(self): example_dags = make_example_dags(example_dags_module) bash_dag = example_dags["example_bash_operator"] - DagCode(bash_dag.fileloc).sync_to_db() + dag_version = DagVersion.get_latest_version("example_bash_operator") + DagCode(dag_version, bash_dag.fileloc).sync_to_db() xcom_dag = example_dags["example_xcom"] - DagCode(xcom_dag.fileloc).sync_to_db() + dag_version = DagVersion.get_latest_version("example_xcom") + DagCode(dag_version, xcom_dag.fileloc).sync_to_db() return [bash_dag, xcom_dag] def _write_example_dags(self): example_dags = make_example_dags(example_dags_module) for dag in example_dags.values(): - dag.sync_to_db() + SDM.write_dag(dag) return example_dags - def test_sync_to_db(self): + def test_write_to_db(self): """Dg code can be written into database.""" example_dags = self._write_example_dags() self._compare_example_dags(example_dags) - def test_bulk_sync_to_db(self): - """Dg code can be bulk written into database.""" - example_dags = make_example_dags(example_dags_module) - files = [dag.fileloc for dag in example_dags.values()] - with create_session() as session: - DagCode.bulk_sync_to_db(files, session=session) - session.commit() - - self._compare_example_dags(example_dags) - - def test_bulk_sync_to_db_half_files(self): - """Dg code can be bulk written into database.""" - example_dags = make_example_dags(example_dags_module) - files = [dag.fileloc for dag in example_dags.values()] - half_files = files[: len(files) // 2] - with create_session() as session: - DagCode.bulk_sync_to_db(half_files, session=session) - session.commit() - with create_session() as session: - DagCode.bulk_sync_to_db(files, session=session) - session.commit() - - self._compare_example_dags(example_dags) - @patch.object(DagCode, "dag_fileloc_hash") def test_detecting_duplicate_key(self, mock_hash): """Dag code detects duplicate key.""" @@ -112,6 +95,8 @@ def _compare_example_dags(self, example_dags): session.query(DagCode.fileloc, DagCode.fileloc_hash, DagCode.source_code) .filter(DagCode.fileloc == dag.fileloc) .filter(DagCode.fileloc_hash == dag_fileloc_hash) + .order_by(DagCode.last_updated.desc()) + .limit(1) .one() ) @@ -126,7 +111,7 @@ def test_code_can_be_read_when_no_access_to_file(self): Source Code should at least exist in one of DB or File. """ example_dag = make_example_dags(example_dags_module).get("example_bash_operator") - example_dag.sync_to_db() + SDM.write_dag(example_dag) # Mock that there is no access to the Dag File with patch("airflow.models.dagcode.open_maybe_zipped") as mock_open: @@ -136,27 +121,50 @@ def test_code_can_be_read_when_no_access_to_file(self): for test_string in ["example_bash_operator", "also_run_this", "run_this_last"]: assert test_string in dag_code - def test_db_code_updated_on_dag_file_change(self): - """Test if DagCode is updated in DB when DAG file is changed""" + def test_db_code_created_on_serdag_change(self, session): + """Test new DagCode is created in DB when DAG file is changed""" example_dag = make_example_dags(example_dags_module).get("example_bash_operator") - example_dag.sync_to_db() - - with create_session() as session: - result = session.query(DagCode).filter(DagCode.fileloc == example_dag.fileloc).one() - - assert result.fileloc == example_dag.fileloc - assert result.source_code is not None + SDM.write_dag(example_dag) - with patch("airflow.models.dagcode.os.path.getmtime") as mock_mtime: - mock_mtime.return_value = (result.last_updated + timedelta(seconds=1)).timestamp() + result = ( + session.query(DagCode) + .filter(DagCode.fileloc == example_dag.fileloc) + .order_by(DagCode.last_updated.desc()) + .limit(1) + .one() + ) - with patch("airflow.models.dagcode.DagCode._get_code_from_file") as mock_code: - mock_code.return_value = "# dummy code" - example_dag.sync_to_db() + assert result.fileloc == example_dag.fileloc + assert result.source_code is not None - with create_session() as session: - new_result = session.query(DagCode).filter(DagCode.fileloc == example_dag.fileloc).one() - - assert new_result.fileloc == example_dag.fileloc - assert new_result.source_code == "# dummy code" - assert new_result.last_updated > result.last_updated + example_dag = make_example_dags(example_dags_module).get("example_bash_operator") + SDM.write_dag(example_dag, processor_subdir="/tmp") + with patch("airflow.models.dagcode.DagCode._get_code_from_file") as mock_code: + mock_code.return_value = "# dummy code" + SDM.write_dag(example_dag) + + new_result = ( + session.query(DagCode) + .filter(DagCode.fileloc == example_dag.fileloc) + .order_by(DagCode.last_updated.desc()) + .limit(1) + .one() + ) + + assert new_result.fileloc == example_dag.fileloc + assert new_result.source_code != result.source_code + assert new_result.last_updated > result.last_updated + + def test_has_dag(self, dag_maker): + """Test has_dag method.""" + with dag_maker("test_has_dag") as dag: + pass + dag.sync_to_db() + SDM.write_dag(dag) + + with dag_maker() as dag2: + pass + dag2.sync_to_db() + SDM.write_dag(dag2) + + assert DagCode.has_dag(dag.fileloc) diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index e8889fb102236..dc5a8ab66e566 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -88,6 +88,7 @@ def _clean_db(): db.clear_db_variables() db.clear_db_assets() db.clear_db_xcom() + db.clear_db_dags() def create_dag_run( self, diff --git a/tests/models/test_serialized_dag.py b/tests/models/test_serialized_dag.py index cdff883760c92..d0bfe37a69cc0 100644 --- a/tests/models/test_serialized_dag.py +++ b/tests/models/test_serialized_dag.py @@ -23,7 +23,7 @@ import pendulum import pytest -from sqlalchemy import select +from sqlalchemy import func, select import airflow.example_dags as example_dags_module from airflow.assets import Asset @@ -31,6 +31,7 @@ from airflow.models.dagbag import DagBag from airflow.models.dagcode import DagCode from airflow.models.serialized_dag import SerializedDagModel as SDM +from airflow.operators.empty import EmptyOperator from airflow.providers.standard.operators.bash import BashOperator from airflow.serialization.serialized_objects import SerializedDAG from airflow.settings import json @@ -47,6 +48,7 @@ def make_example_dags(module): """Loads DAGs from a module for test.""" dagbag = DagBag(module.__path__[0]) + DAG.bulk_write_to_db(dagbag.dags.values()) return dagbag.dags @@ -61,6 +63,7 @@ class TestSerializedDagModel: ], ) def setup_test_cases(self, request, monkeypatch): + db.clear_db_dags() db.clear_db_serialized_dags() with mock.patch("airflow.models.serialized_dag.COMPRESS_SERIALIZED_DAGS", request.param): yield @@ -86,7 +89,7 @@ def test_write_dag(self): assert SDM.has_dag(dag.dag_id) result = session.query(SDM).filter(SDM.dag_id == dag.dag_id).one() - assert result.fileloc == dag.fileloc + assert result.dag_version.dag_code.fileloc == dag.fileloc # Verifies JSON schema. SerializedDAG.validate_schema(result.data) @@ -98,29 +101,28 @@ def test_serialized_dag_is_updated_if_dag_is_changed(self): dag_updated = SDM.write_dag(dag=example_bash_op_dag) assert dag_updated is True - with create_session() as session: - s_dag = session.get(SDM, example_bash_op_dag.dag_id) + s_dag = SDM.get(example_bash_op_dag.dag_id) - # Test that if DAG is not changed, Serialized DAG is not re-written and last_updated - # column is not updated - dag_updated = SDM.write_dag(dag=example_bash_op_dag) - s_dag_1 = session.get(SDM, example_bash_op_dag.dag_id) + # Test that if DAG is not changed, Serialized DAG is not re-written and last_updated + # column is not updated + dag_updated = SDM.write_dag(dag=example_bash_op_dag) + s_dag_1 = SDM.get(example_bash_op_dag.dag_id) - assert s_dag_1.dag_hash == s_dag.dag_hash - assert s_dag.last_updated == s_dag_1.last_updated - assert dag_updated is False + assert s_dag_1.dag_hash == s_dag.dag_hash + assert s_dag.last_updated == s_dag_1.last_updated + assert dag_updated is False - # Update DAG - example_bash_op_dag.tags.add("new_tag") - assert example_bash_op_dag.tags == {"example", "example2", "new_tag"} + # Update DAG + example_bash_op_dag.tags.add("new_tag") + assert example_bash_op_dag.tags == {"example", "example2", "new_tag"} - dag_updated = SDM.write_dag(dag=example_bash_op_dag) - s_dag_2 = session.get(SDM, example_bash_op_dag.dag_id) + dag_updated = SDM.write_dag(dag=example_bash_op_dag) + s_dag_2 = SDM.get(example_bash_op_dag.dag_id) - assert s_dag.last_updated != s_dag_2.last_updated - assert s_dag.dag_hash != s_dag_2.dag_hash - assert s_dag_2.data["dag"]["tags"] == ["example", "example2", "new_tag"] - assert dag_updated is True + assert s_dag.last_updated != s_dag_2.last_updated + assert s_dag.dag_hash != s_dag_2.dag_hash + assert s_dag_2.data["dag"]["tags"] == ["example", "example2", "new_tag"] + assert dag_updated is True @pytest.mark.skip_if_database_isolation_mode # Does not work in db isolation mode def test_serialized_dag_is_updated_if_processor_subdir_changed(self): @@ -131,12 +133,12 @@ def test_serialized_dag_is_updated_if_processor_subdir_changed(self): assert dag_updated is True with create_session() as session: - s_dag = session.get(SDM, example_bash_op_dag.dag_id) + s_dag = SDM.get(example_bash_op_dag.dag_id) # Test that if DAG is not changed, Serialized DAG is not re-written and last_updated # column is not updated dag_updated = SDM.write_dag(dag=example_bash_op_dag, processor_subdir="/tmp/test") - s_dag_1 = session.get(SDM, example_bash_op_dag.dag_id) + s_dag_1 = SDM.get(example_bash_op_dag.dag_id) assert s_dag_1.dag_hash == s_dag.dag_hash assert s_dag.last_updated == s_dag_1.last_updated @@ -145,7 +147,7 @@ def test_serialized_dag_is_updated_if_processor_subdir_changed(self): # Update DAG dag_updated = SDM.write_dag(dag=example_bash_op_dag, processor_subdir="/tmp/other") - s_dag_2 = session.get(SDM, example_bash_op_dag.dag_id) + s_dag_2 = SDM.get(example_bash_op_dag.dag_id) assert s_dag.processor_subdir != s_dag_2.processor_subdir assert dag_updated is True @@ -162,6 +164,19 @@ def test_read_dags(self): assert serialized_dag.dag_id == dag.dag_id assert set(serialized_dag.task_dict) == set(dag.task_dict) + @pytest.mark.skip_if_database_isolation_mode # Does not work in db isolation mode + def test_read_all_dags_only_picks_the_latest_serdags(self, session): + example_dags = self._write_example_dags() + serialized_dags = SDM.read_all_dags() + assert len(example_dags) == len(serialized_dags) + + ex_dags = make_example_dags(example_dags_module) + SDM.write_dag(ex_dags.get("example_bash_operator"), processor_subdir="/tmp/") + serialized_dags2 = SDM.read_all_dags() + sdags = session.query(SDM).all() + # assert only the latest SDM is returned + assert len(sdags) != len(serialized_dags2) + @pytest.mark.skip_if_database_isolation_mode # Does not work in db isolation mode def test_remove_dags_by_id(self): """DAGs can be removed from database.""" @@ -190,26 +205,12 @@ def test_bulk_sync_to_db(self): DAG("dag_2", schedule=None), DAG("dag_3", schedule=None), ] - with assert_queries_count(10): + DAG.bulk_write_to_db(dags) + # we also write to dag_version and dag_code tables + # in dag_version, we search for unique version_name too + with assert_queries_count(24): SDM.bulk_sync_to_db(dags) - @pytest.mark.skip_if_database_isolation_mode # Does not work in db isolation mode - @pytest.mark.parametrize("dag_dependencies_fields", [{"dag_dependencies": None}, {}]) - def test_get_dag_dependencies_default_to_empty(self, dag_dependencies_fields): - """Test a pre-2.1.0 serialized DAG can deserialize DAG dependencies.""" - example_dags = make_example_dags(example_dags_module) - - with create_session() as session: - sdms = [SDM(dag) for dag in example_dags.values()] - # Simulate pre-2.1.0 format. - for sdm in sdms: - del sdm.data["dag"]["dag_dependencies"] - sdm.data["dag"].update(dag_dependencies_fields) - session.bulk_save_objects(sdms) - - expected_dependencies = {dag_id: [] for dag_id in example_dags} - assert SDM.get_dag_dependencies() == expected_dependencies - @pytest.mark.skip_if_database_isolation_mode # Does not work in db isolation mode def test_order_of_dag_params_is_stable(self): """ @@ -284,3 +285,31 @@ def get_hash_set(): first_hashes = get_hash_set() # assert that the hashes are the same assert first_hashes == get_hash_set() + + def test_get_latest_serdag_versions(self, dag_maker, session): + # first dag + with dag_maker("dag1") as dag: + EmptyOperator(task_id="task1") + dag.sync_to_db() + SDM.write_dag(dag) + with dag_maker("dag1") as dag: + EmptyOperator(task_id="task1") + EmptyOperator(task_id="task2") + dag.sync_to_db() + SDM.write_dag(dag) + # second dag + with dag_maker("dag2") as dag: + EmptyOperator(task_id="task1") + dag.sync_to_db() + SDM.write_dag(dag) + with dag_maker("dag2") as dag: + EmptyOperator(task_id="task1") + EmptyOperator(task_id="task2") + dag.sync_to_db() + SDM.write_dag(dag) + + # Total serdags should be 4 + assert session.scalar(select(func.count()).select_from(SDM)) == 4 + + latest_versions = SDM.get_latest_serialized_dags(dag_ids=["dag1", "dag2"], session=session) + assert len(latest_versions) == 2 diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index 8a1df0594e4e9..36b2d22f60fdc 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -103,7 +103,7 @@ from tests_common.test_utils import db from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS from tests_common.test_utils.config import conf_vars -from tests_common.test_utils.db import clear_db_connections, clear_db_runs +from tests_common.test_utils.db import clear_db_connections, clear_db_dags, clear_db_runs from tests_common.test_utils.mock_operators import MockOperator if AIRFLOW_V_3_0_PLUS: @@ -2992,6 +2992,7 @@ def test_changing_of_asset_when_adrq_is_already_populated(self, dag_maker): Test that when a task that produces asset has ran, that changing the consumer dag asset will not cause primary key blank-out """ + clear_db_dags() from airflow.assets import Asset with dag_maker(schedule=None, serialized=True) as dag1: @@ -4013,6 +4014,7 @@ def test_refresh_from_db(self, create_task_instance): "next_method": None, "updated_at": None, "task_display_name": "Test Refresh from DB Task", + "dag_version_id": None, } # Make sure we aren't missing any new value in our expected_values list. expected_keys = {f"task_instance.{key}" for key in expected_values} diff --git a/tests/operators/test_trigger_dagrun.py b/tests/operators/test_trigger_dagrun.py index 52a11d10e5e33..0b40154b21b05 100644 --- a/tests/operators/test_trigger_dagrun.py +++ b/tests/operators/test_trigger_dagrun.py @@ -28,8 +28,6 @@ from airflow.models.dag import DagModel from airflow.models.dagbag import DagBag from airflow.models.dagrun import DagRun -from airflow.models.log import Log -from airflow.models.serialized_dag import SerializedDagModel from airflow.models.taskinstance import TaskInstance from airflow.operators.trigger_dagrun import TriggerDagRunOperator from airflow.settings import TracebackSessionForTests @@ -39,6 +37,8 @@ from airflow.utils.state import DagRunState, State, TaskInstanceState from airflow.utils.types import DagRunType +from tests_common.test_utils.db import clear_db_dags, clear_db_logs, clear_db_runs + pytestmark = pytest.mark.db_test DEFAULT_DATE = datetime(2019, 1, 1, tzinfo=timezone.utc) @@ -82,12 +82,9 @@ def re_sync_triggered_dag_to_db(self, dag, dag_maker): def teardown_method(self): """Cleanup state after testing in DB.""" - with create_session() as session: - session.query(Log).filter(Log.dag_id == TEST_DAG_ID).delete(synchronize_session=False) - for dbmodel in [DagModel, DagRun, TaskInstance, SerializedDagModel]: - session.query(dbmodel).filter(dbmodel.dag_id.in_([TRIGGERED_DAG_ID, TEST_DAG_ID])).delete( - synchronize_session=False - ) + clear_db_logs() + clear_db_runs() + clear_db_dags() # pathlib.Path(self._tmpfile).unlink() diff --git a/tests/sensors/test_external_task_sensor.py b/tests/sensors/test_external_task_sensor.py index 43911c1a41d48..e2246a2f75132 100644 --- a/tests/sensors/test_external_task_sensor.py +++ b/tests/sensors/test_external_task_sensor.py @@ -124,6 +124,7 @@ def add_fake_task_group(self, target_states=None): with self.dag as dag: with TaskGroup(group_id=TEST_TASK_GROUP_ID) as task_group: _ = [EmptyOperator(task_id=f"task{i}") for i in range(len(target_states))] + dag.sync_to_db() SerializedDagModel.write_dag(dag) for idx, task in enumerate(task_group): @@ -146,7 +147,7 @@ def fake_mapped_task(x: int): fake_task() fake_mapped_task.expand(x=list(map_indexes)) - + dag.sync_to_db() SerializedDagModel.write_dag(dag) for task in task_group: diff --git a/tests/utils/test_db_cleanup.py b/tests/utils/test_db_cleanup.py index 47e93c1616d63..5df2c37cddb32 100644 --- a/tests/utils/test_db_cleanup.py +++ b/tests/utils/test_db_cleanup.py @@ -352,6 +352,7 @@ def test_no_models_missing(self): "rendered_task_instance_fields", # foreign key with TI "dag_priority_parsing_request", # Records are purged once per DAG Processing loop, not a # significant source of data. + "dag_version", # self-maintaining } from airflow.utils.db_cleanup import config_dict diff --git a/tests/www/views/test_views_tasks.py b/tests/www/views/test_views_tasks.py index 19caafe55bc63..424fb02979cdc 100644 --- a/tests/www/views/test_views_tasks.py +++ b/tests/www/views/test_views_tasks.py @@ -28,9 +28,9 @@ import time_machine from airflow import settings -from airflow.models.dag import DAG, DagModel +from airflow.models.dag import DAG from airflow.models.dagbag import DagBag -from airflow.models.dagcode import DagCode +from airflow.models.serialized_dag import SerializedDagModel from airflow.models.taskinstance import TaskInstance from airflow.models.taskreschedule import TaskReschedule from airflow.models.xcom import XCom @@ -500,7 +500,7 @@ def test_code(admin_client): def test_code_from_db(admin_client): dag = DagBag(include_examples=True).get_dag("example_bash_operator") - DagCode(dag.fileloc, DagCode._get_code_from_file(dag.fileloc)).sync_to_db() + SerializedDagModel.write_dag(dag) url = "code?dag_id=example_bash_operator" resp = admin_client.get(url, follow_redirects=True) check_content_not_in_response("Failed to load DAG file Code", resp) @@ -510,7 +510,7 @@ def test_code_from_db(admin_client): def test_code_from_db_all_example_dags(admin_client): dagbag = DagBag(include_examples=True) for dag in dagbag.dags.values(): - DagCode(dag.fileloc, DagCode._get_code_from_file(dag.fileloc)).sync_to_db() + SerializedDagModel.write_dag(dag) url = "code?dag_id=example_bash_operator" resp = admin_client.get(url, follow_redirects=True) check_content_not_in_response("Failed to load DAG file Code", resp) @@ -614,23 +614,12 @@ def heartbeat(self): return True -@pytest.fixture -def new_id_example_bash_operator(): - dag_id = "example_bash_operator" - test_dag_id = "non_existent_dag" - with create_session() as session: - dag_query = session.query(DagModel).filter(DagModel.dag_id == dag_id) - dag_query.first().tags = [] # To avoid "FOREIGN KEY constraint" error) - with create_session() as session: - dag_query.update({"dag_id": test_dag_id}) - yield test_dag_id - with create_session() as session: - session.query(DagModel).filter(DagModel.dag_id == test_dag_id).update({"dag_id": dag_id}) - - -def test_delete_dag_button_for_dag_on_scheduler_only(admin_client, new_id_example_bash_operator): +def test_delete_dag_button_for_dag_on_scheduler_only(admin_client, dag_maker): + with dag_maker() as dag: + EmptyOperator(task_id="task") + dag.sync_to_db() # The delete-dag URL should be generated correctly - test_dag_id = new_id_example_bash_operator + test_dag_id = dag.dag_id resp = admin_client.get("/", follow_redirects=True) check_content_in_response(f"/delete?dag_id={test_dag_id}", resp) check_content_in_response(f"return confirmDeleteDag(this, '{test_dag_id}')", resp) @@ -1133,6 +1122,7 @@ def test_task_instances(admin_client): "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), + "dag_version_id": None, }, "run_after_loop": { "custom_operator_name": None, @@ -1169,6 +1159,7 @@ def test_task_instances(admin_client): "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), + "dag_version_id": None, }, "run_this_last": { "custom_operator_name": None, @@ -1205,6 +1196,7 @@ def test_task_instances(admin_client): "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), + "dag_version_id": None, }, "runme_0": { "custom_operator_name": None, @@ -1241,6 +1233,7 @@ def test_task_instances(admin_client): "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), + "dag_version_id": None, }, "runme_1": { "custom_operator_name": None, @@ -1277,6 +1270,7 @@ def test_task_instances(admin_client): "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), + "dag_version_id": None, }, "runme_2": { "custom_operator_name": None, @@ -1313,6 +1307,7 @@ def test_task_instances(admin_client): "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), + "dag_version_id": None, }, "this_will_skip": { "custom_operator_name": None, @@ -1349,5 +1344,6 @@ def test_task_instances(admin_client): "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), + "dag_version_id": None, }, } diff --git a/tests_common/pytest_plugin.py b/tests_common/pytest_plugin.py index 2a35f79de14f5..2a14511ebf239 100644 --- a/tests_common/pytest_plugin.py +++ b/tests_common/pytest_plugin.py @@ -884,11 +884,30 @@ def __exit__(self, type, value, traceback): self.serialized_model = SerializedDagModel( dag, processor_subdir=self.dag_model.processor_subdir ) - self.session.merge(self.serialized_model) + sdm = SerializedDagModel.get(dag.dag_id, session=self.session) + from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS + + if AIRFLOW_V_3_0_PLUS and not sdm: + from airflow.models.dag_version import DagVersion + from airflow.models.dagcode import DagCode + + dagv = DagVersion.write_dag( + dag_id=dag.dag_id, + session=self.session, + version_name=dag.version_name, + ) + dag_code = DagCode(dagv, dag.fileloc, "Source") + self.session.merge(dag_code) + self.serialized_model.dag_version = dagv + if self.want_activate_assets: + self._activate_assets() + if sdm: + self.serialized_model = sdm + else: + self.session.merge(self.serialized_model) serialized_dag = self._serialized_dag() self._bag_dag_compat(serialized_dag) - if AIRFLOW_V_3_0_PLUS and self.want_activate_assets: - self._activate_assets() + self.session.flush() else: self._bag_dag_compat(self.dag) @@ -1007,16 +1026,30 @@ def cleanup(self): return # To isolate problems here with problems from elsewhere on the session object self.session.rollback() - - self.session.query(SerializedDagModel).filter( - SerializedDagModel.dag_id.in_(dag_ids) - ).delete(synchronize_session=False) - self.session.query(DagRun).filter(DagRun.dag_id.in_(dag_ids)).delete( - synchronize_session=False, - ) - self.session.query(TaskInstance).filter(TaskInstance.dag_id.in_(dag_ids)).delete( - synchronize_session=False, - ) + from tests_common.test_utils.compat import AIRFLOW_V_3_0_PLUS + + if AIRFLOW_V_3_0_PLUS: + from airflow.models.dag_version import DagVersion + + self.session.query(DagRun).filter(DagRun.dag_id.in_(dag_ids)).delete( + synchronize_session=False, + ) + self.session.query(TaskInstance).filter(TaskInstance.dag_id.in_(dag_ids)).delete( + synchronize_session=False, + ) + self.session.query(DagVersion).filter(DagVersion.dag_id.in_(dag_ids)).delete( + synchronize_session=False + ) + else: + self.session.query(SerializedDagModel).filter( + SerializedDagModel.dag_id.in_(dag_ids) + ).delete(synchronize_session=False) + self.session.query(DagRun).filter(DagRun.dag_id.in_(dag_ids)).delete( + synchronize_session=False, + ) + self.session.query(TaskInstance).filter(TaskInstance.dag_id.in_(dag_ids)).delete( + synchronize_session=False, + ) self.session.query(XCom).filter(XCom.dag_id.in_(dag_ids)).delete( synchronize_session=False, ) diff --git a/tests_common/test_utils/db.py b/tests_common/test_utils/db.py index d37a8e942e111..14f1007af9a8b 100644 --- a/tests_common/test_utils/db.py +++ b/tests_common/test_utils/db.py @@ -114,6 +114,7 @@ def clear_db_dags(): session.query(DagTag).delete() session.query(DagOwnerAttributes).delete() session.query(DagModel).delete() + session.query(DagCode).delete() def drop_tables_with_prefix(prefix):