Skip to content

Commit

Permalink
Use production image for k8s tests (#9038)
Browse files Browse the repository at this point in the history
    * Use production image for k8s tests

    The CI image has become too large to load into KinD,

    it also only really makes sense to use the production image for
    integration tests

    * nit

    Co-authored-by: Daniel Imberman <daniel@astronomer.io>
    (cherry picked from commit e4d811d)
  • Loading branch information
astro-sql-decorator committed May 28, 2020
1 parent a09d5ec commit 77e6311
Show file tree
Hide file tree
Showing 13 changed files with 215 additions and 151 deletions.
40 changes: 39 additions & 1 deletion .github/workflows/ci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ jobs:
- name: Cache pre-commit env
uses: actions/cache@v1
env:
cache-name: cache-pre-commit
cache-name: cache-pre-commit-epoch2
with:
path: ~/.cache/pre-commit
key: ${{ env.cache-name }}-${{ github.job }}-${{ hashFiles('.pre-commit-config.yaml') }}
Expand Down Expand Up @@ -96,6 +96,44 @@ jobs:
- name: "Build PROD image ${{ matrix.python-version }}"
run: ./scripts/ci/ci_build_production_images.sh

tests-kubernetes:
timeout-minutes: 80
name: "kubernetes-test-${{matrix.kube-mode}}-kube"
runs-on: ubuntu-latest
needs: [static-checks]
strategy:
matrix:
kube-mode:
- persistent_mode
# #TODO These tests will be unblocked when k8s tests run on host
# - git_mode
kubernetes-version:
- "v1.15.3"
fail-fast: false
env:
BACKEND: postgres
TEST_TYPE: ${{ matrix.test-type }}
RUN_TESTS: "true"
CI_JOB_TYPE: "Tests"
PYTHON_MAJOR_MINOR_VERSION: "3.6"
SKIP_CI_IMAGE_CHECK: "true"
RUNTIME: "kubernetes"
ENABLE_KIND_CLUSTER: "true"
KUBERNETES_MODE: "${{ matrix.kube-mode }}"
KUBERNETES_VERSION: "${{ matrix.kubernetes-version }}"
steps:
- uses: actions/checkout@master
- uses: actions/setup-python@v1
with:
python-version: '3.x'
- name: "Free space"
run: ./scripts/ci/ci_free_space_on_ci.sh
- name: "Build CI image ${{ matrix.python-version }}"
run: ./scripts/ci/ci_prepare_image_on_ci.sh
- name: "Build PROD image ${{ matrix.python-version }}"
run: ./scripts/ci/ci_build_production_images.sh
- name: "Tests"
run: ./scripts/ci/ci_run_airflow_testing.sh
tests-postgres:
timeout-minutes: 80
name: "${{matrix.test-type}}:Pg${{matrix.postgres-version}},Py${{matrix.python-version}}"
Expand Down
81 changes: 0 additions & 81 deletions .travis.yml

This file was deleted.

2 changes: 1 addition & 1 deletion airflow/config_templates/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -2120,7 +2120,7 @@
version_added: 1.10.3
type: string
example: ~
default: ""
default: "50000"
- name: fs_group
description: |
Specifies a gid to associate with all containers in the worker pods
Expand Down
2 changes: 1 addition & 1 deletion airflow/config_templates/default_airflow.cfg
Original file line number Diff line number Diff line change
Expand Up @@ -1005,7 +1005,7 @@ tolerations =
kube_client_request_args =

# Specifies the uid to run the first process of the worker pods containers as
run_as_user =
run_as_user = 50000

# Specifies a gid to associate with all containers in the worker pods
# if using a git_ssh_key_secret_name use an fs_group
Expand Down
40 changes: 30 additions & 10 deletions scripts/ci/in_container/kubernetes/app/deploy_app.sh
Original file line number Diff line number Diff line change
Expand Up @@ -56,19 +56,19 @@ rm -f "${BUILD_DIRNAME}"/*
if [[ "${KUBERNETES_MODE}" == "persistent_mode" ]]; then
INIT_DAGS_VOLUME_NAME=airflow-dags
POD_AIRFLOW_DAGS_VOLUME_NAME=airflow-dags
CONFIGMAP_DAGS_FOLDER=/root/airflow/dags
CONFIGMAP_DAGS_FOLDER=/opt/airflow/dags
CONFIGMAP_GIT_DAGS_FOLDER_MOUNT_POINT=
CONFIGMAP_DAGS_VOLUME_CLAIM=airflow-dags
else
INIT_DAGS_VOLUME_NAME=airflow-dags-fake
POD_AIRFLOW_DAGS_VOLUME_NAME=airflow-dags-git
CONFIGMAP_DAGS_FOLDER=/root/airflow/dags/repo/airflow/contrib/example_dags
CONFIGMAP_GIT_DAGS_FOLDER_MOUNT_POINT=/root/airflow/dags
CONFIGMAP_DAGS_FOLDER=/opt/airflow/dags/repo/airflow/example_dags
CONFIGMAP_GIT_DAGS_FOLDER_MOUNT_POINT=/opt/airflow/dags
CONFIGMAP_DAGS_VOLUME_CLAIM=
fi

CONFIGMAP_GIT_REPO=${CI_TARGET_REPO:-apache/airflow}
CONFIGMAP_BRANCH=${CI_TARGET_BRANCH:=master}
CONFIGMAP_GIT_REPO=${GITHUB_REPOSITORY:-apache/airflow}
CONFIGMAP_BRANCH=${GITHUB_BASE_REF:=master}

if [[ "${KUBERNETES_MODE}" == "persistent_mode" ]]; then
sed -e "s/{{INIT_GIT_SYNC}}//g" \
Expand Down Expand Up @@ -111,9 +111,28 @@ kubectl apply -f "${MY_DIR}/secrets.yaml"
kubectl apply -f "${BUILD_DIRNAME}/configmaps.yaml"
kubectl apply -f "${MY_DIR}/postgres.yaml"
kubectl apply -f "${MY_DIR}/volumes.yaml"

set +x
set +o pipefail
PODS_ARE_READY="0"
for i in {1..150}; do
echo "------- Running kubectl get pods: $i -------"
PODS=$(kubectl get pods | awk 'NR>1 {print $0}')
echo "$PODS"
NUM_POSTGRES_READY=$(echo "${PODS}" | grep postgres | awk '{print $2}' | grep -cE '([1-9])\/(\1)' | xargs)
if [[ "${NUM_POSTGRES_READY}" == "1" ]]; then
PODS_ARE_READY="1"
break
fi
sleep 4
done

sleep 7

kubectl apply -f "${BUILD_DIRNAME}/airflow.yaml"

dump_logs() {
echo "dumping logs"
POD=$(kubectl get pods -o go-template --template '{{range .items}}{{.metadata.name}}{{"\n"}}{{end}}' | grep airflow | head -1)
echo "------- pod description -------"
kubectl describe pod "${POD}"
Expand All @@ -134,20 +153,20 @@ set +x
set +o pipefail
# wait for up to 10 minutes for everything to be deployed
PODS_ARE_READY="0"
for i in {1..150}; do
for i in {1..20}; do
echo "------- Running kubectl get pods: $i -------"
PODS=$(kubectl get pods | awk 'NR>1 {print $0}')
echo "$PODS"
NUM_AIRFLOW_READY=$(echo "${PODS}" | grep airflow | awk '{print $2}' | grep -cE '([0-9])\/(\1)' | xargs)
NUM_POSTGRES_READY=$(echo "${PODS}" | grep postgres | awk '{print $2}' | grep -cE '([0-9])\/(\1)' | xargs)
NUM_AIRFLOW_READY=$(echo "${PODS}" | grep airflow | awk '{print $2}' | grep -cE '([2-9])\/(\1)' | xargs)
NUM_POSTGRES_READY=$(echo "${PODS}" | grep postgres | awk '{print $2}' | grep -cE '([1-9])\/(\1)' | xargs)
if [[ "${NUM_AIRFLOW_READY}" == "1" && "${NUM_POSTGRES_READY}" == "1" ]]; then
PODS_ARE_READY="1"
break
fi
sleep 4
done
POD=$(kubectl get pods -o go-template --template '{{range .items}}{{.metadata.name}}{{"\n"}}{{end}}' | grep airflow | head -1)

dump_logs
if [[ "${PODS_ARE_READY}" == "1" ]]; then
echo "PODS are ready."
else
Expand All @@ -160,7 +179,7 @@ fi
KUBERNETES_HOST=${CLUSTER_NAME}-worker
AIRFLOW_WEBSERVER_IS_READY="0"
CONSECUTIVE_SUCCESS_CALLS=0
for i in {1..30}; do
for i in {1..20}; do
echo "------- Wait until webserver is up: $i -------"
PODS=$(kubectl get pods | awk 'NR>1 {print $0}')
echo "$PODS"
Expand All @@ -182,5 +201,6 @@ if [[ "${AIRFLOW_WEBSERVER_IS_READY}" == "1" ]]; then
echo "Airflow webserver is ready."
else
echo >&2 "Airflow webserver is not ready after waiting for a long time. Exiting..."
dump_logs
exit 1
fi
Original file line number Diff line number Diff line change
Expand Up @@ -41,18 +41,20 @@ spec:
labels:
name: airflow
spec:
securityContext:
runAsUser: 50000
runAsGroup: 50000
fsGroup: 50000
initContainers:
- name: "init"
image: {{AIRFLOW_KUBERNETES_IMAGE}}
imagePullPolicy: IfNotPresent
volumeMounts:
- name: airflow-configmap
mountPath: /root/airflow/airflow.cfg
mountPath: /opt/airflow/airflow.cfg
subPath: airflow.cfg
- name: {{INIT_DAGS_VOLUME_NAME}}
mountPath: /root/airflow/dags
- name: test-volume
mountPath: /root/test_volume
mountPath: /opt/airflow/dags
env:
- name: SQL_ALCHEMY_CONN
valueFrom:
Expand Down Expand Up @@ -85,12 +87,12 @@ spec:
key: sql_alchemy_conn
volumeMounts:
- name: airflow-configmap
mountPath: /root/airflow/airflow.cfg
mountPath: /opt/airflow/airflow.cfg
subPath: airflow.cfg
- name: {{POD_AIRFLOW_DAGS_VOLUME_NAME}}
mountPath: /root/airflow/dags
mountPath: /opt/airflow/dags
- name: airflow-logs
mountPath: /root/airflow/logs
mountPath: /opt/airflow/logs
- name: scheduler
image: {{AIRFLOW_KUBERNETES_IMAGE}}
imagePullPolicy: IfNotPresent
Expand All @@ -107,12 +109,12 @@ spec:
key: sql_alchemy_conn
volumeMounts:
- name: airflow-configmap
mountPath: /root/airflow/airflow.cfg
mountPath: /opt/airflow/airflow.cfg
subPath: airflow.cfg
- name: {{POD_AIRFLOW_DAGS_VOLUME_NAME}}
mountPath: /root/airflow/dags
mountPath: /opt/airflow/dags
- name: airflow-logs
mountPath: /root/airflow/logs
mountPath: /opt/airflow/logs
volumes:
- name: airflow-dags
persistentVolumeClaim:
Expand All @@ -121,12 +123,8 @@ spec:
emptyDir: {}
- name: airflow-dags-git
emptyDir: {}
- name: test-volume
persistentVolumeClaim:
claimName: test-volume
- name: airflow-logs
persistentVolumeClaim:
claimName: airflow-logs
emptyDir: {}
- name: airflow-configmap
configMap:
name: airflow-configmap
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,18 +24,18 @@ data:
airflow.cfg: |
[core]
dags_folder = {{CONFIGMAP_DAGS_FOLDER}}
base_log_folder = /root/airflow/logs
base_log_folder = /opt/airflow/logs
logging_level = INFO
executor = KubernetesExecutor
parallelism = 32
load_examples = False
load_default_connections = True
plugins_folder = /root/airflow/plugins
plugins_folder = /opt/airflow/plugins
sql_alchemy_conn = $SQL_ALCHEMY_CONN
[scheduler]
dag_dir_list_interval = 300
child_process_log_directory = /root/airflow/logs/scheduler
child_process_log_directory = /opt/airflow/logs/scheduler
# Task instances listen for external kill signal (when you clear tasks
# from the CLI or the UI), this defines the frequency at which they should
# listen (in seconds).
Expand Down Expand Up @@ -193,12 +193,13 @@ data:
git_dags_folder_mount_point = {{CONFIGMAP_GIT_DAGS_FOLDER_MOUNT_POINT}}
dags_volume_claim = {{CONFIGMAP_DAGS_VOLUME_CLAIM}}
dags_volume_subpath =
logs_volume_claim = airflow-logs
logs_volume_claim =
logs_volume_subpath =
dags_volume_host =
logs_volume_host =
in_cluster = True
namespace = default
run_as_user = 50000
gcp_service_account_keys =
# Example affinity and toleration definitions.
Expand Down
4 changes: 3 additions & 1 deletion scripts/ci/in_container/kubernetes/app/volumes.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@ kind: PersistentVolume
apiVersion: v1
metadata:
name: airflow-logs
annotations:
pv.beta.kubernetes.io/gid: "50000"
spec:
accessModes:
- ReadWriteMany
Expand All @@ -67,7 +69,7 @@ metadata:
name: test-volume
spec:
accessModes:
- ReadWriteOnce
- ReadWriteMany
capacity:
storage: 2Gi
hostPath:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,6 @@

set -x

cd /opt/airflow/airflow && \
cp -R example_dags/* /root/airflow/dags/ && \
cp -R contrib/example_dags/example_kubernetes_*.py /root/airflow/dags/ && \
cp -a contrib/example_dags/libs /root/airflow/dags/ && \
cd /opt/airflow && \
airflow initdb && \
alembic upgrade heads && \
(airflow create_user -u airflow -l airflow -f jon -e airflow@apache.org -r Admin -p airflow || true) && \
echo "retrieved from mount" > /root/test_volume/test.txt
(airflow create_user -u airflow -l airflow -f jon -e airflow@apache.org -r Admin -p airflow || true)
Empty file modified scripts/ci/in_container/kubernetes/docker/bootstrap.sh
100644 → 100755
Empty file.

0 comments on commit 77e6311

Please sign in to comment.