You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by as...@apache.org on 2020/10/18 07:20:05 UTC

[airflow] branch master updated: Make DagRunType inherit from `str` too for easier use. (#11621)

This is an automated email from the ASF dual-hosted git repository.

ash pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/master by this push:
     new f507180  Make DagRunType inherit from `str` too for easier use. (#11621)
f507180 is described below

commit f507180b902d0ff5d5fca4af4c3a4ba6ca7db254
Author: Ash Berlin-Taylor <as...@firemirror.com>
AuthorDate: Sun Oct 18 08:18:48 2020 +0100

    Make DagRunType inherit from `str` too for easier use. (#11621)
    
    This approach is documented in https://docs.python.org/3.6/library/enum.html#others:
    
    ```
    While IntEnum is part of the enum module, it would be very simple to
    implement independently:
    
    class IntEnum(int, Enum):
        pass
    ```
    
    We just extend this to a str -- this means the SQLAlchemy has no trouble
    putting these in to queries, and `"scheduled" == DagRunType.SCHEDULED`
    is true.
    
    This change makes it simpler to use `dagrun.run_type`.
---
 airflow/api_connexion/endpoints/dag_run_endpoint.py    |  2 +-
 airflow/jobs/backfill_job.py                           |  4 ++--
 airflow/jobs/scheduler_job.py                          |  4 ++--
 airflow/models/dag.py                                  |  6 +++---
 airflow/models/dagrun.py                               | 10 +++++-----
 airflow/ti_deps/deps/dagrun_id_dep.py                  |  6 +++---
 airflow/utils/types.py                                 |  2 +-
 airflow/www/forms.py                                   |  2 +-
 tests/api/common/experimental/test_delete_dag.py       |  2 +-
 tests/api_connexion/endpoints/test_dag_run_endpoint.py | 18 +++++++++---------
 .../endpoints/test_extra_link_endpoint.py              |  2 +-
 tests/api_connexion/endpoints/test_log_endpoint.py     |  2 +-
 tests/api_connexion/endpoints/test_xcom_endpoint.py    |  6 +++---
 tests/models/test_dag.py                               |  4 ++--
 tests/models/test_dagrun.py                            |  8 ++++----
 tests/operators/test_python.py                         | 12 ++++++------
 tests/ti_deps/deps/test_dagrun_id_dep.py               |  2 +-
 tests/www/test_views.py                                |  8 ++++----
 18 files changed, 50 insertions(+), 50 deletions(-)

diff --git a/airflow/api_connexion/endpoints/dag_run_endpoint.py b/airflow/api_connexion/endpoints/dag_run_endpoint.py
index 81cc69d..daf12d6 100644
--- a/airflow/api_connexion/endpoints/dag_run_endpoint.py
+++ b/airflow/api_connexion/endpoints/dag_run_endpoint.py
@@ -233,7 +233,7 @@ def post_dag_run(dag_id, session):
         session.query(DagRun).filter(DagRun.dag_id == dag_id, DagRun.run_id == post_body["run_id"]).first()
     )
     if not dagrun_instance:
-        dag_run = DagRun(dag_id=dag_id, run_type=DagRunType.MANUAL.value, **post_body)
+        dag_run = DagRun(dag_id=dag_id, run_type=DagRunType.MANUAL, **post_body)
         session.add(dag_run)
         session.commit()
         return dagrun_schema.dump(dag_run)
diff --git a/airflow/jobs/backfill_job.py b/airflow/jobs/backfill_job.py
index de7c53b..ed51596 100644
--- a/airflow/jobs/backfill_job.py
+++ b/airflow/jobs/backfill_job.py
@@ -334,7 +334,7 @@ class BackfillJob(BaseJob):
         # explicitly mark as backfill and running
         run.state = State.RUNNING
         run.run_id = run.generate_run_id(DagRunType.BACKFILL_JOB, run_date)
-        run.run_type = DagRunType.BACKFILL_JOB.value
+        run.run_type = DagRunType.BACKFILL_JOB
         run.verify_integrity(session=session)
         return run
 
@@ -865,7 +865,7 @@ class BackfillJob(BaseJob):
                 .filter(
                     # pylint: disable=comparison-with-callable
                     DagRun.state == State.RUNNING,
-                    DagRun.run_type != DagRunType.BACKFILL_JOB.value,
+                    DagRun.run_type != DagRunType.BACKFILL_JOB,
                     TaskInstance.state.in_(resettable_states))).all()
         else:
             resettable_tis = filter_by_dag_run.get_task_instances(state=resettable_states,
diff --git a/airflow/jobs/scheduler_job.py b/airflow/jobs/scheduler_job.py
index ba472c4..ac0abf9 100644
--- a/airflow/jobs/scheduler_job.py
+++ b/airflow/jobs/scheduler_job.py
@@ -955,7 +955,7 @@ class SchedulerJob(BaseJob):  # pylint: disable=too-many-instance-attributes
             .query(TI)
             .outerjoin(TI.dag_run)
             .filter(or_(DR.run_id.is_(None),
-                        DR.run_type != DagRunType.BACKFILL_JOB.value))
+                        DR.run_type != DagRunType.BACKFILL_JOB))
             .join(TI.dag_model)
             .filter(not_(DM.is_paused))
             .filter(TI.state == State.SCHEDULED)
