You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2021/09/14 07:16:12 UTC

[GitHub] [airflow] bhavaniravi opened a new pull request #18226: Add start date to trigger_dagrun operator

bhavaniravi opened a new pull request #18226:
URL: https://github.com/apache/airflow/pull/18226


   closes: #18082  
   
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/main/UPDATING.md).
   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#issuecomment-919887197


   Looks like something went wrong in your last rebase. Please rebase again


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] alexInhert commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
alexInhert commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r709121803



##########
File path: airflow/api/common/experimental/trigger_dag.py
##########
@@ -82,6 +82,7 @@ def _trigger_dag(
     for _dag in dags_to_trigger:
         trigger = _dag.create_dagrun(
             run_id=run_id,
+            start_date=timezone.utcnow(),

Review comment:
       If the dag you trigger is timezone aware does it matter?
   I mean you consider here the default timezone but the dag you are triggering may be in a diffrent timezone than the default




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] bhavaniravi commented on pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
bhavaniravi commented on pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#issuecomment-919774252


   Hey, @kaxil, I cannot recreate the failing tests using breeze env or pytest with the same python and Postgres versions. Is there an alternative way to simulate the CI builds?


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
uranusjr commented on pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#issuecomment-924573995


   Failure seems unrelated
   
   ```
      _____________________ test_external_task_marker_exception ______________________
     
     dag_bag_ext = <airflow.models.dagbag.DagBag object at 0x7fec5c30c730>
     
         def test_external_task_marker_exception(dag_bag_ext):
             """
             Clearing across multiple DAGs should raise AirflowException if more levels are being cleared
             than allowed by the recursion_depth of the first ExternalTaskMarker being cleared.
             """
     >       run_tasks(dag_bag_ext)
     ...
     >       assert task_instance.state == state
     E       AssertionError: assert None == <TaskInstanceState.SUCCESS: 'success'>
     E        +  where None = <TaskInstance: dag_0.task_b_0 manual__2015-01-01T00:00:00+00:00 [None]>.state
   ```


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#issuecomment-918971069


   > I think that you made the change in the wrong place. Here's where I think the change should be:
   > https://github.com/apache/airflow/blob/23a68fa4e61ba795340f095ab64fbcaeebd57db4/airflow/operators/trigger_dagrun.py#L132
   > 
   > 
   > Add `start_date=self.start_date` after the line above.
   
   But it seems the trigger_dag doesn't take start_date argument


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r711271422



##########
File path: tests/operators/test_trigger_dagrun.py
##########
@@ -255,3 +255,34 @@ def test_trigger_dagrun_with_wait_for_completion_true_fail(self):
         )
         with pytest.raises(AirflowException):
             task.run(start_date=execution_date, end_date=execution_date)
+
+    def test_trigger_dagrun_triggering_itself(self):
+        """Test TriggerDagRunOperator that triggers itself"""
+        execution_date = DEFAULT_DATE
+        task = TriggerDagRunOperator(
+            task_id="test_task",
+            trigger_dag_id=self.dag.dag_id,
+            allowed_states=[State.RUNNING, State.SUCCESS],
+            dag=self.dag,
+        )
+        task.run(start_date=execution_date, end_date=execution_date)
+
+        with create_session() as session:
+            dagruns = session.query(DagRun).filter(DagRun.dag_id == self.dag.dag_id).all()
+            assert len(dagruns) == 2
+            dagruns[1].set_state()
+            # assert isinstance(dagruns[1].start_date, datetime)

Review comment:
       You can just assert that state of 2nd DagRun -- that it is marked as `State.QUEUED`.
   




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r710014300



