Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Handle partially updated trigger/task_instance data in triggerr jobs #32092

Merged

Conversation

tomrutter
Copy link
Contributor

@tomrutter tomrutter commented Jun 23, 2023

closes: #32091

Updates to triggerer job to improve robustness.

Context:
Triggerer runs two threads, one synchronous and one asynchronous. The synchronous thread handles db updates, and the asynchronous thread handles custom trigger code. The heartbeat update for the job happens in the synchronous thread.

Details (two updates):

  1. Added a check for missing associated task_instance when adding a trigger. This prevents a crash in the triggerer async thread when updating with partially updated data. The trigger concerned is ignored, and will later deleted by Trigger.clean_unused() in the synchronous thread.

  2. Added a check that the async thread is still running before updating triggerer heartbeat. This results in the triggerer being reported as unhealthy (and subsequently restarted) when the async thread crashes.


^ Add meaningful description above

Read the Pull Request Guidelines for more information.
In case of fundamental code changes, an Airflow Improvement Proposal (AIP) is needed.
In case of a new dependency, check compliance with the ASF 3rd Party License Policy.
In case of backwards incompatible changes please leave a note in a newsfragment file, named {pr_number}.significant.rst or {issue_number}.significant.rst, in newsfragments.

@tomrutter tomrutter requested review from kaxil, ashb and XD-DENG as code owners June 23, 2023 11:16
@boring-cyborg boring-cyborg bot added the area:Scheduler including HA (high availability) scheduler label Jun 23, 2023
@boring-cyborg
Copy link

boring-cyborg bot commented Jun 23, 2023

Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst)
Here are some useful points:

  • Pay attention to the quality of your code (ruff, mypy and type annotations). Our pre-commits will help you with that.
  • In case of a new feature add useful documentation (in docstrings or in docs/ directory). Adding a new operator? Check this short guide Consider adding an example DAG that shows how users should use it.
  • Consider using Breeze environment for testing locally, it's a heavy docker but it ships with a working Airflow and a lot of integrations.
  • Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
  • Please follow ASF Code of Conduct for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
  • Be sure to read the Airflow Coding style.
    Apache Airflow is a community-driven project and together we are making it better 🚀.
    In case of doubts contact the developers at:
    Mailing List: [email protected]
    Slack: https://s.apache.org/airflow-slack

Copy link
Member

@hussein-awala hussein-awala left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you add new unit tests for these two changes?

Copy link
Member

@hussein-awala hussein-awala left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For the case where the task_instance is None, I think this could be fixed/improved in a separate PR, and for the blocked triggers, I think we don't need to continue the loop if the thread is dead (regardless the reason), and also we can add an try/catch to stop the Triggerer when it fails:

diff --git a/airflow/jobs/triggerer_job_runner.py b/airflow/jobs/triggerer_job_runner.py
index 79b5e953dc..7b2670340c 100644
--- a/airflow/jobs/triggerer_job_runner.py
+++ b/airflow/jobs/triggerer_job_runner.py
@@ -351,6 +351,9 @@ class TriggererJobRunner(BaseJobRunner["Job | JobPydantic"], LoggingMixin):
         This runs synchronously and handles all database reads/writes.
         """
         while not self.trigger_runner.stop:
+            if not self.trigger_runner.is_alive():
+                self.log.error("Trigger runner thread has died! Exiting.")
+                break
             # Clean out unused triggers
             Trigger.clean_unused()
             # Load/delete triggers
@@ -467,17 +470,21 @@ class TriggerRunner(threading.Thread, LoggingMixin):
         watchdog = asyncio.create_task(self.block_watchdog())
         last_status = time.time()
         while not self.stop:
-            # Run core logic
-            await self.create_triggers()
-            await self.cancel_triggers()
-            await self.cleanup_finished_triggers()
-            # Sleep for a bit
-            await asyncio.sleep(1)
-            # Every minute, log status
-            if time.time() - last_status >= 60:
-                count = len(self.triggers)
-                self.log.info("%i triggers currently running", count)
-                last_status = time.time()
+            try:
+                # Run core logic
+                await self.create_triggers()
+                await self.cancel_triggers()
+                await self.cleanup_finished_triggers()
+                # Sleep for a bit
+                await asyncio.sleep(1)
+                # Every minute, log status
+                if time.time() - last_status >= 60:
+                    count = len(self.triggers)
+                    self.log.info("%i triggers currently running", count)
+                    last_status = time.time()
+            except Exception:
+                self.stop = True
+                raise
         # Wait for watchdog to complete
         await watchdog

WDYT?

@tomrutter
Copy link
Contributor Author

LGTM. A more complete approach to any such errors. Only change I would suggest would be to log any exceptions raised in async thread. Either in the async thread try block, or should we perhaps await the async thread after it dies to pick up any exceptions re-raised?

@tomrutter
Copy link
Contributor Author

Ah, sorry. I guess they're logged when the async thread dies. All good.

@tomrutter tomrutter force-pushed the feature/triggerer_thread_robustness branch from 17fe372 to 038e19f Compare June 26, 2023 08:27
@tomrutter tomrutter force-pushed the feature/triggerer_thread_robustness branch from 038e19f to 39296b7 Compare June 26, 2023 11:12
@tomrutter
Copy link
Contributor Author

I've updated the PR as per suggestions, added a test, and rebased to main. Could I request a review? Thanks!

Copy link
Member

@potiuk potiuk left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM . @hussein-awala ?

Copy link
Member

@hussein-awala hussein-awala left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

@hussein-awala hussein-awala merged commit e585b58 into apache:main Jun 27, 2023
@boring-cyborg
Copy link

boring-cyborg bot commented Jun 27, 2023

Awesome work, congrats on your first merged pull request! You are invited to check our Issue Tracker for additional contributions.

@hussein-awala
Copy link
Member

@tomrutter Congrats on your first merged PR!

@ephraimbuddy ephraimbuddy added the type:bug-fix Changelog: Bug Fixes label Jul 6, 2023
@ephraimbuddy ephraimbuddy added this to the Airlfow 2.6.3 milestone Jul 6, 2023
ephraimbuddy pushed a commit that referenced this pull request Jul 6, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area:Scheduler including HA (high availability) scheduler type:bug-fix Changelog: Bug Fixes
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Triggerer intermittent failure when running many triggerers
4 participants