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 2022/10/18 08:34:43 UTC

[GitHub] [airflow] tirkarthi opened a new pull request, #27106: Fix warning in airflow tasks test command regarding absence of data_interval

tirkarthi opened a new pull request, #27106:
URL: https://github.com/apache/airflow/pull/27106

   Fixes below warning in `airflow tasks test` command along with fixing test cases. Similar to PR https://github.com/apache/airflow/pull/20781 and https://github.com/apache/airflow/issues/20579
   
   ```
   /home/karthikeyan/stuff/python/airflow/airflow/cli/commands/task_command.py:129 RemovedInAirflow3Warning: Calling `DAG.create_dagrun()` without an explicit data interval is deprecated
   ```
   
   cc: @uranusjr 


-- 
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 diff in pull request #27106: Fix warning in airflow tasks test command regarding absence of data_interval

Posted by GitBox <gi...@apache.org>.
uranusjr commented on code in PR #27106:
URL: https://github.com/apache/airflow/pull/27106#discussion_r997965953


##########
airflow/cli/commands/task_command.py:
##########
@@ -126,10 +126,12 @@ def _get_dag_run(
         dag_run = DagRun(dag.dag_id, run_id=exec_date_or_run_id, execution_date=dag_run_execution_date)
         return dag_run, True
     elif create_if_necessary == "db":
+        data_interval = dag.infer_automated_data_interval(logical_date=dag_run_execution_date)

Review Comment:
   This would not always work since infer_automated_data_interval` is only designed to provide backward compatibility for DAG runs that are created before the data interval concept was introduced, and a DAG using a custom timetable would not work correctly with this method. I think `infer_manual_data_interval` is probably a better choice; it is not technically correct (the call below created a scheduled run), but the exact value of the data interval does not actually matter much here anyway since the run is only supposed to be a stub.



##########
airflow/cli/commands/task_command.py:
##########
@@ -126,10 +126,12 @@ def _get_dag_run(
         dag_run = DagRun(dag.dag_id, run_id=exec_date_or_run_id, execution_date=dag_run_execution_date)
         return dag_run, True
     elif create_if_necessary == "db":
+        data_interval = dag.infer_automated_data_interval(logical_date=dag_run_execution_date)

Review Comment:
   This would not always work since `infer_automated_data_interval` is only designed to provide backward compatibility for DAG runs that are created before the data interval concept was introduced, and a DAG using a custom timetable would not work correctly with this method. I think `infer_manual_data_interval` is probably a better choice; it is not technically correct (the call below created a scheduled run), but the exact value of the data interval does not actually matter much here anyway since the run is only supposed to be a stub.



-- 
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 diff in pull request #27106: Fix warning in airflow tasks test command regarding absence of data_interval

Posted by "uranusjr (via GitHub)" <gi...@apache.org>.
uranusjr commented on code in PR #27106:
URL: https://github.com/apache/airflow/pull/27106#discussion_r1138174686


##########
airflow/cli/commands/task_command.py:
##########
@@ -126,10 +126,12 @@ def _get_dag_run(
         dag_run = DagRun(dag.dag_id, run_id=exec_date_or_run_id, execution_date=dag_run_execution_date)
         return dag_run, True
     elif create_if_necessary == "db":
+        data_interval = dag.infer_automated_data_interval(logical_date=dag_run_execution_date)

Review Comment:
   I took the liberty to change this myself.



-- 
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] potiuk commented on pull request #27106: Fix warning in airflow tasks test command regarding absence of data_interval

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #27106:
URL: https://github.com/apache/airflow/pull/27106#issuecomment-1289735370

   Conflicts needs resolving


-- 
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] potiuk commented on a diff in pull request #27106: Fix warning in airflow tasks test command regarding absence of data_interval

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on code in PR #27106:
URL: https://github.com/apache/airflow/pull/27106#discussion_r1138322655


##########
tests/cli/commands/test_task_command.py:
##########
@@ -178,11 +183,14 @@ def test_cli_test_different_path(self, session):
             dag = dagbag.get_dag("test_dags_folder")
             dagbag.sync_to_db(session=session)
 
+        execution_date = pendulum.now('UTC')

Review Comment:
   ```suggestion
           execution_date = pendulum.now("UTC")
   ```



-- 
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] potiuk merged pull request #27106: Fix warning in airflow tasks test command regarding absence of data_interval

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk merged PR #27106:
URL: https://github.com/apache/airflow/pull/27106


-- 
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] potiuk commented on pull request #27106: Fix warning in airflow tasks test command regarding absence of data_interval

Posted by GitBox <gi...@apache.org>.
potiuk commented on PR #27106:
URL: https://github.com/apache/airflow/pull/27106#issuecomment-1336263508

   Well. Not really all fixed.


-- 
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] tirkarthi commented on pull request #27106: Fix warning in airflow tasks test command regarding absence of data_interval

Posted by GitBox <gi...@apache.org>.
tirkarthi commented on PR #27106:
URL: https://github.com/apache/airflow/pull/27106#issuecomment-1321176364

   @potiuk Rebased with latest main branch and resolved conflicts.


-- 
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] tirkarthi commented on a diff in pull request #27106: Fix warning in airflow tasks test command regarding absence of data_interval

Posted by GitBox <gi...@apache.org>.
tirkarthi commented on code in PR #27106:
URL: https://github.com/apache/airflow/pull/27106#discussion_r998055964


##########
airflow/cli/commands/task_command.py:
##########
@@ -126,10 +126,12 @@ def _get_dag_run(
         dag_run = DagRun(dag.dag_id, run_id=exec_date_or_run_id, execution_date=dag_run_execution_date)
         return dag_run, True
     elif create_if_necessary == "db":
+        data_interval = dag.infer_automated_data_interval(logical_date=dag_run_execution_date)

Review Comment:
   yes, since `run_type` is not passed here this just calls the same part as underlying code to have same behavior. It's infer_manual_data_interval in tests since they pass run_type as manual. The dagrun is also deleted in the end so I guess it's ok.
   
   https://github.com/apache/airflow/blob/ea55626d79fdbd96b6d5f371883ac1df2a6313ec/airflow/models/dag.py#L2582-L2585



-- 
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 #27106: Fix warning in airflow tasks test command regarding absence of data_interval

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

   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 5 days if no further activity occurs. Thank you for your contributions.


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