You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/12/17 23:52:01 UTC

[jira] [Commented] (AIRFLOW-1491) Celery executor restarts result in duplicate tasks

    [ https://issues.apache.org/jira/browse/AIRFLOW-1491?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16723486#comment-16723486 ] 

ASF GitHub Bot commented on AIRFLOW-1491:
-----------------------------------------

stale[bot] closed pull request #2503: AIRFLOW-1491 Check celery queue before scheduling commands
URL: https://github.com/apache/incubator-airflow/pull/2503
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py
index 7a4065eb07..8a3a71ad46 100644
--- a/airflow/executors/base_executor.py
+++ b/airflow/executors/base_executor.py
@@ -132,7 +132,7 @@ def heartbeat(self):
         self.sync()
 
     def change_state(self, key, state):
-        self.running.pop(key)
+        self.running.pop(key, None)
         self.event_buffer[key] = state
 
     def fail(self, key):
diff --git a/airflow/executors/celery_executor.py b/airflow/executors/celery_executor.py
index 17c343bd4a..5654685815 100644
--- a/airflow/executors/celery_executor.py
+++ b/airflow/executors/celery_executor.py
@@ -13,6 +13,7 @@
 # limitations under the License.
 
 from builtins import object
+import itertools
 import logging
 import subprocess
 import ssl
@@ -20,6 +21,7 @@
 import traceback
 
 from celery import Celery
+from celery.result import AsyncResult
 from celery import states as celery_states
 
 from airflow.exceptions import AirflowConfigException, AirflowException
@@ -83,6 +85,26 @@ def execute_command(command):
         raise AirflowException('Celery command failed')
 
 
+def recover_command(command):
+    """Recover the celery AsyncResult object for a command, if possible.
+
+    Note:
+        - We use iterables and generators to minimize backend calls.
+        - Functionality is dependent on features presented by the celery
+          backend in use.
+    """
+    insp = app.control.inspect()
+    task_providers = [insp.active, insp.scheduled, insp.reserved]
+    task_lists = (
+        itertools.chain.from_iterable(tp().values())
+        for tp in task_providers)
+    tasks = itertools.chain.from_iterable(task_lists)
+    for task in tasks:
+        if task['args'] == [command]:
+            return AsyncResult(id=task['id'], app=app)
+    return None
+
+
 class CeleryExecutor(BaseExecutor):
     """
     CeleryExecutor is recommended for production use of Airflow. It allows
@@ -98,17 +120,29 @@ def start(self):
         self.last_state = {}
 
     def execute_async(self, key, command, queue=DEFAULT_QUEUE):
-        self.logger.info( "[celery] queuing {key} through celery, "
-                       "queue={queue}".format(**locals()))
-        self.tasks[key] = execute_command.apply_async(
-            args=[command], queue=queue)
+        if key in self.tasks:
+            self.logger.warning('[celery] existing command scheduled '
+                                'for {key}'.format(key=key))
+            return
+
+        async_command = recover_command(command)
+        if async_command is not None:
+            self.logger.warning('[celery] recovering already scheduled '
+                                'command {command}'.format(command=command))
+        else:
+            self.logger.info('[celery] queuing {key} through celery, '
+                             'queue={queue}'.format(key=key, queue=queue))
+            async_command = execute_command.apply_async(
+                args=[command], queue=queue)
+
+        self.tasks[key] = async_command
         self.last_state[key] = celery_states.PENDING
 
     def sync(self):
 
         self.logger.debug(
             "Inquiring about {} celery task(s)".format(len(self.tasks)))
-        for key, async in list(self.tasks.items()):
+        for key, async in self.tasks.items():
             try:
                 state = async.state
                 if self.last_state[key] != state:
@@ -129,7 +163,7 @@ def sync(self):
                     self.last_state[key] = async.state
             except Exception as e:
                 logging.error("Error syncing the celery executor, ignoring "
-                              "it:\n{}\n".format(e, traceback.format_exc()))
+                              "it:\n{}\n{}".format(e, traceback.format_exc()))
 
     def end(self, synchronous=False):
         if synchronous:
diff --git a/tests/executors/celery_executor.py b/tests/executors/celery_executor.py
new file mode 100644
index 0000000000..61c62eabd8
--- /dev/null
+++ b/tests/executors/celery_executor.py
@@ -0,0 +1,143 @@
+# -*- coding: utf-8 -*-
+#
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+import unittest
+
+from celery import states as cstates
+
+from airflow.executors.celery_executor import CeleryExecutor
+
+try:
+    from unittest import mock
+except ImportError:
+    try:
+        import mock
+    except ImportError:
+        mock = None
+
+
+class CeleryExecutorTest(unittest.TestCase):
+    """Unit tests for the CeleryExecutor."""
+
+    @unittest.skipIf(mock is None, 'mock package not present')
+    @mock.patch('airflow.executors.celery_executor.time')
+    @mock.patch('airflow.executors.celery_executor.execute_command')
+    @mock.patch('airflow.executors.celery_executor.app')
+    def test_executor(self, app, execute_command, time):
+        insp = mock.MagicMock(name='inspector')
+        insp.active.return_value = {}
+        insp.scheduled.return_value = {}
+        insp.reserved.return_value = {}
+        app.control.inspect.return_value = insp
+
+        async1 = mock.MagicMock(name='async1')
+        async1.state = cstates.PENDING
+        async2 = mock.MagicMock(name='async2')
+        async2.state = cstates.FAILURE
+        execute_command.apply_async.side_effect = [async1, async2]
+
+        executor = CeleryExecutor()
+        executor.start()
+
+        executor.execute_async(key='one', command='echo 1')
+        executor.execute_async(key='two', command='echo 2', queue='other')
+
+        self.assertEquals(executor.tasks, {'one': async1, 'two': async2})
+        self.assertEquals(executor.last_state,
+                          {'one': cstates.PENDING, 'two': cstates.PENDING})
+
+        self.assertEquals(execute_command.mock_calls, [
+            mock.call.apply_async(args=['echo 1'], queue='default'),
+            mock.call.apply_async(args=['echo 2'], queue='other'),
+        ])
+        self.assertFalse(async1.mock_calls)
+        self.assertFalse(async2.mock_calls)
+        self.assertFalse(time.mock_calls)
+
+        executor.sync()
+
+        self.assertFalse(async1.mock_calls)
+        self.assertFalse(async2.mock_calls)
+
+        self.assertEquals(executor.tasks, {'one': async1})
+        self.assertEquals(executor.last_state,
+                          {'one': cstates.PENDING, 'two': cstates.FAILURE})
+
+    @unittest.skipIf(mock is None, 'mock package not present')
+    @mock.patch('airflow.executors.celery_executor.execute_command')
+    @mock.patch('airflow.executors.celery_executor.app')
+    def test_repeat_key(self, app, execute_command):
+        insp = mock.MagicMock(name='inspector')
+        insp.active.return_value = {}
+        insp.scheduled.return_value = {}
+        insp.reserved.return_value = {}
+        app.control.inspect.return_value = insp
+
+        executor = CeleryExecutor()
+        executor.start()
+
+        executor.execute_async(key='one', command='echo 1')
+        executor.execute_async(key='two', command='echo 2', queue='other')
+        executor.execute_async(key='one', command='echo 3', queue='other')
+
+        self.assertEquals(execute_command.mock_calls, [
+            mock.call.apply_async(args=['echo 1'], queue='default'),
+            mock.call.apply_async(args=['echo 2'], queue='other'),
+        ])
+
+    @unittest.skipIf(mock is None, 'mock package not present')
+    @mock.patch('airflow.executors.celery_executor.AsyncResult')
+    @mock.patch('airflow.executors.celery_executor.execute_command')
+    @mock.patch('airflow.executors.celery_executor.app')
+    def test_existing_command(self, app, execute_command, result):
+        insp = mock.MagicMock(name='inspector')
+        insp.active.return_value = {
+            'worker1': [{
+                'id': 'id1',
+                'args': ['echo 1']}],
+        }
+        insp.scheduled.return_value = {
+            'worker2': [{
+                'id': 'id2',
+                'args': ['echo 2']}],
+        }
+        insp.reserved.return_value = {
+            'worker3': [{
+                'id': 'id3',
+                'args': ['echo 3']}],
+        }
+        app.control.inspect.return_value = insp
+
+        executor = CeleryExecutor()
+        executor.start()
+
+        executor.execute_async(key='one', command='echo 1')
+        executor.execute_async(key='two', command='echo 2', queue='other')
+        executor.execute_async(key='three', command='echo 3', queue='other')
+        executor.execute_async(key='four', command='echo 4')
+
+        self.assertEquals(execute_command.mock_calls, [
+            mock.call.apply_async(args=['echo 4'], queue='default'),
+        ])
+        self.assertEquals(result.mock_calls, [
+            mock.call(id='id1', app=app),
+            mock.call(id='id2', app=app),
+            mock.call(id='id3', app=app),
+        ])
+        self.assertEquals(executor.tasks, {
+            'one': result.return_value,
+            'two': result.return_value,
+            'three': result.return_value,
+            'four': execute_command.apply_async.return_value,
+        })


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


> Celery executor restarts result in duplicate tasks
> --------------------------------------------------
>
>                 Key: AIRFLOW-1491
>                 URL: https://issues.apache.org/jira/browse/AIRFLOW-1491
>             Project: Apache Airflow
>          Issue Type: Bug
>            Reporter: George Leslie-Waksman
>            Assignee: George Leslie-Waksman
>            Priority: Major
>
> Since the Celery executor uses internal state to determine what it has queued already, anything that wipes the executor state (such as a scheduler restart) will result in rescheduling of tasks.
> This bug exacerbates a number of task scheduling race conditions and can result in duplicate task execution.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)