You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "dstandish (via GitHub)" <gi...@apache.org> on 2024/04/30 21:21:42 UTC

[PR] Remove try_number shenanigans [airflow]

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

   Previously, there was a lot of bad stuff happening around try_number.  We incremented it when task started running.  And because of that, we had this logic to return "_try_number + 1" when task *not* running.  And, since it was naively incremented when task starts running -- i.e. without regard to why it is running -- we *decremented* it when deferring or exiting on a reschedule.
   
   What I do here is try to remove *all* of that stuff:
   * no more private _try_number attr
   * no more getter logic
   * no more decrementing
   * no more incrementing as part of task execution
   
   Now what we do is increment only when the task is set to *scheduled* and only when it's not coming out of deferral or "up_for_reschedule".
   


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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

Posted by "dstandish (via GitHub)" <gi...@apache.org>.
dstandish commented on PR #39336:
URL: https://github.com/apache/airflow/pull/39336#issuecomment-2101842090

   alright, yolo, here we go
   
   thanks @ashb @hussein-awala @jedcunningham 


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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/utils/openlineage.py:
##########
@@ -21,13 +21,25 @@
 from contextlib import suppress
 from typing import TYPE_CHECKING
 
+from airflow import __version__ as airflow_version
+
 if TYPE_CHECKING:
     from airflow.models.taskinstance import TaskInstance
     from airflow.providers.dbt.cloud.operators.dbt import DbtCloudRunJobOperator
     from airflow.providers.dbt.cloud.sensors.dbt import DbtCloudJobRunSensor
     from airflow.providers.openlineage.extractors.base import OperatorLineage
 
 
+def _get_try_number(val):
+    # todo: remove when min airflow version >= 2.10.0
+    from packaging.version import parse
+
+    if parse(parse(airflow_version).base_version) < parse("2.10.0"):
+        return val.try_number - 1
+    else:
+        return val.try_number

Review Comment:
   I don't love this, but other than deleting the `_try_number` property (and using that to detect the older version, possibly breaking back compat) I don't have any other ideas.



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


Re: [PR] Remove try_number shenanigans [airflow]

