You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by di...@apache.org on 2020/05/28 16:54:40 UTC

[airflow] 01/01: Use production image for k8s tests (#9038)

This is an automated email from the ASF dual-hosted git repository.

dimberman pushed a commit to branch v1-10-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 77e63116a03abafb8aeeacc09fc4f9411e1832cb
Author: Daniel Imberman <da...@astronomer.io>
AuthorDate: Thu May 28 09:53:03 2020 -0700

        Use production image for k8s tests (#9038)
    
        * 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 <da...@astronomer.io>
        (cherry picked from commit e4d811db8613fdbca9c6af0a2ab90ed755bef7e2)
---
 .github/workflows/ci.yml                           |  40 +++++++-
 .travis.yml                                        |  81 ---------------
 airflow/config_templates/config.yml                |   2 +-
 airflow/config_templates/default_airflow.cfg       |   2 +-
 .../example_kubernetes_executor_config.py          |   0
 .../ci/in_container/kubernetes/app/deploy_app.sh   |  40 ++++++--
 .../kubernetes/app/templates/airflow.template.yaml |  28 +++---
 .../app/templates/configmaps.template.yaml         |   9 +-
 .../ci/in_container/kubernetes/app/volumes.yaml    |   4 +-
 .../kubernetes/docker/airflow-test-env-init.sh     |   9 +-
 .../ci/in_container/kubernetes/docker/bootstrap.sh |   0
 .../kubernetes/docker/rebuild_airflow_image.sh     |  41 +++-----
 scripts/docker/entrypoint.sh                       | 110 +++++++++++++++++++++
 13 files changed, 215 insertions(+), 151 deletions(-)

diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml
index a18f64a..e2490d6 100644
--- a/.github/workflows/ci.yml
+++ b/.github/workflows/ci.yml
@@ -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') }}
@@ -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}}"
diff --git a/.travis.yml b/.travis.yml
deleted file mode 100644
index 5c4e655..0000000
--- a/.travis.yml
+++ /dev/null
@@ -1,81 +0,0 @@
-# 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.
-#
----
-dist: bionic
-language: python
-os: linux
-env:
-  global:
-    - MOUNT_LOCAL_SOURCES="false"
-    - MOUNT_HOST_AIRFLOW_VOLUME="true"
-    - FORCE_ANSWER_TO_QUESTIONS="yes"
-    - SKIP_CHECK_REMOTE_IMAGE="true"
-    - DB_RESET="true"
-    - VERBOSE="true"
-    - CI_JOB_TYPE="Kubernetes tests"
-python: "3.6"
-stages:
-  - test
-services:
-  - docker
-jobs:
-  include:
-    - name: "Tests [Py3.6][Kubernetes][persistent]"
-      env: >-
-        BACKEND=postgres
-        PYTHON_MAJOR_MINOR_VERSION=3.6
-        RUNTIME=kubernetes
-        ENABLE_KIND_CLUSTER=true
-        KUBERNETES_MODE=persistent_mode
-        KUBERNETES_VERSION=v1.15.3
-      python: "3.6"
-      stage: test
-    - name: "Tests [Py3.5][Kubernetes][git]"
-      env: >-
-        BACKEND=postgres
-        PYTHON_MAJOR_MINOR_VERSION=3.5
-        RUNTIME=kubernetes
-        ENABLE_KIND_CLUSTER=true
-        KUBERNETES_MODE=git_mode
-        KUBERNETES_VERSION=v1.15.3
-      python: "3.6"
-      stage: test
-    - name: "Tests [Py2.7][Kubernetes][persistent]"
-      env: >-
-        BACKEND=postgres
-        PYTHON_MAJOR_MINOR_VERSION=2.7
-        RUNTIME=kubernetes
-        ENABLE_KIND_CLUSTER=true
-        KUBERNETES_MODE=persistent_mode
-        KUBERNETES_VERSION=v1.15.3
-      python: "3.6"
-      stage: test
-    - name: "Tests [Py2.7][Kubernetes][git]"
-      env: >-
-        BACKEND=postgres
-        PYTHON_MAJOR_MINOR_VERSION=2.7
-        RUNTIME=kubernetes
-        ENABLE_KIND_CLUSTER=true
-        KUBERNETES_MODE=git_mode
-        KUBERNETES_VERSION=v1.15.3
-      python: "3.6"
-      stage: test
-install: skip
-before_install:
-  - ./scripts/ci/ci_prepare_image_on_ci.sh
-script: ./scripts/ci/ci_run_airflow_testing.sh
diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml
index 4ae7f6c..26cfae8 100644
--- a/airflow/config_templates/config.yml
+++ b/airflow/config_templates/config.yml
@@ -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
diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg
index 6f3f813..5f240de 100644
--- a/airflow/config_templates/default_airflow.cfg
+++ b/airflow/config_templates/default_airflow.cfg
@@ -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
diff --git a/airflow/contrib/example_dags/example_kubernetes_executor_config.py b/airflow/example_dags/example_kubernetes_executor_config.py
similarity index 100%
rename from airflow/contrib/example_dags/example_kubernetes_executor_config.py
rename to airflow/example_dags/example_kubernetes_executor_config.py
diff --git a/scripts/ci/in_container/kubernetes/app/deploy_app.sh b/scripts/ci/in_container/kubernetes/app/deploy_app.sh
index 9bf61c7..6ebc7b0 100755
--- a/scripts/ci/in_container/kubernetes/app/deploy_app.sh
+++ b/scripts/ci/in_container/kubernetes/app/deploy_app.sh
@@ -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" \
@@ -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}"
@@ -134,12 +153,12 @@ 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
@@ -147,7 +166,7 @@ for i in {1..150}; do
     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
