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/29 17:06:05 UTC

[airflow] 01/17: Support creation of configmaps & secrets and extra env & envFrom configuration in Helm Chart (#12164)

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 680986bbcb6cd685497b68a3bd27490114257207
Author: Florent Chehab <fc...@meilleursagents.com>
AuthorDate: Tue Nov 17 10:11:53 2020 +0100

    Support creation of configmaps & secrets and extra env & envFrom configuration in Helm Chart (#12164)
    
    * Enable provisionning of extra secrets and configmaps in helm chart
    
    Added 2 new values:
    *  extraSecrets
    *  extraConfigMaps
    
    Those values enable the provisionning of ConfigMaps
    and secrets directly from the airflow chart.
    
    Those objects could be used for storing airflow variables
    or (secret) connections info for instance
    (the plan is to add support for extraEnv and extraEnvFrom later).
    
    Docs and tests updated accordingly.
    
    * Add support for extra env and envFrom items in helm chart
    
    Added 2 new values:
    *  extraEnv
    *  extraEnvFrom
    
    Those values will be added to the defintion of
    airflow containers. They are expected to be string
    (they can be templated).
    
    Those new values won't be supported by "legacy" kubernetes
    executor configuration (you must use the pod template).
    
    Therefore, the value 'env' is also deprecated as it's kind
    of a duplicate for extraEnv.
    
    Docs and tests updated accordingly.
    
    (cherry picked from commit 56ee2bb3cb6838df0181d753c24c72d0f4938b0a)
---
 chart/README.md                                    |   6 +-
 chart/files/pod-template-file.kubernetes-helm-yaml |  11 +-
 chart/templates/_helpers.yaml                      |  15 ++-
 chart/templates/{ => configmaps}/configmap.yaml    |   0
 chart/templates/configmaps/extra-configmaps.yaml   |  45 ++++++++
 chart/templates/create-user-job.yaml               |   2 +
 chart/templates/flower/flower-deployment.yaml      |   2 +-
 chart/templates/migrate-database-job.yaml          |   2 +
 .../templates/scheduler/scheduler-deployment.yaml  |  10 +-
 chart/templates/secrets/extra-secrets.yaml         |  51 +++++++++
 .../templates/webserver/webserver-deployment.yaml  |   8 +-
 chart/templates/workers/worker-deployment.yaml     |  10 +-
 chart/tests/helm_template_generator.py             |  12 +++
 chart/tests/test_extra_configmaps_secrets.py       | 110 +++++++++++++++++++
 chart/tests/test_extra_env_env_from.py             | 117 +++++++++++++++++++++
 chart/values.schema.json                           |  44 ++++++++
 chart/values.yaml                                  |  51 +++++++++
 17 files changed, 485 insertions(+), 11 deletions(-)

diff --git a/chart/README.md b/chart/README.md
index d56f114..c5106be 100644
--- a/chart/README.md
+++ b/chart/README.md
@@ -158,8 +158,12 @@ The following tables lists the configurable parameters of the Airflow chart and
 | `images.pgbouncerExporter.repository`                 | Docker repository to pull image from. Update this to deploy a custom image                                   | `apache/airflow`                                  |
 | `images.pgbouncerExporter.tag`                        | Docker image tag to pull image from. Update this to deploy a new custom image tag                            | `airflow-pgbouncer-exporter-2020.09.25-0.5.0`     |
 | `images.pgbouncerExporter.pullPolicy`                 | PullPolicy for pgbouncer-exporter image                                                                      | `IfNotPresent`                                    |
-| `env`                                                 | Environment variables key/values to mount into Airflow pods                                                  | `[]`                                              |
+| `env`                                                 | Environment variables key/values to mount into Airflow pods (deprecated, prefer using extraEnv)              | `[]`                                              |
 | `secret`                                              | Secret name/key pairs to mount into Airflow pods                                                             | `[]`                                              |
+| `extraEnv`                                            | Extra env 'items' that will be added to the definition of airflow containers                                 | `~`                                               |
+| `extraEnvFrom`                                        | Extra envFrom 'items' that will be added to the definition of airflow containers                             | `~`                                               |
+| `extraSecrets`                                        | Extra Secrets that will be managed by the chart                                                              | `{}`                                              |
+| `extraConfigMaps`                                     | Extra ConfigMaps that will be managed by the chart                                                           | `{}`                                              |
 | `data.metadataSecretName`                             | Secret name to mount Airflow connection string from                                                          | `~`                                               |
 | `data.resultBackendSecretName`                        | Secret name to mount Celery result backend connection string from                                            | `~`                                               |
 | `data.metadataConection`                              | Field separated connection data (alternative to secret name)                                                 | `{}`                                              |
diff --git a/chart/files/pod-template-file.kubernetes-helm-yaml b/chart/files/pod-template-file.kubernetes-helm-yaml
index 5c4fb92..33ae7b5 100644
--- a/chart/files/pod-template-file.kubernetes-helm-yaml
+++ b/chart/files/pod-template-file.kubernetes-helm-yaml
@@ -27,12 +27,13 @@ spec:
   containers:
     - args: []
       command: []
+      envFrom:
+      {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 6 }}
       env:
-      - name: AIRFLOW__CORE__EXECUTOR
-        value: LocalExecutor
-{{- include "standard_airflow_environment" . | indent 4 }}
-{{- include "custom_airflow_environment" . | indent 4 }}
-      envFrom: []
+        - name: AIRFLOW__CORE__EXECUTOR
+          value: LocalExecutor
+{{- include "standard_airflow_environment" . | indent 6}}
+{{- include "custom_airflow_environment" . | indent 6 }}
       image: {{ template "pod_template_image" . }}
       imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
       name: base
diff --git a/chart/templates/_helpers.yaml b/chart/templates/_helpers.yaml
index 059d64d..df7b158 100644
--- a/chart/templates/_helpers.yaml
+++ b/chart/templates/_helpers.yaml
@@ -85,12 +85,25 @@
         name: {{ $config.secretName }}
         key: {{ default "value" $config.secretKey }}
   {{- end }}
-    {{- if or (eq $.Values.executor "KubernetesExecutor") (eq $.Values.executor "CeleryKubernetesExecutor") }}
+  {{- if or (eq $.Values.executor "KubernetesExecutor") (eq $.Values.executor "CeleryKubernetesExecutor") }}
     {{- range $i, $config := .Values.secret }}
   - name: AIRFLOW__KUBERNETES_SECRETS__{{ $config.envName }}
     value: {{ printf "%s=%s" $config.secretName $config.secretKey }}
     {{- end }}
   {{ end }}
+  # Extra env
+  {{- $Global := . }}
+  {{- with .Values.extraEnv }}
+  {{- tpl . $Global | nindent 2 }}
+  {{- end }}
+{{- end }}
+
+{{/* User defined Airflow environment from */}}
+{{- define "custom_airflow_environment_from" }}
+  {{- $Global := . }}
+  {{- with .Values.extraEnvFrom }}
+  {{- tpl . $Global | nindent 2 }}
+  {{- end }}
 {{- end }}
 
 {{/*  Git ssh key volume */}}
diff --git a/chart/templates/configmap.yaml b/chart/templates/configmaps/configmap.yaml
similarity index 100%
rename from chart/templates/configmap.yaml
rename to chart/templates/configmaps/configmap.yaml
diff --git a/chart/templates/configmaps/extra-configmaps.yaml b/chart/templates/configmaps/extra-configmaps.yaml
new file mode 100644
index 0000000..a186aba
--- /dev/null
+++ b/chart/templates/configmaps/extra-configmaps.yaml
@@ -0,0 +1,45 @@
+# 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.
+
+####################################################
+## Extra ConfigMaps provisioned via the chart values
+####################################################
+{{- $Global := . }}
+{{- range $configMapName, $configMapContent := .Values.extraConfigMaps }}
+---
+apiVersion: v1
+kind: ConfigMap
+metadata:
+  name: {{ tpl $configMapName $Global | quote }}
+  labels:
+    release: {{ $Global.Release.Name }}
+    chart: "{{ $Global.Chart.Name }}-{{ $Global.Chart.Version }}"
+    heritage: {{ $Global.Release.Service }}
+  annotations:
+    "helm.sh/hook": "pre-install,pre-upgrade"
+    "helm.sh/hook-delete-policy": "before-hook-creation"
+    "helm.sh/hook-weight": "0"
+{{- with $Global.Values.labels }}
+{{ toYaml . | indent 4 }}
+{{- end }}
+{{- if $configMapContent.data }}
+data:
+  {{- with $configMapContent.data }}
+  {{- tpl . $Global | nindent 2 }}
+  {{- end }}
+{{- end }}
+{{- end }}
diff --git a/chart/templates/create-user-job.yaml b/chart/templates/create-user-job.yaml
index 27a0363..4df7dd6 100644
--- a/chart/templates/create-user-job.yaml
+++ b/chart/templates/create-user-job.yaml
@@ -79,6 +79,8 @@ spec:
             - {{ .Values.webserver.defaultUser.lastName }}
             - "-p"
             - {{ .Values.webserver.defaultUser.password }}
+          envFrom:
+          {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
diff --git a/chart/templates/flower/flower-deployment.yaml b/chart/templates/flower/flower-deployment.yaml
index c5d1f91..3a33369 100644
--- a/chart/templates/flower/flower-deployment.yaml
+++ b/chart/templates/flower/flower-deployment.yaml
@@ -33,7 +33,7 @@ metadata:
 {{ toYaml . | indent 4 }}
 {{- end }}
   annotations:
-    checksum/airflow-config: {{ include (print $.Template.BasePath "/configmap.yaml") . | sha256sum }}
+    checksum/airflow-config: {{ include (print $.Template.BasePath "/configmaps/configmap.yaml") . | sha256sum }}
 spec:
   replicas: 1
   selector:
diff --git a/chart/templates/migrate-database-job.yaml b/chart/templates/migrate-database-job.yaml
index 37a9b2d..8639648 100644
--- a/chart/templates/migrate-database-job.yaml
+++ b/chart/templates/migrate-database-job.yaml
@@ -62,6 +62,8 @@ spec:
           imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
           # Support running against 1.10.x and 2.0.0dev/master
           args: ["bash", "-c", "airflow upgradedb || airflow db upgrade"]
+          envFrom:
+          {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
diff --git a/chart/templates/scheduler/scheduler-deployment.yaml b/chart/templates/scheduler/scheduler-deployment.yaml
index 9a928a6..61dcade 100644
--- a/chart/templates/scheduler/scheduler-deployment.yaml
+++ b/chart/templates/scheduler/scheduler-deployment.yaml
@@ -65,7 +65,9 @@ spec:
         checksum/metadata-secret: {{ include (print $.Template.BasePath "/secrets/metadata-connection-secret.yaml") . | sha256sum }}
         checksum/result-backend-secret: {{ include (print $.Template.BasePath "/secrets/result-backend-connection-secret.yaml") . | sha256sum }}
         checksum/pgbouncer-config-secret: {{ include (print $.Template.BasePath "/secrets/pgbouncer-config-secret.yaml") . | sha256sum }}
-        checksum/airflow-config: {{ include (print $.Template.BasePath "/configmap.yaml") . | sha256sum }}
+        checksum/airflow-config: {{ include (print $.Template.BasePath "/configmaps/configmap.yaml") . | sha256sum }}
+        checksum/extra-configmaps: {{ include (print $.Template.BasePath "/configmaps/extra-configmaps.yaml") . | sha256sum }}
+        checksum/extra-secrets: {{ include (print $.Template.BasePath "/secrets/extra-secrets.yaml") . | sha256sum }}
         {{- if .Values.scheduler.safeToEvict }}
         cluster-autoscaler.kubernetes.io/safe-to-evict: "true"
         {{- end }}
@@ -95,6 +97,8 @@ spec:
           imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
           args:
           {{- include "wait-for-migrations-command" . | indent 10 }}
+          envFrom:
+          {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
@@ -104,6 +108,8 @@ spec:
           image: {{ template "airflow_image" . }}
           imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
           args: ["scheduler"]
+          envFrom:
+          {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
@@ -184,6 +190,8 @@ spec:
               mountPath: {{ template "airflow_config_path" . }}
               subPath: airflow.cfg
               readOnly: true
+          envFrom:
+          {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
diff --git a/chart/templates/secrets/extra-secrets.yaml b/chart/templates/secrets/extra-secrets.yaml
new file mode 100644
index 0000000..1326aa2
--- /dev/null
+++ b/chart/templates/secrets/extra-secrets.yaml
@@ -0,0 +1,51 @@
+# 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.
+
+#################################################
+## Extra Secrets provisioned via the chart values
+#################################################
+{{- $Global := . }}
+{{- range $secretName, $secretContent := .Values.extraSecrets }}
+---
+apiVersion: v1
+kind: Secret
+metadata:
+  name: {{ tpl $secretName $Global | quote }}
+  labels:
+    release: {{ $Global.Release.Name }}
+    chart: "{{ $Global.Chart.Name }}-{{ $Global.Chart.Version }}"
+    heritage: {{ $Global.Release.Service }}
+  annotations:
+    "helm.sh/hook": "pre-install,pre-upgrade"
+    "helm.sh/hook-delete-policy": "before-hook-creation"
+    "helm.sh/hook-weight": "0"
+{{- with $Global.Values.labels }}
+{{ toYaml . | indent 4 }}
+{{- end }}
+{{- if $secretContent.data }}
+data:
+  {{- with $secretContent.data }}
+  {{- tpl . $Global | nindent 2 }}
+  {{- end }}
+{{- end }}
+{{- if $secretContent.stringData }}
+stringData:
+  {{- with $secretContent.stringData }}
+  {{- tpl . $Global | nindent 2 }}
+  {{- end }}
+{{- end }}
+{{- end }}
diff --git a/chart/templates/webserver/webserver-deployment.yaml b/chart/templates/webserver/webserver-deployment.yaml
index a3c42c0..25b6b63 100644
--- a/chart/templates/webserver/webserver-deployment.yaml
+++ b/chart/templates/webserver/webserver-deployment.yaml
@@ -54,7 +54,9 @@ spec:
       annotations:
         checksum/metadata-secret: {{ include (print $.Template.BasePath "/secrets/metadata-connection-secret.yaml") . | sha256sum }}
         checksum/pgbouncer-config-secret: {{ include (print $.Template.BasePath "/secrets/pgbouncer-config-secret.yaml") . | sha256sum }}
-        checksum/airflow-config: {{ include (print $.Template.BasePath "/configmap.yaml") . | sha256sum }}
+        checksum/airflow-config: {{ include (print $.Template.BasePath "/configmaps/configmap.yaml") . | sha256sum }}
+        checksum/extra-configmaps: {{ include (print $.Template.BasePath "/configmaps/extra-configmaps.yaml") . | sha256sum }}
+        checksum/extra-secrets: {{ include (print $.Template.BasePath "/secrets/extra-secrets.yaml") . | sha256sum }}
         {{- if .Values.airflowPodAnnotations }}
         {{- toYaml .Values.airflowPodAnnotations | nindent 8 }}
         {{- end }}
@@ -80,6 +82,8 @@ spec:
           imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
           args:
           {{- include "wait-for-migrations-command" . | indent 10 }}
+          envFrom:
+          {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
@@ -136,6 +140,8 @@ spec:
             timeoutSeconds: {{ .Values.webserver.readinessProbe.timeoutSeconds | default 30 }}
             failureThreshold: {{ .Values.webserver.readinessProbe.failureThreshold | default 20 }}
             periodSeconds: {{ .Values.webserver.readinessProbe.periodSeconds | default 5 }}
+          envFrom:
+          {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
diff --git a/chart/templates/workers/worker-deployment.yaml b/chart/templates/workers/worker-deployment.yaml
index 77d5fe2..40fbbe1 100644
--- a/chart/templates/workers/worker-deployment.yaml
+++ b/chart/templates/workers/worker-deployment.yaml
@@ -56,7 +56,9 @@ spec:
         checksum/metadata-secret: {{ include (print $.Template.BasePath "/secrets/metadata-connection-secret.yaml") . | sha256sum }}
         checksum/result-backend-secret: {{ include (print $.Template.BasePath "/secrets/result-backend-connection-secret.yaml") . | sha256sum }}
         checksum/pgbouncer-config-secret: {{ include (print $.Template.BasePath "/secrets/pgbouncer-config-secret.yaml") . | sha256sum }}
-        checksum/airflow-config: {{ include (print $.Template.BasePath "/configmap.yaml") . | sha256sum }}
+        checksum/airflow-config: {{ include (print $.Template.BasePath "/configmaps/configmap.yaml") . | sha256sum }}
+        checksum/extra-configmaps: {{ include (print $.Template.BasePath "/configmaps/extra-configmaps.yaml") . | sha256sum }}
+        checksum/extra-secrets: {{ include (print $.Template.BasePath "/secrets/extra-secrets.yaml") . | sha256sum }}
         {{- if .Values.workers.safeToEvict }}
         cluster-autoscaler.kubernetes.io/safe-to-evict: "true"
         {{- end }}
@@ -101,6 +103,8 @@ spec:
           imagePullPolicy: {{ .Values.images.airflow.pullPolicy }}
           args:
           {{- include "wait-for-migrations-command" . | indent 10 }}
+          envFrom:
+          {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
@@ -146,6 +150,8 @@ spec:
             - name: dags
               mountPath: {{ template "airflow_dags_mount_path" . }}
 {{- end }}
+          envFrom:
+          {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
           {{- include "custom_airflow_environment" . | indent 10 }}
           {{- include "standard_airflow_environment" . | indent 10 }}
@@ -195,6 +201,8 @@ spec:
             - name: kerberos-ccache
               mountPath: {{ .Values.kerberos.ccacheMountPath | quote }}
               readOnly: false
+          envFrom:
+          {{- include "custom_airflow_environment_from" . | default "\n  []" | indent 10 }}
           env:
             - name: KRB5_CONFIG
               value:  {{ .Values.kerberos.configPath | quote }}
diff --git a/chart/tests/helm_template_generator.py b/chart/tests/helm_template_generator.py
index ba870ed..d8e3f49 100644
--- a/chart/tests/helm_template_generator.py
+++ b/chart/tests/helm_template_generator.py
@@ -19,6 +19,7 @@ import subprocess
 import sys
 from functools import lru_cache
 from tempfile import NamedTemporaryFile
+from typing import Any, Dict, Tuple
 
 import jmespath
 import jsonschema
@@ -81,6 +82,17 @@ def render_chart(name="RELEASE-NAME", values=None, show_only=None):
         return k8s_objects
 
 
+def prepare_k8s_lookup_dict(k8s_objects) -> Dict[Tuple[str, str], Dict[str, Any]]:
+    """
+    Helper to create a lookup dict from k8s_objects.
+    The keys of the dict are the k8s object's kind and name
+    """
+    k8s_obj_by_key = {
+        (k8s_object["kind"], k8s_object["metadata"]["name"]): k8s_object for k8s_object in k8s_objects
+    }
+    return k8s_obj_by_key
+
+
 def render_k8s_object(obj, type_to_render):
     """
     Function that renders dictionaries into k8s objects. For helm chart testing only.
diff --git a/chart/tests/test_extra_configmaps_secrets.py b/chart/tests/test_extra_configmaps_secrets.py
new file mode 100644
index 0000000..378d80e
--- /dev/null
+++ b/chart/tests/test_extra_configmaps_secrets.py
@@ -0,0 +1,110 @@
+# 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 textwrap
+import unittest
+from base64 import b64encode
+
+import yaml
+
+from tests.helm_template_generator import prepare_k8s_lookup_dict, render_chart
+
+RELEASE_NAME = "TEST-EXTRA-CONFIGMAPS-SECRETS"
+
+
+class ExtraConfigMapsSecretsTest(unittest.TestCase):
+    def test_extra_configmaps(self):
+        values_str = textwrap.dedent(
+            """
+            extraConfigMaps:
+              "{{ .Release.Name }}-airflow-variables":
+                data: |
+                  AIRFLOW_VAR_HELLO_MESSAGE: "Hi!"
+                  AIRFLOW_VAR_KUBERNETES_NAMESPACE: "{{ .Release.Namespace }}"
+              "{{ .Release.Name }}-other-variables":
+                data: |
+                  HELLO_WORLD: "Hi again!"
+            """
+        )
+        values = yaml.safe_load(values_str)
+        k8s_objects = render_chart(
+            RELEASE_NAME, values=values, show_only=["templates/configmaps/extra-configmaps.yaml"]
+        )
+        k8s_objects_by_key = prepare_k8s_lookup_dict(k8s_objects)
+
+        all_expected_keys = [
+            ("ConfigMap", f"{RELEASE_NAME}-airflow-variables"),
+            ("ConfigMap", f"{RELEASE_NAME}-other-variables"),
+        ]
+        self.assertEqual(set(k8s_objects_by_key.keys()), set(all_expected_keys))
+
+        all_expected_data = [
+            {"AIRFLOW_VAR_HELLO_MESSAGE": "Hi!", "AIRFLOW_VAR_KUBERNETES_NAMESPACE": "default"},
+            {"HELLO_WORLD": "Hi again!"},
+        ]
+        for expected_key, expected_data in zip(all_expected_keys, all_expected_data):
+            configmap_obj = k8s_objects_by_key[expected_key]
+            self.assertEqual(configmap_obj["data"], expected_data)
+
+    def test_extra_secrets(self):
+        values_str = textwrap.dedent(
+            """
+            extraSecrets:
+              "{{ .Release.Name }}-airflow-connections":
+                data: |
+                  AIRFLOW_CON_AWS: {{ printf "aws_connection_string" | b64enc }}
+                stringData: |
+                  AIRFLOW_CON_GCP: "gcp_connection_string"
+              "{{ .Release.Name }}-other-secrets":
+                data: |
+                  MY_SECRET_1: {{ printf "MY_SECRET_1" | b64enc }}
+                  MY_SECRET_2: {{ printf "MY_SECRET_2" | b64enc }}
+                stringData: |
+                  MY_SECRET_3: "MY_SECRET_3"
+                  MY_SECRET_4: "MY_SECRET_4"
+            """
+        )
+        values = yaml.safe_load(values_str)
+        k8s_objects = render_chart(
+            RELEASE_NAME, values=values, show_only=["templates/secrets/extra-secrets.yaml"]
+        )
+        k8s_objects_by_key = prepare_k8s_lookup_dict(k8s_objects)
+
+        all_expected_keys = [
+            ("Secret", f"{RELEASE_NAME}-airflow-connections"),
+            ("Secret", f"{RELEASE_NAME}-other-secrets"),
+        ]
+        self.assertEqual(set(k8s_objects_by_key.keys()), set(all_expected_keys))
+
+        all_expected_data = [
+            {"AIRFLOW_CON_AWS": b64encode(b"aws_connection_string").decode("utf-8")},
+            {
+                "MY_SECRET_1": b64encode(b"MY_SECRET_1").decode("utf-8"),
+                "MY_SECRET_2": b64encode(b"MY_SECRET_2").decode("utf-8"),
+            },
+        ]
+
+        all_expected_string_data = [
+            {"AIRFLOW_CON_GCP": "gcp_connection_string"},
+            {"MY_SECRET_3": "MY_SECRET_3", "MY_SECRET_4": "MY_SECRET_4"},
+        ]
+        for expected_key, expected_data, expected_string_data in zip(
+            all_expected_keys, all_expected_data, all_expected_string_data
+        ):
+            configmap_obj = k8s_objects_by_key[expected_key]
+            self.assertEqual(configmap_obj["data"], expected_data)
+            self.assertEqual(configmap_obj["stringData"], expected_string_data)
diff --git a/chart/tests/test_extra_env_env_from.py b/chart/tests/test_extra_env_env_from.py
new file mode 100644
index 0000000..7e1b28d
--- /dev/null
+++ b/chart/tests/test_extra_env_env_from.py
@@ -0,0 +1,117 @@
+# 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 textwrap
+import unittest
+
+import jmespath
+import yaml
+from parameterized import parameterized
+
+from tests.helm_template_generator import prepare_k8s_lookup_dict, render_chart
+
+RELEASE_NAME = "TEST-EXTRA-ENV-ENV-FROM"
+
+# Test Params: k8s object key and paths with expected env / envFrom
+PARAMS = [
+    (
+        ("Job", f"{RELEASE_NAME}-create-user"),
+        ("spec.template.spec.containers[0]",),
+    ),
+    (
+        ("Job", f"{RELEASE_NAME}-run-airflow-migrations"),
+        ("spec.template.spec.containers[0]",),
+    ),
+    (
+        ("Deployment", f"{RELEASE_NAME}-scheduler"),
+        (
+            "spec.template.spec.initContainers[0]",
+            "spec.template.spec.containers[0]",
+        ),
+    ),
+    (
+        ("StatefulSet", f"{RELEASE_NAME}-worker"),
+        (
+            "spec.template.spec.initContainers[0]",
+            "spec.template.spec.containers[0]",
+        ),
+    ),
+    (
+        ("Deployment", f"{RELEASE_NAME}-webserver"),
+        ("spec.template.spec.initContainers[0]", "spec.template.spec.containers[0]"),
+    ),
+]
+
+
+class ExtraEnvEnvFromTest(unittest.TestCase):
+    @classmethod
+    def setUpClass(cls) -> None:
+        values_str = textwrap.dedent(
+            """
+            executor: "CeleryExecutor"
+            extraEnvFrom: |
+              - secretRef:
+                  name: '{{ .Release.Name }}-airflow-connections'
+              - configMapRef:
+                  name: '{{ .Release.Name }}-airflow-variables'
+            extraEnv: |
+              - name: PLATFORM
+                value: FR
+              - name: TEST
+                valueFrom:
+                  secretKeyRef:
+                    name: '{{ .Release.Name }}-some-secret'
+                    key: connection
+            """
+        )
+        values = yaml.safe_load(values_str)
+        cls.k8s_objects = render_chart(RELEASE_NAME, values=values)
+        cls.k8s_objects_by_key = prepare_k8s_lookup_dict(cls.k8s_objects)
+
+    @parameterized.expand(PARAMS)
+    def test_extra_env(self, k8s_obj_key, env_paths):
+        expected_env_as_str = textwrap.dedent(
+            f"""
+            - name: PLATFORM
+              value: FR
+            - name: TEST
+              valueFrom:
+                secretKeyRef:
+                  key: connection
+                  name: {RELEASE_NAME}-some-secret
+            """
+        ).lstrip()
+        k8s_object = self.k8s_objects_by_key[k8s_obj_key]
+        for path in env_paths:
+            env = jmespath.search(f"{path}.env", k8s_object)
+            self.assertIn(expected_env_as_str, yaml.dump(env))
+
+    @parameterized.expand(PARAMS)
+    def test_extra_env_from(self, k8s_obj_key, env_from_paths):
+        expected_env_from_as_str = textwrap.dedent(
+            f"""
+            - secretRef:
+                name: {RELEASE_NAME}-airflow-connections
+            - configMapRef:
+                name: {RELEASE_NAME}-airflow-variables
+            """
+        ).lstrip()
+
+        k8s_object = self.k8s_objects_by_key[k8s_obj_key]
+        for path in env_from_paths:
+            env_from = jmespath.search(f"{path}.envFrom", k8s_object)
+            self.assertIn(expected_env_from_as_str, yaml.dump(env_from))
diff --git a/chart/values.schema.json b/chart/values.schema.json
index 7881c82..f1d8271 100644
--- a/chart/values.schema.json
+++ b/chart/values.schema.json
@@ -343,6 +343,50 @@
             "description": "Secrets for all airflow containers.",
             "type": "array"
         },
+        "extraEnv": {
+          "description": "Extra env 'items' that will be added to the definition of airflow containers ; a string is expected (can be templated).",
+          "type": ["null", "string"]
+        },
+        "extraEnvFrom": {
+          "description": "Extra envFrom 'items' that will be added to the definition of airflow containers ; a string is expected (can be templated).",
+          "type": ["null", "string"]
+        },
+        "extraSecrets": {
+          "description": "Extra secrets that will be managed by the chart.",
+          "type": "object",
+          "additionalProperties": {
+            "description": "Name of the secret (can be templated).",
+            "type": "object",
+            "minProperties": 1,
+            "additionalProperties": false,
+            "properties": {
+              "data": {
+                "description": "Content **as string** for the 'data' item of the secret (can be templated)",
+                "type": "string"
+              },
+              "stringData": {
+                "description": "Content **as string** for the 'stringData' item of the secret (can be templated)",
+                "type": "string"
+              }
+            }
+          }
+        },
+        "extraConfigMaps": {
+          "description": "Extra configMaps that will be managed by the chart.",
+          "type": "object",
+          "additionalProperties": {
+            "description": "Name of the configMap (can be templated).",
+            "type": "object",
+            "minProperties": 1,
+            "additionalProperties": false,
+            "properties": {
+              "data": {
+                "description": "Content **as string** for the 'data' item of the secret (can be templated)",
+                "type": "string"
+              }
+            }
+          }
+        },
         "data": {
             "description": "Airflow database configuration.",
             "type": "object",
diff --git a/chart/values.yaml b/chart/values.yaml
index 0f5b313..091a0c9 100644
--- a/chart/values.yaml
+++ b/chart/values.yaml
@@ -163,6 +163,57 @@ secret: []
 #   secretName: ""
 #   secretKey: ""
 
+# Extra secrets that will be managed by the chart
+# (You can use them with extraEnv or extraEnvFrom or some of the extraVolumes values).
+# The format is "key/value" where
+#    * key (can be templated) is the the name the secret that will be created
+#    * value: an object with the standard 'data' or 'stringData' key (or both).
+#          The value associated with those keys must be a string (can be templated)
+extraSecrets: {}
+# eg:
+# extraSecrets:
+#   {{ .Release.Name }}-airflow-connections:
+#     data: |
+#       AIRFLOW_CONN_GCP: 'base64_encoded_gcp_conn_string'
+#       AIRFLOW_CONN_AWS: 'base64_encoded_aws_conn_string'
+#     stringData: |
+#       AIRFLOW_CONN_OTHER: 'other_conn'
+#   {{ .Release.Name }}-other-secret-name-suffix: |
+#     data: |
+#        ...
+
+# Extra ConfigMaps that will be managed by the chart
+# (You can use them with extraEnv or extraEnvFrom or some of the extraVolumes values).
+# The format is "key/value" where
+#    * key (can be templated) is the the name the configmap that will be created
+#    * value: an object with the standard 'data' key.
+#          The value associated with this keys must be a string (can be templated)
+extraConfigMaps: {}
+# eg:
+# extraConfigMaps:
+#   {{ .Release.Name }}-airflow-variables:
+#     data: |
+#       AIRFLOW_VAR_HELLO_MESSAGE: "Hi!"
+#       AIRFLOW_VAR_KUBERNETES_NAMESPACE: "{{ .Release.Namespace }}"
+
+# Extra env 'items' that will be added to the definition of airflow containers
+# a string is expected (can be templated).
+extraEnv: ~
+# eg:
+# extraEnv: |
+#   - name: PLATFORM
+#     value: FR
+
+# Extra envFrom 'items' that will be added to the definition of airflow containers
+# A string is expected (can be templated).
+extraEnvFrom: ~
+# eg:
+# extraEnvFrom: |
+#   - secretRef:
+#       name: '{{ .Release.Name }}-airflow-connections'
+#   - configMapRef:
+#       name: '{{ .Release.Name }}-airflow-variables'
+
 # Airflow database config
 data:
   # If secret names are provided, use those secrets