You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by ka...@apache.org on 2021/05/19 22:53:05 UTC

[airflow] branch master updated: Mount DAGs read only when using ``gitsync`` (#15953)

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

kaxilnaik pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/master by this push:
     new e01b4e6  Mount DAGs read only when using ``gitsync`` (#15953)
e01b4e6 is described below

commit e01b4e60d1bfbccce614ce8674c5d8f3580431ef
Author: Jed Cunningham <66...@users.noreply.github.com>
AuthorDate: Wed May 19 16:52:51 2021 -0600

    Mount DAGs read only when using ``gitsync`` (#15953)
    
    We will mount the DAGs as read only when we are using gitsync, but not
    when we are only using persistence. This also moves the whole mount definition
    into helpers instead of just the mount path, making it easier to modify
    the DAGs mount everywhere it is used in the future.
---
 chart/.gitignore                                   |  3 +
 chart/files/pod-template-file.kubernetes-helm-yaml |  4 +-
 chart/templates/_helpers.yaml                      |  8 +-
 .../templates/scheduler/scheduler-deployment.yaml  | 11 ++-
 .../templates/webserver/webserver-deployment.yaml  |  9 +--
 chart/templates/workers/worker-deployment.yaml     | 11 ++-
 chart/tests/test_airflow_common.py                 | 92 ++++++++++++++++++++++
 chart/tests/test_git_sync_webserver.py             |  4 +-
 chart/tests/test_pod_template_file.py              | 31 +++-----
 9 files changed, 126 insertions(+), 47 deletions(-)

diff --git a/chart/.gitignore b/chart/.gitignore
index 6cd3d91..f14564f 100644
--- a/chart/.gitignore
+++ b/chart/.gitignore
@@ -7,3 +7,6 @@ repository
 
 # Chart dependencies
 **/charts/*.tgz
+
+# Never check in tmpcharts
+tmpcharts
diff --git a/chart/files/pod-template-file.kubernetes-helm-yaml b/chart/files/pod-template-file.kubernetes-helm-yaml
index f47016c..115a033 100644
--- a/chart/files/pod-template-file.kubernetes-helm-yaml
+++ b/chart/files/pod-template-file.kubernetes-helm-yaml
@@ -66,9 +66,7 @@ spec:
           subPath: ssh
 {{- end }}
 {{- if or .Values.dags.gitSync.enabled .Values.dags.persistence.enabled }}
-        - mountPath: {{ include "airflow_dags_mount_path" . }}
-          name: dags
-          readOnly: true
+        {{- include "airflow_dags_mount" . | nindent 8 }}
 {{- end }}
 {{- if .Values.workers.extraVolumeMounts }}
 {{ toYaml .Values.workers.extraVolumeMounts | indent 8 }}
diff --git a/chart/templates/_helpers.yaml b/chart/templates/_helpers.yaml
index 8fd7bf0..fa0746c 100644
--- a/chart/templates/_helpers.yaml
+++ b/chart/templates/_helpers.yaml
@@ -377,9 +377,11 @@ server_tls_key_file = /etc/pgbouncer/server.key
 {{- end -}}
 {{- end -}}
 
-{{ define "airflow_dags_mount_path" -}}
-{{ (printf "%s/dags" .Values.airflowHome) }}
-{{- end }}
+{{ define "airflow_dags_mount" -}}
+- name: dags
+  mountPath: {{ (printf "%s/dags" .Values.airflowHome) }}
+  readOnly: {{ .Values.dags.gitSync.enabled | ternary "True" "False" }}
+{{- end -}}
 
 {{ define "airflow_config_path" -}}
 {{ (printf "%s/airflow.cfg" .Values.airflowHome) | quote }}
diff --git a/chart/templates/scheduler/scheduler-deployment.yaml b/chart/templates/scheduler/scheduler-deployment.yaml
index 21ddcec..da67975 100644
--- a/chart/templates/scheduler/scheduler-deployment.yaml
+++ b/chart/templates/scheduler/scheduler-deployment.yaml
@@ -162,16 +162,15 @@ spec:
               mountPath: {{ template "airflow_config_path" . }}
               subPath: airflow.cfg
               readOnly: true
-{{- if .Values.airflowLocalSettings }}
+            {{- if .Values.airflowLocalSettings }}
             - name: config
               mountPath: {{ template "airflow_local_setting_path" . }}
               subPath: airflow_local_settings.py
               readOnly: true
-{{- end }}
-{{- if or .Values.dags.gitSync.enabled .Values.dags.persistence.enabled }}
-            - name: dags
-              mountPath: {{ template "airflow_dags_mount_path" . }}
-{{- end }}
+            {{- end }}
+            {{- if or .Values.dags.persistence.enabled .Values.dags.gitSync.enabled }}
+            {{- include "airflow_dags_mount" . | nindent 12 }}
+            {{- end }}
 {{- if .Values.scheduler.extraVolumeMounts }}
 {{ toYaml .Values.scheduler.extraVolumeMounts | indent 12 }}
 {{- end }}
diff --git a/chart/templates/webserver/webserver-deployment.yaml b/chart/templates/webserver/webserver-deployment.yaml
index eec95df5..f17679f 100644
--- a/chart/templates/webserver/webserver-deployment.yaml
+++ b/chart/templates/webserver/webserver-deployment.yaml
@@ -108,9 +108,6 @@ spec:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
       containers:
-{{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) (semverCompare "<2.0.0" .Values.airflowVersion) }}
-{{- include "git_sync_container" . | indent 8 }}
-{{- end }}
         - name: webserver
           image: {{ template "airflow_image" . }}
           imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
@@ -135,8 +132,7 @@ spec:
               readOnly: true
 {{- end }}
 {{- if or (and .Values.dags.gitSync.enabled (semverCompare "<2.0.0" .Values.airflowVersion)) .Values.dags.persistence.enabled }}
-            - name: dags
-              mountPath: {{ template "airflow_dags_mount_path" . }}
+            {{- include "airflow_dags_mount" . | nindent 12 }}
 {{- end }}
 {{- if .Values.logs.persistence.enabled }}
             - name: logs
@@ -179,6 +175,9 @@ spec:
           env:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
+{{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) (semverCompare "<2.0.0" .Values.airflowVersion) }}
+{{- include "git_sync_container" . | indent 8 }}
+{{- end }}
 {{- if .Values.webserver.extraContainers }}
 {{- toYaml .Values.webserver.extraContainers | nindent 8 }}
 {{- end }}
diff --git a/chart/templates/workers/worker-deployment.yaml b/chart/templates/workers/worker-deployment.yaml
index 38e4e6d..b4897fd 100644
--- a/chart/templates/workers/worker-deployment.yaml
+++ b/chart/templates/workers/worker-deployment.yaml
@@ -156,16 +156,15 @@ spec:
               mountPath: {{ .Values.kerberos.ccacheMountPath | quote }}
               readOnly: true
             {{- end }}
-{{- if .Values.airflowLocalSettings }}
+            {{- if .Values.airflowLocalSettings }}
             - name: config
               mountPath: {{ template "airflow_local_setting_path" . }}
               subPath: airflow_local_settings.py
               readOnly: true
-{{- end }}
-{{- if or .Values.dags.persistence.enabled .Values.dags.gitSync.enabled }}
-            - name: dags
-              mountPath: {{ template "airflow_dags_mount_path" . }}
-{{- end }}
+            {{- end }}
+            {{- if or .Values.dags.persistence.enabled .Values.dags.gitSync.enabled }}
+            {{- include "airflow_dags_mount" . | nindent 12 }}
+            {{- end }}
           envFrom:
           {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
diff --git a/chart/tests/test_airflow_common.py b/chart/tests/test_airflow_common.py
new file mode 100644
index 0000000..e34b1b1
--- /dev/null
+++ b/chart/tests/test_airflow_common.py
@@ -0,0 +1,92 @@
+# 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.
+
+import unittest
+
+import jmespath
+from parameterized import parameterized
+
+from tests.helm_template_generator import render_chart
+
+
+class AirflowCommon(unittest.TestCase):
+    """
+    This class holds tests that apply to more than 1 Airflow component so
+    we don't have to repeat tests everywhere
+
+    The one general exception will be the KubernetesExecutor PodTemplateFile,
+    as it requires extra test setup.
+    """
+
+    @parameterized.expand(
+        [
+            ({"gitSync": {"enabled": True}}, True),
+            ({"persistence": {"enabled": True}}, False),
+            (
+                {
+                    "gitSync": {"enabled": True},
+                    "persistence": {"enabled": True},
+                },
+                True,
+            ),
+        ]
+    )
+    def test_dags_mount(self, dag_values, expected_read_only):
+        docs = render_chart(
+            values={
+                "dags": dag_values,
+                "airflowVersion": "1.10.15",
+            },  # airflowVersion is present so webserver gets the mount
+            show_only=[
+                "templates/scheduler/scheduler-deployment.yaml",
+                "templates/workers/worker-deployment.yaml",
+                "templates/webserver/webserver-deployment.yaml",
+            ],
+        )
+
+        assert 3 == len(docs)
+        for doc in docs:
+            expected_mount = {
+                "mountPath": "/opt/airflow/dags",
+                "name": "dags",
+                "readOnly": expected_read_only,
+            }
+            assert expected_mount in jmespath.search("spec.template.spec.containers[0].volumeMounts", doc)
+
+    def test_annotations(self):
+        """
+        Test Annotations are correctly applied on all pods created Scheduler, Webserver & Worker
+        deployments.
+        """
+        release_name = "TEST-BASIC"
+        k8s_objects = render_chart(
+            name=release_name,
+            values={"airflowPodAnnotations": {"test-annotation/safe-to-evict": "true"}},
+            show_only=[
+                "templates/scheduler/scheduler-deployment.yaml",
+                "templates/workers/worker-deployment.yaml",
+                "templates/webserver/webserver-deployment.yaml",
+                "templates/flower/flower-deployment.yaml",
+            ],
+        )
+
+        assert 4 == len(k8s_objects)
+
+        for k8s_object in k8s_objects:
+            annotations = k8s_object["spec"]["template"]["metadata"]["annotations"]
+            assert "test-annotation/safe-to-evict" in annotations
+            assert "true" in annotations["test-annotation/safe-to-evict"]
diff --git a/chart/tests/test_git_sync_webserver.py b/chart/tests/test_git_sync_webserver.py
index 1d00845..4c6f544 100644
--- a/chart/tests/test_git_sync_webserver.py
+++ b/chart/tests/test_git_sync_webserver.py
@@ -58,7 +58,7 @@ class GitSyncWebserverTest(unittest.TestCase):
             show_only=["templates/webserver/webserver-deployment.yaml"],
         )
 
-        assert "git-sync" == jmespath.search("spec.template.spec.containers[0].name", docs[0])
+        assert "git-sync" == jmespath.search("spec.template.spec.containers[1].name", docs[0])
 
     def test_should_have_service_account_defined(self):
         docs = render_chart(
@@ -146,5 +146,5 @@ class GitSyncWebserverTest(unittest.TestCase):
         )
 
         assert {"name": "FOO", "value": "bar"} in jmespath.search(
-            "spec.template.spec.containers[0].env", docs[0]
+            "spec.template.spec.containers[1].env", docs[0]
         )
diff --git a/chart/tests/test_pod_template_file.py b/chart/tests/test_pod_template_file.py
index 791134b..d302a3e 100644
--- a/chart/tests/test_pod_template_file.py
+++ b/chart/tests/test_pod_template_file.py
@@ -122,42 +122,29 @@ class PodTemplateFileTest(unittest.TestCase):
 
     @parameterized.expand(
         [
-            ({"gitSync": {"enabled": True}},),
-            ({"persistence": {"enabled": True}},),
+            ({"gitSync": {"enabled": True}}, True),
+            ({"persistence": {"enabled": True}}, False),
             (
                 {
                     "gitSync": {"enabled": True},
                     "persistence": {"enabled": True},
                 },
+                True,
             ),
         ]
     )
-    def test_dags_mount(self, dag_values):
+    def test_dags_mount(self, dag_values, expected_read_only):
         docs = render_chart(
             values={"dags": dag_values},
             show_only=["templates/pod-template-file.yaml"],
             chart_dir=self.temp_chart_dir,
         )
 
-        assert {"mountPath": "/opt/airflow/dags", "name": "dags", "readOnly": True} in jmespath.search(
-            "spec.containers[0].volumeMounts", docs[0]
-        )
-
-    def test_dags_mount_with_gitsync_and_persistence(self):
-        docs = render_chart(
-            values={
-                "dags": {
-                    "gitSync": {"enabled": True},
-                    "persistence": {"enabled": True},
-                }
-            },
-            show_only=["templates/pod-template-file.yaml"],
-            chart_dir=self.temp_chart_dir,
-        )
-
-        assert {"mountPath": "/opt/airflow/dags", "name": "dags", "readOnly": True} in jmespath.search(
-            "spec.containers[0].volumeMounts", docs[0]
-        )
+        assert {
+            "mountPath": "/opt/airflow/dags",
+            "name": "dags",
+            "readOnly": expected_read_only,
+        } in jmespath.search("spec.containers[0].volumeMounts", docs[0])
 
     def test_validate_if_ssh_params_are_added(self):
         docs = render_chart(