Removes TASK_ERROR_EMAILS/mail_admins
This commit is contained in:
Родитель
c4de6da107
Коммит
b5d8c1cebd
|
@ -808,24 +808,6 @@ class Celery(object):
|
|||
:class:`~datetime.datetime` object."""
|
||||
return self.loader.now(utc=self.conf.enable_utc)
|
||||
|
||||
def mail_admins(self, subject, body, fail_silently=False):
|
||||
"""Sends an email to the admins in the :setting:`admins` setting."""
|
||||
conf = self.conf
|
||||
if conf.admins:
|
||||
to = [admin_email for _, admin_email in conf.admins]
|
||||
return self.loader.mail_admins(
|
||||
subject, body, fail_silently, to=to,
|
||||
sender=conf.server_email,
|
||||
host=conf.email_host,
|
||||
port=conf.email_port,
|
||||
user=conf.email_host_user,
|
||||
password=conf.email_host_password,
|
||||
timeout=conf.email_timeout,
|
||||
use_ssl=conf.email_use_ssl,
|
||||
use_tls=conf.email_use_tls,
|
||||
charset=conf.email_charset,
|
||||
)
|
||||
|
||||
def select_queues(self, queues=None):
|
||||
"""Select a subset of queues, where queues must be a list of queue
|
||||
names to keep."""
|
||||
|
|
|
@ -85,7 +85,6 @@ NAMESPACES = Namespace(
|
|||
enable_utc=Option(True, type='bool'),
|
||||
imports=Option((), type='tuple', old=OLD_NS),
|
||||
include=Option((), type='tuple', old=OLD_NS),
|
||||
server_email=Option('celery@localhost'),
|
||||
timezone=Option(type='string', old=OLD_NS),
|
||||
beat=Namespace(
|
||||
__old__=OLD_NS_BEAT,
|
||||
|
@ -140,16 +139,6 @@ NAMESPACES = Namespace(
|
|||
|
||||
backend_settings=Option(None, type='dict'),
|
||||
),
|
||||
email=Namespace(
|
||||
charset=Option('utf-8'),
|
||||
host=Option('localhost'),
|
||||
host_user=Option(),
|
||||
host_password=Option(),
|
||||
port=Option(25, type='int'),
|
||||
timeout=Option(2, type='float'),
|
||||
use_ssl=Option(False, type='bool'),
|
||||
use_tls=Option(False, type='bool'),
|
||||
),
|
||||
mongodb=Namespace(
|
||||
__old__=old_ns('celery_mongodb'),
|
||||
|
||||
|
@ -247,9 +236,6 @@ NAMESPACES = Namespace(
|
|||
reject_on_worker_lost=Option(type='bool'),
|
||||
remote_tracebacks=Option(False, type='bool'),
|
||||
routes=Option(type='any'),
|
||||
send_error_emails=Option(
|
||||
False, type='bool', old={'celery_send_task_error_emails'},
|
||||
),
|
||||
send_sent_event=Option(
|
||||
False, type='bool', old={'celery_send_task_sent_event'},
|
||||
),
|
||||
|
|
|
@ -23,7 +23,6 @@ from celery.utils import abstract
|
|||
from celery.utils import uuid, maybe_reraise
|
||||
from celery.utils.functional import mattrgetter, maybe_list
|
||||
from celery.utils.imports import instantiate
|
||||
from celery.utils.mail import ErrorMail
|
||||
|
||||
from .annotations import resolve_all as resolve_all_annotations
|
||||
from .registry import _unpickle_task_v2
|
||||
|
@ -133,7 +132,6 @@ class Task(object):
|
|||
__trace__ = None
|
||||
__v2_compat__ = False # set by old base in celery.task.base
|
||||
|
||||
ErrorMail = ErrorMail
|
||||
MaxRetriesExceededError = MaxRetriesExceededError
|
||||
|
||||
#: Execution strategy used, or the qualified name of one.
|
||||
|
@ -175,10 +173,6 @@ class Task(object):
|
|||
#: configured to ignore results.
|
||||
store_errors_even_if_ignored = None
|
||||
|
||||
#: If enabled an email will be sent to :setting:`admins` whenever a task
|
||||
#: of this type fails.
|
||||
send_error_emails = None
|
||||
|
||||
#: The name of a serializer that are registered with
|
||||
#: :mod:`kombu.serialization.registry`. Default is `'pickle'`.
|
||||
serializer = None
|
||||
|
@ -263,7 +257,6 @@ class Task(object):
|
|||
__bound__ = False
|
||||
|
||||
from_config = (
|
||||
('send_error_emails', 'task_send_error_emails'),
|
||||
('serializer', 'task_serializer'),
|
||||
('rate_limit', 'task_default_rate_limit'),
|
||||
('track_started', 'task_track_started'),
|
||||
|
@ -904,11 +897,6 @@ class Task(object):
|
|||
"""
|
||||
pass
|
||||
|
||||
def send_error_email(self, context, exc, **kwargs):
|
||||
if self.send_error_emails and \
|
||||
not getattr(self, 'disable_error_emails', None):
|
||||
self.ErrorMail(self, **kwargs).send(context, exc)
|
||||
|
||||
def add_trail(self, result):
|
||||
if self.trail:
|
||||
self.request.children.append(result)
|
||||
|
|
|
@ -236,9 +236,6 @@ class TraceInfo(object):
|
|||
exc_info=exc_info if policy.traceback else None,
|
||||
extra={'data': context})
|
||||
|
||||
if policy.mail:
|
||||
task.send_error_email(context, einfo.exception)
|
||||
|
||||
|
||||
def build_tracer(name, task, loader=None, hostname=None, store_errors=True,
|
||||
Info=TraceInfo, eager=False, propagate=False, app=None,
|
||||
|
|
|
@ -60,7 +60,6 @@ class DjangoFixup(object):
|
|||
|
||||
self._settings = symbol_by_name('django.conf:settings')
|
||||
self.app.loader.now = self.now
|
||||
self.app.loader.mail_admins = self.mail_admins
|
||||
|
||||
signals.import_modules.connect(self.on_import_modules)
|
||||
signals.worker_init.connect(self.on_worker_init)
|
||||
|
@ -86,9 +85,6 @@ class DjangoFixup(object):
|
|||
def now(self, utc=False):
|
||||
return datetime.utcnow() if utc else self._now()
|
||||
|
||||
def mail_admins(self, subject, body, fail_silently=False, **kwargs):
|
||||
return self._mail_admins(subject, body, fail_silently=fail_silently)
|
||||
|
||||
def autodiscover_tasks(self):
|
||||
try:
|
||||
from django.apps import apps
|
||||
|
@ -97,10 +93,6 @@ class DjangoFixup(object):
|
|||
else:
|
||||
return [config.name for config in apps.get_app_configs()]
|
||||
|
||||
@cached_property
|
||||
def _mail_admins(self):
|
||||
return symbol_by_name('django.core.mail:mail_admins')
|
||||
|
||||
@cached_property
|
||||
def _now(self):
|
||||
try:
|
||||
|
|
|
@ -223,20 +223,6 @@ class BaseLoader(object):
|
|||
return ns_key, value
|
||||
return dict(getarg(arg) for arg in args)
|
||||
|
||||
def mail_admins(self, subject, body, fail_silently=False,
|
||||
sender=None, to=None, host=None, port=None,
|
||||
user=None, password=None, timeout=None,
|
||||
use_ssl=False, use_tls=False, charset='us-ascii'):
|
||||
message = self.mail.Message(sender=sender, to=to,
|
||||
subject=safe_str(subject),
|
||||
body=safe_str(body),
|
||||
charset=charset)
|
||||
mailer = self.mail.Mailer(host=host, port=port,
|
||||
user=user, password=password,
|
||||
timeout=timeout, use_ssl=use_ssl,
|
||||
use_tls=use_tls)
|
||||
mailer.send(message, fail_silently=fail_silently)
|
||||
|
||||
def read_configuration(self, env='CELERY_CONFIG_MODULE'):
|
||||
try:
|
||||
custom_config = os.environ[env]
|
||||
|
@ -259,10 +245,6 @@ class BaseLoader(object):
|
|||
self._conf = self.read_configuration()
|
||||
return self._conf
|
||||
|
||||
@cached_property
|
||||
def mail(self):
|
||||
return self.import_module('celery.utils.mail')
|
||||
|
||||
|
||||
def autodiscover_tasks(packages, related_name='tasks'):
|
||||
global _RACE_PROTECTION
|
||||
|
|
|
@ -150,7 +150,6 @@ class Task(BaseTask):
|
|||
immediate = False # XXX deprecated
|
||||
priority = None
|
||||
type = 'regular'
|
||||
disable_error_emails = False
|
||||
|
||||
from_config = BaseTask.from_config + (
|
||||
('exchange_type', 'task_default_exchange_type'),
|
||||
|
|
|
@ -33,7 +33,6 @@ from celery.tests.case import (
|
|||
patch,
|
||||
)
|
||||
from celery.utils import uuid
|
||||
from celery.utils.mail import ErrorMail
|
||||
from celery.utils.objects import Bunch
|
||||
|
||||
THIS_IS_A_KEY = 'this is a value'
|
||||
|
@ -708,19 +707,6 @@ class test_App(AppCase):
|
|||
self.app.start()
|
||||
execute.assert_called()
|
||||
|
||||
def test_mail_admins(self):
|
||||
|
||||
class Loader(BaseLoader):
|
||||
|
||||
def mail_admins(*args, **kwargs):
|
||||
return args, kwargs
|
||||
|
||||
self.app.loader = Loader(app=self.app)
|
||||
self.app.conf.admins = None
|
||||
self.assertFalse(self.app.mail_admins('Subject', 'Body'))
|
||||
self.app.conf.admins = [('George Costanza', 'george@vandelay.com')]
|
||||
self.assertTrue(self.app.mail_admins('Subject', 'Body'))
|
||||
|
||||
def test_amqp_get_broker_info(self):
|
||||
self.assertDictContainsSubset(
|
||||
{'hostname': 'localhost',
|
||||
|
@ -901,21 +887,6 @@ class test_App(AppCase):
|
|||
exchange='bar_exchange', routing_key='bar_exchange',
|
||||
)
|
||||
|
||||
def test_error_mail_sender(self):
|
||||
x = ErrorMail.subject % {'name': 'task_name',
|
||||
'id': uuid(),
|
||||
'exc': 'FOOBARBAZ',
|
||||
'hostname': 'lana'}
|
||||
self.assertTrue(x)
|
||||
|
||||
def test_error_mail_disabled(self):
|
||||
task = Mock()
|
||||
x = ErrorMail(task)
|
||||
x.should_send = Mock()
|
||||
x.should_send.return_value = False
|
||||
x.send(Mock(), Mock())
|
||||
task.app.mail_admins.assert_not_called()
|
||||
|
||||
def test_select_queues(self):
|
||||
self.app.amqp = Mock(name='amqp')
|
||||
self.app.select_queues({'foo', 'bar'})
|
||||
|
|
|
@ -61,7 +61,6 @@ class test_defaults(AppCase):
|
|||
def test_find(self):
|
||||
find = self.defaults.find
|
||||
|
||||
self.assertEqual(find('server_email')[2].default, 'celery@localhost')
|
||||
self.assertEqual(find('default_queue')[2].default, 'celery')
|
||||
self.assertEqual(find('task_default_exchange')[2], 'celery')
|
||||
|
||||
|
|
|
@ -11,7 +11,6 @@ from celery.loaders import base
|
|||
from celery.loaders import default
|
||||
from celery.loaders.app import AppLoader
|
||||
from celery.utils.imports import NotAPackage
|
||||
from celery.utils.mail import SendmailWarning
|
||||
|
||||
from celery.tests.case import AppCase, Case, Mock, mock, patch
|
||||
|
||||
|
@ -98,33 +97,6 @@ class test_LoaderBase(AppCase):
|
|||
self.loader.import_from_cwd('foo', imp=imp)
|
||||
imp.assert_called()
|
||||
|
||||
@patch('celery.utils.mail.Mailer._send')
|
||||
def test_mail_admins_errors(self, send):
|
||||
send.side_effect = KeyError()
|
||||
opts = dict(self.message_options, **self.server_options)
|
||||
|
||||
with self.assertWarnsRegex(SendmailWarning, r'KeyError'):
|
||||
self.loader.mail_admins(fail_silently=True, **opts)
|
||||
|
||||
with self.assertRaises(KeyError):
|
||||
self.loader.mail_admins(fail_silently=False, **opts)
|
||||
|
||||
@patch('celery.utils.mail.Mailer._send')
|
||||
def test_mail_admins(self, send):
|
||||
opts = dict(self.message_options, **self.server_options)
|
||||
self.loader.mail_admins(**opts)
|
||||
self.assertTrue(send.call_args)
|
||||
message = send.call_args[0][0]
|
||||
self.assertEqual(message.to, [self.message_options['to']])
|
||||
self.assertEqual(message.subject, self.message_options['subject'])
|
||||
self.assertEqual(message.sender, self.message_options['sender'])
|
||||
self.assertEqual(message.body, self.message_options['body'])
|
||||
|
||||
def test_mail_attribute(self):
|
||||
from celery.utils import mail
|
||||
loader = base.BaseLoader(app=self.app)
|
||||
self.assertIs(loader.mail, mail)
|
||||
|
||||
def test_cmdline_config_ValueError(self):
|
||||
with self.assertRaises(ValueError):
|
||||
self.loader.cmdline_config_parser(['broker.port=foobar'])
|
||||
|
|
|
@ -57,7 +57,6 @@ CELERY_TEST_CONFIG = {
|
|||
#: Don't want log output when running suite.
|
||||
'worker_hijack_root_logger': False,
|
||||
'worker_log_color': False,
|
||||
'task_send_error_emails': False,
|
||||
'task_default_queue': 'testcelery',
|
||||
'task_default_exchange': 'testcelery',
|
||||
'task_default_routing_key': 'testcelery',
|
||||
|
|
|
@ -110,7 +110,6 @@ class test_DjangoFixup(FixupCase):
|
|||
f.install()
|
||||
self.sigs.worker_init.connect.assert_called_with(f.on_worker_init)
|
||||
self.assertEqual(self.app.loader.now, f.now)
|
||||
self.assertEqual(self.app.loader.mail_admins, f.mail_admins)
|
||||
self.p.append.assert_called_with('/opt/vandelay')
|
||||
|
||||
def test_now(self):
|
||||
|
@ -120,13 +119,6 @@ class test_DjangoFixup(FixupCase):
|
|||
self.assertTrue(f.now(utc=False))
|
||||
f._now.assert_called()
|
||||
|
||||
def test_mail_admins(self):
|
||||
with self.fixup_context(self.app) as (f, _, _):
|
||||
f.mail_admins('sub', 'body', True)
|
||||
f._mail_admins.assert_called_with(
|
||||
'sub', 'body', fail_silently=True,
|
||||
)
|
||||
|
||||
def test_on_worker_init(self):
|
||||
with self.fixup_context(self.app) as (f, _, _):
|
||||
with patch('celery.fixups.django.DjangoWorkerFixup') as DWF:
|
||||
|
|
|
@ -452,17 +452,6 @@ class test_tasks(TasksCase):
|
|||
with self.assertRaises(Ignore):
|
||||
self.mytask.replace(c)
|
||||
|
||||
def test_send_error_email_enabled(self):
|
||||
mytask = self.increment_counter._get_current_object()
|
||||
mytask.send_error_emails = True
|
||||
mytask.disable_error_emails = False
|
||||
mytask.ErrorMail = Mock(name='ErrorMail')
|
||||
context = Mock(name='context')
|
||||
exc = Mock(name='context')
|
||||
mytask.send_error_email(context, exc, foo=1)
|
||||
mytask.ErrorMail.assert_called_with(mytask, foo=1)
|
||||
mytask.ErrorMail().send.assert_called_with(context, exc)
|
||||
|
||||
def test_add_trail__no_trail(self):
|
||||
mytask = self.increment_counter._get_current_object()
|
||||
mytask.trail = False
|
||||
|
|
|
@ -1,83 +0,0 @@
|
|||
from __future__ import absolute_import, unicode_literals
|
||||
|
||||
from celery.utils.mail import Message, Mailer, SSLError, ErrorMail
|
||||
|
||||
from celery.tests.case import Case, Mock, patch
|
||||
|
||||
|
||||
msg = Message(to='george@vandelay.com', sender='elaine@pendant.com',
|
||||
subject="What's up with Jerry?", body='???!')
|
||||
|
||||
|
||||
class test_Message(Case):
|
||||
|
||||
def test_repr(self):
|
||||
self.assertTrue(repr(msg))
|
||||
|
||||
def test_str(self):
|
||||
self.assertTrue(str(msg))
|
||||
|
||||
|
||||
class test_Mailer(Case):
|
||||
|
||||
def test_send_wrapper(self):
|
||||
mailer = Mailer()
|
||||
mailer._send = Mock()
|
||||
mailer.send(msg)
|
||||
mailer._send.assert_called_with(msg)
|
||||
|
||||
@patch('smtplib.SMTP_SSL', create=True)
|
||||
def test_send_ssl_tls(self, SMTP_SSL):
|
||||
mailer = Mailer(use_ssl=True, use_tls=True)
|
||||
client = SMTP_SSL.return_value = Mock()
|
||||
mailer._send(msg)
|
||||
client.starttls.assert_called()
|
||||
self.assertEqual(client.ehlo.call_count, 2)
|
||||
client.quit.assert_called_with()
|
||||
client.sendmail.assert_called_with(msg.sender, msg.to, str(msg))
|
||||
mailer = Mailer(use_ssl=True, use_tls=True, user='foo',
|
||||
password='bar')
|
||||
mailer._send(msg)
|
||||
client.login.assert_called_with('foo', 'bar')
|
||||
|
||||
@patch('smtplib.SMTP')
|
||||
def test_send(self, SMTP):
|
||||
client = SMTP.return_value = Mock()
|
||||
mailer = Mailer(use_ssl=False, use_tls=False)
|
||||
mailer._send(msg)
|
||||
|
||||
client.sendmail.assert_called_with(msg.sender, msg.to, str(msg))
|
||||
|
||||
client.quit.side_effect = SSLError()
|
||||
mailer._send(msg)
|
||||
client.close.assert_called_with()
|
||||
|
||||
|
||||
class test_ErrorMail(Case):
|
||||
|
||||
def setUp(self):
|
||||
self.task = Mock(name='task')
|
||||
self.mailer = ErrorMail(
|
||||
self.task, subject='foo{foo} ', body='bar{bar} ',
|
||||
)
|
||||
|
||||
def test_should_send(self):
|
||||
self.assertTrue(self.mailer.should_send(Mock(), Mock()))
|
||||
|
||||
def test_format_subject(self):
|
||||
self.assertEqual(
|
||||
self.mailer.format_subject({'foo': 'FOO'}),
|
||||
'fooFOO',
|
||||
)
|
||||
|
||||
def test_format_body(self):
|
||||
self.assertEqual(
|
||||
self.mailer.format_body({'bar': 'BAR'}),
|
||||
'barBAR',
|
||||
)
|
||||
|
||||
def test_send(self):
|
||||
self.mailer.send({'foo': 'FOO', 'bar': 'BAR'}, KeyError())
|
||||
self.task.app.mail_admins.assert_called_with(
|
||||
'fooFOO', 'barBAR', fail_silently=True,
|
||||
)
|
|
@ -1,193 +0,0 @@
|
|||
# -*- coding: utf-8 -*-
|
||||
"""
|
||||
celery.utils.mail
|
||||
~~~~~~~~~~~~~~~~~
|
||||
|
||||
How task error emails are formatted and sent.
|
||||
|
||||
"""
|
||||
from __future__ import absolute_import, unicode_literals
|
||||
|
||||
import smtplib
|
||||
import socket
|
||||
import traceback
|
||||
import warnings
|
||||
|
||||
from email.mime.text import MIMEText
|
||||
|
||||
from celery.five import python_2_unicode_compatible
|
||||
|
||||
from .functional import maybe_list
|
||||
|
||||
try:
|
||||
from ssl import SSLError
|
||||
except ImportError: # pragma: no cover
|
||||
class SSLError(Exception): # noqa
|
||||
"""fallback used when ssl module not compiled."""
|
||||
|
||||
__all__ = ['SendmailWarning', 'Message', 'Mailer', 'ErrorMail']
|
||||
|
||||
_local_hostname = None
|
||||
|
||||
|
||||
def get_local_hostname():
|
||||
global _local_hostname
|
||||
if _local_hostname is None:
|
||||
_local_hostname = socket.getfqdn()
|
||||
return _local_hostname
|
||||
|
||||
|
||||
class SendmailWarning(UserWarning):
|
||||
"""Problem happened while sending the email message."""
|
||||
|
||||
|
||||
@python_2_unicode_compatible
|
||||
class Message(object):
|
||||
|
||||
def __init__(self, to=None, sender=None, subject=None,
|
||||
body=None, charset='utf-8'):
|
||||
self.to = maybe_list(to)
|
||||
self.sender = sender
|
||||
self.subject = subject
|
||||
self.body = body
|
||||
self.charset = charset
|
||||
|
||||
def __repr__(self):
|
||||
return '<Email: To:{0.to!r} Subject:{0.subject!r}>'.format(self)
|
||||
|
||||
def __str__(self):
|
||||
msg = MIMEText(self.body, 'plain', self.charset)
|
||||
msg['Subject'] = self.subject
|
||||
msg['From'] = self.sender
|
||||
msg['To'] = ', '.join(self.to)
|
||||
return msg.as_string()
|
||||
|
||||
|
||||
class Mailer(object):
|
||||
|
||||
def __init__(self, host='localhost', port=0, user=None, password=None,
|
||||
timeout=2, use_ssl=False, use_tls=False):
|
||||
self.host = host
|
||||
self.port = port
|
||||
self.user = user
|
||||
self.password = password
|
||||
self.timeout = timeout
|
||||
self.use_ssl = use_ssl
|
||||
self.use_tls = use_tls
|
||||
|
||||
def send(self, message, fail_silently=False, **kwargs):
|
||||
try:
|
||||
self._send(message, **kwargs)
|
||||
except Exception as exc:
|
||||
if not fail_silently:
|
||||
raise
|
||||
warnings.warn(SendmailWarning(
|
||||
'Mail could not be sent: {0!r} {1!r}\n{2!r}'.format(
|
||||
exc, {'To': ', '.join(message.to),
|
||||
'Subject': message.subject},
|
||||
traceback.format_stack())))
|
||||
|
||||
def _send(self, message, **kwargs):
|
||||
Client = smtplib.SMTP_SSL if self.use_ssl else smtplib.SMTP
|
||||
client = Client(self.host, self.port, timeout=self.timeout,
|
||||
local_hostname=get_local_hostname(), **kwargs)
|
||||
|
||||
if self.use_tls:
|
||||
client.ehlo()
|
||||
client.starttls()
|
||||
client.ehlo()
|
||||
|
||||
if self.user and self.password:
|
||||
client.login(self.user, self.password)
|
||||
|
||||
client.sendmail(message.sender, message.to, str(message))
|
||||
try:
|
||||
client.quit()
|
||||
except SSLError:
|
||||
client.close()
|
||||
|
||||
|
||||
class ErrorMail(object):
|
||||
"""Defines how and when task error e-mails should be sent.
|
||||
|
||||
:param task: The task instance that raised the error.
|
||||
|
||||
:attr:`subject` and :attr:`body` are format strings which
|
||||
are passed a context containing the following keys:
|
||||
|
||||
* name
|
||||
|
||||
Name of the task.
|
||||
|
||||
* id
|
||||
|
||||
UUID of the task.
|
||||
|
||||
* exc
|
||||
|
||||
String representation of the exception.
|
||||
|
||||
* args
|
||||
|
||||
Positional arguments.
|
||||
|
||||
* kwargs
|
||||
|
||||
Keyword arguments.
|
||||
|
||||
* traceback
|
||||
|
||||
String representation of the traceback.
|
||||
|
||||
* hostname
|
||||
|
||||
Worker nodename.
|
||||
|
||||
"""
|
||||
|
||||
# pep8.py borks on a inline signature separator and
|
||||
# says "trailing whitespace" ;)
|
||||
EMAIL_SIGNATURE_SEP = '-- '
|
||||
|
||||
#: Format string used to generate error email subjects.
|
||||
subject = """\
|
||||
[{hostname}] Error: Task {name} ({id}): {exc!r}
|
||||
"""
|
||||
|
||||
#: Format string used to generate error email content.
|
||||
body = """
|
||||
Task {{name}} with id {{id}} raised exception:\n{{exc!r}}
|
||||
|
||||
|
||||
Task was called with args: {{args}} kwargs: {{kwargs}}.
|
||||
|
||||
The contents of the full traceback was:
|
||||
|
||||
{{traceback}}
|
||||
|
||||
{EMAIL_SIGNATURE_SEP}
|
||||
Just to let you know,
|
||||
py-celery at {{hostname}}.
|
||||
""".format(EMAIL_SIGNATURE_SEP=EMAIL_SIGNATURE_SEP)
|
||||
|
||||
def __init__(self, task, **kwargs):
|
||||
self.task = task
|
||||
self.subject = kwargs.get('subject', self.subject)
|
||||
self.body = kwargs.get('body', self.body)
|
||||
|
||||
def should_send(self, context, exc):
|
||||
"""Return true or false depending on if a task error mail
|
||||
should be sent for this type of error."""
|
||||
return True
|
||||
|
||||
def format_subject(self, context):
|
||||
return self.subject.strip().format(**context)
|
||||
|
||||
def format_body(self, context):
|
||||
return self.body.strip().format(**context)
|
||||
|
||||
def send(self, context, exc, fail_silently=True):
|
||||
if self.should_send(context, exc):
|
||||
self.task.app.mail_admins(self.format_subject(context),
|
||||
self.format_body(context),
|
||||
fail_silently=fail_silently)
|
|
@ -34,7 +34,7 @@ APPDIRECT = {
|
|||
'add_defaults', 'config_from_object', 'config_from_envvar',
|
||||
'config_from_cmdline', 'setup_security', 'autodiscover_tasks',
|
||||
'send_task', 'connection', 'connection_or_acquire',
|
||||
'producer_or_acquire', 'prepare_config', 'now', 'mail_admins',
|
||||
'producer_or_acquire', 'prepare_config', 'now',
|
||||
'select_queues', 'either', 'bugreport', 'create_task_cls',
|
||||
'subclass_with_self', 'annotations', 'current_task', 'oid',
|
||||
'timezone', '__reduce_keys__', 'fixups', 'finalized', 'configured',
|
||||
|
|
|
@ -58,7 +58,6 @@ rush in moving to the new settings format.
|
|||
``CELERY_ENABLE_UTC`` :setting:`enable_utc`
|
||||
``CELERY_IMPORTS`` :setting:`imports`
|
||||
``CELERY_INCLUDE`` :setting:`include`
|
||||
``SERVER_EMAIL`` :setting:`server_email`
|
||||
``CELERY_TIMEZONE`` :setting:`timezone`
|
||||
``CELERYBEAT_MAX_LOOP_INTERVAL`` :setting:`beat_max_loop_interval`
|
||||
``CELERYBEAT_SCHEDULE`` :setting:`beat_schedule`
|
||||
|
@ -86,13 +85,6 @@ rush in moving to the new settings format.
|
|||
``CASSANDRA_SERVERS`` :setting:`cassandra_servers`
|
||||
``CASSANDRA_WRITE_CONSISTENCY`` :setting:`cassandra_write_consistency`
|
||||
``CELERY_COUCHBASE_BACKEND_SETTINGS`` :setting:`couchbase_backend_settings`
|
||||
``EMAIL_HOST`` :setting:`email_host`
|
||||
``EMAIL_HOST_USER`` :setting:`email_host_user`
|
||||
``EMAIL_HOST_PASSWORD`` :setting:`email_host_password`
|
||||
``EMAIL_PORT`` :setting:`email_port`
|
||||
``EMAIL_TIMEOUT`` :setting:`email_timeout`
|
||||
``EMAIL_USE_SSL`` :setting:`email_use_ssl`
|
||||
``EMAIL_USE_TLS`` :setting:`email_use_tls`
|
||||
``CELERY_MONGODB_BACKEND_SETTINGS`` :setting:`mongodb_backend_settings`
|
||||
``CELERY_EVENT_QUEUE_EXPIRES`` :setting:`event_queue_expires`
|
||||
``CELERY_EVENT_QUEUE_TTL`` :setting:`event_queue_ttl`
|
||||
|
@ -135,7 +127,6 @@ rush in moving to the new settings format.
|
|||
``CELERY_TASK_PUBLISH_RETRY_POLICY`` :setting:`task_publish_retry_policy`
|
||||
``CELERY_QUEUES`` :setting:`task_queues`
|
||||
``CELERY_ROUTES`` :setting:`task_routes`
|
||||
``CELERY_SEND_TASK_ERROR_EMAILS`` :setting:`task_send_error_emails`
|
||||
``CELERY_SEND_TASK_SENT_EVENT`` :setting:`task_send_sent_event`
|
||||
``CELERY_TASK_SERIALIZER`` :setting:`task_serializer`
|
||||
``CELERYD_TASK_SOFT_TIME_LIMIT`` :setting:`task_soft_time_limit`
|
||||
|
@ -1885,127 +1876,6 @@ Specify if remote control of the workers is enabled.
|
|||
|
||||
Default is :const:`True`.
|
||||
|
||||
.. _conf-error-mails:
|
||||
|
||||
Error E-Mails
|
||||
-------------
|
||||
|
||||
.. setting:: task_send_error_emails
|
||||
|
||||
``task_send_error_emails``
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
|
||||
The default value for the `Task.send_error_emails` attribute, which if
|
||||
set to :const:`True` means errors occurring during task execution will be
|
||||
sent to :setting:`admins` by email.
|
||||
|
||||
Disabled by default.
|
||||
|
||||
.. setting:: admins
|
||||
|
||||
``admins``
|
||||
~~~~~~~~~~
|
||||
|
||||
List of `(name, email_address)` tuples for the administrators that should
|
||||
receive error emails.
|
||||
|
||||
.. setting:: server_email
|
||||
|
||||
``server_email``
|
||||
~~~~~~~~~~~~~~~~
|
||||
|
||||
The email address this worker sends emails from.
|
||||
Default is celery@localhost.
|
||||
|
||||
.. setting:: email_host
|
||||
|
||||
``email_host``
|
||||
~~~~~~~~~~~~~~
|
||||
|
||||
The mail server to use. Default is ``localhost``.
|
||||
|
||||
.. setting:: email_host_user
|
||||
|
||||
``email_host_user``
|
||||
~~~~~~~~~~~~~~~~~~~
|
||||
|
||||
User name (if required) to log on to the mail server with.
|
||||
|
||||
.. setting:: email_host_password
|
||||
|
||||
``email_host_password``
|
||||
~~~~~~~~~~~~~~~~~~~~~~~
|
||||
|
||||
Password (if required) to log on to the mail server with.
|
||||
|
||||
.. setting:: email_port
|
||||
|
||||
``email_port``
|
||||
~~~~~~~~~~~~~~
|
||||
|
||||
The port the mail server is listening on. Default is `25`.
|
||||
|
||||
.. setting:: email_use_ssl
|
||||
|
||||
``email_use_ssl``
|
||||
~~~~~~~~~~~~~~~~~
|
||||
|
||||
Use SSL when connecting to the SMTP server. Disabled by default.
|
||||
|
||||
.. setting:: email_use_tls
|
||||
|
||||
``email_use_tls``
|
||||
~~~~~~~~~~~~~~~~~
|
||||
|
||||
Use TLS when connecting to the SMTP server. Disabled by default.
|
||||
|
||||
.. setting:: email_timeout
|
||||
|
||||
``email_timeout``
|
||||
~~~~~~~~~~~~~~~~~
|
||||
|
||||
Timeout in seconds for when we give up trying to connect
|
||||
to the SMTP server when sending emails.
|
||||
|
||||
The default is 2 seconds.
|
||||
|
||||
.. setting:: email_charset
|
||||
|
||||
``email_charset``
|
||||
~~~~~~~~~~~~~~~~~
|
||||
.. versionadded:: 4.0
|
||||
|
||||
Character set for outgoing emails. Default is ``"utf-8"``.
|
||||
|
||||
.. _conf-example-error-mail-config:
|
||||
|
||||
Example E-Mail configuration
|
||||
~~~~~~~~~~~~~~~~~~~~~~~~~~~~
|
||||
|
||||
This configuration enables the sending of error emails to
|
||||
george@vandelay.com and kramer@vandelay.com:
|
||||
|
||||
.. code-block:: python
|
||||
|
||||
# Enables error emails.
|
||||
task_send_error_emails = True
|
||||
|
||||
# Name and email addresses of recipients
|
||||
admins = (
|
||||
('George Costanza', 'george@vandelay.com'),
|
||||
('Cosmo Kramer', 'kosmo@vandelay.com'),
|
||||
)
|
||||
|
||||
# Email address used as sender (From field).
|
||||
server_email = 'no-reply@vandelay.com'
|
||||
|
||||
# Mailserver configuration
|
||||
email_host = 'mail.vandelay.com'
|
||||
email_port = 25
|
||||
email_charset = 'utf-8'
|
||||
# email_host_user = 'servers'
|
||||
# email_host_password = 's3cr3t'
|
||||
|
||||
.. _conf-events:
|
||||
|
||||
Events
|
||||
|
|
|
@ -277,7 +277,6 @@ Quick Jump
|
|||
- :ref:`see a list of event message types <event-reference>`
|
||||
- :ref:`contribute to Celery <contributing>`
|
||||
- :ref:`learn about available configuration settings <configuration>`
|
||||
- :ref:`receive email when a task fails <conf-error-mails>`
|
||||
- :ref:`get a list of people and companies using Celery <res-using-celery>`
|
||||
- :ref:`write my own remote control command <worker-custom-control-commands>`
|
||||
- :ref:`change worker queues at runtime <worker-queues>`
|
||||
|
|
|
@ -194,7 +194,6 @@ Settings
|
|||
===================================== =====================================
|
||||
**Setting name** **Replace with**
|
||||
===================================== =====================================
|
||||
``CELERY_TASK_ERROR_WITELIST`` Annotate ``Task.ErrorMail``
|
||||
``CELERY_AMQP_TASK_RESULT_EXPIRES`` :setting:`result_expires`
|
||||
===================================== =====================================
|
||||
|
||||
|
|
|
@ -105,8 +105,6 @@ and creating Celery applications.
|
|||
|
||||
.. automethod:: producer_or_acquire
|
||||
|
||||
.. automethod:: mail_admins
|
||||
|
||||
.. automethod:: select_queues
|
||||
|
||||
.. automethod:: now
|
||||
|
|
|
@ -1,11 +0,0 @@
|
|||
====================================
|
||||
``celery.utils.mail``
|
||||
====================================
|
||||
|
||||
.. contents::
|
||||
:local:
|
||||
.. currentmodule:: celery.utils.mail
|
||||
|
||||
.. automodule:: celery.utils.mail
|
||||
:members:
|
||||
:undoc-members:
|
|
@ -26,7 +26,6 @@
|
|||
celery.signals
|
||||
celery.security
|
||||
celery.utils.debug
|
||||
celery.utils.mail
|
||||
celery.exceptions
|
||||
celery.loaders
|
||||
celery.loaders.app
|
||||
|
|
|
@ -460,7 +460,7 @@ chain breaks:
|
|||
from celery.registry import tasks
|
||||
|
||||
class Hello(Task):
|
||||
send_error_emails = True
|
||||
queue = 'hipri'
|
||||
|
||||
def run(self, to):
|
||||
return 'hello {0}'.format(to)
|
||||
|
@ -477,7 +477,7 @@ chain breaks:
|
|||
|
||||
from celery.task import task
|
||||
|
||||
@task(send_error_emails=True)
|
||||
@task(queue='hipri')
|
||||
def hello(x):
|
||||
return 'hello {0}'.format(to)
|
||||
|
||||
|
@ -538,7 +538,7 @@ by changing its :meth:`@Task` attribute:
|
|||
>>> app = Celery()
|
||||
|
||||
>>> class MyBaseTask(Task):
|
||||
... send_error_emails = True
|
||||
... queue = 'hipri'
|
||||
|
||||
>>> app.Task = MyBaseTask
|
||||
>>> app.Task
|
||||
|
|
|
@ -737,17 +737,6 @@ General
|
|||
If :const:`True`, errors will be stored even if the task is configured
|
||||
to ignore results.
|
||||
|
||||
.. attribute:: Task.send_error_emails
|
||||
|
||||
Send an email whenever a task of this type fails.
|
||||
Defaults to the :setting:`task_send_error_emails` setting.
|
||||
See :ref:`conf-error-mails` for more information.
|
||||
|
||||
.. attribute:: Task.ErrorMail
|
||||
|
||||
If the sending of error emails is enabled for this task, then
|
||||
this is the class defining the logic to send error mails.
|
||||
|
||||
.. attribute:: Task.serializer
|
||||
|
||||
A string identifying the default serialization
|
||||
|
|
|
@ -216,7 +216,6 @@ a few special ones:
|
|||
``CELERY_TASK_PUBLISH_RETRY_POLICY`` :setting:`task_publish_retry_policy`
|
||||
``CELERY_QUEUES`` :setting:`task_queues`
|
||||
``CELERY_ROUTES`` :setting:`task_routes`
|
||||
``CELERY_SEND_TASK_ERROR_EMAILS`` :setting:`task_send_error_emails`
|
||||
``CELERY_SEND_TASK_SENT_EVENT`` :setting:`task_send_sent_event`
|
||||
``CELERY_TASK_SERIALIZER`` :setting:`task_serializer`
|
||||
``CELERYD_TASK_SOFT_TIME_LIMIT`` :setting:`task_soft_time_limit`
|
||||
|
@ -1005,11 +1004,6 @@ In Other News
|
|||
|
||||
Contributed by Alexander Oblovatniy.
|
||||
|
||||
- **Tasks**: New :setting:`email_charset` setting allows for changing
|
||||
the character set used for outgoing error emails.
|
||||
|
||||
Contributed by Vladimir Gorbunov.
|
||||
|
||||
- **Worker**: Now respects :setting:`broker_connection_retry` setting.
|
||||
|
||||
Fix contributed by Nat Williams.
|
||||
|
@ -1064,9 +1058,6 @@ In Other News
|
|||
|
||||
Fix contributed by Feanil Patel.
|
||||
|
||||
- **Tasks**: Task error email character set now set to ``utf-8`` by default
|
||||
(Issue #2737).
|
||||
|
||||
- Apps can now define how tasks are named (:meth:`@gen_task_name`).
|
||||
|
||||
Contributed by Dmitry Malinovsky
|
||||
|
|
|
@ -5,7 +5,6 @@ import os
|
|||
|
||||
broker_url = os.environ.get('BROKER_URL') or 'amqp://'
|
||||
result_backend = 'amqp://'
|
||||
send_task_error_emails = False
|
||||
|
||||
default_queue = 'testcelery'
|
||||
default_exchange = 'testcelery'
|
||||
|
|
Загрузка…
Ссылка в новой задаче