@@ -1790,7 +1790,7 @@ class SchedulerJob(BaseJob):  # pylint: disable=too-many-instance-attributes
             .outerjoin(TI.queued_by_job)
             .filter(or_(TI.queued_by_job_id.is_(None), SchedulerJob.state != State.RUNNING))
             .join(TI.dag_run)
-            .filter(DagRun.run_type != DagRunType.BACKFILL_JOB.value,
+            .filter(DagRun.run_type != DagRunType.BACKFILL_JOB,
                     # pylint: disable=comparison-with-callable
                     DagRun.state == State.RUNNING)
             .options(load_only(TI.dag_id, TI.task_id, TI.execution_date))
diff --git a/airflow/models/dag.py b/airflow/models/dag.py
index 1d928d2..45bd398 100644
--- a/airflow/models/dag.py
+++ b/airflow/models/dag.py
@@ -1704,7 +1704,7 @@ class DAG(BaseDag, LoggingMixin):
             external_trigger=external_trigger,
             conf=conf,
             state=state,
-            run_type=run_type.value,
+            run_type=run_type,
             dag_hash=dag_hash,
             creating_job_id=creating_job_id
         )
@@ -1773,8 +1773,8 @@ class DAG(BaseDag, LoggingMixin):
         most_recent_dag_runs = dict(session.query(DagRun.dag_id, func.max_(DagRun.execution_date)).filter(
             DagRun.dag_id.in_(existing_dag_ids),
             or_(
-                DagRun.run_type == DagRunType.BACKFILL_JOB.value,
-                DagRun.run_type == DagRunType.SCHEDULED.value,
+                DagRun.run_type == DagRunType.BACKFILL_JOB,
+                DagRun.run_type == DagRunType.SCHEDULED,
                 DagRun.external_trigger.is_(True),
             ),
         ).group_by(DagRun.dag_id).all())
diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py
index 07d83c5..2108f45 100644
--- a/airflow/models/dagrun.py
+++ b/airflow/models/dagrun.py
@@ -179,7 +179,7 @@ class DagRun(Base, LoggingMixin):
         # TODO: Bake this query, it is run _A lot_
         query = session.query(cls).filter(
             cls.state == State.RUNNING,
-            cls.run_type != DagRunType.BACKFILL_JOB.value
+            cls.run_type != DagRunType.BACKFILL_JOB
         ).join(
             DagModel,
             DagModel.dag_id == cls.dag_id,
@@ -259,9 +259,9 @@ class DagRun(Base, LoggingMixin):
         if external_trigger is not None:
             qry = qry.filter(DR.external_trigger == external_trigger)
         if run_type:
-            qry = qry.filter(DR.run_type == run_type.value)
+            qry = qry.filter(DR.run_type == run_type)
         if no_backfills:
-            qry = qry.filter(DR.run_type != DagRunType.BACKFILL_JOB.value)
+            qry = qry.filter(DR.run_type != DagRunType.BACKFILL_JOB)
 
         dr = qry.order_by(DR.execution_date).all()
 
@@ -270,7 +270,7 @@ class DagRun(Base, LoggingMixin):
     @staticmethod
     def generate_run_id(run_type: DagRunType, execution_date: datetime) -> str:
         """Generate Run ID based on Run Type and Execution Date"""
-        return f"{run_type.value}__{execution_date.isoformat()}"
+        return f"{run_type}__{execution_date.isoformat()}"
 
     @provide_session
     def get_task_instances(self, state=None, session=None):
@@ -617,7 +617,7 @@ class DagRun(Base, LoggingMixin):
 
     @property
     def is_backfill(self):
-        return self.run_type == DagRunType.BACKFILL_JOB.value
+        return self.run_type == DagRunType.BACKFILL_JOB
 
     @classmethod
     @provide_session
diff --git a/airflow/ti_deps/deps/dagrun_id_dep.py b/airflow/ti_deps/deps/dagrun_id_dep.py
index 8e40c49..3549fa9 100644
--- a/airflow/ti_deps/deps/dagrun_id_dep.py
+++ b/airflow/ti_deps/deps/dagrun_id_dep.py
@@ -46,11 +46,11 @@ class DagrunIdDep(BaseTIDep):
         """
         dagrun = ti.get_dagrun(session)
 
-        if not dagrun or not dagrun.run_id or dagrun.run_type != DagRunType.BACKFILL_JOB.value:
+        if not dagrun or not dagrun.run_id or dagrun.run_type != DagRunType.BACKFILL_JOB:
             yield self._passing_status(
                 reason=f"Task's DagRun doesn't exist or run_id is either NULL "
-                       f"or run_type is not {DagRunType.BACKFILL_JOB.value}")
+                       f"or run_type is not {DagRunType.BACKFILL_JOB}")
         else:
             yield self._failing_status(
                 reason=f"Task's DagRun run_id is not NULL "
-                       f"and run type is {DagRunType.BACKFILL_JOB.value}")
+                       f"and run type is {DagRunType.BACKFILL_JOB}")
diff --git a/airflow/utils/types.py b/airflow/utils/types.py
index 1c46a75..9dfee00 100644
--- a/airflow/utils/types.py
+++ b/airflow/utils/types.py
@@ -17,7 +17,7 @@
 import enum
 
 
-class DagRunType(enum.Enum):
+class DagRunType(str, enum.Enum):
     """Class with DagRun types"""
 
     BACKFILL_JOB = "backfill"
diff --git a/airflow/www/forms.py b/airflow/www/forms.py
index 9b6c31d..20b35a3 100644
--- a/airflow/www/forms.py
+++ b/airflow/www/forms.py
@@ -153,7 +153,7 @@ class DagRunForm(DynamicForm):
     def populate_obj(self, item):
         """Populates the attributes of the passed obj with data from the form’s fields."""
         super().populate_obj(item)  # pylint: disable=no-member
-        item.run_type = DagRunType.from_run_id(item.run_id).value
+        item.run_type = DagRunType.from_run_id(item.run_id)
         if item.conf:
             item.conf = json.loads(item.conf)
 
diff --git a/tests/api/common/experimental/test_delete_dag.py b/tests/api/common/experimental/test_delete_dag.py
index 762e0d1..0005871 100644
--- a/tests/api/common/experimental/test_delete_dag.py
+++ b/tests/api/common/experimental/test_delete_dag.py
@@ -67,7 +67,7 @@ class TestDeleteDAGSuccessfulDelete(unittest.TestCase):
         test_date = days_ago(1)
         with create_session() as session:
             session.add(DM(dag_id=self.key, fileloc=self.dag_file_path, is_subdag=for_sub_dag))
-            session.add(DR(dag_id=self.key, run_type=DagRunType.MANUAL.value))
+            session.add(DR(dag_id=self.key, run_type=DagRunType.MANUAL))
             session.add(TI(task=task,
                            execution_date=test_date,
                            state=State.SUCCESS))
diff --git a/tests/api_connexion/endpoints/test_dag_run_endpoint.py b/tests/api_connexion/endpoints/test_dag_run_endpoint.py
index f00060b..0ad0c63 100644
--- a/tests/api_connexion/endpoints/test_dag_run_endpoint.py
+++ b/tests/api_connexion/endpoints/test_dag_run_endpoint.py
@@ -85,7 +85,7 @@ class TestDagRunEndpoint(unittest.TestCase):
         dagrun_model_1 = DagRun(
             dag_id="TEST_DAG_ID",
             run_id="TEST_DAG_RUN_ID_1",
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
             execution_date=timezone.parse(self.default_time),
             start_date=timezone.parse(self.default_time),
             external_trigger=True,
@@ -95,7 +95,7 @@ class TestDagRunEndpoint(unittest.TestCase):
         dagrun_model_2 = DagRun(
             dag_id="TEST_DAG_ID",
             run_id="TEST_DAG_RUN_ID_2",
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
             execution_date=timezone.parse(self.default_time_2),
             start_date=timezone.parse(self.default_time),
             external_trigger=True,
@@ -109,7 +109,7 @@ class TestDagRunEndpoint(unittest.TestCase):
                     DagRun(
                         dag_id='TEST_DAG_ID_' + str(i),
                         run_id='TEST_DAG_RUN_ID_' + str(i),
-                        run_type=DagRunType.MANUAL.value,
+                        run_type=DagRunType.MANUAL,
                         execution_date=timezone.parse(self.default_time_2),
                         start_date=timezone.parse(self.default_time),
                         external_trigger=True,
@@ -177,7 +177,7 @@ class TestGetDagRun(TestDagRunEndpoint):
         dagrun_model = DagRun(
             dag_id="TEST_DAG_ID",
             run_id="TEST_DAG_RUN_ID",
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
             execution_date=timezone.parse(self.default_time),
             start_date=timezone.parse(self.default_time),
             external_trigger=True,
@@ -220,7 +220,7 @@ class TestGetDagRun(TestDagRunEndpoint):
         dagrun_model = DagRun(
             dag_id="TEST_DAG_ID",
             run_id="TEST_DAG_RUN_ID",
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
             execution_date=timezone.parse(self.default_time),
             start_date=timezone.parse(self.default_time),
             external_trigger=True,
@@ -369,7 +369,7 @@ class TestGetDagRunsPagination(TestDagRunEndpoint):
             DagRun(
                 dag_id="TEST_DAG_ID",
                 run_id="TEST_DAG_RUN_ID" + str(i),
-                run_type=DagRunType.MANUAL.value,
+                run_type=DagRunType.MANUAL,
                 execution_date=timezone.parse(self.default_time) + timedelta(minutes=i),
                 start_date=timezone.parse(self.default_time),
                 external_trigger=True,
@@ -453,7 +453,7 @@ class TestGetDagRunsPaginationFilters(TestDagRunEndpoint):
             DagRun(
                 dag_id="TEST_DAG_ID",
                 run_id="TEST_START_EXEC_DAY_1" + str(i),
-                run_type=DagRunType.MANUAL.value,
+                run_type=DagRunType.MANUAL,
                 execution_date=timezone.parse(dates[i]),
                 start_date=timezone.parse(dates[i]),
                 external_trigger=True,
@@ -647,7 +647,7 @@ class TestGetDagRunBatchPagination(TestDagRunEndpoint):
             DagRun(
                 dag_id="TEST_DAG_ID",
                 run_id="TEST_DAG_RUN_ID" + str(i),
-                run_type=DagRunType.MANUAL.value,
+                run_type=DagRunType.MANUAL,
                 execution_date=timezone.parse(self.default_time) + timedelta(minutes=i),
                 start_date=timezone.parse(self.default_time),
                 external_trigger=True,
@@ -729,7 +729,7 @@ class TestGetDagRunBatchDateFilters(TestDagRunEndpoint):
             DagRun(
                 dag_id="TEST_DAG_ID",
                 run_id="TEST_START_EXEC_DAY_1" + str(i),
-                run_type=DagRunType.MANUAL.value,
+                run_type=DagRunType.MANUAL,
                 execution_date=timezone.parse(dates[i]),
                 start_date=timezone.parse(dates[i]),
                 external_trigger=True,
diff --git a/tests/api_connexion/endpoints/test_extra_link_endpoint.py b/tests/api_connexion/endpoints/test_extra_link_endpoint.py
index 3cb0506..032e5d8 100644
--- a/tests/api_connexion/endpoints/test_extra_link_endpoint.py
+++ b/tests/api_connexion/endpoints/test_extra_link_endpoint.py
@@ -80,7 +80,7 @@ class TestGetExtraLinks(unittest.TestCase):
             dag_id=self.dag.dag_id,
             run_id="TEST_DAG_RUN_ID",
             execution_date=self.default_time,
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
         )
         session.add(dr)
         session.commit()
diff --git a/tests/api_connexion/endpoints/test_log_endpoint.py b/tests/api_connexion/endpoints/test_log_endpoint.py
index 2a38862..1cab4fb 100644
--- a/tests/api_connexion/endpoints/test_log_endpoint.py
+++ b/tests/api_connexion/endpoints/test_log_endpoint.py
@@ -84,7 +84,7 @@ class TestGetLog(unittest.TestCase):
         dagrun_model = DagRun(
             dag_id=self.DAG_ID,
             run_id='TEST_DAG_RUN_ID',
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
             execution_date=timezone.parse(self.default_time),
             start_date=timezone.parse(self.default_time),
             external_trigger=True,
diff --git a/tests/api_connexion/endpoints/test_xcom_endpoint.py b/tests/api_connexion/endpoints/test_xcom_endpoint.py
index 517286f..1993cfc 100644
--- a/tests/api_connexion/endpoints/test_xcom_endpoint.py
+++ b/tests/api_connexion/endpoints/test_xcom_endpoint.py
@@ -161,7 +161,7 @@ class TestGetXComEntry(TestXComEndpoint):
             run_id=dag_run_id,
             execution_date=execution_date,
             start_date=execution_date,
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
         )
         session.add(dagrun)
 
@@ -344,7 +344,7 @@ class TestGetXComEntries(TestXComEndpoint):
             run_id=dag_run_id,
             execution_date=execution_date,
             start_date=execution_date,
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
         )
         session.add(dagrun)
 
@@ -418,7 +418,7 @@ class TestPaginationGetXComEntries(TestXComEndpoint):
             run_id=self.dag_run_id,
             execution_date=self.execution_date_parsed,
             start_date=self.execution_date_parsed,
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
         )
         xcom_models = self._create_xcoms(10)
         session.add_all(xcom_models)
diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py
index 16ee3e0..cb484f0 100644
--- a/tests/models/test_dag.py
+++ b/tests/models/test_dag.py
@@ -1339,11 +1339,11 @@ class TestDag(unittest.TestCase):
     def test_create_dagrun_run_id_is_generated(self):
         dag = DAG(dag_id="run_id_is_generated")
         dr = dag.create_dagrun(run_type=DagRunType.MANUAL, execution_date=DEFAULT_DATE, state=State.NONE)
-        assert dr.run_id == f"{DagRunType.MANUAL.value}__{DEFAULT_DATE.isoformat()}"
+        assert dr.run_id == f"manual__{DEFAULT_DATE.isoformat()}"
 
     def test_create_dagrun_run_type_is_obtained_from_run_id(self):
         dag = DAG(dag_id="run_type_is_obtained_from_run_id")
-        dr = dag.create_dagrun(run_id=f"{DagRunType.SCHEDULED.value}__", state=State.NONE)
+        dr = dag.create_dagrun(run_id="scheduled__", state=State.NONE)
         assert dr.run_type == DagRunType.SCHEDULED.value
 
         dr = dag.create_dagrun(run_id="custom_is_set_to_manual", state=State.NONE)
diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py
index 4d1dce8..86413ef 100644
--- a/tests/models/test_dagrun.py
+++ b/tests/models/test_dagrun.py
@@ -106,7 +106,7 @@ class TestDagRun(unittest.TestCase):
         dag_id1 = "test_dagrun_find_externally_triggered"
         dag_run = models.DagRun(
             dag_id=dag_id1,
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
             execution_date=now,
             start_date=now,
             state=State.RUNNING,
@@ -117,7 +117,7 @@ class TestDagRun(unittest.TestCase):
         dag_id2 = "test_dagrun_find_not_externally_triggered"
         dag_run = models.DagRun(
             dag_id=dag_id2,
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
             execution_date=now,
             start_date=now,
             state=State.RUNNING,
@@ -583,7 +583,7 @@ class TestDagRun(unittest.TestCase):
         # don't want
         dag_run = models.DagRun(
             dag_id=dag.dag_id,
-            run_type=DagRunType.MANUAL.value,
+            run_type=DagRunType.MANUAL,
             execution_date=now,
             start_date=now,
             state=State.RUNNING,
@@ -612,7 +612,7 @@ class TestDagRun(unittest.TestCase):
         dag = DAG(dag_id='test_is_backfill', start_date=DEFAULT_DATE)
 
         dagrun = self.create_dag_run(dag, execution_date=DEFAULT_DATE)
-        dagrun.run_type = DagRunType.BACKFILL_JOB.value
+        dagrun.run_type = DagRunType.BACKFILL_JOB
 
         dagrun2 = self.create_dag_run(
             dag, execution_date=DEFAULT_DATE + datetime.timedelta(days=1))
diff --git a/tests/operators/test_python.py b/tests/operators/test_python.py
index 7674361..ac3b82d 100644
--- a/tests/operators/test_python.py
+++ b/tests/operators/test_python.py
@@ -386,7 +386,7 @@ class TestAirflowTaskDecorator(TestPythonBase):
         with self.dag:
             ret = add_number(2)
         self.dag.create_dagrun(
-            run_id=DagRunType.MANUAL.value,
+            run_id=DagRunType.MANUAL,
             execution_date=DEFAULT_DATE,
             start_date=DEFAULT_DATE,
             state=State.RUNNING
@@ -404,7 +404,7 @@ class TestAirflowTaskDecorator(TestPythonBase):
         with self.dag:
             ret = add_number(2)
         self.dag.create_dagrun(
-            run_id=DagRunType.MANUAL.value,
+            run_id=DagRunType.MANUAL,
             execution_date=DEFAULT_DATE,
             start_date=DEFAULT_DATE,
             state=State.RUNNING
@@ -431,7 +431,7 @@ class TestAirflowTaskDecorator(TestPythonBase):
         ret = task(4, date(2019, 1, 1), "dag {{dag.dag_id}} ran on {{ds}}.", named_tuple)
 
         self.dag.create_dagrun(
-            run_id=DagRunType.MANUAL.value,
+            run_id=DagRunType.MANUAL,
             execution_date=DEFAULT_DATE,
             start_date=DEFAULT_DATE,
             state=State.RUNNING
@@ -460,7 +460,7 @@ class TestAirflowTaskDecorator(TestPythonBase):
         )
         ret = task(an_int=4, a_date=date(2019, 1, 1), a_templated_string="dag {{dag.dag_id}} ran on {{ds}}.")
         self.dag.create_dagrun(
-            run_id=DagRunType.MANUAL.value,
+            run_id=DagRunType.MANUAL,
             execution_date=DEFAULT_DATE,
             start_date=DEFAULT_DATE,
             state=State.RUNNING
@@ -533,7 +533,7 @@ class TestAirflowTaskDecorator(TestPythonBase):
             ret = return_dict(test_number)
 
         dr = self.dag.create_dagrun(
-            run_id=DagRunType.MANUAL.value,
+            run_id=DagRunType.MANUAL,
             start_date=timezone.utcnow(),
             execution_date=DEFAULT_DATE,
             state=State.RUNNING
@@ -575,7 +575,7 @@ class TestAirflowTaskDecorator(TestPythonBase):
             ret = add_num(bigger_number, XComArg(bigger_number.operator))  # pylint: disable=maybe-no-member
 
         dr = self.dag.create_dagrun(
-            run_id=DagRunType.MANUAL.value,
+            run_id=DagRunType.MANUAL,
             start_date=timezone.utcnow(),
             execution_date=DEFAULT_DATE,
             state=State.RUNNING
diff --git a/tests/ti_deps/deps/test_dagrun_id_dep.py b/tests/ti_deps/deps/test_dagrun_id_dep.py
index 2a0a7e4..3686603 100644
--- a/tests/ti_deps/deps/test_dagrun_id_dep.py
+++ b/tests/ti_deps/deps/test_dagrun_id_dep.py
@@ -32,7 +32,7 @@ class TestDagrunRunningDep(unittest.TestCase):
         """
         dagrun = DagRun()
         dagrun.run_id = "anything"
-        dagrun.run_type = DagRunType.BACKFILL_JOB.value
+        dagrun.run_type = DagRunType.BACKFILL_JOB
         ti = Mock(get_dagrun=Mock(return_value=dagrun))
         self.assertFalse(DagrunIdDep().is_met(ti=ti))
 
diff --git a/tests/www/test_views.py b/tests/www/test_views.py
index cf9fef2..6ea7310 100644
--- a/tests/www/test_views.py
+++ b/tests/www/test_views.py
@@ -2409,8 +2409,8 @@ class TestTriggerDag(TestBase):
 
         run = self.session.query(DR).filter(DR.dag_id == test_dag_id).first()
         self.assertIsNotNone(run)
-        self.assertIn(DagRunType.MANUAL.value, run.run_id)
-        self.assertEqual(run.run_type, DagRunType.MANUAL.value)
+        self.assertIn(DagRunType.MANUAL, run.run_id)
+        self.assertEqual(run.run_type, DagRunType.MANUAL)
 
     @pytest.mark.quarantined
     def test_trigger_dag_conf(self):
@@ -2426,8 +2426,8 @@ class TestTriggerDag(TestBase):
 
         run = self.session.query(DR).filter(DR.dag_id == test_dag_id).first()
         self.assertIsNotNone(run)
-        self.assertIn(DagRunType.MANUAL.value, run.run_id)
-        self.assertEqual(run.run_type, DagRunType.MANUAL.value)
+        self.assertIn(DagRunType.MANUAL, run.run_id)
+        self.assertEqual(run.run_type, DagRunType.MANUAL)
         self.assertEqual(run.conf, conf_dict)
 
     def test_trigger_dag_conf_malformed(self):