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

[TWTR][CX-17516] Requeue tasks in the queued state #27

Merged
merged 1 commit into from
Dec 18, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions airflow/executors/base_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -54,9 +54,9 @@ def queue_command(self, task_instance, command, priority=1, queue=None):
key = task_instance.key
if key not in self.queued_tasks and key not in self.running:
self.log.info("Adding to queue: %s", command)
self.queued_tasks[key] = (command, priority, queue, task_instance)
else:
self.log.info("could not queue task {}".format(key))
self.log.info("Adding to queue even though already queued or running {}".format(command, key))
self.queued_tasks[key] = (command, priority, queue, task_instance)
Copy link

Choose a reason for hiding this comment

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

Does this make any effect? if queued_task has the key, you are just effectively refreshing the value, not sure about the motivation for this change.

Copy link
Author

Choose a reason for hiding this comment

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

The reason I did this is in case the task was in the running state. It doesn't hurt even for the queued state becase task instance/etc might have changed, and I wanted to minimize the size of the fork (i.e. could have broken out the if state == running then queue the task into a separate elif).


def queue_task_instance(
self,
Expand Down
7 changes: 6 additions & 1 deletion airflow/jobs.py
Original file line number Diff line number Diff line change
Expand Up @@ -1728,8 +1728,13 @@ def _execute_helper(self, processor_manager):
scheduled_dag_ids = ", ".join(simple_dag_bag.dag_ids)
self.log.info('DAGs to be executed: {}'.format(scheduled_dag_ids))

# TODO(CX-17516): State.QUEUED has been added here which is a hack as the Celery
# Executor does not reliably enqueue tasks with the my MySQL broker, and we have
# seen tasks hang after they get queued. The effect of this hack is queued tasks
# will constantly be requeued and resent to the executor (Celery).
# This should be removed when we switch away from the MySQL Celery backend.
self._execute_task_instances(simple_dag_bag,
(State.SCHEDULED,))
(State.SCHEDULED, State.QUEUED))

# Call heartbeats
self.log.debug("Heartbeating the executor")
Expand Down
2 changes: 1 addition & 1 deletion airflow/version.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,4 +18,4 @@
# under the License.
#

version = '1.10.0+twtr24'
version = '1.10.0+twtr25'