Skip to content

Commit

Permalink
AIRFLOW-2952 Fix Kubernetes CI (#3922)
Browse files Browse the repository at this point in the history
The current dockerised CI pipeline doesn't run minikube and the
Kubernetes integration tests. This starts a Kubernetes cluster 
using minikube and runs k8s integration tests using docker-compose.
  • Loading branch information
odracci authored and Fokko committed Sep 21, 2018
1 parent 4bad731 commit 8038f88
Show file tree
Hide file tree
Showing 17 changed files with 175 additions and 47 deletions.
32 changes: 20 additions & 12 deletions .travis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -26,29 +26,37 @@ env:
- TRAVIS_CACHE=$HOME/.travis_cache/
matrix:
- TOX_ENV=flake8
- TOX_ENV=py27-backend_mysql
- TOX_ENV=py27-backend_sqlite
- TOX_ENV=py27-backend_postgres
- TOX_ENV=py35-backend_mysql PYTHON_VERSION=3
- TOX_ENV=py35-backend_sqlite PYTHON_VERSION=3
- TOX_ENV=py35-backend_postgres PYTHON_VERSION=3
- TOX_ENV=py27-backend_postgres KUBERNETES_VERSION=v1.9.0
- TOX_ENV=py35-backend_postgres KUBERNETES_VERSION=v1.10.0 PYTHON_VERSION=3
- TOX_ENV=py27-backend_mysql-env_docker
- TOX_ENV=py27-backend_sqlite-env_docker
- TOX_ENV=py27-backend_postgres-env_docker
- TOX_ENV=py35-backend_mysql-env_docker PYTHON_VERSION=3
- TOX_ENV=py35-backend_sqlite-env_docker PYTHON_VERSION=3
- TOX_ENV=py35-backend_postgres-env_docker PYTHON_VERSION=3
- TOX_ENV=py27-backend_postgres-env_kubernetes KUBERNETES_VERSION=v1.9.0
- TOX_ENV=py35-backend_postgres-env_kubernetes KUBERNETES_VERSION=v1.10.0 PYTHON_VERSION=3

cache:
directories:
- $HOME/.wheelhouse/
- $HOME/.cache/pip
- $HOME/.travis_cache/
before_install:
- sudo ls -lh $HOME/.cache/pip/
- sudo rm -rf $HOME/.cache/pip/* $HOME/.wheelhouse/*
- sudo chown -R travis:travis $HOME/.cache/pip
# Required for K8s v1.10.x. See
# https://github.com/kubernetes/kubernetes/issues/61058#issuecomment-372764783
- if [ ! -z "$KUBERNETES_VERSION" ]; then sudo mount --make-shared / && sudo service docker restart; fi
install:
# Use recent docker-compose version
- sudo rm /usr/local/bin/docker-compose
- curl -L https://github.com/docker/compose/releases/download/${DOCKER_COMPOSE_VERSION}/docker-compose-`uname -s`-`uname -m` > docker-compose
- chmod +x docker-compose
- sudo mv docker-compose /usr/local/bin
- pip install --upgrade pip
- if [ ! -z "$KUBERNETES_VERSION" ]; then ./scripts/ci/kubernetes/setup_kubernetes.sh; fi
script:
- docker-compose --log-level ERROR -f scripts/ci/docker-compose.yml run airflow-testing /app/scripts/ci/run-ci.sh
- if [ -z "$KUBERNETES_VERSION" ]; then docker-compose --log-level ERROR -f scripts/ci/docker-compose.yml run airflow-testing /app/scripts/ci/run-ci.sh; fi
- if [ ! -z "$KUBERNETES_VERSION" ]; then
./scripts/ci/kubernetes/kube/deploy.sh &&
MINIKUBE_IP=$(minikube ip) docker-compose --log-level ERROR -f scripts/ci/docker-compose.yml -f scripts/ci/docker-compose-kubernetes.yml run airflow-testing /app/scripts/ci/run-ci.sh;
fi
before_cache:
- sudo chown -R travis:travis $HOME/.cache/pip $HOME/.wheelhouse/
18 changes: 18 additions & 0 deletions airflow/example_dags_kubernetes/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
# -*- coding: utf-8 -*-
#
# 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.
47 changes: 47 additions & 0 deletions airflow/example_dags_kubernetes/example_kubernetes_annotation.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
# -*- coding: utf-8 -*-
#
# 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.
from __future__ import print_function
import airflow
from airflow.operators.python_operator import PythonOperator
from airflow.models import DAG

args = {
'owner': 'airflow',
'start_date': airflow.utils.dates.days_ago(2)
}

dag = DAG(
dag_id='example_kubernetes_annotation', default_args=args,
schedule_interval=None
)


def print_stuff():
print("annotated!")


# You can use annotations on your kubernetes pods!
start_task = PythonOperator(
task_id="start_task", python_callable=print_stuff, dag=dag,
executor_config={
"KubernetesExecutor": {
"annotations": {"test": "annotation"}
}
}
)
14 changes: 9 additions & 5 deletions scripts/ci/5-run-tests.sh
Original file line number Diff line number Diff line change
Expand Up @@ -58,9 +58,11 @@ which airflow > /dev/null || python setup.py develop
# (which contains /usr/local/bin)
sudo ln -sf "${VIRTUAL_ENV}/bin/airflow" /usr/local/bin/

echo "Initializing the DB"
yes | airflow initdb
yes | airflow resetdb
if [ -z "$KUBERNETES_VERSION" ]; then
echo "Initializing the DB"
yes | airflow initdb
yes | airflow resetdb
fi

if [ -z "$nose_args" ]; then
nose_args="--with-coverage \
Expand All @@ -75,8 +77,10 @@ if [ -z "$nose_args" ]; then
--logging-level=DEBUG"
fi

# kdc init happens in setup_kdc.sh
kinit -kt ${KRB5_KTNAME} airflow
if [ -z "$KUBERNETES_VERSION" ]; then
# kdc init happens in setup_kdc.sh
kinit -kt ${KRB5_KTNAME} airflow
fi

# For impersonation tests running on SQLite on Travis, make the database world readable so other
# users can update it
Expand Down
28 changes: 28 additions & 0 deletions scripts/ci/docker-compose-kubernetes.yml
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
#
# 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.

version: "2.2"
services:
airflow-testing:
network_mode: host
environment:
- KUBERNETES_VERSION
- MINIKUBE_IP
volumes:
- /usr/local/bin/kubectl:/usr/local/bin/kubectl
- /usr/local/bin/minikube:/usr/local/bin/minikube
- ~/.kube:/home/airflow/.kube
- ~/.minikube:/home/airflow/.minikube
2 changes: 1 addition & 1 deletion scripts/ci/kubernetes/docker/airflow-test-env-init.sh
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
# under the License.

cd /usr/local/lib/python2.7/dist-packages/airflow && \
cp -R example_dags/* /root/airflow/dags/ && \
cp -R example_dags_kubernetes/* /root/airflow/dags/ && \
airflow initdb && \
alembic upgrade heads && \
(airflow create_user -u airflow -l airflow -f jon -e [email protected] -r Admin -p airflow || true) && \
Expand Down
10 changes: 7 additions & 3 deletions scripts/ci/kubernetes/docker/build.sh
Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,13 @@ TAG=${2:-latest}
DIRNAME=$(cd "$(dirname "$0")"; pwd)
AIRFLOW_ROOT="$DIRNAME/../../../.."

ENVCONFIG=$(minikube docker-env)
if [ $? -eq 0 ]; then
eval $ENVCONFIG
set -e

if [ "${VM_DRIVER:-none}" != "none" ]; then
ENVCONFIG=$(minikube docker-env)
if [ $? -eq 0 ]; then
eval $ENVCONFIG
fi
fi

echo "Airflow directory $AIRFLOW_ROOT"
Expand Down
1 change: 0 additions & 1 deletion scripts/ci/kubernetes/kube/airflow.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -151,4 +151,3 @@ spec:
nodePort: 30809
selector:
name: airflow

2 changes: 1 addition & 1 deletion scripts/ci/kubernetes/kube/configmaps.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ data:
logging_level = INFO
executor = KubernetesExecutor
parallelism = 32
load_examples = True
load_examples = False
plugins_folder = /root/airflow/plugins
sql_alchemy_conn = $SQL_ALCHEMY_CONN
Expand Down
2 changes: 2 additions & 0 deletions scripts/ci/kubernetes/kube/deploy.sh
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@
# specific language governing permissions and limitations *
# under the License. *

set -x

IMAGE=${1:-airflow/ci}
TAG=${2:-latest}
DIRNAME=$(cd "$(dirname "$0")"; pwd)
Expand Down
5 changes: 3 additions & 2 deletions scripts/ci/kubernetes/minikube/start_minikube.sh
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,13 @@

#!/usr/bin/env bash

set -x

_MY_SCRIPT="${BASH_SOURCE[0]}"
_MY_DIR=$(cd "$(dirname "$_MY_SCRIPT")" && pwd)
# Avoids 1.7.x because of https://github.com/kubernetes/minikube/issues/2240
_KUBERNETES_VERSION="${KUBERNETES_VERSION}"
_MINIKUBE_VERSION="${MINIKUBE_VERSION:-v0.26.0}"
_MINIKUBE_VERSION="${MINIKUBE_VERSION:-v0.28.2}"

echo "setting up kubernetes ${_KUBERNETES_VERSION}, using minikube ${_MINIKUBE_VERSION}"

Expand Down Expand Up @@ -116,7 +117,7 @@ echo Showing kube-system pods
kubectl get -n kube-system pods

(k8s_single_pod_ready -n kube-system -l component=kube-addon-manager) ||
(_ADDON=$(kubectl get pod -n kube-system -l component=kube-addon-manager
(_ADDON=$(kubectl get pod -n kube-system -l component=kube-addon-manager \
--no-headers -o name| cut -d/ -f2);
echo Addon-manager describe:;
kubectl describe pod -n kube-system $_ADDON;
Expand Down
1 change: 0 additions & 1 deletion scripts/ci/kubernetes/setup_kubernetes.sh
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,5 @@ DIRNAME=$(cd "$(dirname "$0")"; pwd)

$DIRNAME/minikube/start_minikube.sh
$DIRNAME/docker/build.sh
$DIRNAME/kube/deploy.sh

echo "Airflow environment on kubernetes is good to go!"
10 changes: 8 additions & 2 deletions scripts/ci/run-ci.sh
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ DIRNAME=$(cd "$(dirname "$0")"; pwd)
AIRFLOW_ROOT="$DIRNAME/../.."

# Fix file permissions
sudo chown -R airflow.airflow . $HOME/.wheelhouse/ $HOME/.cache/pip
sudo chown -R airflow.airflow . $HOME/.cache $HOME/.wheelhouse/ $HOME/.cache/pip $HOME/.kube $HOME/.minikube

if [[ $PYTHON_VERSION == '3' ]]; then
PIP=pip3
Expand All @@ -41,7 +41,13 @@ if [ -z "$KUBERNETES_VERSION" ];
then
tox -e $TOX_ENV
else
KUBERNETES_VERSION=${KUBERNETES_VERSION} $DIRNAME/kubernetes/setup_kubernetes.sh && \
# This script runs inside a container, the path of the kubernetes certificate
# is /home/travis/.minikube/client.crt but the user in the container is `airflow`
if [ ! -d /home/travis ]; then
sudo mkdir -p /home/travis
fi
sudo ln -s /home/airflow/.minikube /home/travis/.minikube

tox -e $TOX_ENV -- tests.contrib.minikube \
--with-coverage \
--cover-erase \
Expand Down
24 changes: 16 additions & 8 deletions tests/contrib/minikube/test_kubernetes_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
# under the License.


import os
import unittest
from subprocess import check_call, check_output
import requests.exceptions
Expand All @@ -25,18 +26,25 @@
import re

try:
check_call(["kubectl", "get", "pods"])
check_call(["/usr/local/bin/kubectl", "get", "pods"])
except Exception as e:
raise unittest.SkipTest(
"Kubernetes integration tests require a minikube cluster;"
"Skipping tests {}".format(e)
)
if os.environ.get('KUBERNETES_VERSION'):
raise e
else:
raise unittest.SkipTest(
"Kubernetes integration tests require a minikube cluster;"
"Skipping tests {}".format(e)
)


def get_minikube_host():
host_ip = check_output(['minikube', 'ip'])
if six.PY3:
host_ip = host_ip.decode('UTF-8')
if "MINIKUBE_IP" in os.environ:
host_ip = os.environ['MINIKUBE_IP']
else:
host_ip = check_output(['/usr/local/bin/minikube', 'ip'])
if six.PY3:
host_ip = host_ip.decode('UTF-8')

host = '{}:30809'.format(host_ip.strip())
return host

Expand Down
13 changes: 8 additions & 5 deletions tests/contrib/minikube/test_kubernetes_pod_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,12 +29,15 @@
from airflow.contrib.kubernetes.volume import Volume

try:
check_call(["kubectl", "get", "pods"])
check_call(["/usr/local/bin/kubectl", "get", "pods"])
except Exception as e:
raise unittest.SkipTest(
"Kubernetes integration tests require a minikube cluster;"
"Skipping tests {}".format(e)
)
if os.environ.get('KUBERNETES_VERSION'):
raise e
else:
raise unittest.SkipTest(
"Kubernetes integration tests require a minikube cluster;"
"Skipping tests {}".format(e)
)


class KubernetesPodOperatorTest(unittest.TestCase):
Expand Down
3 changes: 2 additions & 1 deletion tests/core.py
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,8 @@ def reset(dag_id=TEST_DAG_ID):


configuration.conf.load_test_config()
reset()
if os.environ.get('KUBERNETES_VERSION') is None:
reset()


class OperatorSubclass(BaseOperator):
Expand Down
10 changes: 5 additions & 5 deletions tox.ini
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
# under the License.

[tox]
envlist = flake8,{py27,py35}-backend_{mysql,sqlite,postgres}
envlist = flake8,{py27,py35}-backend_{mysql,sqlite,postgres}-env_{docker,kubernetes}
skipsdist = True

[global]
Expand Down Expand Up @@ -57,10 +57,10 @@ passenv = *
commands =
pip wheel --progress-bar off -w {homedir}/.wheelhouse -f {homedir}/.wheelhouse -e .[devel_ci]
pip install --progress-bar off --find-links={homedir}/.wheelhouse --no-index -e .[devel_ci]
{toxinidir}/scripts/ci/1-setup-env.sh
{toxinidir}/scripts/ci/2-setup-kdc.sh
{toxinidir}/scripts/ci/3-setup-databases.sh
{toxinidir}/scripts/ci/4-load-data.sh
env_docker: {toxinidir}/scripts/ci/1-setup-env.sh
env_docker: {toxinidir}/scripts/ci/2-setup-kdc.sh
env_docker: {toxinidir}/scripts/ci/3-setup-databases.sh
env_docker: {toxinidir}/scripts/ci/4-load-data.sh
{toxinidir}/scripts/ci/5-run-tests.sh []
{toxinidir}/scripts/ci/6-check-license.sh
codecov -e TOXENV
Expand Down

0 comments on commit 8038f88

Please sign in to comment.