tanelk opened a new issue, #37902:
URL: https://github.com/apache/airflow/issues/37902

   ### Apache Airflow version
   
   main (development)
   
   ### If "Other Airflow 2 version" selected, which one?
   
   _No response_
   
   ### What happened?
   
   The `/dags/<string:dag_id>/calendar` view does not open for DAGs, that have 
`schedule='@continuous'`.
   
   It happens because this loop in `airflow.www.views.Airflow.calendar` exits 
at the end of the year.
   ```python
   while True:
       curr_info = dag.timetable.next_dagrun_info(
           last_automated_data_interval=last_automated_data_interval,
           restriction=restriction,
       )
       if curr_info is None:
           break  # Reached the end.
       if curr_info.logical_date <= prev_logical_date:
           break  # We're not progressing. Maybe a malformed timetable? Give up.
       if curr_info.logical_date.year != year:
           break  # Crossed the year boundary.
       last_automated_data_interval = curr_info.data_interval
       dates[curr_info.logical_date] += 1
       prev_logical_date = curr_info.logical_date
   ```
   
   The `prev_logical_date` and `curr_info.logical_date` are both set to 
`timezone.utcnow()` at consecutive loop iterations. This is why 
`curr_info.logical_date <= prev_logical_date` never fails and 
`curr_info.logical_date.year != year` waits for the end of the year.
   
   
   ### What you think should happen instead?
   
   I'm not sure, what is the best way to solve this:
   1.  Check if `dag.timetable` is `ContinuousTimetable`
   2. Limit the number of iterations
   3. Check that the diff between `curr_info.logical_date` and 
`prev_logical_date` is at least some amount - 1 minute or so.
   
   ### How to reproduce
   
   Make @continuous DAG and check the calendar view.
   
   ### Operating System
   
   -
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Other
   
   ### Deployment details
   
   _No response_
   
   ### Anything else?
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of 
Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to