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 2020/11/13 23:05:48 UTC

[GitHub] [airflow] ashb opened a new pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

ashb opened a new pull request #12353:
URL: https://github.com/apache/airflow/pull/12353


   `time.time() - start`, or `timezone.utcnow() - start_dttm` will work
   fine in 99% of cases, but it has one fatal flaw:
   
   They both operate on system time, and that can go backwards.
   
   While this might be surprising, it can happen -- usually due to clocks
   being adjusted.
   
   And while it is might seem rare, for long running processes it is more
   common than we might expect. Most of these durations are harmless to get
   wrong (just being logs) it is better to be safe than sorry.
   
   Also the `utcnow()` style I have replaced will be much lighter weight -
   creating a date time object is a comparatively expensive operation, and
   computing a diff between two even more so, _especially_ when compared to
   just subtracting two floats.
   
   To make the "common" case easier of wanting to compute a duration for a
   block, I have made `Stats.timer()` return an object that has a
   `duration` field.
   
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/master/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/master/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.

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



[GitHub] [airflow] ashb commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   (This will likely fail tests at first. I likely messed up some calculations 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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/379360242) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/379354322) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


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

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



[GitHub] [airflow] ashb commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   Hopefully got the tests sorted now (just one last instance of mocking `time.time` when it now needs to mock `time.monotonic`)


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

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



[GitHub] [airflow] ashb commented on a change in pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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



##########
File path: CONTRIBUTING.rst
##########
@@ -806,6 +806,45 @@ If this function is designed to be called by "end-users" (i.e. DAG authors) then
       ...
       # You SHOULD not commit the session here. The wrapper will take care of commit()/rollback() if exception
 
+Don't use time() for duration calcuations
+-----------------------------------------
+
+If you wish to compute the time difference between two events with in the same process, use
+``time.monotonic()``, not ``time.time()`` nor ``timzeone.utcnow()``.
+
+If you are measuring duration for performance reasons, then ``time.perf_counter()`` should be used. (On many
+platforms, this uses the same underlying clock mechanism as monotonic, but ``perf_counter`` is guaranteed to be
+the highest accuracy clock on the system, monotonic is simply "guaranteed" to not go backwards.)
+
+If you wish to time how long a block of code takes, use ``Stats.timer()`` -- either with a metric name, which
+will be timed and submitted automatically:
+
+.. code-block:: python
+
+    from airflow.stats import Stats
+
+    ...
+
+    with Stats.timer("my_timer_metric"):
+        ...
+
+or without, to access the ``duration`` property:
+
+.. code-block:: python
+
+    from airflow.stats import Stats
+
+    ...
+
+    with Stats.timer("my_timer_metric") as timer:

Review comment:
       Left these two common cases here (with a tidy up as suggested), and linked to airflow/stats.py.




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

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



[GitHub] [airflow] XD-DENG commented on a change in pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

Posted by GitBox <gi...@apache.org>.
XD-DENG commented on a change in pull request #12353:
URL: https://github.com/apache/airflow/pull/12353#discussion_r523399124



##########
File path: airflow/stats.py
##########
@@ -72,20 +73,71 @@ def timer(cls, *args, **kwargs) -> TimerProtocol:
         """Timer metric that can be cancelled"""
 
 
-class DummyTimer:
-    """No-op timer"""
+class Timer:
+    """
+    Timer that records duration, and optional sends to statsd backend.
+
+    This class lets us have an accurate timer with the logic in one place (so
+    that we don't use datetime math for duration -- it is error prone).
+
+    Example usage:
+
+    .. code-block:: python
+
+        with Stats.timer() as t:
+            # Something to time
+            frob_the_foos()
+
+        log.info("Frobbing the foos took %.2f", t.duration)
+
+    To send a metric:
+
+    .. code-block:: python
+
+        with Stats.timer("foos.frob"):
+            # Something to time
+            frob_the_foos()
+
+
+    Or both:
+
+    .. code-block:: python
+
+        with Stats.timer("foos.frob") as t:
+            # Something to time
+            frob_the_foos()
+
+        log.info("Frobbing the foos took %.2f", t.duration)
+
+    """
+
+    # pystatsd and dogstatsd both have a timer class, but present different API
+    # so we can't use this as a mixin on those, instead this class is contains the "real" timer
+
+    _start_time: Optional[int]
+    duration: Optional[int]
+    ms: Optional[int]
+    _send: bool = False

Review comment:
       How is this `_send` attribute behaving here? Or only for compatibility with `Timer` in `pystatsd `? 




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

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



[GitHub] [airflow] kaxil commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   One failure
   
   ```
   self = <Task(PythonSensor): test_reschedule_handling_sensor>
   context = {'conf': <airflow.configuration.AirflowConfigParser object at 0x7f383eda7860>, 'dag': <DAG: test_reschedule_handling>,...36:50.699432+00:00: scheduled__2020-11-14T12:36:50.699432+00:00, externally triggered: False>, 'ds': '2020-11-14', ...}
   
       def execute(self, context: Dict) -> Any:
           started_at = time.monotonic()
           try_number = 1
           log_dag_id = self.dag.dag_id if self.has_dag() else ""
           if self.reschedule:
               # If reschedule, use first start date of current try
               task_reschedules = TaskReschedule.find_for_task_instance(context['ti'])
               if task_reschedules:
                   started_at = task_reschedules[0].start_date
                   try_number = len(task_reschedules) + 1
           while not self.poke(context):
   >           if time.monotonic() - started_at > self.timeout:
   E           TypeError: unsupported operand type(s) for -: 'float' and 'FakeDatetime'
   
   airflow/sensors/base_sensor_operator.py:210: TypeError
   ```


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

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



