uranusjr commented on a change in pull request #18724:
URL: https://github.com/apache/airflow/pull/18724#discussion_r765347284



##########
File path: airflow/models/dag.py
##########
@@ -179,6 +179,20 @@ def get_last_dagrun(dag_id, session, 
include_externally_triggered=False):
     return query.first()
 
 
+def get_first_dagrun(dag_id, session, include_externally_triggered=False):
+    """
+    Returns the first dag run for a dag, None if there was none.
+    First dag run can be any type of run eg. scheduled or backfilled.
+    Overridden DagRuns are ignored.
+    """
+    DR = DagRun
+    query = session.query(DR).filter(DR.dag_id == dag_id)
+    if not include_externally_triggered:
+        query = query.filter(DR.external_trigger == expression.false())
+    query = query.order_by(DR.execution_date.asc())
+    return query.first()

Review comment:
       Do we _need_ this as a separate function? (I don’t like 
`get_last_dagrun` to begin with and would prefer we not adding more things like 
it unless needed.)

##########
File path: airflow/models/dag.py
##########
@@ -1630,7 +1650,7 @@ def _get_task_instances(
     def set_task_instance_state(
         self,
         task_id: str,
-        execution_date: datetime,
+        dag_run_id: str,

Review comment:
       I think we might need to keep compatibility here.

##########
File path: airflow/models/dagrun.py
##########
@@ -336,7 +336,10 @@ def find(
         if dag_ids:
             qry = qry.filter(cls.dag_id.in_(dag_ids))
         if run_id:
-            qry = qry.filter(cls.run_id == run_id)
+            if isinstance(run_id, list):
+                qry = qry.filter(cls.run_id.in_(run_id))
+            else:
+                qry = qry.filter(cls.run_id == run_id)

Review comment:
       ```python
   if is_container(run_id):
       ...
   elif run_id is not None:
       ...
   ```
   
   and we can relax the type of `run_id` to `Iterable`. (And yes the same 
applies to `execution_date` but that’s existing code.)




-- 
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]


Reply via email to