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 2020/06/03 01:52:25 UTC

[GitHub] [airflow] dimberman commented on a change in pull request #8265: Kubernetes on host

dimberman commented on a change in pull request #8265:
URL: https://github.com/apache/airflow/pull/8265#discussion_r434263901



##########
File path: kubernetes_tests/test_kubernetes_pod_operator.py
##########
@@ -141,98 +143,11 @@ def test_config_path_move(self):
             do_xcom_push=False,
             config_file=new_config_path,
         )
-        context = self.create_context(k)
+        context = create_context(k)
         k.execute(context)
         actual_pod = self.api_client.sanitize_for_serialization(k.pod)
         self.assertEqual(self.expected_pod, actual_pod)
 
-    @mock.patch("airflow.kubernetes.pod_launcher.PodLauncher.start_pod")
-    @mock.patch("airflow.kubernetes.pod_launcher.PodLauncher.monitor_pod")
-    @mock.patch("airflow.kubernetes.kube_client.get_kube_client")
-    def test_config_path(self, client_mock, monitor_mock, start_mock):  # pylint: disable=unused-argument

Review comment:
       Why are these tests being deleted?

##########
File path: scripts/ci/_utils.sh
##########
@@ -1721,3 +1748,436 @@ function run_prepare_backport_readme() {
 function get_airflow_version_from_production_image() {
      docker run --entrypoint /bin/bash "${AIRFLOW_PROD_IMAGE}" -c 'echo "${AIRFLOW_VERSION}"'
 }
+
+function dump_kind_logs() {

Review comment:
       Let's move all KinD functions into a "_kind.sh" that can be imported by "_utils.sh". It's really hard to find functions with the current setup.

##########
File path: scripts/ci/kubernetes/app/templates/airflow.template.yaml
##########
@@ -46,31 +46,53 @@ spec:
         runAsGroup: 50000
         fsGroup: 50000
       initContainers:
-      - name: "init"
-        image: {{AIRFLOW_KUBERNETES_IMAGE}}
-        imagePullPolicy: IfNotPresent
-        volumeMounts:
-        - name: airflow-configmap
-          mountPath: /opt/airflow/airflow.cfg
-          subPath: airflow.cfg
-        - name: {{INIT_DAGS_VOLUME_NAME}}
-          mountPath: /opt/airflow/dags
-        env:
-        - name: SQL_ALCHEMY_CONN
-          valueFrom:
-            secretKeyRef:
-              name: airflow-secrets
-              key: sql_alchemy_conn
-        command:
-          - "bash"
-        args:
-          - "-cx"
-          - "/tmp/airflow-test-env-init.sh"
+        - name: "init-dags"

Review comment:
       good idea! Should make things move faster

##########
File path: scripts/ci/_utils.sh
##########
@@ -1721,3 +1748,436 @@ function run_prepare_backport_readme() {
 function get_airflow_version_from_production_image() {
      docker run --entrypoint /bin/bash "${AIRFLOW_PROD_IMAGE}" -c 'echo "${AIRFLOW_VERSION}"'
 }
+
+function dump_kind_logs() {
+    echo "###########################################################################################"
+    echo "                   Dumping logs from KIND"
+    echo "###########################################################################################"
+
+    FILE_NAME="${1}"
+    kind --name "${KIND_CLUSTER_NAME}" export logs "${FILE_NAME}"
+}
+
+
+function send_kubernetes_logs_to_file_io() {
+    echo "##############################################################################"
+    echo
+    echo "   DUMPING LOG FILES FROM KIND AND SENDING THEM TO file.io"
+    echo
+    echo "##############################################################################"
+    DUMP_DIR_NAME=$(date "+%Y-%m-%d")_kind_${CI_BUILD_ID:="default"}_${CI_JOB_ID:="default"}
+    DUMP_DIR=/tmp/${DUMP_DIR_NAME}
+    dump_kind_logs "${DUMP_DIR}"
+    tar -cvzf "${DUMP_DIR}.tar.gz" -C /tmp "${DUMP_DIR_NAME}"
+    echo
+    echo "   Logs saved to ${DUMP_DIR}.tar.gz"
+    echo
+    echo "##############################################################################"
+    curl -F "file=@${DUMP_DIR}.tar.gz" https://file.io
+}
+
+function check_kind_and_kubectl_are_installed() {
+    SYSTEM=$(uname -s| tr '[:upper:]' '[:lower:]')
+    KIND_VERSION="v0.7.0"
+    KIND_URL="https://github.com/kubernetes-sigs/kind/releases/download/${KIND_VERSION}/kind-${SYSTEM}-amd64"
+    KIND_PATH="${BUILD_CACHE_DIR}/bin/kind"
+    KUBECTL_VERSION="v1.15.3"
+    KUBECTL_URL="https://storage.googleapis.com/kubernetes-release/release/${KUBECTL_VERSION}/bin/${SYSTEM}/amd64/kubectl"
+    KUBECTL_PATH="${BUILD_CACHE_DIR}/bin/kubectl"
+    mkdir -pv "${BUILD_CACHE_DIR}/bin"
+    if [[ ! -f "${KIND_PATH}" ]]; then
+        echo
+        echo "Downloading Kind version ${KIND_VERSION}"
+        echo
+        curl --fail --location "${KIND_URL}" --output "${KIND_PATH}"
+        chmod +x "${KIND_PATH}"
+    fi
+    if [[ ! -f "${KUBECTL_PATH}" ]]; then
+        echo
+        echo "Downloading Kubectl version ${KUBECTL_VERSION}"
+        echo
+        curl --fail --location "${KUBECTL_URL}" --output "${KUBECTL_PATH}"
+        chmod +x "${KUBECTL_PATH}"
+    fi
+    PATH=${PATH}:${BUILD_CACHE_DIR}/bin
+}
+
+function create_cluster() {
+    if [[ "${TRAVIS:="false"}" == "true" ]]; then
+        # Travis CI does not handle the nice output of Kind well, so we need to capture it
+        # And display only if kind fails to start
+        start_output_heartbeat "Creating kubernetes cluster" 10
+        set +e
+        if ! OUTPUT=$(kind create cluster \
+                        --name "${KIND_CLUSTER_NAME}" \
+                        --config "${AIRFLOW_SOURCES}/scripts/ci/kubernetes/kind-cluster-conf.yaml" \
+                        --image "kindest/node:${KUBERNETES_VERSION}" 2>&1); then
+            echo "${OUTPUT}"
+        fi
+        stop_output_heartbeat
+    else
+        kind create cluster \
+            --name "${KIND_CLUSTER_NAME}" \
+            --config "${AIRFLOW_SOURCES}/scripts/ci/kubernetes/kind-cluster-conf.yaml" \
+            --image "kindest/node:${KUBERNETES_VERSION}"
+    fi
+    echo
+    echo "Created cluster ${KIND_CLUSTER_NAME}"
+    echo
+
+}
+
+function delete_cluster() {
+    kind delete cluster --name "${KIND_CLUSTER_NAME}"
+    echo
+    echo "Deleted cluster ${KIND_CLUSTER_NAME}"
+    echo
+    rm -rf "${HOME}/.kube/*"
+}
+
+function perform_kind_cluster_operation() {
+    OPERATION="${1}"
+    ALL_CLUSTERS=$(kind get clusters || true)
+
+    echo
+    echo "Kubernetes mode: ${KUBERNETES_MODE}"
+    echo
+
+    if [[ ${OPERATION} == "status" ]]; then
+        if [[ ${ALL_CLUSTERS} == *"${KIND_CLUSTER_NAME}"* ]]; then
+            echo
+            echo "Cluster name: ${KIND_CLUSTER_NAME}"
+            echo
+            kind get nodes --name "${KIND_CLUSTER_NAME}"
+            echo
+            exit
+        else
+            echo
+            echo "Cluster ${KIND_CLUSTER_NAME} is not running"
+            echo
+            exit
+        fi
+    fi
+    if [[ ${ALL_CLUSTERS} == *"${KIND_CLUSTER_NAME}"* ]]; then
+        if [[ ${OPERATION} == "start" ]]; then
+            echo
+            echo "Cluster ${KIND_CLUSTER_NAME} is already created"
+            echo "Reusing previously created cluster"
+            echo
+        elif [[ ${OPERATION} == "restart" ]]; then
+            echo
+            echo "Recreating cluster"
+            echo
+            delete_cluster
+            create_cluster
+        elif [[ ${OPERATION} == "stop" ]]; then
+            echo
+            echo "Deleting cluster"
+            echo
+            delete_cluster
+            exit
+        elif [[ ${OPERATION} == "deploy" ]]; then
+            echo
+            echo "Deploying Airflow to KinD"
+            echo
+            get_ci_environment
+            check_kind_and_kubectl_are_installed
+            build_kubernetes_image
+            load_image_to_kind_cluster
+            prepare_kubernetes_app_variables
+            prepare_kubernetes_resources
+            apply_kubernetes_resources
+            wait_for_airflow_pods_up_and_running
+            wait_for_airflow_webserver_up_and_running
+        elif [[ ${OPERATION} == "test" ]]; then
+            echo
+            echo "Testing with kind to KinD"
+            echo
+            "${AIRFLOW_SOURCES}/scripts/ci/ci_run_kubernetes_tests.sh"
+        else
+            echo
+            echo "Wrong cluster operation: ${OPERATION}. Should be one of:"
+            echo "${FORMATTED_KIND_OPERATIONS}"
+            echo
+            exit 1
+        fi
+    else
+        if [[ ${OPERATION} == "start" ]]; then
+            echo
+            echo "Creating cluster"
+            echo
+            create_cluster
+        elif [[ ${OPERATION} == "recreate" ]]; then
+            echo
+            echo "Cluster ${KIND_CLUSTER_NAME} does not exist. Creating rather than recreating"
+            echo "Creating cluster"
+            echo
+            create_cluster
+        elif [[ ${OPERATION} == "stop" || ${OEPRATON} == "deploy" || ${OPERATION} == "test" ]]; then
+            echo
+            echo "Cluster ${KIND_CLUSTER_NAME} does not exist. It should exist for ${OPERATION} operation"
+            echo
+            exit 1
+        else
+            echo
+            echo "Wrong cluster operation: ${OPERATION}. Should be one of:"
+            echo "${FORMATTED_KIND_OPERATIONS}"
+            echo
+            exit 1
+        fi
+    fi
+}
+
+function check_cluster_ready_for_airflow() {
+    kubectl cluster-info --cluster "${KUBECTL_CLUSTER_NAME}"
+    kubectl get nodes --cluster "${KUBECTL_CLUSTER_NAME}"
+    echo
+    echo "Showing storageClass"
+    echo
+    kubectl get storageclass --cluster "${KUBECTL_CLUSTER_NAME}"
+    echo
+    echo "Showing kube-system pods"
+    echo
+    kubectl get -n kube-system pods --cluster "${KUBECTL_CLUSTER_NAME}"
+    echo
+    echo "Airflow environment on kubernetes is good to go!"
+    echo
+    kubectl create namespace test-namespace --cluster "${KUBECTL_CLUSTER_NAME}"
+}
+
+
+function build_kubernetes_image() {
+    cd "${AIRFLOW_SOURCES}" || exit 1
+    prepare_prod_build
+    if [[ $(docker images -q "${AIRFLOW_PROD_IMAGE}") == "" ||  "${FORCE_BUILD_IMAGES}" == "true" ]]; then
+        build_prod_image
+    else
+        echo
+        echo "Skip rebuilding prod image. Use --force-build-images to rebuild prod image."
+        echo
+    fi
+    echo
+    echo "Adding kubernetes-specific scripts to prod image."
+    echo "Building ${AIRFLOW_KUBERNETES_IMAGE} from ${AIRFLOW_PROD_IMAGE} with latest sources."
+    echo
+    docker build \
+        --build-arg AIRFLOW_PROD_IMAGE="${AIRFLOW_PROD_IMAGE}" \
+        --cache-from "${AIRFLOW_PROD_IMAGE}" \
+        --tag="${AIRFLOW_KUBERNETES_IMAGE}" \
+        -f- . << 'EOF'
+    ARG AIRFLOW_PROD_IMAGE
+    FROM ${AIRFLOW_PROD_IMAGE}
+
+    ARG AIRFLOW_SOURCES=/home/airflow/airflow_sources/
+    ENV AIRFLOW_SOURCES=${AIRFLOW_SOURCES}
+
+    USER root
+
+    COPY --chown=airflow:airflow . ${AIRFLOW_SOURCES}
+
+    COPY scripts/ci/kubernetes/docker/airflow-test-env-init-db.sh /tmp/airflow-test-env-init-db.sh
+    COPY scripts/ci/kubernetes/docker/airflow-test-env-init-dags.sh /tmp/airflow-test-env-init-dags.sh
+    COPY scripts/ci/kubernetes/docker/bootstrap.sh /bootstrap.sh
+
+    RUN chmod +x /bootstrap.sh
+
+
+    USER airflow
+
+
+    ENTRYPOINT ["/bootstrap.sh"]
+EOF
+
+    echo "The ${AIRFLOW_KUBERNETES_IMAGE} is prepared for deployment."
+}
+
+function load_image_to_kind_cluster() {
+    echo
+    echo "Loading ${AIRFLOW_KUBERNETES_IMAGE} to ${KIND_CLUSTER_NAME}"
+    echo
+    kind load docker-image --name "${KIND_CLUSTER_NAME}" "${AIRFLOW_KUBERNETES_IMAGE}"
+}
+
+function prepare_kubernetes_app_variables() {
+    echo
+    echo "Preparing kubernetes variables"
+    echo
+    KUBERNETES_APP_DIR="${AIRFLOW_SOURCES}/scripts/ci/kubernetes/app"
+    TEMPLATE_DIRNAME="${KUBERNETES_APP_DIR}/templates"
+    BUILD_DIRNAME="${KUBERNETES_APP_DIR}/build"
+
+    # shellcheck source=common/_image_variables.sh
+    . "${AIRFLOW_SOURCES}/common/_image_variables.sh"
+
+    # Source branch will be set in DockerHub
+    SOURCE_BRANCH=${SOURCE_BRANCH:=${DEFAULT_BRANCH}}
+    BRANCH_NAME=${BRANCH_NAME:=${SOURCE_BRANCH}}
+
+    if [[ ! -d "${BUILD_DIRNAME}" ]]; then
+        mkdir -p "${BUILD_DIRNAME}"
+    fi
+
+    rm -f "${BUILD_DIRNAME}"/*
+    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=/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=/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_SOURCE_REPO}
+    CONFIGMAP_BRANCH=${CI_SOURCE_BRANCH}
+}
+
+function prepare_kubernetes_resources() {

Review comment:
       this can be in a "_kubernetes_deploy.sh"

##########
File path: scripts/ci/_utils.sh
##########
@@ -1721,3 +1748,436 @@ function run_prepare_backport_readme() {
 function get_airflow_version_from_production_image() {
      docker run --entrypoint /bin/bash "${AIRFLOW_PROD_IMAGE}" -c 'echo "${AIRFLOW_VERSION}"'
 }
+
+function dump_kind_logs() {
+    echo "###########################################################################################"
+    echo "                   Dumping logs from KIND"
+    echo "###########################################################################################"
+
+    FILE_NAME="${1}"
+    kind --name "${KIND_CLUSTER_NAME}" export logs "${FILE_NAME}"
+}
+
+
+function send_kubernetes_logs_to_file_io() {
+    echo "##############################################################################"
+    echo
+    echo "   DUMPING LOG FILES FROM KIND AND SENDING THEM TO file.io"
+    echo
+    echo "##############################################################################"
+    DUMP_DIR_NAME=$(date "+%Y-%m-%d")_kind_${CI_BUILD_ID:="default"}_${CI_JOB_ID:="default"}
+    DUMP_DIR=/tmp/${DUMP_DIR_NAME}
+    dump_kind_logs "${DUMP_DIR}"
+    tar -cvzf "${DUMP_DIR}.tar.gz" -C /tmp "${DUMP_DIR_NAME}"
+    echo
+    echo "   Logs saved to ${DUMP_DIR}.tar.gz"
+    echo
+    echo "##############################################################################"
+    curl -F "file=@${DUMP_DIR}.tar.gz" https://file.io
+}
+
+function check_kind_and_kubectl_are_installed() {
+    SYSTEM=$(uname -s| tr '[:upper:]' '[:lower:]')
+    KIND_VERSION="v0.7.0"
+    KIND_URL="https://github.com/kubernetes-sigs/kind/releases/download/${KIND_VERSION}/kind-${SYSTEM}-amd64"
+    KIND_PATH="${BUILD_CACHE_DIR}/bin/kind"
+    KUBECTL_VERSION="v1.15.3"

Review comment:
       why are these values hard set?

##########
File path: scripts/ci/in_container/entrypoint_ci.sh
##########
@@ -143,53 +142,31 @@ if [[ ${INTEGRATION_KERBEROS:="false"} == "true" ]]; then
 fi
 
 
-if [[ "${RUNTIME}" == "" ]]; then
-    # Start MiniCluster
-    java -cp "/opt/minicluster-1.1-SNAPSHOT/*" com.ing.minicluster.MiniCluster \
-        >"${AIRFLOW_HOME}/logs/minicluster.log" 2>&1 &
+# Start MiniCluster
+java -cp "/opt/minicluster-1.1-SNAPSHOT/*" com.ing.minicluster.MiniCluster \

Review comment:
       are you always starting minicluster as now the assumption is no one would run the k8s tests in the image?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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