[GitHub] [airflow] ashb merged pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/362503890) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


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

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



[GitHub] [airflow] ashb commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   YAY GREEN FINALLY!


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   The PR needs to run all tests because it modifies core of Airflow! Please rebase it to latest master or ask committer to re-run 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.

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



[GitHub] [airflow] ashb commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   https://github.com/apache/airflow/pull/12353/files#diff-f0e400bd7a526279c114ce0bd87e80f1f9344885b749f2da495d4f7bb670432eR228
   
   When statsd was _actually_ enabled that line was failing, but we don't test with statsd on anywhere outside of the kube tests (thankfully?) -- but to get the error we had to download the kube logs artifact and trawl through the logs until I found the actual error.


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

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



[GitHub] [airflow] XD-DENG commented on a change in pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

Posted by GitBox <gi...@apache.org>.
XD-DENG commented on a change in pull request #12353:
URL: https://github.com/apache/airflow/pull/12353#discussion_r523399682



##########
File path: airflow/stats.py
##########
@@ -72,20 +73,71 @@ def timer(cls, *args, **kwargs) -> TimerProtocol:
         """Timer metric that can be cancelled"""
 
 
-class DummyTimer:
-    """No-op timer"""
+class Timer:
+    """
+    Timer that records duration, and optional sends to statsd backend.
+
+    This class lets us have an accurate timer with the logic in one place (so
+    that we don't use datetime math for duration -- it is error prone).
+
+    Example usage:
+
+    .. code-block:: python
+
+        with Stats.timer() as t:
+            # Something to time
+            frob_the_foos()
+
+        log.info("Frobbing the foos took %.2f", t.duration)
+
+    To send a metric:
+
+    .. code-block:: python
+
+        with Stats.timer("foos.frob"):
+            # Something to time
+            frob_the_foos()
+
+
+    Or both:
+
+    .. code-block:: python
+
+        with Stats.timer("foos.frob") as t:
+            # Something to time
+            frob_the_foos()
+
+        log.info("Frobbing the foos took %.2f", t.duration)
+
+    """
+
+    # pystatsd and dogstatsd both have a timer class, but present different API
+    # so we can't use this as a mixin on those, instead this class is contains the "real" timer
+
+    _start_time: Optional[int]
+    duration: Optional[int]
+    ms: Optional[int]
+    _send: bool = False

Review comment:
       And the `ms` attribute above.
   
   May be dumb question ;-)




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

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



[GitHub] [airflow] ashb commented on a change in pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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



##########
File path: airflow/stats.py
##########
@@ -72,20 +73,71 @@ def timer(cls, *args, **kwargs) -> TimerProtocol:
         """Timer metric that can be cancelled"""
 
 
-class DummyTimer:
-    """No-op timer"""
+class Timer:
+    """
+    Timer that records duration, and optional sends to statsd backend.
+
+    This class lets us have an accurate timer with the logic in one place (so
+    that we don't use datetime math for duration -- it is error prone).
+
+    Example usage:
+
+    .. code-block:: python
+
+        with Stats.timer() as t:
+            # Something to time
+            frob_the_foos()
+
+        log.info("Frobbing the foos took %.2f", t.duration)
+
+    To send a metric:
+
+    .. code-block:: python
+
+        with Stats.timer("foos.frob"):
+            # Something to time
+            frob_the_foos()
+
+
+    Or both:
+
+    .. code-block:: python
+
+        with Stats.timer("foos.frob") as t:
+            # Something to time
+            frob_the_foos()
+
+        log.info("Frobbing the foos took %.2f", t.duration)
+
+    """
+
+    # pystatsd and dogstatsd both have a timer class, but present different API
+    # so we can't use this as a mixin on those, instead this class is contains the "real" timer
+
+    _start_time: Optional[int]
+    duration: Optional[int]
+    ms: Optional[int]
+    _send: bool = False

Review comment:
       Oh yes, these aren't needed anymore. I was planning on making this a subclass of the timer, bit pystatsd and dogstatsd are too different to work. So I went with encapsulation instead.
   
   These can be removed




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

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



[GitHub] [airflow] ashb commented on a change in pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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



##########
File path: CONTRIBUTING.rst
##########
@@ -806,6 +806,45 @@ If this function is designed to be called by "end-users" (i.e. DAG authors) then
       ...
       # You SHOULD not commit the session here. The wrapper will take care of commit()/rollback() if exception
 