@@ -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"
@@ -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
diff --git a/scripts/ci/in_container/kubernetes/app/templates/airflow.template.yaml b/scripts/ci/in_container/kubernetes/app/templates/airflow.template.yaml
index 08b5afb..0ee9492 100644
--- a/scripts/ci/in_container/kubernetes/app/templates/airflow.template.yaml
+++ b/scripts/ci/in_container/kubernetes/app/templates/airflow.template.yaml
@@ -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:
@@ -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
@@ -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:
@@ -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
diff --git a/scripts/ci/in_container/kubernetes/app/templates/configmaps.template.yaml b/scripts/ci/in_container/kubernetes/app/templates/configmaps.template.yaml
index 7af3332..868da7c 100644
--- a/scripts/ci/in_container/kubernetes/app/templates/configmaps.template.yaml
+++ b/scripts/ci/in_container/kubernetes/app/templates/configmaps.template.yaml
@@ -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).
@@ -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.
diff --git a/scripts/ci/in_container/kubernetes/app/volumes.yaml b/scripts/ci/in_container/kubernetes/app/volumes.yaml
index ea1b40c..5d366a1 100644
--- a/scripts/ci/in_container/kubernetes/app/volumes.yaml
+++ b/scripts/ci/in_container/kubernetes/app/volumes.yaml
@@ -42,6 +42,8 @@ kind: PersistentVolume
 apiVersion: v1
 metadata:
   name: airflow-logs
+  annotations:
+    pv.beta.kubernetes.io/gid: "50000"
 spec:
   accessModes:
     - ReadWriteMany
@@ -67,7 +69,7 @@ metadata:
   name: test-volume
 spec:
   accessModes:
-    - ReadWriteOnce
+    - ReadWriteMany
   capacity:
     storage: 2Gi
   hostPath:
diff --git a/scripts/ci/in_container/kubernetes/docker/airflow-test-env-init.sh b/scripts/ci/in_container/kubernetes/docker/airflow-test-env-init.sh
index 1f8223a..8e15f0f 100755
--- a/scripts/ci/in_container/kubernetes/docker/airflow-test-env-init.sh
+++ b/scripts/ci/in_container/kubernetes/docker/airflow-test-env-init.sh
@@ -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)
diff --git a/scripts/ci/in_container/kubernetes/docker/bootstrap.sh b/scripts/ci/in_container/kubernetes/docker/bootstrap.sh
old mode 100644
new mode 100755
diff --git a/scripts/ci/in_container/kubernetes/docker/rebuild_airflow_image.sh b/scripts/ci/in_container/kubernetes/docker/rebuild_airflow_image.sh
index 60ebffa..1df822e 100755
--- a/scripts/ci/in_container/kubernetes/docker/rebuild_airflow_image.sh
+++ b/scripts/ci/in_container/kubernetes/docker/rebuild_airflow_image.sh
@@ -38,44 +38,26 @@ cp /entrypoint.sh scripts/docker/
 echo
 echo "Building image from ${AIRFLOW_CI_IMAGE} with latest sources"
 echo
