forked from apache/airflow
-
Notifications
You must be signed in to change notification settings - Fork 2
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[AIRFLOW-1527] Refactor celery config
The celery config is currently part of the celery executor definition. This is really inflexible for users wanting to change it. In addition Celery 4 is moving to lowercase. Closes apache#2542 from bolkedebruin/upgrade_celery
- Loading branch information
1 parent
87df670
commit f4437be
Showing
7 changed files
with
131 additions
and
45 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,6 +19,7 @@ jdk: | |
services: | ||
- mysql | ||
- postgresql | ||
- rabbitmq | ||
addons: | ||
apt: | ||
packages: | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,58 @@ | ||
# -*- 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 ssl | ||
|
||
from airflow.exceptions import AirflowConfigException, AirflowException | ||
from airflow import configuration | ||
from airflow.utils.log.logging_mixin import LoggingMixin | ||
|
||
|
||
DEFAULT_CELERY_CONFIG = { | ||
'accept_content': ['json', 'pickle'], | ||
'event_serializer': 'json', | ||
'result_serializer': 'pickle', | ||
'worker_prefetch_multiplier': 1, | ||
'task_acks_late': True, | ||
'task_default_queue': configuration.get('celery', 'DEFAULT_QUEUE'), | ||
'task_default_exchange': configuration.get('celery', 'DEFAULT_QUEUE'), | ||
'broker_url': configuration.get('celery', 'BROKER_URL'), | ||
'broker_transport_options': {'visibility_timeout': 21600}, | ||
'result_backend': configuration.get('celery', 'CELERY_RESULT_BACKEND'), | ||
'worker_concurrency': configuration.getint('celery', 'CELERYD_CONCURRENCY'), | ||
} | ||
|
||
celery_ssl_active = False | ||
try: | ||
celery_ssl_active = configuration.getboolean('celery', 'CELERY_SSL_ACTIVE') | ||
except AirflowConfigException as e: | ||
log = LoggingMixin().log | ||
log.warning("Celery Executor will run without SSL") | ||
|
||
try: | ||
if celery_ssl_active: | ||
broker_use_ssl = {'keyfile': configuration.get('celery', 'CELERY_SSL_KEY'), | ||
'certfile': configuration.get('celery', 'CELERY_SSL_CERT'), | ||
'ca_certs': configuration.get('celery', 'CELERY_SSL_CACERT'), | ||
'cert_reqs': ssl.CERT_REQUIRED} | ||
DEFAULT_CELERY_CONFIG['broker_use_ssl'] = broker_use_ssl | ||
except AirflowConfigException as e: | ||
raise AirflowException('AirflowConfigException: CELERY_SSL_ACTIVE is True, ' | ||
'please ensure CELERY_SSL_KEY, ' | ||
'CELERY_SSL_CERT and CELERY_SSL_CACERT are set') | ||
except Exception as e: | ||
raise AirflowException('Exception: There was an unknown Celery SSL Error. ' | ||
'Please ensure you want to use ' | ||
'SSL and/or have all necessary certs and key ({}).'.format(e)) | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -44,8 +44,8 @@ smtp_mail_from = [email protected] | |
celery_app_name = airflow.executors.celery_executor | ||
celeryd_concurrency = 16 | ||
worker_log_server_port = 8793 | ||
broker_url = sqla+mysql://airflow:airflow@localhost:3306/airflow | ||
celery_result_backend = db+mysql://airflow:airflow@localhost:3306/airflow | ||
broker_url = amqp://guest:guest@localhost:5672/ | ||
celery_result_backend = db+mysql://root@localhost/airflow | ||
flower_port = 5555 | ||
default_queue = default | ||
|
||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
# -*- 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 | ||
import sys | ||
|
||
from airflow.executors.celery_executor import app | ||
from airflow.executors.celery_executor import CeleryExecutor | ||
from airflow.utils.state import State | ||
from celery.contrib.testing.worker import start_worker | ||
|
||
# leave this it is used by the test worker | ||
import celery.contrib.testing.tasks | ||
|
||
|
||
class CeleryExecutorTest(unittest.TestCase): | ||
|
||
def test_celery_integration(self): | ||
executor = CeleryExecutor() | ||
executor.start() | ||
with start_worker(app=app, logfile=sys.stdout, loglevel='debug'): | ||
|
||
success_command = 'echo 1' | ||
fail_command = 'exit 1' | ||
|
||
executor.execute_async(key='success', command=success_command) | ||
# errors are propagated for some reason | ||
try: | ||
executor.execute_async(key='fail', command=fail_command) | ||
except: | ||
pass | ||
executor.running['success'] = True | ||
executor.running['fail'] = True | ||
|
||
executor.end(synchronous=True) | ||
|
||
self.assertTrue(executor.event_buffer['success'], State.SUCCESS) | ||
self.assertTrue(executor.event_buffer['fail'], State.FAILED) | ||
|
||
self.assertNotIn('success', executor.tasks) | ||
self.assertNotIn('fail', executor.tasks) | ||
|
||
|
||
if __name__ == '__main__': | ||
unittest.main() |