##########
File path: airflow/api/common/experimental/trigger_dag.py
##########
@@ -82,6 +82,7 @@ def _trigger_dag(
     for _dag in dags_to_trigger:
         trigger = _dag.create_dagrun(
             run_id=run_id,
+            start_date=timezone.utcnow(),

Review comment:
       We should only change the one identified here. Looks like others are more involved change and requires a separate PR(that's if it should be changed)




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r709804091



##########
File path: airflow/api/common/experimental/trigger_dag.py
##########
@@ -82,6 +82,7 @@ def _trigger_dag(
     for _dag in dags_to_trigger:
         trigger = _dag.create_dagrun(
             run_id=run_id,
+            start_date=timezone.utcnow(),

Review comment:
       You’re correct. Although maybe we shouldn’t set this to `RUNNING` in the first place, but should set it to `QUEUED` and let the scheduler start the run (and set `start_date`) instead. See how this is implemented in the new REST API (in `airflow/api_connexion/endpoints/dag_run_endpoint.py::post_dag_run`).




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] uranusjr commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
uranusjr commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r713582288



##########
File path: tests/operators/test_trigger_dagrun.py
##########
@@ -250,8 +250,36 @@ def test_trigger_dagrun_with_wait_for_completion_true_fail(self):
             execution_date=execution_date,
             wait_for_completion=True,
             poke_interval=10,
-            failed_states=[State.RUNNING],
+            failed_states=[State.QUEUED],
             dag=self.dag,
         )
         with pytest.raises(AirflowException):
             task.run(start_date=execution_date, end_date=execution_date)
+
+    def test_trigger_dagrun_triggering_itself(self):
+        """Test TriggerDagRunOperator that triggers itself"""
+        execution_date = DEFAULT_DATE
+        task = TriggerDagRunOperator(
+            task_id="test_task",
+            trigger_dag_id=self.dag.dag_id,
+            dag=self.dag,
+        )
+        task.run(start_date=execution_date, end_date=execution_date)
+
+        with create_session() as session:
+            dagruns = session.query(DagRun).filter(DagRun.dag_id == self.dag.dag_id).all()

Review comment:
       You probably want to add an explicit `order_by` here, the db does not always return rows in a deterministic order.




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#issuecomment-919621292


   ```
   =========================== short test summary info ============================
     FAILED tests/models/test_cleartasks.py::TestClearTasks::test_clear_task_instances
     FAILED tests/models/test_cleartasks.py::TestClearTasks::test_clear_task_instances_with_task_reschedule
     FAILED tests/models/test_cleartasks.py::TestClearTasks::test_dag_clear - asse...
     = 3 failed, 2445 passed, 61 skipped, 3 xfailed, 4 warnings in 508.01s (0:08:28) =
   ```
   
   tests are failing


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
kaxil commented on pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#issuecomment-926494616


   Thanks @bhavaniravi 🎉 


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
ephraimbuddy commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r709906016



##########
File path: airflow/api/common/experimental/trigger_dag.py
##########
@@ -82,6 +82,7 @@ def _trigger_dag(
     for _dag in dags_to_trigger:
         trigger = _dag.create_dagrun(
             run_id=run_id,
+            start_date=timezone.utcnow(),

Review comment:
       Supported. Setting it to queued will save us some complaints especially about max_active_runs




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] github-actions[bot] commented on pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#issuecomment-918979206


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r708553715



##########
File path: tests/operators/test_trigger_dagrun.py
##########
@@ -255,3 +255,33 @@ def test_trigger_dagrun_with_wait_for_completion_true_fail(self):
         )
         with pytest.raises(AirflowException):
             task.run(start_date=execution_date, end_date=execution_date)
+
+    def test_trigger_dagrun_triggering_itself(self):
+        """Test TriggerDagRunOperator that triggers itself"""
+        execution_date = DEFAULT_DATE
+        task = TriggerDagRunOperator(
+            task_id="test_task",
+            trigger_dag_id=self.dag.dag_id,
+            allowed_states=[State.RUNNING, State.SUCCESS],
+            dag=self.dag,
+        )
+        task.run(start_date=execution_date, end_date=execution_date)
+
+        with create_session() as session:
+            dagruns = session.query(DagRun).filter(DagRun.dag_id == self.dag.dag_id).all()
+            assert len(dagruns) == 2
+            assert isinstance(dagruns[1].start_date, datetime)
+
+    def test_trigger_dagrun_triggering_itself_with_execution_date(self):
+        """Test TriggerDagRunOperator that triggers itself with execution date, fails with DagRunAlreadyExists"""
+        execution_date = DEFAULT_DATE
+        task = TriggerDagRunOperator(
+            task_id="test_task",
+            trigger_dag_id=self.dag.dag_id,
+            execution_date=execution_date,
+            allowed_states=[State.RUNNING, State.SUCCESS],
+            dag=self.dag,
+        )
+        with pytest.raises(DagRunAlreadyExists):
+            task.run(start_date=execution_date, end_date=execution_date)
+

Review comment:
       ```suggestion
   ```




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r708553250



##########
File path: tests/operators/test_trigger_dagrun.py
##########
@@ -255,3 +255,33 @@ def test_trigger_dagrun_with_wait_for_completion_true_fail(self):
         )
         with pytest.raises(AirflowException):
             task.run(start_date=execution_date, end_date=execution_date)
+

Review comment:
       ```suggestion
   ```




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil merged pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
kaxil merged pull request #18226:
URL: https://github.com/apache/airflow/pull/18226


   


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] bhavaniravi commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
bhavaniravi commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r710122853



##########
File path: airflow/api/common/experimental/trigger_dag.py
##########
@@ -82,6 +82,7 @@ def _trigger_dag(
     for _dag in dags_to_trigger:
         trigger = _dag.create_dagrun(
             run_id=run_id,
+            start_date=timezone.utcnow(),

Review comment:
       This change will affect three places. All these use the `trigger_dag` function. 
   
   1. TriggerDAGRunOperator
   2. APIClient - `dag_runs`
   3. CLI trigger dag
   
   Is that okay?




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] ephraimbuddy removed a comment on pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
ephraimbuddy removed a comment on pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#issuecomment-918971069


   > I think that you made the change in the wrong place. Here's where I think the change should be:
   > https://github.com/apache/airflow/blob/23a68fa4e61ba795340f095ab64fbcaeebd57db4/airflow/operators/trigger_dagrun.py#L132
   > 
   > 
   > Add `start_date=self.start_date` after the line above.
   
   But it seems the trigger_dag doesn't take start_date argument


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r708160130



##########
File path: tests/operators/test_trigger_dagrun.py
##########
@@ -255,3 +255,34 @@ def test_trigger_dagrun_with_wait_for_completion_true_fail(self):
         )
         with pytest.raises(AirflowException):
             task.run(start_date=execution_date, end_date=execution_date)
+
+    def test_trigger_dagrun_triggering_itself(self):
+        """Test TriggerDagRunOperator that triggers itself"""
+        execution_date = DEFAULT_DATE
+        task = TriggerDagRunOperator(
+            task_id="test_task",
+            trigger_dag_id=self.dag.dag_id,
+            allowed_states=[State.RUNNING, State.SUCCESS],
+            dag=self.dag,
+        )
+        task.run(start_date=execution_date, end_date=execution_date)
+
+        with create_session() as session:
+            dagruns = session.query(DagRun).filter(DagRun.dag_id == self.dag.dag_id).all()
+            assert len(dagruns) == 2
+            assert isinstance(dagruns[1].start_date, datetime)
+
+    def test_trigger_dagrun_triggering_itself_with_execution_date(self):
+        """Test TriggerDagRunOperator that triggers itself with execution date, fails with DagRunAlreadyExists"""
+        execution_date = DEFAULT_DATE
+        task = TriggerDagRunOperator(
+            task_id="test_task",
+            trigger_dag_id=self.dag.dag_id,
+            execution_date=execution_date,
+            allowed_states=[State.RUNNING, State.SUCCESS],
+            dag=self.dag,
+        )
+        with pytest.raises(DagRunAlreadyExists):
+            task.run(start_date=execution_date, end_date=execution_date)
+
+

Review comment:
       ```suggestion
   ```




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] bhavaniravi edited a comment on pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
bhavaniravi edited a comment on pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#issuecomment-919774252


   Hey, @kaxil, I cannot recreate the failing tests using breeze env or pytest with the same python and Postgres versions. Looks like a persistent issue across PRs, 
   
   https://github.com/apache/airflow/pull/18243/checks?check_run_id=3604451375
   Let me rebase and push again


