You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by po...@apache.org on 2020/11/15 02:52:02 UTC

[airflow] 19/32: Validate airflow chart values.yaml & values.schema.json (#11990)

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

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

commit 6a12758b0e9503977da1b71fde1ebc2a4691622a
Author: Florent Chehab <fc...@meilleursagents.com>
AuthorDate: Sat Oct 31 18:19:32 2020 +0100

    Validate airflow chart values.yaml & values.schema.json (#11990)
    
    * Correct type for multiNamespaceMode chart value
    * Updated values.schema.json to reflect the latest change and to be stricter
    * Fixed current test
    * Added a test to validate the values file against the schema
    
    (cherry picked from commit e238b882a8568532829be80e96e54856d7a0018d)
---
 chart/README.md                      |   2 +-
 chart/tests/test_basic_helm_chart.py |   4 +-
 chart/tests/test_chart_quality.py    |  42 +++++++
 chart/values.schema.json             | 220 ++++++++++++++++++++++++-----------
 chart/values.yaml                    |  79 ++++++-------
 5 files changed, 229 insertions(+), 118 deletions(-)

diff --git a/chart/README.md b/chart/README.md
index 7bbe63c..df8ea5d 100644
--- a/chart/README.md
+++ b/chart/README.md
@@ -169,7 +169,7 @@ The following tables lists the configurable parameters of the Airflow chart and
 | `kerberos.keytabPath`                                 | Path for the Kerberos keytab file                                                                            | `/etc/airflow.keytab`                             |
 | `kerberos.principal`                                  | Name of the Kerberos principal                                                                               | `airflow`                                         |
 | `kerberos.reinitFrequency`                            | Frequency of reinitialization of the Kerberos token                                                          | `3600`                                            |
-| `kerberos.confg`                                      | Content of the configuration file for kerberos (might be templated using Helm templates)                     | `<see values.yaml>`                               |
+| `kerberos.config`                                      | Content of the configuration file for kerberos (might be templated using Helm templates)                     | `<see values.yaml>`                               |
 | `workers.replicas`                                    | Replica count for Celery workers (if applicable)                                                             | `1`                                               |
 | `workers.keda.enabled`                                | Enable KEDA autoscaling features                                                                             | `false`                                           |
 | `workers.keda.pollingInverval`                        | How often KEDA should poll the backend database for metrics in seconds                                       | `5`                                               |
diff --git a/chart/tests/test_basic_helm_chart.py b/chart/tests/test_basic_helm_chart.py
index dcf20cf..e535ac9 100644
--- a/chart/tests/test_basic_helm_chart.py
+++ b/chart/tests/test_basic_helm_chart.py
@@ -38,8 +38,8 @@ class TestBaseChartTest(unittest.TestCase):
                 ('Secret', 'TEST-BASIC-airflow-metadata'),
                 ('Secret', 'TEST-BASIC-airflow-result-backend'),
                 ('ConfigMap', 'TEST-BASIC-airflow-config'),
-                ('ClusterRole', 'TEST-BASIC-pod-launcher-role'),
-                ('ClusterRoleBinding', 'TEST-BASIC-pod-launcher-rolebinding'),
+                ('Role', 'TEST-BASIC-pod-launcher-role'),
+                ('RoleBinding', 'TEST-BASIC-pod-launcher-rolebinding'),
                 ('Service', 'TEST-BASIC-postgresql-headless'),
                 ('Service', 'TEST-BASIC-postgresql'),
                 ('Service', 'TEST-BASIC-statsd'),
diff --git a/chart/tests/test_chart_quality.py b/chart/tests/test_chart_quality.py
new file mode 100644
index 0000000..3898941
--- /dev/null
+++ b/chart/tests/test_chart_quality.py
@@ -0,0 +1,42 @@
+# 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 json
+import os
+import unittest
+import yaml
+
+from jsonschema import validate
+
+
+CHART_FOLDER = os.path.dirname(os.path.dirname(__file__))
+
+
+class ChartQualityTest(unittest.TestCase):
+    def test_values_validate_schema(self):
+        with open(os.path.join(CHART_FOLDER, "values.yaml"), "r") as f:
+            values = yaml.safe_load(f)
+        with open(os.path.join(CHART_FOLDER, "values.schema.json"), "r") as f:
+            schema = json.load(f)
+
+        # Add extra restrictions just for the tests to make sure
+        # we don't forget to update the schema if we add a new property
+        schema["additionalProperties"] = False
+        schema["minProperties"] = len(schema["properties"].keys())
+
+        # shouldn't raise
+        validate(instance=values, schema=schema)
diff --git a/chart/values.schema.json b/chart/values.schema.json
index 4395372..2f25892 100644
--- a/chart/values.schema.json
+++ b/chart/values.schema.json
@@ -23,10 +23,6 @@
             "description": "Default airflow tag to deploy.",
             "type": "string"
         },
-        "multi_namespaceMode": {
-          "description": "Whether the KubernetesExecutor can launch workers in multiple namespaces",
-          "type": "boolean"
-        },
         "nodeSelector": {
             "description": "Select certain nodes for airflow pods.",
             "type": "object",
@@ -60,6 +56,7 @@
                 "web": {
                     "description": "Configuration for the Ingress of the web Service.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "annotations": {
                             "description": "Annotations for the web Ingress.",
@@ -76,6 +73,7 @@
                         "tls": {
                             "description": "Configuration for web Ingress TLS.",
                             "type": "object",
+                            "additionalProperties": false,
                             "properties": {
                                 "enabled": {
                                     "description": "Enable TLS termination for the web Ingress.",
@@ -100,6 +98,7 @@
                 "flower": {
                     "description": "Configuration for the Ingress of the flower Service.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "annotations": {
                             "description": "Annotations for the flower Ingress.",
@@ -116,6 +115,7 @@
                         "tls": {
                             "description": "Configuration for flower Ingress TLS.",
                             "type": "object",
+                            "additionalProperties": false,
                             "properties": {
                                 "enabled": {
                                     "description": "Enable TLS termination for the flower Ingress.",
@@ -142,6 +142,7 @@
         "networkPolicies": {
             "description": "Network policy configuration.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "enabled": {
                     "description": "Enabled network policies.",
@@ -168,6 +169,7 @@
         "images": {
             "description": "Images.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "airflow": {
                     "description": "Configuration of the airflow image.",
@@ -193,6 +195,30 @@
                         }
                     }
                 },
+                "pod_template": {
+                    "description": "Configuration of the pod_template image.",
+                    "type": "object",
+                    "properties": {
+                        "repository": {
+                            "description": "The pod_template image repository.",
+                            "type": [
+                                "string",
+                                "null"
+                            ]
+                        },
+                        "tag": {
+                            "description": "The pod_template image tag.",
+                            "type": [
+                                "string",
+                                "null"
+                            ]
+                        },
+                        "pullPolicy": {
+                            "description": "The pod_template image pull policy.",
+                            "type": "string"
+                        }
+                    }
+                },
                 "flower": {
                     "description": "Configuration of the flower image.",
                     "type": "object",
@@ -320,6 +346,7 @@
         "data": {
             "description": "Airflow database configuration.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "metadataSecretName": {
                     "description": "Metadata connection string secret.",
@@ -338,6 +365,7 @@
                 "metadataConnection": {
                     "description": "Metadata connection configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "user": {
                             "description": "The database user.",
@@ -371,6 +399,7 @@
                 "resultBackendConnection": {
                     "description": "Result backend connection configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "user": {
                             "description": "The database user.",
@@ -417,9 +446,48 @@
                 "null"
             ]
         },
+        "kerberos": {
+            "description": "Kerberos configurations for airflow",
+            "type": "object",
+            "properties": {
+                "enabled": {
+                    "description": "Enable kerberos.",
+                    "type": "boolean"
+                },
+                "ccacheMountPath": {
+                    "description": "Path to mount shared volume for kerberos credentials cache.",
+                    "type": "string"
+                },
+                "ccacheFileName": {
+                    "description": "Name for kerberos credentials cache file.",
+                    "type": "string"
+                },
+                "configPath":{
+                    "description": "Path to mount krb5.conf kerberos configuration file.",
+                    "type": "string"
+                },
+                "keytabPath":{
+                    "description": "Path to mount the keytab for refreshing credentials in the kerberos sidecar.",
+                    "type": "string"
+                },
+                "principal":{
+                    "description": "Principal to use when refreshing kerberos credentials.",
+                    "type": "string"
+                },
+                "reinitFrequency": {
+                    "description": "How often (in seconds) airflow kerberos will reinitialize the credentials cache.",
+                    "type": "integer"
+                },
+                "config": {
+                    "description": "Contents of krb5.conf.",
+                    "type": "string"
+                }
+            }
+        },
         "workers": {
             "description": "Airflow Worker configuration.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "replicas": {
                     "description": "Number of airflow celery workers in StatefulSet.",
@@ -428,6 +496,7 @@
                 "keda": {
                     "description": "KEDA configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "enabled": {
                             "description": "Allow KEDA autoscaling. `Persistence.enabled` must be set to false to use KEDA.",
@@ -453,6 +522,7 @@
                 "persistence": {
                     "description": "Persistence configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "enabled": {
                             "description": "Enable persistent volumes.",
@@ -475,6 +545,16 @@
                         }
                     }
                 },
+                "kerberosSidecar": {
+                    "type": "object",
+                    "additionalProperties": false,
+                    "properties": {
+                        "enabled": {
+                            "description": "Enable Kerberos sidecar for the worker.",
+                            "type": "boolean"
+                        }
+                    }
+                },
                 "resources": {
                     "type": "object"
                 },
@@ -489,20 +569,21 @@
                 "serviceAccountAnnotations": {
                   "description": "Annotations to add to the worker kubernetes service account.",
                   "type": "object"
+                },
+                "extraVolumes": {
+                    "description": "Mount additional volumes into workers.",
+                    "type": "array"
+                },
+                "extraVolumeMounts": {
+                    "description": "Mount additional volumes into workers.",
+                    "type": "array"
                 }
-              },
-          "extraVolumes": {
-            "description": "Mount additional volumes into workers.",
-            "type": "array"
-          },
-          "extraVolumeMounts": {
-            "description": "Mount additional volumes into workers.",
-            "type": "array"
-          }
+            }
         },
         "scheduler": {
             "description": "Airflow scheduler settings.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "replicas": {
                   "description": "Airflow 2.0 allows users to run multiple schedulers. This feature is only recommended for Mysql 8+ and postgres",
@@ -511,6 +592,7 @@
                 "podDisruptionBudget": {
                     "description": "Scheduler pod disruption budget.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "enabled": {
                             "description": "Enable pod disruption budget.",
@@ -519,6 +601,7 @@
                         "config": {
                             "description": "Disruption budget configuration.",
                             "type": "object",
+                            "additionalProperties": false,
                             "properties": {
                                 "maxUnavailable": {
                                     "description": "Max unavailable pods for scheduler.",
@@ -545,24 +628,26 @@
                 "serviceAccountAnnotations": {
                   "description": "Annotations to add to the scheduler kubernetes service account.",
                   "type": "object"
+                },
+                "extraVolumes": {
+                    "description": "Mount additional volumes into scheduler.",
+                    "type": "array"
+                },
+                "extraVolumeMounts": {
+                    "description": "Mount additional volumes into scheduler.",
+                    "type": "array"
                 }
-            },
-          "extraVolumes": {
-            "description": "Mount additional volumes into scheduler.",
-            "type": "array"
-          },
-          "extraVolumeMounts": {
-            "description": "Mount additional volumes into scheduler.",
-            "type": "array"
-          }
+            }
         },
         "webserver": {
             "description": "Airflow webserver settings.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "livenessProbe": {
                     "description": "Liveness probe configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "initialDelaySeconds": {
                             "description": "Webserver Liveness probe initial delay.",
@@ -585,6 +670,7 @@
                 "readinessProbe": {
                     "description": "Readiness probe configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "initialDelaySeconds": {
                             "description": "Webserver Readiness probe initial delay.",
@@ -618,6 +704,7 @@
                 "defaultUser": {
                     "description": "Optional default airflow user information",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "enabled": {
                             "description": "Enable default user creation.",
@@ -667,6 +754,7 @@
                 "service": {
                     "description": "Webserver service configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "type": {
                             "description": "Webserver service type.",
@@ -687,6 +775,7 @@
         "flower": {
             "description": "Flower settings.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "extraNetworkPolicies": {
                     "description": "Additional network policies as needed.",
@@ -698,6 +787,7 @@
                 "service": {
                     "description": "Flower service configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "type": {
                             "description": "Flower service type.",
@@ -710,6 +800,7 @@
         "statsd": {
             "description": "Statsd settings.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "enabled": {
                     "description": "Enable statsd.",
@@ -725,6 +816,7 @@
                 "service": {
                     "description": "Statsd service configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "extraAnnotations": {
                             "description": "Extra annotations for the statsd service.",
@@ -737,6 +829,7 @@
         "pgbouncer": {
             "description": "Pgbouncer settings.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "enabled": {
                     "description": "Enable pgbouncer.",
@@ -761,6 +854,7 @@
                 "podDisruptionBudget": {
                     "description": "Pgbouner pod disruption budget.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "enabled": {
                             "description": "Enabled pod distribution budget.",
@@ -769,6 +863,7 @@
                         "config": {
                             "description": "Pod distribution configuration.",
                             "type": "object",
+                            "additionalProperties": false,
                             "properties": {
                                 "maxUnavailable": {
                                     "description": "Max unavailable pods for pgbouncer.",
@@ -784,6 +879,7 @@
                 "service": {
                     "description": "Pgbouncer service configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "extraAnnotations": {
                             "description": "Extra annotations for the pgbouncer service.",
@@ -835,6 +931,7 @@
         "redis": {
             "description": "",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "terminationGracePeriodSeconds": {
                     "description": "Grace period for tasks to finish after SIGTERM is sent from Kubernetes.",
@@ -843,6 +940,7 @@
                 "persistence": {
                     "description": "Persistence configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "enabled": {
                             "description": "Enable persistent volumes.",
@@ -861,16 +959,6 @@
                         }
                     }
                 },
-                "kerberosSidecar": {
-                    "description": "Run a side car in each worker pod to refresh Kerberos ccache with `airflow kerberos` according to the airflow security configuration",
-                    "type": "object",
-                    "properties": {
-                        "enabled": {
-                            "description": "Enable Kerberos side car on worker pods.",
-                            "type": "boolean"
-                        }
-                    }
-                },
                 "resources": {
                     "type": "object"
                 },
@@ -904,6 +992,7 @@
         "registry": {
             "description": "Auth secret for a private registry. This is used if pulling airflow images from a private registry.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "secretName": {
                     "description": "Registry connection string secret.",
@@ -921,6 +1010,7 @@
         "elasticsearch": {
             "description": "Elasticsearch logging configuration.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "enabled": {
                     "description": "Enable elasticsearch task logging.",
@@ -942,6 +1032,7 @@
         "ports": {
             "description": "All ports used by chart.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "flowerUI": {
                     "description": "Flower UI port.",
@@ -988,6 +1079,7 @@
         "cleanup": {
             "description": "This runs as a CronJob to cleanup old pods.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "enabled": {
                     "description": "Enable cleanup.",
@@ -1032,13 +1124,22 @@
                 }
             }
         },
+        "multiNamespaceMode": {
+            "description": "Whether the KubernetesExecutor can launch workers and pods in multiple namespaces",
+            "type": "boolean"
+        },
+        "podTemplate": {
+            "description": "TODO ; also add type if you know it"
+        },
         "dags": {
             "description": "DAGs settings.",
             "type": "object",
+            "additionalProperties": false,
             "properties": {
                 "persistence": {
                     "description": "Persistence configuration.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "enabled": {
                             "description": "Enable persistent volume for storing dags.",
@@ -1071,6 +1172,7 @@
                 "gitSync": {
                     "description": "Git sync settings.",
                     "type": "object",
+                    "additionalProperties": false,
                     "properties": {
                         "enabled": {
                             "description": "Enable Git sync.",
@@ -1115,48 +1217,26 @@
                         "containerName": {
                             "description": "Git sync container name.",
                             "type": "string"
+                        },
+                        "uid": {
+                            "description": "Git sync container run as user parameter.",
+                            "type": "integer"
+                        },
+                        "credentialsSecret": {
+                            "description": "TODO",
+                            "type": ["string", "null"]
+                        },
+                        "sshKeySecret": {
+                            "description": "TODO",
+                            "type": ["string", "null"]
+                        },
+                        "knownHosts": {
+                            "description": "TODO",
+                            "type": ["string", "null"]
                         }
                     }
                 }
             }
-        },
-        "kerberos": {
-            "description": "Kerberos configurations for airflow",
-            "type": "object",
-            "properties": {
-                "enabled": {
-                    "description": "Enable kerberos.",
-                    "type": "boolean"
-                },
-                "ccacheMountPath": {
-                    "description": "Path to mount shared volume for kerberos credentials cache.",
-                    "type": "string"
-                },
-                "ccacheFileName": {
-                    "description": "Name for kerberos credentials cache file.",
-                    "type": "string"
-                },
-                "configPath":{
-                    "description": "Path to mount krb5.conf kerberos configuration file.",
-                    "type": "string"
-                },
-                "keytabPath":{
-                    "description": "Path to mount the keytab for refreshing credentials in the kerberos sidecar.",
-                    "type": "string"
-                },
-                "principal":{
-                    "description": "Principal to use when refreshing kerberos credentials.",
-                    "type": "string"
-                },
-                "reinitFrequency": {
-                    "description": "How often (in seconds) airflow kerberos will reinitialize the credentials cache.",
-                    "type": "integer"
-                },
-                "config": {
-                    "description": "Contents of krb5.conf.",
-                    "type": "string"
-                }
-              }
         }
     }
 }
diff --git a/chart/values.yaml b/chart/values.yaml
index 54a78d6..4855214 100644
--- a/chart/values.yaml
+++ b/chart/values.yaml
@@ -112,7 +112,7 @@ rbacEnabled: true
 # Options: SequentialExecutor, LocalExecutor, CeleryExecutor, KubernetesExecutor
 executor: "KubernetesExecutor"
 
-# If this is true and using LocalExecutor/SequentialExecutor/KubernetesExecutor, the scheudler's
+# If this is true and using LocalExecutor/SequentialExecutor/KubernetesExecutor, the scheduler's
 # service account will have access to communicate with the api-server and launch pods.
 # If this is true and using the CeleryExecutor, the workers will be able to launch pods.
 allowPodLaunching: true
@@ -450,13 +450,21 @@ pgbouncer:
     config:
       maxUnavailable: 1
 
+  # Limit the resources to pgbouncerExported.
+  # When you specify the resource request the scheduler uses this information to decide which node to place
+  # the Pod on. When you specify a resource limit for a Container, the kubelet enforces those limits so
+  # that the running container is not allowed to use more of that resource than the limit you set.
+  # See: https://kubernetes.io/docs/concepts/configuration/manage-resources-containers/
+  # Example:
+  #
+  # resource:
+  #   limits:
+  #     cpu: 100m
+  #     memory: 128Mi
+  #   requests:
+  #     cpu: 100m
+  #     memory: 128Mi
   resources: {}
-  # limits:
-  #   cpu: 100m
-  #   memory: 128Mi
-  # requests:
-  #   cpu: 100m
-  #   memory: 128Mi
 
   service:
     extraAnnotations: {}
@@ -510,11 +518,13 @@ redis:
 registry:
   secretName: ~
 
+  # Example:
+  # connection:
+  #   user: ~
+  #   pass: ~
+  #   host: ~
+  #   email: ~
   connection: {}
-  # user: ~
-  # pass: ~
-  # host: ~
-  # email: ~
 
 # Elasticsearch logging configuration
 elasticsearch:
@@ -523,11 +533,14 @@ elasticsearch:
   # A secret containing the connection
   secretName: ~
   # Or an object representing the connection
+  # Example:
+  # connection:
+  #   user: ~
+  #   pass: ~
+  #   host: ~
+  #   port: ~
   connection: {}
-  # user: ~
-  # pass: ~
-  # host: ~
-  # port: ~
+
 
 # All ports used by chart
 ports:
@@ -563,7 +576,7 @@ postgresql:
 #
 # Please note that these values are passed through the `tpl` function, so are
 # all subject to being rendered as go templates. If you need to include a
-# litera `{{` in a value, it must be expessed like this:
+# literal `{{` in a value, it must be expressed like this:
 #
 #    a: '{{ "{{ not a template }}" }}'
 #
@@ -619,38 +632,11 @@ config:
     pod_template_file: '{{ include "airflow_pod_template_file" . }}/pod_template_file.yaml'
     worker_container_repository: '{{ .Values.images.airflow.repository | default .Values.defaultAirflowRepository }}'
     worker_container_tag: '{{ .Values.images.airflow.tag | default .Values.defaultAirflowTag }}'
-    worker_container_image_pull_policy: '{{ .Values.images.airflow.pullPolicy }}'
-    worker_service_account_name: '{{ .Release.Name }}-worker-serviceaccount'
-    image_pull_secrets: '{{ template "registry_secret" . }}'
-    dags_in_image: '{{ ternary "False" "True" (or .Values.dags.gitSync.enabled .Values.dags.persistence.enabled) }}'
     delete_worker_pods: 'True'
-    multi_namespace_mode: '{{ .Values.multiNamespaceMode }}'
-    delete_worker_pods_on_failure: 'False'
-    run_as_user: '{{ .Values.uid }}'
-    fs_group: '{{ .Values.gid }}'
-    git_dags_folder_mount_point: '{{- if or .Values.dags.gitSync.enabled .Values.dags.persistence.enabled }}{{ include "airflow_dags_mount_path" . }}{{end}}'
-    dags_volume_mount_point: '{{- if or .Values.dags.gitSync.enabled .Values.dags.persistence.enabled }}{{ include "airflow_dags_mount_path" . }}{{ end }}'
-    dags_volume_claim: '{{- if .Values.dags.persistence.enabled }}{{ include "airflow_dags_volume_claim" . }}{{ end }}'
-    dags_volume_subpath: '{{- if .Values.dags.persistence.enabled }}{{.Values.dags.gitSync.dest }}/{{ .Values.dags.gitSync.subPath }}{{ end }}'
-    git_repo: '{{- if and .Values.dags.gitSync.enabled (not .Values.dags.persistence.enabled) }}{{ .Values.dags.gitSync.repo }}{{ end }}'
-    git_branch: '{{ .Values.dags.gitSync.branch }}'
-    git_sync_rev: '{{ .Values.dags.gitSync.rev }}'
-    git_sync_depth: '{{ .Values.dags.gitSync.depth }}'
-    git_sync_root: '{{ .Values.dags.gitSync.root }}'
-    git_sync_dest: '{{ .Values.dags.gitSync.dest }}'
-    git_sync_container_repository: '{{ .Values.dags.gitSync.containerRepository }}'
-    git_sync_container_tag: '{{ .Values.dags.gitSync.containerTag }}'
-    git_sync_init_container_name: '{{ .Values.dags.gitSync.containerName }}'
-    git_sync_run_as_user: '{{ .Values.uid }}'
-    git_ssh_known_hosts_configmap_name: '{{- if .Values.dags.gitSync.knownHosts }}{{ include "airflow_config" . }}{{ end }}'
-    git_ssh_key_secret_name: '{{- if .Values.dags.gitSync.sshKeySecret }}{{ .Values.dags.gitSync.sshKeySecret }}{{ end }}'
-    git_sync_credentials_secret: '{{- if .Values.dags.gitSync.credentialsSecret }}{{ .Values.dags.gitSync.credentialsSecret }}{{ end }}'
-  kubernetes_secrets:
-    AIRFLOW__CORE__SQL_ALCHEMY_CONN: '{{ printf "%s=connection" (include "airflow_metadata_secret" .) }}'
-    AIRFLOW__CORE__FERNET_KEY: '{{ printf "%s=fernet-key" (include "fernet_key_secret" .) }}'
+    multi_namespace_mode: '{{ if .Values.multiNamespaceMode }}True{{ else }}False{{ end }}'
 # yamllint enable rule:line-length
 
-multiNamespaceMode: 'False'
+multiNamespaceMode: false
 
 podTemplate: ~
 
@@ -695,6 +681,7 @@ dags:
     #     GIT_SYNC_USERNAME: <base64_encoded_git_username>
     #     GIT_SYNC_PASSWORD: <base64_encoded_git_password>
     # and specify the name of the secret below
+    #
     # credentialsSecret: git-credentials
     #
     #
@@ -710,8 +697,10 @@ dags:
     #     gitSshKey: <base64_encoded_data>
     # and specify the name of the secret below
     # sshKeySecret: airflow-ssh-secret
+    #
     # If you are using an ssh private key, you can additionally
     # specify the content of your known_hosts file, example:
+    #
     # knownHosts: |
     #    <host1>,<ip1> <key1>
     #    <host2>,<ip2> <key2>