Posted by "dstandish (via GitHub)" <gi...@apache.org>.
dstandish commented on PR #39336:
URL: https://github.com/apache/airflow/pull/39336#issuecomment-2099400302

   @mobuchowski there are some OL files affected here. would appreciate you having a look


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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/jobs/backfill_job_runner.py:
##########
@@ -245,7 +245,16 @@ def _update_counters(self, ti_status: _DagRunTaskStatus, session: Session) -> No
             session.execute(
                 update(TI)
                 .where(filter_for_tis)
-                .values(state=TaskInstanceState.SCHEDULED)
+                .values(
+                    state=TaskInstanceState.SCHEDULED,
+                    try_number=case(
+                        (
+                            or_(TI.state.is_(None), TI.state != TaskInstanceState.UP_FOR_RESCHEDULE),

Review Comment:
   to be clear i _also_ thought it should work with positive check on up for...
   
   i.e.`where not state = 'up_for_reschedule'` but it did not work. i don't recall if maybe sqlalchemy was just being too "helpful" or what.
   
   snowflake has `equal_null` function that handles this case.  alas we don't run on snowflake 🤷 



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/utils/openlineage.py:
##########
@@ -21,13 +21,25 @@
 from contextlib import suppress
 from typing import TYPE_CHECKING
 
+from airflow import __version__ as airflow_version
+
 if TYPE_CHECKING:
     from airflow.models.taskinstance import TaskInstance
     from airflow.providers.dbt.cloud.operators.dbt import DbtCloudRunJobOperator
     from airflow.providers.dbt.cloud.sensors.dbt import DbtCloudJobRunSensor
     from airflow.providers.openlineage.extractors.base import OperatorLineage
 
 
+def _get_try_number(val):
+    # todo: remove when min airflow version >= 2.10.0
+    from packaging.version import parse
+
+    if parse(parse(airflow_version).base_version) < parse("2.10.0"):
+        return val.try_number - 1
+    else:
+        return val.try_number

Review Comment:
   SHould we maybe raise deprecation warnings if this is used in the code somewhere?



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

Posted by "dstandish (via GitHub)" <gi...@apache.org>.
dstandish commented on PR #39336:
URL: https://github.com/apache/airflow/pull/39336#issuecomment-2101995616

   I do plan to do more manual testing.  I had some trouble with backfill but thought i got it no worse than main (backfill has issues and doesn’t work right in all scenarios).  But plan to do a little more. I don’t personally object to including it. 


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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/CHANGELOG.rst:
##########
@@ -28,6 +28,11 @@
 Changelog
 ---------
 
+main
+.....
+
+In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state.  Importantly, after the task is done, it no longer shows current_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior.
+

Review Comment:
   We are discussing this very thing in here with @kacpermuda and @mobuchowski :
   
   * https://github.com/apache/airflow/pull/39513#discussion_r1596005178
   
   But also I need @dstandish to confirm if the SMTP change https://github.com/apache/airflow/pull/39513#discussion_r1596008489 - seem that the DBT change was good actually - as it handles back-compatibility nicely.
   
   This is the case for which I started to implement #39513 (for 2.9.0 compatibility) and follow-ups for 2.8 (#39606) and 2.7 (will be next) will be really helpful, because we are going to run the full suite of provider tests for past versions of airlfow once it is complete, so any kind of changes like that when we change both airflow and provider behaviour will have to be handled either as separate PRs or (if they are implemented as single PR) it will have to pass the back-compatibility tests with supported versions of Airflow. 
   
   So those kind of changes will be caught very early in the process - in the PR that modifies tests of provider to accomodate for changed behaviour of Airflow (and it will need to be handled by the author).
   
   
   



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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
tests/jobs/test_backfill_job.py:
##########
@@ -1525,7 +1544,7 @@ def test_update_counters(self, dag_maker, session):
         # match what's in the in-memory ti_status.running map. This is the same
         # for skipped, failed and retry states.
         ti_status.running[ti.key] = ti  # Task is queued and marked as running
-        ti._try_number += 1  # Try number is increased during ti.run()
+        ti.try_number += 1  # Try number is increased during ti.run()

Review Comment:
   yep
   
   ```suggestion
           ti.try_number += 1
   ```



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #39336:
URL: https://github.com/apache/airflow/pull/39336#issuecomment-2101991720

   Maybe just a bit more manual test scrutiny for 2.9.2 around try_num will be `enough` of stability ? I think the sooner we get it out, the easier will be to fix any potential problems there. 


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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/jobs/backfill_job_runner.py:
##########
@@ -245,7 +245,16 @@ def _update_counters(self, ti_status: _DagRunTaskStatus, session: Session) -> No
             session.execute(
                 update(TI)
                 .where(filter_for_tis)
-                .values(state=TaskInstanceState.SCHEDULED)
+                .values(
+                    state=TaskInstanceState.SCHEDULED,
+                    try_number=case(
+                        (
+                            or_(TI.state.is_(None), TI.state != TaskInstanceState.UP_FOR_RESCHEDULE),

Review Comment:
   to be clear i thought it should work with positive check on up for...
   
   i.e.`where not state = 'up_for_reschedule'` but it did not work. i don't recall if maybe sqlalchemy was just being too "helpful" or what.
   
   snowflake has `equal_null` function that handles this case.  alas we don't run on snowflake 🤷 



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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
airflow/models/taskinstance.py:
##########
@@ -1549,53 +1519,22 @@ def init_on_load(self) -> None:
         """Initialize the attributes that aren't stored in the DB."""
         self.test_mode = False  # can be changed when calling 'run'
 
-    @hybrid_property
-    def try_number(self):
-        """
-        Return the try number that a task number will be when it is actually run.
-
-        If the TaskInstance is currently running, this will match the column in the
-        database, in all other cases this will be incremented.
-
-        This is designed so that task logs end up in the right file.
-        """
-        return _get_try_number(task_instance=self)
-
-    @try_number.expression
-    def try_number(cls):
-        """
-        Return the expression to be used by SQLAlchemy when filtering on try_number.
-
-        This is required because the override in the get_try_number function causes
-        try_number values to be off by one when listing tasks in the UI.
-
-        :meta private:
-        """
-        return cls._try_number
-
-    @try_number.setter
-    def try_number(self, value: int) -> None:
-        """
-        Set a task try number.
-
-        :param value: the try number
-        """
-        _set_try_number(task_instance=self, value=value)
-
     @property
+    @deprecated(reason="Use try_number instead.", version="2.10.0", category=RemovedInAirflow3Warning)
     def prev_attempted_tries(self) -> int:
         """
-        Calculate the number of previously attempted tries, defaulting to 0.
+        Calculate the total number of attempted tries, defaulting to 0.
+
+        This used to be necessary because try_number did not always tell the truth.
 
-        Expose this for the Task Tries and Gantt graph views.
-        Using `try_number` throws off the counts for non-running tasks.
-        Also useful in error logging contexts to get the try number for the last try that was attempted.
+        :meta private:
         """
-        return self._try_number
+        return self.try_number
 
     @property
     def next_try_number(self) -> int:
-        return self._try_number + 1
+        # todo (dstandish): deprecate this property; we don't need a property that is just + 1

Review Comment:
   leaving that for the future @jedcunningham . it's got some other sequelae so i considered it out of scope



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

Posted by "hussein-awala (via GitHub)" <gi...@apache.org>.
hussein-awala commented on code in PR #39336:
URL: https://github.com/apache/airflow/pull/39336#discussion_r1594675865


##########
airflow/jobs/backfill_job_runner.py:
##########
@@ -245,7 +245,16 @@ def _update_counters(self, ti_status: _DagRunTaskStatus, session: Session) -> No
             session.execute(
                 update(TI)
                 .where(filter_for_tis)
-                .values(state=TaskInstanceState.SCHEDULED)
+                .values(
+                    state=TaskInstanceState.SCHEDULED,
+                    try_number=case(
+                        (
+                            or_(TI.state.is_(None), TI.state != TaskInstanceState.UP_FOR_RESCHEDULE),

Review Comment:
   Do we need the or if you reverse it? (Positive check on up_for_reschedule)



##########
airflow/models/dagrun.py:
##########
@@ -1567,7 +1568,16 @@ def schedule_tis(
                         TI.run_id == self.run_id,
                         tuple_in_condition((TI.task_id, TI.map_index), schedulable_ti_ids_chunk),
                     )
-                    .values(state=TaskInstanceState.SCHEDULED)
+                    .values(
+                        state=TaskInstanceState.SCHEDULED,
+                        try_number=case(
+                            (
+                                or_(TI.state.is_(None), TI.state != TaskInstanceState.UP_FOR_RESCHEDULE),

Review Comment:
   Same here



##########
airflow/jobs/backfill_job_runner.py:
##########
@@ -425,6 +434,8 @@ def _task_instances_for_dag_run(
         try:
             for ti in dag_run.get_task_instances(session=session):
                 if ti in schedulable_tis:
+                    if not ti.state == TaskInstanceState.UP_FOR_RESCHEDULE:

Review Comment:
   Why not !=?



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/CHANGELOG.rst:
##########
@@ -28,6 +28,11 @@
 Changelog
 ---------
 
+main
+.....
+
+In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state.  Importantly, after the task is done, it no longer shows current_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior.
+

Review Comment:
   We are discussing this very thing in here with @kacpermuda and @mobuchowski :
   
   * https://github.com/apache/airflow/pull/39513#discussion_r1596005178
   
   But also I need @dstandish to confirm if the SMTP change https://github.com/apache/airflow/pull/39513#discussion_r1596008489 - seem that the DBT change was good actually - as it handles back-compatibility nicely.
   
   This is the case for which I started to implement #39513 (for 2.9.0 compatibility) and follow-ups for 2.8 (#39606) and 2.7 (will be next) will be really helpful, because we are going to run the full suite of provider tests for past versions of airlfow once it is complete, so any kind of changes like that when we change both airflow and provider behaviour will have to be handled either as separate PRs or it will have to pass the back-compatibility tests with supported versions of Airflow. 
   
   So those kind of changes will be caught very early in the process - in the PR that modifies tests of provider to accomodate for changed behaviour of Airflow (and it will need to be handled by the author).
   
   
   



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/CHANGELOG.rst:
##########
@@ -28,6 +28,11 @@
 Changelog
 ---------
 
+main
+.....
+
+In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state.  Importantly, after the task is done, it no longer shows current_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior.
+

Review Comment:
   BTW. Most likely those kind of changes will have to be done as single PRs, because otherwise it will be difficult to coordinate such change in case provider test implicitly depends on some "airflow" internals - like it was in this case where provider behaviour (and tests) are depending on when try_number gets updated. 



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

Posted by "dstandish (via GitHub)" <gi...@apache.org>.
dstandish commented on PR #39336:
URL: https://github.com/apache/airflow/pull/39336#issuecomment-2101987113

   I feel like it is not unreasonable to think of it as bug fix. But I think the point of that convention is that patch releases should get more stable not less. So since this has some stability risk it may violate spirit of convention to include it. Though there could also be cherry pick convenience arguments that force it. If it doesn’t go in 2.9.x then incidentally it may not ever see a release because of Jed’s coming changes re TIs :) 


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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
tests/jobs/test_backfill_job.py:
##########
@@ -185,7 +185,7 @@ def test_trigger_controller_dag(self, session):
 
         assert task_instances_list
 
-    @pytest.mark.backend("postgres", "mysql")
+    # @pytest.mark.backend("postgres", "mysql")

Review Comment:
   nope
   ```suggestion
       @pytest.mark.backend("postgres", "mysql")
   ```



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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
airflow/www/views.py:
##########
@@ -5196,7 +5196,7 @@ class TaskInstanceModelView(AirflowModelView):
         "pool",
         "queued_by_job_id",
     ]
-
+    # todo: don't use prev_attempted_tries; just use try_number

Review Comment:
   deferred / out of scope



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
kubernetes_tests/test_kubernetes_pod_operator.py:
##########
@@ -112,7 +112,7 @@ def setup_tests(self, test_label):
                     "run_id": "manual__2016-01-01T0100000100-da4d1ce7b",
                     "dag_id": "dag",
                     "task_id": ANY,
-                    "try_number": "1",
+                    "try_number": "0",

Review Comment:
   talked with ash about this.  in this case, we're synthesizing a TI so we may as well set its try_number to be 1 so it looks more "normal"



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

Posted by "jedcunningham (via GitHub)" <gi...@apache.org>.
jedcunningham commented on PR #39336:
URL: https://github.com/apache/airflow/pull/39336#issuecomment-2105361519

   #protm


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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/utils/openlineage.py:
##########
@@ -21,13 +21,25 @@
 from contextlib import suppress
 from typing import TYPE_CHECKING
 
+from airflow import __version__ as airflow_version
+
 if TYPE_CHECKING:
     from airflow.models.taskinstance import TaskInstance
     from airflow.providers.dbt.cloud.operators.dbt import DbtCloudRunJobOperator
     from airflow.providers.dbt.cloud.sensors.dbt import DbtCloudJobRunSensor
     from airflow.providers.openlineage.extractors.base import OperatorLineage
 
 
+def _get_try_number(val):
+    # todo: remove when min airflow version >= 2.10.0
+    from packaging.version import parse
+
+    if parse(parse(airflow_version).base_version) < parse("2.10.0"):
+        return val.try_number - 1
+    else:
+        return val.try_number

Review Comment:
   e.g. when _try_number is a property then Class._try_number will be a property object
   and when _try_number is an instance attribute then Class._try_number will not be present
   but yeah... addressed why looking at version seems better above



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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
newsfragments/39336.significant.rst:
##########
@@ -0,0 +1,5 @@
+``try_number`` is no longer incremented during task execution
+
+Previously, the try_number was incremented at the beginning of task execution on the worker.  This was problematic for many reasons. For one it meant that the try_number was incremented when it was not supposed to, namely when resuming from reschedule or deferral. And it also resulted in the try_number being "wrong" when the task had not yet started.  The workarounds for these two issues caused a lot of confusion.  What we do instead now, is we only increment the try_number when it is scheduled by the scheduler.  So the try number for a task run is determined in advanced, and does not change in flight, and it is never decremented.  So after the task runs, the observed try_number be remain the same as it was when the task was running; only when there is a "new try" will the try_number be incremented again.
+
+One consequence of this change is, if users were "manually" running tasks (e.g. by calling ``ti.run()`` directly, or command line ``airflow tasks run``), try_number will no longer be incremented.  Airflow assumes that tasks are always run after being scheduled by the scheduler, so we do not regard this as a breaking change.

Review Comment:
   ```suggestion
   Previously, the try number (``try_number``) was incremented at the beginning of task execution on the worker. This was problematic for many reasons. For one it meant that the try number was incremented when it was not supposed to, namely when resuming from reschedule or deferral. And it also resulted in the try number being "wrong" when the task had not yet started. The workarounds for these two issues caused a lot of confusion.
   
   Now, instead, the try number for a task run is determined at the time the task is scheduled, and does not change in flight, and it is never decremented. So after the task runs, the observed try number remains the same as it was when the task was running; only when there is a "new try" will the try number be incremented again.
   
   One consequence of this change is, if users were "manually" running tasks (e.g. by calling ``ti.run()`` directly, or command line ``airflow tasks run``), try number will no longer be incremented. Airflow assumes that tasks are always run after being scheduled by the scheduler, so we do not regard this as a breaking change.
   ```
   
   We could convert a few ``try_number``s to "try number" if we want less inline 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: commits-unsubscribe@airflow.apache.org

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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/CHANGELOG.rst:
##########
@@ -28,6 +28,11 @@
 Changelog
 ---------
 
+main
+.....
+
+In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state.  Importantly, after the task is done, it no longer shows current_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior.
+

Review Comment:
   Why is this here?
   How is this PR affect bdt provider?
   Providers are supported for older versions of Airflow as well. I'm out of context for what this PR achieves but I am not sure if this note in change log is the right place.
   
   What if I upgrade to provider version while I was on Airflow 2.7? Months later when I will upgrade to Airflow 2.10 I might miss that.



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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
newsfragments/39336.significant.rst:
##########
@@ -0,0 +1,5 @@
+``try_number`` is no longer incremented during task execution
+
+Previously, the try_number was incremented at the beginning of task execution on the worker.  This was problematic for many reasons. For one it meant that the try_number was incremented when it was not supposed to, namely when resuming from reschedule or deferral. And it also resulted in the try_number being "wrong" when the task had not yet started.  The workarounds for these two issues caused a lot of confusion.  What we do instead now, is we only increment the try_number when it is scheduled by the scheduler.  So the try number for a task run is determined in advanced, and does not change in flight, and it is never decremented.  So after the task runs, the observed try_number be remain the same as it was when the task was running; only when there is a "new try" will the try_number be incremented again.
+
+One consequence of this change is, if users were "manually" running tasks (e.g. by calling ``ti.run()`` directly, or command line ``airflow tasks run``), try_number will no longer be incremented.  Airflow assumes that tasks are always run after being scheduled by the scheduler, so we do not regard this as a breaking change.

Review Comment:
   ```suggestion
   Previously, the try number (``try_number``) was incremented at the beginning of task execution on the worker. This was problematic for many reasons. For one it meant that the try number was incremented when it was not supposed to, namely when resuming from reschedule or deferral. And it also resulted in the try number being "wrong" when the task had not yet started. The workarounds for these two issues caused a lot of confusion.
   
   Now, instead, the try_number for a task run is determined at the time the task is scheduled, and does not change in flight, and it is never decremented. So after the task runs, the observed try number remains the same as it was when the task was running; only when there is a "new try" will the try number be incremented again.
   
   One consequence of this change is, if users were "manually" running tasks (e.g. by calling ``ti.run()`` directly, or command line ``airflow tasks run``), try number will no longer be incremented. Airflow assumes that tasks are always run after being scheduled by the scheduler, so we do not regard this as a breaking change.
   ```
   
   We could convert a few ``try_number``s to "try number" if we want less inline 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: commits-unsubscribe@airflow.apache.org

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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
airflow/www/views.py:
##########
@@ -5196,7 +5196,7 @@ class TaskInstanceModelView(AirflowModelView):
         "pool",
         "queued_by_job_id",
     ]
-
+    # todo: don't use prev_attempted_tries; just use try_number

Review Comment:
   deferred



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

Posted by "potiuk (via GitHub)" <gi...@apache.org>.
potiuk commented on PR #39336:
URL: https://github.com/apache/airflow/pull/39336#issuecomment-2101983614

   > A few nits, otherwise it looks good, and I don't know why we (I) did it this terrible hacky way in the first place!
   
   I think we will find out when it is released :). But I also think it's worth to take risk to finally get rid of that one.
   
   BTW. Shall we mark it for 2.9.2 ? I **think** we can treat it as a bug-fix ?


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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/jobs/backfill_job_runner.py:
##########
@@ -245,7 +245,16 @@ def _update_counters(self, ti_status: _DagRunTaskStatus, session: Session) -> No
             session.execute(
                 update(TI)
                 .where(filter_for_tis)
-                .values(state=TaskInstanceState.SCHEDULED)
+                .values(
+                    state=TaskInstanceState.SCHEDULED,
+                    try_number=case(
+                        (
+                            or_(TI.state.is_(None), TI.state != TaskInstanceState.UP_FOR_RESCHEDULE),

Review Comment:
   you do need it because sql.



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/CHANGELOG.rst:
##########
@@ -28,6 +28,11 @@
 Changelog
 ---------
 
+main
+.....
+
+In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state.  Importantly, after the task is done, it no longer shows current_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior.
+

Review Comment:
   Why is this here?
   How is this PR affect dbt provider?
   Providers are supported for older versions of Airflow as well. I'm out of context for what this PR achieves but I am not sure if this note in change log is the right place.
   
   What if I upgrade to provider version while I was on Airflow 2.7? Months later when I will upgrade to Airflow 2.10 I might miss that.



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/CHANGELOG.rst:
##########
@@ -28,6 +28,11 @@
 Changelog
 ---------
 
+main
+.....
+
+In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state.  Importantly, after the task is done, it no longer shows current_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior.
+

Review Comment:
   We are discussing this very thing in here with @kacpermuda and @mobuchowski :
   
   * https://github.com/apache/airflow/pull/39513#discussion_r1596005178
   
   But also I need @dstandish to confirm if the SMTP change https://github.com/apache/airflow/pull/39513#discussion_r1596008489 - seem that the DBT change was good actually - as it handles back-compatibility nicely.
   
   This is the case for which I started to implement #39513 (for 2.9.0 compatibility) and follow-ups for 2.8 (39606) and 2.7 (will be next) will be really helpful, because we are going to run the full suite of provider tests for past versions of airlfow once it is complete, so any kind of changes like that when we change both airflow and provider behaviour will have to be handled either as separate PRs or it will have to pass the back-compatibility tests with supported versions of Airflow. 
   
   So those kind of changes will be caught very early in the process - in the PR that modifies tests of provider to accomodate for changed behaviour of Airflow (and it will need to be handled by the author).
   
   
   



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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
tests/sensors/test_base.py:
##########
@@ -382,9 +398,12 @@ def _get_tis():
         # Task is cleared
         sensor.clear()
         sensor_ti, dummy_ti = _get_tis()
-        assert sensor_ti.try_number == 2

Review Comment:
   we change this (and similar in this file) from 2 to 1 because it used to give us the artificially increased number (the getter returned +1 when task not running)



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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
airflow/providers/amazon/aws/executors/ecs/ecs_executor.py:
##########
@@ -527,7 +527,7 @@ def try_adopt_task_instances(self, tis: Sequence[TaskInstance]) -> Sequence[Task
                         ti.queue,
                         ti.command_as_list(),
                         ti.executor_config,
-                        ti.prev_attempted_tries,
+                        ti.try_number,  # prev attempted tries

Review Comment:
   yeah.....



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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
airflow/providers/dbt/cloud/utils/openlineage.py:
##########
@@ -21,13 +21,28 @@
 from contextlib import suppress
 from typing import TYPE_CHECKING
 
+try:
+    from airflow import __version__ as airflow_version

Review Comment:
   #39497



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/models/dagrun.py:
##########
@@ -1567,7 +1568,16 @@ def schedule_tis(
                         TI.run_id == self.run_id,
                         tuple_in_condition((TI.task_id, TI.map_index), schedulable_ti_ids_chunk),
                     )
-                    .values(state=TaskInstanceState.SCHEDULED)
+                    .values(
+                        state=TaskInstanceState.SCHEDULED,
+                        try_number=case(
+                            (
+                                or_(TI.state.is_(None), TI.state != TaskInstanceState.UP_FOR_RESCHEDULE),

Review Comment:
   what do you mean same here? please clarify what you think it should be



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
airflow/models/taskinstancekey.py:
##########
@@ -37,6 +37,7 @@ def primary(self) -> tuple[str, str, str, int]:
     @property
     def reduced(self) -> TaskInstanceKey:
         """Remake the key by subtracting 1 from try number to match in memory information."""
+        # todo (dstandish): remove this property

Review Comment:
   todo reminder :)



##########
airflow/models/taskinstance.py:
##########
@@ -1549,53 +1519,22 @@ def init_on_load(self) -> None:
         """Initialize the attributes that aren't stored in the DB."""
         self.test_mode = False  # can be changed when calling 'run'
 
-    @hybrid_property
-    def try_number(self):
-        """
-        Return the try number that a task number will be when it is actually run.
-
-        If the TaskInstance is currently running, this will match the column in the
-        database, in all other cases this will be incremented.
-
-        This is designed so that task logs end up in the right file.
-        """
-        return _get_try_number(task_instance=self)
-
-    @try_number.expression
-    def try_number(cls):
-        """
-        Return the expression to be used by SQLAlchemy when filtering on try_number.
-
-        This is required because the override in the get_try_number function causes
-        try_number values to be off by one when listing tasks in the UI.
-
-        :meta private:
-        """
-        return cls._try_number
-
-    @try_number.setter
-    def try_number(self, value: int) -> None:
-        """
-        Set a task try number.
-
-        :param value: the try number
-        """
-        _set_try_number(task_instance=self, value=value)
-
     @property
+    @deprecated(reason="Use try_number instead.", version="2.10.0", category=RemovedInAirflow3Warning)
     def prev_attempted_tries(self) -> int:
         """
-        Calculate the number of previously attempted tries, defaulting to 0.
+        Calculate the total number of attempted tries, defaulting to 0.
+
+        This used to be necessary because try_number did not always tell the truth.
 
-        Expose this for the Task Tries and Gantt graph views.
-        Using `try_number` throws off the counts for non-running tasks.
-        Also useful in error logging contexts to get the try number for the last try that was attempted.
+        :meta private:
         """
-        return self._try_number
+        return self.try_number
 
     @property
     def next_try_number(self) -> int:
-        return self._try_number + 1
+        # todo (dstandish): deprecate this property; we don't need a property that is just + 1

Review Comment:
   todo reminder :)



##########
airflow/models/taskinstance.py:
##########
@@ -1508,6 +1458,26 @@ def __init__(
     def __hash__(self):
         return hash((self.task_id, self.dag_id, self.run_id, self.map_index))
 
+    @property
+    @deprecated(reason="Use try_number instead.", version="2.10.0", category=RemovedInAirflow3Warning)
+    def _try_number(self):
+        """
+        Do not use.  For semblance of backcompat.
+
+        :meta private:
+        """
+        return self.try_number
+
+    @_try_number.setter
+    @deprecated(reason="Use try_number instead.", version="2.10.0", category=RemovedInAirflow3Warning)
+    def _try_number(self, val):
+        """
+        Do not use.  For semblance of backcompat.

Review Comment:
   ```suggestion
           Do not use. For semblance of backcompat.
   ```
   
   nit



##########
airflow/providers/dbt/cloud/utils/openlineage.py:
##########
@@ -21,13 +21,28 @@
 from contextlib import suppress
 from typing import TYPE_CHECKING
 
+try:
+    from airflow import __version__ as airflow_version

Review Comment:
   This always exists in 2.7+. In fact, I think it works all the back to 2.4 :)



##########
tests/jobs/test_backfill_job.py:
##########
@@ -1525,7 +1544,7 @@ def test_update_counters(self, dag_maker, session):
         # match what's in the in-memory ti_status.running map. This is the same
         # for skipped, failed and retry states.
         ti_status.running[ti.key] = ti  # Task is queued and marked as running
-        ti._try_number += 1  # Try number is increased during ti.run()
+        ti.try_number += 1  # Try number is increased during ti.run()

Review Comment:
   This comment isn't right anymore, right?



##########
newsfragments/39336.significant.rst:
##########
@@ -0,0 +1,5 @@
+``try_number`` is no longer incremented during task execution
+
+Previously, the try_number was incremented at the beginning of task execution on the worker.  This was problematic for many reasons. For one it meant that the try_number was incremented when it was not supposed to, namely when resuming from reschedule or deferral. And it also resulted in the try_number being "wrong" when the task had not yet started.  The workarounds for these two issues caused a lot of confusion.  What we do instead now, is we only increment the try_number when it is scheduled by the scheduler.  So the try number for a task run is determined in advanced, and does not change in flight, and it is never decremented.  So after the task runs, the observed try_number be remain the same as it was when the task was running; only when there is a "new try" will the try_number be incremented again.
+
+One consequence of this change is, if users were "manually" running tasks (e.g. by calling ``ti.run()`` directly, or command line ``airflow tasks run``), try_number will no longer be incremented.  Airflow assumes that tasks are always run after being scheduled by the scheduler, so we do not regard this as a breaking change.

Review Comment:
   ```suggestion
   Previously, the ``try_number`` was incremented at the beginning of task execution on the worker. This was problematic for many reasons. For one it meant that the ``try_number`` was incremented when it was not supposed to, namely when resuming from reschedule or deferral. And it also resulted in the ``try_number`` being "wrong" when the task had not yet started. The workarounds for these two issues caused a lot of confusion. Not instead, we only increment the ``try_number`` when the task is scheduled by the scheduler. The ``try_number`` for a task run is determined in advanced, and does not change in flight, and it is never decremented. So after the task runs, the observed ``try_number`` remains the same as it was when the task was running; only when there is a "new try" will the ``try_number`` be incremented again.
   
   One consequence of this change is, if users were "manually" running tasks (e.g. by calling ``ti.run()`` directly, or command line ``airflow tasks run``), ``try_number`` will no longer be incremented. Airflow assumes that tasks are always run after being scheduled by the scheduler, so we do not regard this as a breaking change.
   ```
   
   We could convert a few ``try_number``s to "try number" if we want less inline code.



##########
airflow/providers/amazon/aws/executors/ecs/ecs_executor.py:
##########
@@ -527,7 +527,7 @@ def try_adopt_task_instances(self, tis: Sequence[TaskInstance]) -> Sequence[Task
                         ti.queue,
                         ti.command_as_list(),
                         ti.executor_config,
-                        ti.prev_attempted_tries,
+                        ti.try_number,  # prev attempted tries

Review Comment:
   ```suggestion
                           ti.try_number,
   ```
   
   Not sure we need this comment?



##########
airflow/www/views.py:
##########
@@ -5196,7 +5196,7 @@ class TaskInstanceModelView(AirflowModelView):
         "pool",
         "queued_by_job_id",
     ]
-
+    # todo: don't use prev_attempted_tries; just use try_number

Review Comment:
   todo reminder :)



##########
airflow/models/taskinstance.py:
##########
@@ -1508,6 +1458,26 @@ def __init__(
     def __hash__(self):
         return hash((self.task_id, self.dag_id, self.run_id, self.map_index))
 
+    @property
+    @deprecated(reason="Use try_number instead.", version="2.10.0", category=RemovedInAirflow3Warning)
+    def _try_number(self):
+        """
+        Do not use.  For semblance of backcompat.

Review Comment:
   ```suggestion
           Do not use. For semblance of backcompat.
   ```
   
   nit!



##########
tests/jobs/test_backfill_job.py:
##########
@@ -185,7 +185,7 @@ def test_trigger_controller_dag(self, session):
 
         assert task_instances_list
 
-    @pytest.mark.backend("postgres", "mysql")
+    # @pytest.mark.backend("postgres", "mysql")

Review Comment:
   Did you mean to leave this commented out?



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
kubernetes_tests/test_kubernetes_pod_operator.py:
##########
@@ -112,7 +112,7 @@ def setup_tests(self, test_label):
                     "run_id": "manual__2016-01-01T0100000100-da4d1ce7b",
                     "dag_id": "dag",
                     "task_id": ANY,
-                    "try_number": "1",
+                    "try_number": "0",

Review Comment:
   We shouldn't expect a try_number of 0 to be the label, should we? This might be a case where we need to have `try_number + 1` when generating the labels for the pod?



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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
airflow/models/taskinstance.py:
##########
@@ -280,14 +280,13 @@ def clear_task_instances(
                 ti.refresh_from_task(task)
                 if TYPE_CHECKING:
                     assert ti.task
-                task_retries = task.retries
-                ti.max_tries = ti.try_number + task_retries - 1

Review Comment:
   we no longer subtract 1 here because we're no longer _adding_ 1 when task is not running.  i assume this old logic would give the wrong result if task was running.....



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


Re: [PR] Remove try_number shenanigans [airflow]

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


##########
airflow/models/taskinstancekey.py:
##########
@@ -37,6 +37,7 @@ def primary(self) -> tuple[str, str, str, int]:
     @property
     def reduced(self) -> TaskInstanceKey:
         """Remake the key by subtracting 1 from try number to match in memory information."""
+        # todo (dstandish): remove this property

Review Comment:
   deferred 
   ![Uploading image.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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/utils/openlineage.py:
##########
@@ -21,13 +21,25 @@
 from contextlib import suppress
 from typing import TYPE_CHECKING
 
+from airflow import __version__ as airflow_version
+
 if TYPE_CHECKING:
     from airflow.models.taskinstance import TaskInstance
     from airflow.providers.dbt.cloud.operators.dbt import DbtCloudRunJobOperator
     from airflow.providers.dbt.cloud.sensors.dbt import DbtCloudJobRunSensor
     from airflow.providers.openlineage.extractors.base import OperatorLineage
 
 
+def _get_try_number(val):
+    # todo: remove when min airflow version >= 2.10.0
+    from packaging.version import parse
+
+    if parse(parse(airflow_version).base_version) < parse("2.10.0"):
+        return val.try_number - 1
+    else:
+        return val.try_number

Review Comment:
   yeah.  @ashb originally i was just deleting _try_number.  but the reason we added that setter is just elasticsearch task handler.  there's one code path where it will set `_try_number`  since it's in providers, there will be users who upgrade airflow without upgrading the provider, and they will hit it. ideally should not have been referencing _try_number in the first place.  (my mistake actually). so this is just going out of the way to be nice to such users.
   
   re this ugly compat shim.... we could probably still check by looking at the object... but i think @jedcunningham's argument was that comparing version is more explicit / self-documenting, and i think i agree.



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/CHANGELOG.rst:
##########
@@ -28,6 +28,11 @@
 Changelog
 ---------
 
+main
+.....
+
+In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state.  Importantly, after the task is done, it no longer shows current_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior.
+

Review Comment:
   Why is this here?
   How is this PR affect bdt provider?



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/jobs/backfill_job_runner.py:
##########
@@ -245,7 +245,16 @@ def _update_counters(self, ti_status: _DagRunTaskStatus, session: Session) -> No
             session.execute(
                 update(TI)
                 .where(filter_for_tis)
-                .values(state=TaskInstanceState.SCHEDULED)
+                .values(
+                    state=TaskInstanceState.SCHEDULED,
+                    try_number=case(
+                        (
+                            or_(TI.state.is_(None), TI.state != TaskInstanceState.UP_FOR_RESCHEDULE),

Review Comment:
   when it's null, comparisons evaluate to false....



##########
airflow/jobs/backfill_job_runner.py:
##########
@@ -425,6 +434,8 @@ def _task_instances_for_dag_run(
         try:
             for ti in dag_run.get_task_instances(session=session):
                 if ti in schedulable_tis:
+                    if not ti.state == TaskInstanceState.UP_FOR_RESCHEDULE:

Review Comment:
   i think i'm just in sql brain here.....



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/models/dagrun.py:
##########
@@ -1567,7 +1568,16 @@ def schedule_tis(
                         TI.run_id == self.run_id,
                         tuple_in_condition((TI.task_id, TI.map_index), schedulable_ti_ids_chunk),
                     )
-                    .values(state=TaskInstanceState.SCHEDULED)
+                    .values(
+                        state=TaskInstanceState.SCHEDULED,
+                        try_number=case(
+                            (
+                                or_(TI.state.is_(None), TI.state != TaskInstanceState.UP_FOR_RESCHEDULE),

Review Comment:
   oh i see i think you're saying same as https://github.com/apache/airflow/pull/39336#discussion_r1594675865 not same as the comment immediately above.  yeah this is needed because sql comparisons with null



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/utils/openlineage.py:
##########
@@ -21,13 +21,25 @@
 from contextlib import suppress
 from typing import TYPE_CHECKING
 
+from airflow import __version__ as airflow_version
+
 if TYPE_CHECKING:
     from airflow.models.taskinstance import TaskInstance
     from airflow.providers.dbt.cloud.operators.dbt import DbtCloudRunJobOperator
     from airflow.providers.dbt.cloud.sensors.dbt import DbtCloudJobRunSensor
     from airflow.providers.openlineage.extractors.base import OperatorLineage
 
 
+def _get_try_number(val):
+    # todo: remove when min airflow version >= 2.10.0
+    from packaging.version import parse
+
+    if parse(parse(airflow_version).base_version) < parse("2.10.0"):
+        return val.try_number - 1
+    else:
+        return val.try_number

Review Comment:
   That's what this is currently doing, `ti._try_number` is currently raising the deprecation warning. But that means we can't use `if hasattr(val, "_try_number")` as a way of detecting the old version, cos we've done the "right" thing.



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


Re: [PR] Scheduler to handle incrementing of try_number [airflow]

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


##########
airflow/providers/dbt/cloud/CHANGELOG.rst:
##########
@@ -28,6 +28,11 @@
 Changelog
 ---------
 
+main
+.....
+
+In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state.  Importantly, after the task is done, it no longer shows current_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior.
+

Review Comment:
   BTW. Most likely those kind of tests will have to be done as single PRs, because otherwise it will be difficult to coordinate such change in case provider test implicitly depends on some "airflow" internals - like it was in this case where provider behaviour (and tests) are depending on when try_number gets updated. 



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