|  | @@ -11,39 +11,16 @@ from __future__ import absolute_import
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  from collections import deque
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -from celery._state import get_current_worker_task
 | 
	
		
			
				|  |  | +from celery._state import get_current_worker_task, connect_on_app_finalize
 | 
	
		
			
				|  |  |  from celery.utils import uuid
 | 
	
		
			
				|  |  |  from celery.utils.log import get_logger
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -__all__ = ['shared_task', 'load_shared_tasks']
 | 
	
		
			
				|  |  | +__all__ = []
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  logger = get_logger(__name__)
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -#: global list of functions defining tasks that should be
 | 
	
		
			
				|  |  | -#: added to all apps.
 | 
	
		
			
				|  |  | -_shared_tasks = set()
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -def shared_task(constructor):
 | 
	
		
			
				|  |  | -    """Decorator that specifies a function that generates a built-in task.
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    The function will then be called for every new app instance created
 | 
	
		
			
				|  |  | -    (lazily, so more exactly when the task registry for that app is needed).
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -    The function must take a single ``app`` argument.
 | 
	
		
			
				|  |  | -    """
 | 
	
		
			
				|  |  | -    _shared_tasks.add(constructor)
 | 
	
		
			
				|  |  | -    return constructor
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -def load_shared_tasks(app):
 | 
	
		
			
				|  |  | -    """Create built-in tasks for an app instance."""
 | 
	
		
			
				|  |  | -    constructors = set(_shared_tasks)
 | 
	
		
			
				|  |  | -    for constructor in constructors:
 | 
	
		
			
				|  |  | -        constructor(app)
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -
 | 
	
		
			
				|  |  | -@shared_task
 | 
	
		
			
				|  |  | +@connect_on_app_finalize
 | 
	
		
			
				|  |  |  def add_backend_cleanup_task(app):
 | 
	
		
			
				|  |  |      """The backend cleanup task can be used to clean up the default result
 | 
	
		
			
				|  |  |      backend.
 | 
	
	
		
			
				|  | @@ -60,7 +37,7 @@ def add_backend_cleanup_task(app):
 | 
	
		
			
				|  |  |      return backend_cleanup
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -@shared_task
 | 
	
		
			
				|  |  | +@connect_on_app_finalize
 | 
	
		
			
				|  |  |  def add_unlock_chord_task(app):
 | 
	
		
			
				|  |  |      """This task is used by result backends without native chord support.
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -127,7 +104,7 @@ def add_unlock_chord_task(app):
 | 
	
		
			
				|  |  |      return unlock_chord
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -@shared_task
 | 
	
		
			
				|  |  | +@connect_on_app_finalize
 | 
	
		
			
				|  |  |  def add_map_task(app):
 | 
	
		
			
				|  |  |      from celery.canvas import signature
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -138,7 +115,7 @@ def add_map_task(app):
 | 
	
		
			
				|  |  |      return xmap
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -@shared_task
 | 
	
		
			
				|  |  | +@connect_on_app_finalize
 | 
	
		
			
				|  |  |  def add_starmap_task(app):
 | 
	
		
			
				|  |  |      from celery.canvas import signature
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -149,7 +126,7 @@ def add_starmap_task(app):
 | 
	
		
			
				|  |  |      return xstarmap
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -@shared_task
 | 
	
		
			
				|  |  | +@connect_on_app_finalize
 | 
	
		
			
				|  |  |  def add_chunk_task(app):
 | 
	
		
			
				|  |  |      from celery.canvas import chunks as _chunks
 | 
	
		
			
				|  |  |  
 | 
	
	
		
			
				|  | @@ -159,7 +136,7 @@ def add_chunk_task(app):
 | 
	
		
			
				|  |  |      return chunks
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -@shared_task
 | 
	
		
			
				|  |  | +@connect_on_app_finalize
 | 
	
		
			
				|  |  |  def add_group_task(app):
 | 
	
		
			
				|  |  |      _app = app
 | 
	
		
			
				|  |  |      from celery.canvas import maybe_signature, signature
 | 
	
	
		
			
				|  | @@ -226,7 +203,7 @@ def add_group_task(app):
 | 
	
		
			
				|  |  |      return Group
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -@shared_task
 | 
	
		
			
				|  |  | +@connect_on_app_finalize
 | 
	
		
			
				|  |  |  def add_chain_task(app):
 | 
	
		
			
				|  |  |      from celery.canvas import (
 | 
	
		
			
				|  |  |          Signature, chain, chord, group, maybe_signature, maybe_unroll_group,
 | 
	
	
		
			
				|  | @@ -322,7 +299,7 @@ def add_chain_task(app):
 | 
	
		
			
				|  |  |      return Chain
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  |  
 | 
	
		
			
				|  |  | -@shared_task
 | 
	
		
			
				|  |  | +@connect_on_app_finalize
 | 
	
		
			
				|  |  |  def add_chord_task(app):
 | 
	
		
			
				|  |  |      """Every chord is executed in a dedicated task, so that the chord
 | 
	
		
			
				|  |  |      can be used as a signature, and this generates the task
 |