-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] bhavaniravi commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
bhavaniravi commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r709794580



##########
File path: airflow/api/common/experimental/trigger_dag.py
##########
@@ -82,6 +82,7 @@ def _trigger_dag(
     for _dag in dags_to_trigger:
         trigger = _dag.create_dagrun(
             run_id=run_id,
+            start_date=timezone.utcnow(),

Review comment:
       The `start_date` here is not the DAG start_date. It is the dag run starting time of the run. Internally it's all tracked in UTC and rendered in the user's timezone on UI.
   
   Correct me if I am wrong.




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] bhavaniravi commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
bhavaniravi commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r710122853



##########
File path: airflow/api/common/experimental/trigger_dag.py
##########
@@ -82,6 +82,7 @@ def _trigger_dag(
     for _dag in dags_to_trigger:
         trigger = _dag.create_dagrun(
             run_id=run_id,
+            start_date=timezone.utcnow(),

Review comment:
       This change will affect three places. All these use the `trigger_dag` function. 
   
   1. TriggerDAGRunOperator
   2. APIClient - `dag_runs`
   3. CLI trigger dag




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] bhavaniravi commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
bhavaniravi commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r709980251



##########
File path: airflow/api/common/experimental/trigger_dag.py
##########
@@ -82,6 +82,7 @@ def _trigger_dag(
     for _dag in dags_to_trigger:
         trigger = _dag.create_dagrun(
             run_id=run_id,
+            start_date=timezone.utcnow(),

Review comment:
       Should I go ahead and make that change? There are 3 places where we are creating dag_runs with the state running, should this be the same behavior across the board?
   
   ![image](https://user-images.githubusercontent.com/10116000/133594527-9f975155-09a9-431f-a462-17f98236177c.png)
   
   




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [airflow] kaxil commented on a change in pull request #18226: Add start date to trigger_dagrun operator

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #18226:
URL: https://github.com/apache/airflow/pull/18226#discussion_r711269834



##########
File path: tests/operators/test_trigger_dagrun.py
##########
@@ -255,3 +255,34 @@ def test_trigger_dagrun_with_wait_for_completion_true_fail(self):
         )
         with pytest.raises(AirflowException):
             task.run(start_date=execution_date, end_date=execution_date)
+
+    def test_trigger_dagrun_triggering_itself(self):
+        """Test TriggerDagRunOperator that triggers itself"""
+        execution_date = DEFAULT_DATE
+        task = TriggerDagRunOperator(
+            task_id="test_task",
+            trigger_dag_id=self.dag.dag_id,
+            allowed_states=[State.RUNNING, State.SUCCESS],
+            dag=self.dag,
+        )
+        task.run(start_date=execution_date, end_date=execution_date)
+
+        with create_session() as session:
+            dagruns = session.query(DagRun).filter(DagRun.dag_id == self.dag.dag_id).all()
+            assert len(dagruns) == 2
+            dagruns[1].set_state()
+            # assert isinstance(dagruns[1].start_date, datetime)

Review comment:
       Does this need updating?




-- 
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: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org