ephraimbuddy commented on a change in pull request #16352: URL: https://github.com/apache/airflow/pull/16352#discussion_r663722499
########## File path: airflow/migrations/versions/142555e44c17_add_data_interval_start_end_to_dagrun.py ########## @@ -0,0 +1,155 @@ +# +# 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 data_interval_[start|end] to DagRun. + +Revision ID: 142555e44c17 +Revises: e9304a3141f0 +Create Date: 2021-06-09 08:28:02.089817 + +""" + +import json + +from alembic import op +from sqlalchemy import Column, Integer, String, or_ +from sqlalchemy.ext.declarative import declarative_base +from sqlalchemy.orm import foreign, relationship +from sqlalchemy_jsonfield import JSONField + +from airflow.serialization.serialized_objects import SerializedDAG +from airflow.utils.session import create_session +from airflow.utils.sqlalchemy import Interval, UtcDateTime +from airflow.utils.types import DagRunType + +# Revision identifiers, used by Alembic. +revision = "142555e44c17" +down_revision = "30867afad44a" +branch_labels = None +depends_on = None + +ID_LEN = 250 + +Base = declarative_base() + + +class DagModel(Base): + """A partially frozen ``airflow.models.DagModel`` class.""" + + __tablename__ = "dag" + + dag_id = Column(String(ID_LEN), primary_key=True) + schedule_interval = Column(Interval) + + +class DagRun(Base): + """A partially frozen ``airflow.models.DagRun`` class.""" + + __tablename__ = "dag_run" + + id = Column(Integer, primary_key=True) + dag_id = Column(String(ID_LEN)) + execution_date = Column(UtcDateTime) + data_interval_start = Column(UtcDateTime) + data_interval_end = Column(UtcDateTime) + run_id = Column(String(ID_LEN)) + run_type = Column(String(50)) + + dag = relationship(DagModel, primaryjoin=(foreign(DagModel.dag_id) == dag_id)) + + +class SerializedDagModel(Base): + """A partially frozen ``airflow.models.SerializedDagModel`` class.""" + + __tablename__ = "serialized_dag" + + dag_id = Column(String(ID_LEN), primary_key=True) + data = Column(JSONField(json=json)) + + @property + def dag(self): + """Copied from the original model class.""" + SerializedDAG._load_operator_extra_links = self.load_op_links # pylint: disable=protected-access + if isinstance(self.data, dict): + return SerializedDAG.from_dict(self.data) + return SerializedDAG.from_json(self.data) + + +# These kinds of runs don't have a data interval and can be bulk-updated with +# one SQL call, so we do them separately. Other "real" schedule intervals are +# too complicated and need to be populated manually. +NO_SCHEDULE_FILTER = or_( + DagModel.schedule_interval.in_([None, "@once"]), + DagRun.run_type == DagRunType.MANUAL, +) + + +def _populate_simple_dagrun_intervals(session): + """Handle DAG runs with simple schedule intervals.""" + updates = { + DagRun.data_interval_start: DagRun.execution_date, + DagRun.data_interval_end: DagRun.execution_date, + } + # SQLite doesn't support UPDATE ... WHERE with multiple tables, and MySQL + # doesn't support subqeury in UPDATE, so... + if op.get_bind().dialect.name != 'sqlite': + dag_runs = session.query(DagRun).filter(DagRun.dag, NO_SCHEDULE_FILTER) + else: + dag_run_ids = session.query(DagRun.id).filter(NO_SCHEDULE_FILTER) + dag_runs = session.query(DagRun).filter(DagRun.id.in_(dag_run_ids.subquery())) + dag_runs.update(updates, synchronize_session=False) + + +def _populate_complex_dagrun_intervals(session): + """Handle DAG runs with "real" schedule intervals.""" + dag_runs_with_serialized = session.query(DagRun, SerializedDagModel).filter( + DagModel.dag_id == DagRun.dag_id, + SerializedDagModel.dag_id == DagRun.dag_id, + ~NO_SCHEDULE_FILTER, + ) + for dag_run, serialized in dag_runs_with_serialized: + dag = serialized.dag + data_interval_start = dag_run.execution_date + dag_run.data_interval_start = data_interval_start + dag_run.data_interval_end = dag.following_schedule(data_interval_start) + session.merge(dag_run, load=False) + + +def upgrade(): + """Apply add data_interval_[start|end] to DagRun.""" + # Create aolumns with NULL as default. + with op.batch_alter_table("dag_run") as batch_op: + batch_op.add_column(Column("data_interval_start", UtcDateTime)) Review comment: MSSQL needs `mssql.DATETIME2(precision=6)` for the column -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
