You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2018/09/04 07:36:21 UTC

[GitHub] Fokko closed pull request #2187: [AIRFLOW-1042] Easy Unit Testing with Docker

Fokko closed pull request #2187: [AIRFLOW-1042] Easy Unit Testing with Docker
URL: https://github.com/apache/incubator-airflow/pull/2187
 
 
   

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/CONTRIBUTING.md b/CONTRIBUTING.md
index 04b0d7f713..87f0a24cd7 100644
--- a/CONTRIBUTING.md
+++ b/CONTRIBUTING.md
@@ -141,11 +141,26 @@ We *highly* recommend setting up [Travis CI](https://travis-ci.org/) on
 your repo to automate this. It is free for open source projects. If for
 some reason you cannot, you can use the steps below to run tests.
 
-Here are loose guidelines on how to get your environment to run the unit tests.
-We do understand that no one out there can run the full test suite since
-Airflow is meant to connect to virtually any external system and that you most
-likely have only a subset of these in your environment. You should run the
-CoreTests and tests related to things you touched in your PR.
+Unit tests can be run locally using Docker. Running this command:
+
+    docker-compose up -d
+
+builds and starts three Docker containers: one for MySQL, one for Postgres,
+and one for Airflow. Once the Docker containers are built and running you can
+then run:
+
+    ./scripts/docker/unittest/run.sh tests.core:CoreTest
+
+The Airflow container has a volume mapped to the Airflow source directory so
+that any edits made to source files are reflected in the container. You can
+make edits and then run tests specific to the area you're working on.
+
+If you want to run unit tests without Docker, here are loose guidelines on
+how to get your environment to run the unit tests. We do understand that no
+one out there can run the full test suite since Airflow is meant to connect
+to virtually any external system and that you most likely have only a subset
+of these in your environment. You should run the CoreTests and tests related
+to things you touched in your PR.
 
 To set up a unit test environment, first take a look at `run_unit_tests.sh` and
 understand that your ``AIRFLOW_CONFIG`` points to an alternate config file
diff --git a/airflow/config_templates/default_test.cfg b/airflow/config_templates/default_test.cfg
index ecf7f4ebb0..93a4f9fde3 100644
--- a/airflow/config_templates/default_test.cfg
+++ b/airflow/config_templates/default_test.cfg
@@ -70,8 +70,8 @@ smtp_mail_from = airflow@airflow.com
 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 = sqla+mysql://airflow:airflow@{AIRFLOW_MYSQL_HOST}:3306/airflow
+celery_result_backend = db+mysql://airflow:airflow@{AIRFLOW_MYSQL_HOST}:3306/airflow
 flower_host = 0.0.0.0
 flower_port = 5555
 default_queue = default
diff --git a/airflow/configuration.py b/airflow/configuration.py
index f140be2bc1..9ddaf5b4c1 100644
--- a/airflow/configuration.py
+++ b/airflow/configuration.py
@@ -318,6 +318,11 @@ def mkdir_p(path):
 else:
     AIRFLOW_CONFIG = expand_env_var(os.environ['AIRFLOW_CONFIG'])
 
+if 'AIRFLOW_MYSQL_HOST' not in os.environ:
+    AIRFLOW_MYSQL_HOST = 'localhost'
+else:
+    AIRFLOW_MYSQL_HOST = expand_env_var(os.environ['AIRFLOW_MYSQL_HOST'])
+
 # Set up dags folder for unit tests
 # this directory won't exist if users install via pip
 _TEST_DAGS_FOLDER = os.path.join(
diff --git a/airflow/utils/db.py b/airflow/utils/db.py
index 618e00200b..4ca59e704f 100644
--- a/airflow/utils/db.py
+++ b/airflow/utils/db.py
@@ -27,6 +27,7 @@
 
 from airflow import settings
 
+
 def provide_session(func):
     """
     Function decorator that provides a session if it isn't provided.
@@ -94,6 +95,21 @@ def checkout(dbapi_connection, connection_record, connection_proxy):
         )
 
 
+def get_mysql_host(default='localhost'):
+    return default if 'AIRFLOW_MYSQL_HOST' not in os.environ \
+        else os.environ['AIRFLOW_MYSQL_HOST']
+
+
+def get_mysql_login(default='root'):
+    return default if 'AIRFLOW_MYSQL_USER' not in os.environ \
+        else os.environ['AIRFLOW_MYSQL_USER']
+
+
+def get_mysql_password(default=None):
+    return default if 'AIRFLOW_MYSQL_PASSWORD' not in os.environ \
+        else os.environ['AIRFLOW_MYSQL_PASSWORD']
+
+
 def initdb():
     session = settings.Session()
 
@@ -103,12 +119,13 @@ def initdb():
     merge_conn(
         models.Connection(
             conn_id='airflow_db', conn_type='mysql',
-            host='localhost', login='root', password='',
+            login=get_mysql_login(), host=get_mysql_host(), password=get_mysql_password(),
             schema='airflow'))
     merge_conn(
         models.Connection(
             conn_id='airflow_ci', conn_type='mysql',
-            host='localhost', login='root', extra="{\"local_infile\": true}",
+            host=get_mysql_host(), login=get_mysql_login(), password=get_mysql_password(),
+            extra="{\"local_infile\": true}",
             schema='airflow_ci'))
     merge_conn(
         models.Connection(
@@ -121,7 +138,8 @@ def initdb():
     merge_conn(
         models.Connection(
             conn_id='local_mysql', conn_type='mysql',
-            host='localhost', login='airflow', password='airflow',
+            host=get_mysql_host(), login=get_mysql_login('airflow'),
+            password=get_mysql_password('airflow'),
             schema='airflow'))
     merge_conn(
         models.Connection(
@@ -145,14 +163,17 @@ def initdb():
     merge_conn(
         models.Connection(
             conn_id='mysql_default', conn_type='mysql',
-            login='root',
-            host='localhost'))
+            login=get_mysql_login(), host=get_mysql_host(), password=get_mysql_password()))
     merge_conn(
         models.Connection(
             conn_id='postgres_default', conn_type='postgres',
-            login='postgres',
+            login=('postgres' if 'AIRFLOW_POSTGRES_USER' not in os.environ
+                   else os.environ['AIRFLOW_POSTGRES_USER']),
             schema='airflow',
-            host='localhost'))
+            host=('localhost' if 'AIRFLOW_POSTGRES_HOST' not in os.environ
+                  else os.environ['AIRFLOW_POSTGRES_HOST']),
+            password=(None if 'AIRFLOW_POSTGRES_PASSWORD' not in os.environ
+                      else os.environ['AIRFLOW_POSTGRES_PASSWORD'])))
     merge_conn(
         models.Connection(
             conn_id='sqlite_default', conn_type='sqlite',
diff --git a/docker-compose.yml b/docker-compose.yml
new file mode 100644
index 0000000000..cb0fa0cdff
--- /dev/null
+++ b/docker-compose.yml
@@ -0,0 +1,67 @@
+#
+# 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.
+
+# This Docker Compose file is intended to make running Airflow unit tests
+# as easy as possible. Contributors should be able to 'docker-compose up -d'
+# and './scripts/docker/unittest/run.sh'
+#
+# NOTE: If you already have MySQL running on port 3306, Postgres running
+#       on 5432, or Airflow running on 8080 on localhost you will need
+#       to stop those process prior to running docker-compose.
+
+version: '3'
+services:
+  mysql:
+    build:
+      context: .
+      dockerfile: scripts/docker/unittest/Dockerfile-mysql
+    container_name: mysql
+    environment:
+      MYSQL_ROOT_PASSWORD: root
+      MYSQL_DATABASE: airflow
+      MYSQL_USER: airflow
+      MYSQL_PASSWORD: airflow
+    ports:
+      - "3306:3306"
+  postgres:
+    build:
+      context: .
+      dockerfile: scripts/docker/unittest/Dockerfile-postgres
+    container_name: postgres
+    environment:
+      POSTGRES_DB: airflow
+      POSTGRES_USER: airflow
+      POSTGRES_PASSWORD: airflow
+    ports:
+      - "5432:5432"
+  airflow:
+    build:
+      context: .
+      dockerfile: scripts/docker/unittest/Dockerfile-airflow
+    command: "bash -c \"airflow upgradedb && airflow webserver\""
+    container_name: airflow
+    environment:
+      AIRFLOW_MYSQL_HOST: mysql
+      AIRFLOW_MYSQL_USER: airflow
+      AIRFLOW_MYSQL_PASSWORD: airflow
+      AIRFLOW_POSTGRES_HOST: postgres
+      AIRFLOW_POSTGRES_USER: airflow
+      AIRFLOW_POSTGRES_PASSWORD: airflow
+    links:
+      - mysql
+      - postgres
+    ports:
+      - "8080:8080"
+    volumes:
+      - .:/usr/src/airflow
+
diff --git a/scripts/ci/data/mysql_schema.sql b/scripts/ci/data/mysql_schema.sql
index c2c60729d1..a8c8b4a968 100644
--- a/scripts/ci/data/mysql_schema.sql
+++ b/scripts/ci/data/mysql_schema.sql
@@ -24,4 +24,4 @@ CREATE TABLE IF NOT EXISTS baby_names (
   baby_name VARCHAR(25),
   rate FLOAT(7,6),
   sex VARCHAR(4)
-)
\ No newline at end of file
+)
diff --git a/scripts/docker/unittest/Dockerfile-airflow b/scripts/docker/unittest/Dockerfile-airflow
new file mode 100644
index 0000000000..7a7b6ec06d
--- /dev/null
+++ b/scripts/docker/unittest/Dockerfile-airflow
@@ -0,0 +1,40 @@
+#
+# 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.
+
+FROM python:2.7
+
+RUN apt-get update && \
+    apt-get install -y apt-utils python3.4-dev libsasl2-dev \
+                       mysql-client postgresql-client sqlite3 vim openssh-server
+
+RUN pip install redis coverage wheel
+
+ENV AIRFLOW_HOME=/usr/local/airflow
+ENV AIRFLOW_SRC=/usr/src/airflow
+
+RUN mkdir -p ${AIRFLOW_HOME}
+RUN mkdir -p ${AIRFLOW_SRC}
+
+COPY . ${AIRFLOW_SRC}
+COPY scripts/docker/unittest/airflow.cfg ${AIRFLOW_HOME}
+
+RUN ln -s ${AIRFLOW_SRC}/tests/dags ${AIRFLOW_HOME}/dags
+
+WORKDIR ${AIRFLOW_SRC}
+<<<<<<< HEAD
+RUN pip install -e .[devel,celery,crypto,docker,dask,hdfs,hive,kerberos,ldap,postgres,s3]
+=======
+RUN pip install -e .[devel,celery,crypto,docker,dask,hdfs,hive,ldap,postgres,s3]
+>>>>>>> 31e5cc66560244099d735a7a57c535db1a53f302
+
+WORKDIR ${AIRFLOW_HOME}
diff --git a/scripts/docker/unittest/Dockerfile-mysql b/scripts/docker/unittest/Dockerfile-mysql
new file mode 100644
index 0000000000..071a72bb4b
--- /dev/null
+++ b/scripts/docker/unittest/Dockerfile-mysql
@@ -0,0 +1,18 @@
+#
+# 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.
+
+FROM mysql:5.7
+
+COPY scripts/ci/data/baby_names.csv /var/lib/mysql-files/baby_names.csv
+COPY scripts/docker/unittest/grant_airflow.sh /docker-entrypoint-initdb.d/
+COPY scripts/docker/unittest/load_data.sh /docker-entrypoint-initdb.d/
diff --git a/scripts/docker/unittest/Dockerfile-postgres b/scripts/docker/unittest/Dockerfile-postgres
new file mode 100644
index 0000000000..3277c82fd7
--- /dev/null
+++ b/scripts/docker/unittest/Dockerfile-postgres
@@ -0,0 +1,15 @@
+#
+# 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.
+
+FROM postgres:9.5
+
diff --git a/scripts/docker/unittest/grant_airflow.sh b/scripts/docker/unittest/grant_airflow.sh
new file mode 100755
index 0000000000..1cdfe30774
--- /dev/null
+++ b/scripts/docker/unittest/grant_airflow.sh
@@ -0,0 +1,22 @@
+#!/usr/bin/env bash
+
+#
+# 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.
+
+set -x
+
+mysql -u root -proot -e "GRANT ALL PRIVILEGES ON *.* TO 'airflow'@'%';"
+
+
+
+
diff --git a/scripts/docker/unittest/load_data.sh b/scripts/docker/unittest/load_data.sh
new file mode 100755
index 0000000000..4fa1ac4221
--- /dev/null
+++ b/scripts/docker/unittest/load_data.sh
@@ -0,0 +1,21 @@
+#!/usr/bin/env bash
+#  Licensed to the Apache Software Foundation (ASF) under one   *
+#  or more contributor license agreements.  See the NOTICE file *
+#  distributed with this work for additional information        *
+#  regarding copyright ownership.  The ASF licenses this file   *
+#  to you 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.                                           *
+
+mysql -u root -proot -e "CREATE SCHEMA IF NOT EXISTS airflow_ci"
+mysql -u root -proot -e "CREATE TABLE IF NOT EXISTS airflow_ci.baby_names (org_year integer(4), baby_name VARCHAR(25), rate FLOAT(7,6),sex VARCHAR(4));"
+mysqlimport -u root -proot --fields-optionally-enclosed-by="\"" --fields-terminated-by=, --ignore-lines=1 airflow_ci /var/lib/mysql-files/baby_names.csv
diff --git a/scripts/docker/unittest/run.sh b/scripts/docker/unittest/run.sh
new file mode 100755
index 0000000000..1a8500abf7
--- /dev/null
+++ b/scripts/docker/unittest/run.sh
@@ -0,0 +1,20 @@
+#!/usr/bin/env bash
+
+#
+# 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.
+
+set -x
+
+UNITTEST_ARGS="$@"
+docker exec -i -t airflow /bin/sh -c "cd /usr/src/airflow; /usr/src/airflow/run_unit_tests.sh $UNITTEST_ARGS"
+
diff --git a/tests/core.py b/tests/core.py
index 848553af3e..a8febd5d6c 100644
--- a/tests/core.py
+++ b/tests/core.py
@@ -1450,7 +1450,9 @@ def test_csrf_rejection(self):
         ])
         for endpoint in endpoints:
             response = self.app.post(endpoint)
-            self.assertIn('CSRF token is missing', response.data.decode('utf-8'))
+            self.assertTrue('CSRF token is missing' in response.data.decode('utf-8') or
+                            'CSRF token missing or incorrect' in
+                            response.data.decode('utf-8'))
 
     def test_csrf_acceptance(self):
         response = self.app.get("/admin/queryview/")
@@ -2072,9 +2074,11 @@ def test_get_connections_env_var(self):
     def test_get_connections_db(self):
         conns = BaseHook.get_connections(conn_id='airflow_db')
         assert len(conns) == 1
-        assert conns[0].host == 'localhost'
+        assert conns[0].host == ('localhost' if 'AIRFLOW_MYSQL_HOST' not in os.environ
+                                 else os.environ['AIRFLOW_MYSQL_HOST'])
         assert conns[0].schema == 'airflow'
-        assert conns[0].login == 'root'
+        assert conns[0].login == ('root' if 'AIRFLOW_MYSQL_USER' not in os.environ
+                                  else os.environ['AIRFLOW_MYSQL_USER'])
 
 
 class WebHDFSHookTest(unittest.TestCase):


 

----------------------------------------------------------------
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


With regards,
Apache Git Services