-start_output_heartbeat "Rebuilding Kubernetes image" 3
-docker build \
-    --build-arg PYTHON_BASE_IMAGE="${PYTHON_BASE_IMAGE}" \
-    --build-arg PYTHON_MAJOR_MINOR_VERSION="${PYTHON_MAJOR_MINOR_VERSION}" \
-    --build-arg AIRFLOW_VERSION="${AIRFLOW_VERSION}" \
-    --build-arg AIRFLOW_EXTRAS="${AIRFLOW_EXTRAS}" \
-    --build-arg AIRFLOW_BRANCH="${AIRFLOW_BRANCH}" \
-    --build-arg AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD="${AIRFLOW_CONTAINER_CI_OPTIMISED_BUILD}" \
-    --build-arg UPGRADE_TO_LATEST_REQUIREMENTS="${UPGRADE_TO_LATEST_REQUIREMENTS}" \
-    --build-arg HOME="${HOME}" \
-    --cache-from "${AIRFLOW_CI_IMAGE}" \
-    --tag="${AIRFLOW_CI_IMAGE}" \
-    --target="main" \
-    -f Dockerfile.ci . >> "${OUTPUT_LOG}"
-echo
+#export AIRFLOW_PROD_BASE_TAG="${DEFAULT_BRANCH}-python${PYTHON_MAJOR_MINOR_VERSION}"
+#export AIRFLOW_PROD_IMAGE="${DOCKERHUB_USER}/${DOCKERHUB_REPO}:${AIRFLOW_PROD_BASE_TAG}"
+export AIRFLOW_PROD_IMAGE="apache/airflow:v1-10-test-python3.6"
 echo "Adding kubernetes-specific scripts to basic CI image."
-echo "Building ${AIRFLOW_KUBERNETES_IMAGE} from ${AIRFLOW_CI_IMAGE}"
+echo "Building ${AIRFLOW_KUBERNETES_IMAGE} from ${AIRFLOW_PROD_IMAGE}"
 echo
 docker build \
-    --build-arg AIRFLOW_CI_IMAGE="${AIRFLOW_CI_IMAGE}" \
-    --cache-from "${AIRFLOW_CI_IMAGE}" \
+    --build-arg AIRFLOW_PROD_IMAGE="${AIRFLOW_PROD_IMAGE}" \
+    --cache-from "${AIRFLOW_PROD_IMAGE}" \
     --tag="${AIRFLOW_KUBERNETES_IMAGE}" \
-    -f- . >> "${OUTPUT_LOG}" <<EOF
-ARG AIRFLOW_CI_IMAGE
-FROM ${AIRFLOW_CI_IMAGE}
+    -f- .  <<EOF
+ARG AIRFLOW_PROD_IMAGE
+FROM ${AIRFLOW_PROD_IMAGE}
 
 COPY scripts/ci/in_container/kubernetes/docker/airflow-test-env-init.sh /tmp/airflow-test-env-init.sh
-COPY scripts/ci/in_container/kubernetes/docker/bootstrap.sh /bootstrap.sh
-
-RUN chmod +x /bootstrap.sh
-
-COPY airflow/ ${AIRFLOW_SOURCES}/airflow/
 
-ENTRYPOINT ["/bootstrap.sh"]
+ENV AIRFLOW__CORE__LOAD_EXAMPLES="true"
+ENTRYPOINT ["/usr/bin/dumb-init", "--", "/entrypoint"]
 EOF
-stop_output_heartbeat
 
-start_output_heartbeat "Loading image to Kind cluster" 10
 echo
 echo "Loading the ${AIRFLOW_KUBERNETES_IMAGE} to cluster ${CLUSTER_NAME} from docker"
 echo
@@ -88,6 +70,5 @@ echo
 echo "Stopping output heartbeat"
 echo
 
-stop_output_heartbeat
 
 in_container_script_end
