Skip to content

Commit 7115883

Browse files
dima-asanaashb
authored andcommitted
[AIRFLOW-3168] More resillient database use in CI (#4014)
Make sure mysql is available before calling it in CI
1 parent cdbdcae commit 7115883

File tree

6 files changed

+27
-4
lines changed

6 files changed

+27
-4
lines changed

scripts/ci/3-setup-databases.sh scripts/ci/3-setup-mysql.sh

+8-1
Original file line numberDiff line numberDiff line change
@@ -20,4 +20,11 @@ set -exuo pipefail
2020

2121
MYSQL_HOST=mysql
2222

23-
mysql -h ${MYSQL_HOST} -u root -e 'drop database if exists airflow; create database airflow'
23+
retries=3
24+
for ((i=0; i<retries; i++)); do
25+
mysql -h ${MYSQL_HOST} -u root -e 'drop database if exists airflow; create database airflow' && exit 0
26+
echo "mysql db creation did not succeed, waiting 5 seconds to retry"
27+
sleep 5
28+
done
29+
30+
echo "mysql db creation could not succeed" && exit 1

scripts/ci/airflow_travis.cfg

+2-2
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ airflow_home = ~/airflow
2121
dags_folder = ~/airflow/dags
2222
base_log_folder = ~/airflow/logs
2323
executor = LocalExecutor
24-
sql_alchemy_conn = mysql://root@mysql/airflow
24+
sql_alchemy_conn = # overridden by tox.ini
2525
unit_test_mode = True
2626
load_examples = True
2727
donot_pickle = False
@@ -56,7 +56,7 @@ celery_app_name = airflow.executors.celery_executor
5656
worker_concurrency = 16
5757
worker_log_server_port = 8793
5858
broker_url = amqp://guest:guest@rabbitmq:5672/
59-
result_backend = db+mysql://root@mysql/airflow
59+
result_backend = # overridden by tox.ini
6060
flower_port = 5555
6161
default_queue = default
6262
sync_parallelism = 0

tests/configuration.py

+5
Original file line numberDiff line numberDiff line change
@@ -191,4 +191,9 @@ def test_deprecated_options_cmd(self):
191191
conf.set('celery', 'celery_result_backend_cmd', '/bin/echo 99')
192192

193193
with self.assertWarns(DeprecationWarning):
194+
tmp = None
195+
if 'AIRFLOW__CELERY__RESULT_BACKEND' in os.environ:
196+
tmp = os.environ.pop('AIRFLOW__CELERY__RESULT_BACKEND')
194197
self.assertEquals(conf.getint('celery', 'result_backend'), 99)
198+
if tmp:
199+
os.environ['AIRFLOW__CELERY__RESULT_BACKEND'] = tmp

tests/executors/test_celery_executor.py

+5
Original file line numberDiff line numberDiff line change
@@ -26,11 +26,16 @@
2626
from airflow.executors.celery_executor import CELERY_FETCH_ERR_MSG_HEADER
2727
from airflow.utils.state import State
2828

29+
from airflow import configuration
30+
configuration.load_test_config()
31+
2932
# leave this it is used by the test worker
3033
import celery.contrib.testing.tasks # noqa: F401
3134

3235

3336
class CeleryExecutorTest(unittest.TestCase):
37+
@unittest.skipIf('sqlite' in configuration.conf.get('core', 'sql_alchemy_conn'),
38+
"sqlite is configured with SequentialExecutor")
3439
def test_celery_integration(self):
3540
executor = CeleryExecutor()
3641
executor.start()

tests/sensors/test_sql_sensor.py

+4
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,8 @@ def setUp(self):
4040
}
4141
self.dag = DAG(TEST_DAG_ID, default_args=args)
4242

43+
@unittest.skipUnless('mysql' in configuration.conf.get('core', 'sql_alchemy_conn'),
44+
"this is a mysql test")
4345
def test_sql_sensor_mysql(self):
4446
t = SqlSensor(
4547
task_id='sql_sensor_check',
@@ -49,6 +51,8 @@ def test_sql_sensor_mysql(self):
4951
)
5052
t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
5153

54+
@unittest.skipUnless('postgresql' in configuration.conf.get('core', 'sql_alchemy_conn'),
55+
"this is a postgres test")
5256
def test_sql_sensor_postgres(self):
5357
t = SqlSensor(
5458
task_id='sql_sensor_check',

tox.ini

+3-1
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,9 @@ setenv =
4848
KRB5_CONFIG=/etc/krb5.conf
4949
KRB5_KTNAME=/etc/airflow.keytab
5050
backend_mysql: AIRFLOW__CORE__SQL_ALCHEMY_CONN=mysql://root@mysql/airflow
51+
backend_mysql: AIRFLOW__CELERY__RESULT_BACKEND=db+mysql://root@mysql/airflow
5152
backend_postgres: AIRFLOW__CORE__SQL_ALCHEMY_CONN=postgresql+psycopg2://postgres:airflow@postgres/airflow
53+
backend_postgres: AIRFLOW__CELERY__RESULT_BACKEND=db+postgresql://postgres:airflow@postgres/airflow
5254
backend_sqlite: AIRFLOW__CORE__SQL_ALCHEMY_CONN=sqlite:///{homedir}/airflow.db
5355
backend_sqlite: AIRFLOW__CORE__EXECUTOR=SequentialExecutor
5456

@@ -59,7 +61,7 @@ commands =
5961
pip install --progress-bar off --find-links={homedir}/.wheelhouse --no-index -e .[devel_ci]
6062
env_docker: {toxinidir}/scripts/ci/1-setup-env.sh
6163
env_docker: {toxinidir}/scripts/ci/2-setup-kdc.sh
62-
env_docker: {toxinidir}/scripts/ci/3-setup-databases.sh
64+
backend_mysql: {toxinidir}/scripts/ci/3-setup-mysql.sh
6365
{toxinidir}/scripts/ci/5-run-tests.sh []
6466
{toxinidir}/scripts/ci/6-check-license.sh
6567
codecov -e TOXENV

0 commit comments

Comments
 (0)