+Don't use time() for duration calcuations
+-----------------------------------------
+
+If you wish to compute the time difference between two events with in the same process, use
+``time.monotonic()``, not ``time.time()`` nor ``timzeone.utcnow()``.
+
+If you are measuring duration for performance reasons, then ``time.perf_counter()`` should be used. (On many
+platforms, this uses the same underlying clock mechanism as monotonic, but ``perf_counter`` is guaranteed to be
+the highest accuracy clock on the system, monotonic is simply "guaranteed" to not go backwards.)
+
+If you wish to time how long a block of code takes, use ``Stats.timer()`` -- either with a metric name, which
+will be timed and submitted automatically:
+
+.. code-block:: python
+
+    from airflow.stats import Stats
+
+    ...
+
+    with Stats.timer("my_timer_metric"):
+        ...
+
+or without, to access the ``duration`` property:
+
+.. code-block:: python
+
+    from airflow.stats import Stats
+
+    ...
+
+    with Stats.timer("my_timer_metric") as timer:

Review comment:
       Hmmm yeah, I'll tidy this up. I've got duplication between here and in airflow.stats -- I might just link from 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.

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



[GitHub] [airflow] kaxil commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   Master was failing, I have fixed it and rebased this PR


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

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



[GitHub] [airflow] ashb commented on a change in pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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



##########
File path: airflow/stats.py
##########
@@ -72,20 +73,71 @@ def timer(cls, *args, **kwargs) -> TimerProtocol:
         """Timer metric that can be cancelled"""
 
 
-class DummyTimer:
-    """No-op timer"""
+class Timer:
+    """
+    Timer that records duration, and optional sends to statsd backend.
+
+    This class lets us have an accurate timer with the logic in one place (so
+    that we don't use datetime math for duration -- it is error prone).
+
+    Example usage:
+
+    .. code-block:: python
+
+        with Stats.timer() as t:
+            # Something to time
+            frob_the_foos()
+
+        log.info("Frobbing the foos took %.2f", t.duration)
+
+    To send a metric:
+
+    .. code-block:: python
+
+        with Stats.timer("foos.frob"):
+            # Something to time
+            frob_the_foos()
+
+
+    Or both:
+
+    .. code-block:: python
+
+        with Stats.timer("foos.frob") as t:
+            # Something to time
+            frob_the_foos()
+
+        log.info("Frobbing the foos took %.2f", t.duration)
+
+    """
+
+    # pystatsd and dogstatsd both have a timer class, but present different API
+    # so we can't use this as a mixin on those, instead this class is contains the "real" timer
+
+    _start_time: Optional[int]
+    duration: Optional[int]
+    ms: Optional[int]
+    _send: bool = False

Review comment:
       Done.




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

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



[GitHub] [airflow] ashb edited a comment on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   > Should we add note about this approach in https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#coding-style-and-best-practices ?
   
   Done in a7642459d


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

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



[GitHub] [airflow] ashb commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   That failure was an actual bug -- I had broken rescheduled sensors. Yay for tests!


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

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



[GitHub] [airflow] turbaszek commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   Should we add note about this approach in https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#coding-style-and-best-practices ?


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/379361133) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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



##########
File path: CONTRIBUTING.rst
##########
@@ -806,6 +806,45 @@ If this function is designed to be called by "end-users" (i.e. DAG authors) then
       ...
       # You SHOULD not commit the session here. The wrapper will take care of commit()/rollback() if exception
 
+Don't use time() for duration calcuations
+-----------------------------------------
+
+If you wish to compute the time difference between two events with in the same process, use
+``time.monotonic()``, not ``time.time()`` nor ``timzeone.utcnow()``.
+
+If you are measuring duration for performance reasons, then ``time.perf_counter()`` should be used. (On many
+platforms, this uses the same underlying clock mechanism as monotonic, but ``perf_counter`` is guaranteed to be
+the highest accuracy clock on the system, monotonic is simply "guaranteed" to not go backwards.)
+
+If you wish to time how long a block of code takes, use ``Stats.timer()`` -- either with a metric name, which
+will be timed and submitted automatically:
+
+.. code-block:: python
+
+    from airflow.stats import Stats
+
+    ...
+
+    with Stats.timer("my_timer_metric"):
+        ...
+
+or without, to access the ``duration`` property:
+
+.. code-block:: python
+
+    from airflow.stats import Stats
+
+    ...
+
+    with Stats.timer("my_timer_metric") as timer:

Review comment:
       ```suggestion
   or without, to access the ``duration`` property:
   
   .. code-block:: python
   
       from airflow.stats import Stats
   
       ...
   
       with Stats.timer() as timer:
   ```
   
   To comply with "or without..."?




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

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



[GitHub] [airflow] ashb commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   Hmmmm, even with rebasing Kube tests are failing. I wonder what I broke.


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

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



[GitHub] [airflow] ashb commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   > Should we add note about this approach in https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#coding-style-and-best-practices ?
   
   Done


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/383221944) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/379359903) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider packages,^Checks: Helm tests$,^Test OpenAPI*.


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

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



[GitHub] [airflow] turbaszek commented on pull request #12353: Don't use time.time() or timezone.utcnow() for duration calculations

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


   @ashb what exactly was the problem with the tests?


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

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