diff --git a/scripts/docker/entrypoint.sh b/scripts/docker/entrypoint.sh
new file mode 100755
index 0000000..3d436e2
--- /dev/null
+++ b/scripts/docker/entrypoint.sh
@@ -0,0 +1,110 @@
+#!/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.
+
+# Might be empty
+AIRFLOW_COMMAND="${1}"
+
+set -euo pipefail
+
+function verify_db_connection {
+    DB_URL="${1}"
+
+    DB_CHECK_MAX_COUNT=${MAX_DB_CHECK_COUNT:=20}
+    DB_CHECK_SLEEP_TIME=${DB_CHECK_SLEEP_TIME:=3}
+
+    local DETECTED_DB_BACKEND=""
+    local DETECTED_DB_HOST=""
+    local DETECTED_DB_PORT=""
+
+
+    if [[ ${DB_URL} != sqlite* ]]; then
+        # Auto-detect DB parameters
+        [[ ${DB_URL} =~ ([^:]*)://([^@/]*)@?([^/:]*):?([0-9]*)/([^\?]*)\??(.*) ]] && \
+            DETECTED_DB_BACKEND=${BASH_REMATCH[1]} &&
+            # Not used USER match
+            DETECTED_DB_HOST=${BASH_REMATCH[3]} &&
+            DETECTED_DB_PORT=${BASH_REMATCH[4]} &&
+            # Not used SCHEMA match
+            # Not used PARAMS match
+
+        echo DB_BACKEND="${DB_BACKEND:=${DETECTED_DB_BACKEND}}"
+
+        if [[ -z "${DETECTED_DB_PORT}" ]]; then
+            if [[ ${DB_BACKEND} == "postgres"* ]]; then
+                DETECTED_DB_PORT=5432
+            elif [[ ${DB_BACKEND} == "mysql"* ]]; then
+                DETECTED_DB_PORT=3306
+            fi
+        fi
+
+        DETECTED_DB_HOST=${DETECTED_DB_HOST:="localhost"}
+
+        # Allow the DB parameters to be overridden by environment variable
+        echo DB_HOST="${DB_HOST:=${DETECTED_DB_HOST}}"
+        echo DB_PORT="${DB_PORT:=${DETECTED_DB_PORT}}"
+
+        while true
+        do
+            set +e
+            LAST_CHECK_RESULT=$(nc -zvv "${DB_HOST}" "${DB_PORT}" >/dev/null 2>&1)
+            RES=$?
+            set -e
+            if [[ ${RES} == 0 ]]; then
+                echo
+                break
+            else
+                echo -n "."
+                DB_CHECK_MAX_COUNT=$((DB_CHECK_MAX_COUNT-1))
+            fi
+            if [[ ${DB_CHECK_MAX_COUNT} == 0 ]]; then
+                echo
+                echo "ERROR! Maximum number of retries (${DB_CHECK_MAX_COUNT}) reached while checking ${DB_BACKEND} db. Exiting"
+                echo
+                break
+            else
+                sleep "${DB_CHECK_SLEEP_TIME}"
+            fi
+        done
+        if [[ ${RES} != 0 ]]; then
+            echo "        ERROR: ${BACKEND} db could not be reached!"
+            echo
+            echo "${LAST_CHECK_RESULT}"
+            echo
+            export EXIT_CODE=${RES}
+        fi
+    fi
+}
+
+# if no DB configured - use sqlite db by default
+AIRFLOW__CORE__SQL_ALCHEMY_CONN="${AIRFLOW__CORE__SQL_ALCHEMY_CONN:="sqlite:///${AIRFLOW_HOME}/airflow.db"}"
+
+verify_db_connection "${AIRFLOW__CORE__SQL_ALCHEMY_CONN}"
+
+AIRFLOW__CELERY__BROKER_URL=${AIRFLOW__CELERY__BROKER_URL:=}
+
+if [[ -n ${AIRFLOW__CELERY__BROKER_URL} ]] && \
+        [[ ${AIRFLOW_COMMAND} =~ ^(scheduler|worker|flower)$ ]]; then
+    verify_db_connection "${AIRFLOW__CELERY__BROKER_URL}"
+fi
+
+if [[ ${AIRFLOW_COMMAND} == "" ]]; then
+   exec "/bin/bash"
+fi
+
+# Run the command
+exec airflow "${@}"