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 2022/07/01 12:43:31 UTC

[GitHub] [airflow] rishkarajgi opened a new pull request, #24784: [WIP] added extraEnv to scheduler, webserver, triggerer, flower, workers

rishkarajgi opened a new pull request, #24784:
URL: https://github.com/apache/airflow/pull/24784

   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in a newsfragement file, named `{pr_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r918717859


##########
chart/values.schema.json:
##########
@@ -552,6 +552,11 @@
                                 "IfNotPresent"
                             ],
                             "default": "IfNotPresent"
+                        },
+                        "extraEnv": {
+                            "description": "Add additional env vars to flower.",
+                            "type": "array",
+                            "default": []

Review Comment:
   Have added "items" as standard validation for envVar
   ```
   "items": {
                                   "$ref": "#/definitions/io.k8s.api.core.v1.EnvVar"
                               }
   ```



##########
chart/values.schema.json:
##########
@@ -552,6 +552,11 @@
                                 "IfNotPresent"
                             ],
                             "default": "IfNotPresent"
+                        },
+                        "extraEnv": {
+                            "description": "Add additional env vars to flower.",
+                            "type": "array",
+                            "default": []

Review Comment:
   Have added "items" as standard validation for envVar
   ```
   "items": {
                    "$ref": "#/definitions/io.k8s.api.core.v1.EnvVar"
                  }
   ```



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r941406049


##########
tests/charts/test_scheduler.py:
##########
@@ -115,6 +115,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[*].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",

Review Comment:
   done



##########
tests/charts/test_scheduler.py:
##########
@@ -115,6 +115,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[*].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "scheduler": {
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                },
+            },
+            show_only=["templates/scheduler/scheduler-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",

Review Comment:
   done



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r938085718


##########
tests/charts/test_scheduler.py:
##########
@@ -115,6 +115,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[*].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "scheduler": {
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                },
+            },
+            show_only=["templates/scheduler/scheduler-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "scheduler": {
+                    "waitForMigrations": {
+                        "enabled": True,

Review Comment:
   ```suggestion
   ```
   
   Also the default.



##########
tests/charts/test_pod_template_file.py:
##########
@@ -639,6 +639,16 @@ def test_should_add_pod_labels(self):
             "tier": "airflow",
         } == jmespath.search("metadata.labels", docs[0])
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={"env": [{"name": "TEST_ENV_1", "value": "test_env_1"}]},

Review Comment:
   ```suggestion
               values={"worker": {"env": [{"name": "TEST_ENV_1", "value": "test_env_1"}]}},
   ```



##########
tests/charts/test_webserver.py:
##########
@@ -169,6 +169,39 @@ def test_should_add_extra_containers(self):
             "image": "test-registry/test-repo:test-tag",
         } == jmespath.search("spec.template.spec.containers[-1]", docs[0])
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",

Review Comment:
   ```suggestion
   ```



##########
chart/templates/dag-processor/dag-processor-deployment.yaml:
##########
@@ -134,6 +134,9 @@ spec:
         {{- if .Values.dagProcessor.extraInitContainers }}
         {{- toYaml .Values.dagProcessor.extraInitContainers | nindent 8 }}
         {{- end }}
+{{- if .Values.dagProcessor.waitForMigrations.env }}
+{{ tpl (toYaml .Values.dagProcessor.waitForMigrations.env) $ | indent 12 }}
+{{- end }}

Review Comment:
   This isn't in the right spot. Needs to move up to ensure it's in the envs.



##########
chart/values.schema.json:
##########
@@ -2619,6 +2698,14 @@
                     "description": "Specify if you want to use the default Helm Hook annotations",
                     "type": "boolean",
                     "default": true
+                },
+                "env": {
+                    "description": "Add additional env vars to job.",

Review Comment:
   ```suggestion
                       "description": "Add additional env vars to the create user job pod.",
   ```



##########
tests/charts/test_worker.py:
##########
@@ -112,6 +112,36 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[0].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",

Review Comment:
   ```suggestion
   ```



##########
tests/charts/test_worker.py:
##########
@@ -112,6 +112,36 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[0].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "workers": {
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                },
+            },
+            show_only=["templates/workers/worker-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",

Review Comment:
   ```suggestion
   ```



##########
tests/charts/test_triggerer.py:
##########
@@ -131,6 +131,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[0].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "triggerer": {
+                    "enabled": True,

Review Comment:
   ```suggestion
   ```



##########
tests/charts/test_scheduler.py:
##########
@@ -115,6 +115,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[*].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",

Review Comment:
   ```suggestion
   ```
   
   This is the default.



##########
tests/charts/test_webserver.py:
##########
@@ -169,6 +169,39 @@ def test_should_add_extra_containers(self):
             "image": "test-registry/test-repo:test-tag",
         } == jmespath.search("spec.template.spec.containers[-1]", docs[0])
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "webserver": {
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                },
+            },
+            show_only=["templates/webserver/webserver-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "webserver": {
+                    "waitForMigrations": {
+                        "enabled": True,

Review Comment:
   ```suggestion
                   "webserver": {
                       "waitForMigrations": {
   ```



##########
tests/charts/test_scheduler.py:
##########
@@ -115,6 +115,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[*].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "scheduler": {
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                },
+            },
+            show_only=["templates/scheduler/scheduler-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",

Review Comment:
   ```suggestion
   ```



##########
chart/values.yaml:
##########
@@ -597,6 +597,12 @@ workers:
     #   cpu: 100m
     #   memory: 128Mi
 
+  waitForMigrations:
+    # To add env vars specific to wait-for-migrations init container

Review Comment:
   ```suggestion
   ```



##########
chart/templates/workers/worker-deployment.yaml:
##########
@@ -223,6 +226,9 @@ spec:
         {{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) }}
         {{- include "git_sync_container" . | indent 8 }}
         {{- end }}
+{{- if .Values.workers.env }}
+{{ tpl (toYaml .Values.workers.env) $ | indent 12 }}
+{{- end }}

Review Comment:
   Same here, needs to be moved to the right spot.



##########
chart/files/pod-template-file.kubernetes-helm-yaml:
##########
@@ -58,6 +58,9 @@ spec:
           value: LocalExecutor
 {{- include "standard_airflow_environment" . | indent 6}}
 {{- include "custom_airflow_environment" . | indent 6 }}
+{{- if .Values.workers.env }}
+{{ tpl (toYaml .Values.workers.env) $ | indent 12 }}

Review Comment:
   ```suggestion
   {{ tpl (toYaml .Values.workers.env) $ | indent 8 }}
   ```
   
   I think this is the right indent?



##########
tests/charts/test_triggerer.py:
##########
@@ -131,6 +131,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[0].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "triggerer": {
+                    "enabled": True,
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                }
+            },
+            show_only=["templates/triggerer/triggerer-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "triggerer": {
+                    "enabled": True,
+                    "waitForMigrations": {
+                        "enabled": True,

Review Comment:
   ```suggestion
                       "waitForMigrations": {
   ```



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r941407841


##########
chart/templates/dag-processor/dag-processor-deployment.yaml:
##########
@@ -134,6 +134,9 @@ spec:
         {{- if .Values.dagProcessor.extraInitContainers }}
         {{- toYaml .Values.dagProcessor.extraInitContainers | nindent 8 }}
         {{- end }}
+{{- if .Values.dagProcessor.waitForMigrations.env }}
+{{ tpl (toYaml .Values.dagProcessor.waitForMigrations.env) $ | indent 12 }}
+{{- end }}

Review Comment:
   @jedcunningham I ran a helm template and it seems to be aligning right under the env section



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r941407123


##########
tests/charts/test_worker.py:
##########
@@ -112,6 +112,36 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[0].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "workers": {
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                },
+            },
+            show_only=["templates/workers/worker-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",

Review Comment:
   done



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on pull request #24784: [WIP] added extraEnv to scheduler, webserver, triggerer, flower, workers

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on PR #24784:
URL: https://github.com/apache/airflow/pull/24784#issuecomment-1175301741

   Approach looks good. We should also do cleanup and the jobs.


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on PR #24784:
URL: https://github.com/apache/airflow/pull/24784#issuecomment-1183259997

   > Let's keep this PR to just env vars. Labels should be done in a separate PR.
   
   Had to merge the two PRs to generate a helm chart for testing. Shall keep it in two separate PRs


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r920050117


##########
chart/templates/workers/worker-deployment.yaml:
##########
@@ -219,6 +219,9 @@ spec:
         {{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) }}
         {{- include "git_sync_container" . | indent 8 }}
         {{- end }}
+{{- if .Values.workers.extraEnv }}
+{{ tpl (toYaml .Values.workers.extraEnv) $ | indent 12 }}

Review Comment:
   Thanks. Have pushed changes to the pod template file



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r947000141


##########
chart/templates/_helpers.yaml:
##########
@@ -782,3 +782,83 @@ Where `.` is the global variables scope and `.Values.workers` the local variable
     {{- end -}}
   {{- end -}}
 {{- end -}}
+{{- define "workers_extra_envs" -}}

Review Comment:
   You shouldn't need to do a helper per component, just pass the right map in from the template and have a single helper.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] boring-cyborg[bot] commented on pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on PR #24784:
URL: https://github.com/apache/airflow/pull/24784#issuecomment-1219836670

   Awesome work, congrats on your first merged pull request!
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r918389081


##########
chart/values.schema.json:
##########
@@ -552,6 +552,11 @@
                                 "IfNotPresent"
                             ],
                             "default": "IfNotPresent"
+                        },
+                        "extraEnv": {
+                            "description": "Add additional env vars to flower.",
+                            "type": "array",
+                            "default": []

Review Comment:
   You should add "items" here so we can properly validate what the user supplies here. e.g:
   https://github.com/apache/airflow/blob/2af19f16a4d94e749bbf6c7c4704e02aac35fc11/chart/values.schema.json#L695-L710



##########
chart/templates/dag-processor/dag-processor-deployment.yaml:
##########
@@ -130,6 +130,9 @@ spec:
         {{- if .Values.dagProcessor.extraInitContainers }}
         {{- toYaml .Values.dagProcessor.extraInitContainers | nindent 8 }}
         {{- end }}
+{{- if .Values.dagProcessor.waitForMigrations.extraEnv }}
+{{ tpl (toYaml .Values.dagProcessor.waitForMigrations.extraEnv) $ | indent 12 }}

Review Comment:
   Only waitForMigrations? What about the normal container?



##########
chart/templates/workers/worker-deployment.yaml:
##########
@@ -219,6 +219,9 @@ spec:
         {{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) }}
         {{- include "git_sync_container" . | indent 8 }}
         {{- end }}
+{{- if .Values.workers.extraEnv }}
+{{ tpl (toYaml .Values.workers.extraEnv) $ | indent 12 }}

Review Comment:
   Why no waitForMigrations here? Don't forget the k8s pod_template_file too.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r919087688


##########
chart/templates/workers/worker-deployment.yaml:
##########
@@ -219,6 +219,9 @@ spec:
         {{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) }}
         {{- include "git_sync_container" . | indent 8 }}
         {{- end }}
+{{- if .Values.workers.extraEnv }}
+{{ tpl (toYaml .Values.workers.extraEnv) $ | indent 12 }}

Review Comment:
   @jedcunningham There isn't currently a waitForMigrations property for worker pods. I have added the same set of envs used for the worker container for now. Or I could introduce a new property to add custom envs for the migrations container.
   Could you direct me on how to overwrite the base pod_template_file with labels and envs?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r943734196


##########
chart/files/pod-template-file.kubernetes-helm-yaml:
##########
@@ -58,6 +58,9 @@ spec:
           value: LocalExecutor
 {{- include "standard_airflow_environment" . | indent 6}}
 {{- include "custom_airflow_environment" . | indent 6 }}
+{{- if .Values.workers.env }}
+{{ tpl (toYaml .Values.workers.env) $ | indent 12 }}

Review Comment:
   done



##########
chart/templates/workers/worker-deployment.yaml:
##########
@@ -223,6 +226,9 @@ spec:
         {{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) }}
         {{- include "git_sync_container" . | indent 8 }}
         {{- end }}
+{{- if .Values.workers.env }}
+{{ tpl (toYaml .Values.workers.env) $ | indent 12 }}
+{{- end }}

Review Comment:
   done



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r941406773


##########
tests/charts/test_webserver.py:
##########
@@ -169,6 +169,39 @@ def test_should_add_extra_containers(self):
             "image": "test-registry/test-repo:test-tag",
         } == jmespath.search("spec.template.spec.containers[-1]", docs[0])
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",

Review Comment:
   done



##########
tests/charts/test_webserver.py:
##########
@@ -169,6 +169,39 @@ def test_should_add_extra_containers(self):
             "image": "test-registry/test-repo:test-tag",
         } == jmespath.search("spec.template.spec.containers[-1]", docs[0])
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "webserver": {
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                },
+            },
+            show_only=["templates/webserver/webserver-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "webserver": {
+                    "waitForMigrations": {
+                        "enabled": True,

Review Comment:
   done



##########
tests/charts/test_worker.py:
##########
@@ -112,6 +112,36 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[0].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",

Review Comment:
   done



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r941405674


##########
chart/values.schema.json:
##########
@@ -2619,6 +2698,14 @@
                     "description": "Specify if you want to use the default Helm Hook annotations",
                     "type": "boolean",
                     "default": true
+                },
+                "env": {
+                    "description": "Add additional env vars to job.",

Review Comment:
   done



##########
chart/values.yaml:
##########
@@ -597,6 +597,12 @@ workers:
     #   cpu: 100m
     #   memory: 128Mi
 
+  waitForMigrations:
+    # To add env vars specific to wait-for-migrations init container

Review Comment:
   done



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r918718812


##########
chart/templates/dag-processor/dag-processor-deployment.yaml:
##########
@@ -130,6 +130,9 @@ spec:
         {{- if .Values.dagProcessor.extraInitContainers }}
         {{- toYaml .Values.dagProcessor.extraInitContainers | nindent 8 }}
         {{- end }}
+{{- if .Values.dagProcessor.waitForMigrations.extraEnv }}
+{{ tpl (toYaml .Values.dagProcessor.waitForMigrations.extraEnv) $ | indent 12 }}

Review Comment:
   Wasn't aware if the dag-processor container interacted with the airflow db. Have added it for the main container as well



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r948986578


##########
tests/charts/test_dag_processor.py:
##########
@@ -119,6 +119,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[0].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "dagProcessor": {
+                    "enabled": True,
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                },
+            },
+            show_only=["templates/dag-processor/dag-processor-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "dagProcessor": {
+                    "enabled": True,
+                    "waitForMigrations": {
+                        "enabled": True,

Review Comment:
   resolved



##########
chart/values.schema.json:
##########
@@ -561,6 +561,27 @@
                                 "IfNotPresent"
                             ],
                             "default": "IfNotPresent"
+                        },
+                        "env": {
+                            "description": "Add additional env vars to flower.",
+                            "type": "array",
+                            "default": [],
+                            "items": {
+                                "type": "object",
+                                "properties": {
+                                    "name": {
+                                        "type": "string"
+                                    },
+                                    "value": {
+                                        "type": "string"
+                                    }
+                                },
+                                "required": [
+                                    "name",
+                                    "value"
+                                ],
+                                "additionalProperties": false
+                            }

Review Comment:
   done



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r919626769


##########
chart/Chart.yaml:
##########
@@ -19,7 +19,7 @@
 ---
 apiVersion: v2
 name: airflow
-version: 1.7.0-dev
+version: 1.8.0-dev

Review Comment:
   ```suggestion
   version: 1.7.0-dev
   ```
   
   This gets bumped as part of the release process.



##########
chart/Chart.yaml:
##########
@@ -19,7 +19,7 @@
 ---
 apiVersion: v2
 name: airflow
-version: 1.7.0-dev
+version: 1.8.0-dev

Review Comment:
   ```suggestion
   version: 1.7.0-dev
   ```
   
   This gets bumped as part of the chart release process.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r919624242


##########
chart/templates/workers/worker-deployment.yaml:
##########
@@ -219,6 +219,9 @@ spec:
         {{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) }}
         {{- include "git_sync_container" . | indent 8 }}
         {{- end }}
+{{- if .Values.workers.extraEnv }}
+{{ tpl (toYaml .Values.workers.extraEnv) $ | indent 12 }}

Review Comment:
   Here is the pod_template_file: https://github.com/apache/airflow/blob/main/chart/files/pod-template-file.kubernetes-helm-yaml



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] boring-cyborg[bot] commented on pull request #24784: [WIP] added extraEnv to scheduler, webserver, triggerer, flower, workers

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on PR #24784:
URL: https://github.com/apache/airflow/pull/24784#issuecomment-1172306700

   Congratulations on your first Pull Request and welcome to the Apache Airflow community! If you have any issues or are unsure about any anything please check our Contribution Guide (https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst)
   Here are some useful points:
   - Pay attention to the quality of your code (flake8, mypy and type annotations). Our [pre-commits]( https://github.com/apache/airflow/blob/main/STATIC_CODE_CHECKS.rst#prerequisites-for-pre-commit-hooks) will help you with that.
   - In case of a new feature add useful documentation (in docstrings or in `docs/` directory). Adding a new operator? Check this short [guide](https://github.com/apache/airflow/blob/main/docs/apache-airflow/howto/custom-operator.rst) Consider adding an example DAG that shows how users should use it.
   - Consider using [Breeze environment](https://github.com/apache/airflow/blob/main/BREEZE.rst) for testing locally, itโ€™s a heavy docker but it ships with a working Airflow and a lot of integrations.
   - Be patient and persistent. It might take some time to get a review or get the final approval from Committers.
   - Please follow [ASF Code of Conduct](https://www.apache.org/foundation/policies/conduct) for all communication including (but not limited to) comments on Pull Requests, Mailing list and Slack.
   - Be sure to read the [Airflow Coding style]( https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#coding-style-and-best-practices).
   Apache Airflow is a community-driven project and together we are making it better ๐Ÿš€.
   In case of doubts contact the developers at:
   Mailing List: dev@airflow.apache.org
   Slack: https://s.apache.org/airflow-slack
   


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r948321002


##########
chart/values.schema.json:
##########
@@ -561,6 +561,27 @@
                                 "IfNotPresent"
                             ],
                             "default": "IfNotPresent"
+                        },
+                        "env": {
+                            "description": "Add additional env vars to flower.",
+                            "type": "array",
+                            "default": [],
+                            "items": {
+                                "type": "object",
+                                "properties": {
+                                    "name": {
+                                        "type": "string"
+                                    },
+                                    "value": {
+                                        "type": "string"
+                                    }
+                                },
+                                "required": [
+                                    "name",
+                                    "value"
+                                ],
+                                "additionalProperties": false
+                            }

Review Comment:
   ```suggestion
   ```
   
   Don't need this, wrong section.



##########
tests/charts/test_dag_processor.py:
##########
@@ -119,6 +119,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[0].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "dagProcessor": {
+                    "enabled": True,
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                },
+            },
+            show_only=["templates/dag-processor/dag-processor-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "dagProcessor": {
+                    "enabled": True,
+                    "waitForMigrations": {
+                        "enabled": True,

Review Comment:
   ```suggestion
   ```
   
   This is the default.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r935748018


##########
chart/templates/dag-processor/dag-processor-deployment.yaml:
##########
@@ -130,6 +130,9 @@ spec:
         {{- if .Values.dagProcessor.extraInitContainers }}
         {{- toYaml .Values.dagProcessor.extraInitContainers | nindent 8 }}
         {{- end }}
+{{- if .Values.dagProcessor.waitForMigrations.env }}
+{{ tpl (toYaml .Values.dagProcessor.waitForMigrations.env) $ | indent 12 }}
+{{- end }}
       containers:
         - name: dag-processor

Review Comment:
   You've missed adding `dagProcessor.env` to the main container here.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
jedcunningham commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r941606170


##########
chart/templates/dag-processor/dag-processor-deployment.yaml:
##########
@@ -134,6 +134,9 @@ spec:
         {{- if .Values.dagProcessor.extraInitContainers }}
         {{- toYaml .Values.dagProcessor.extraInitContainers | nindent 8 }}
         {{- end }}
+{{- if .Values.dagProcessor.waitForMigrations.env }}
+{{ tpl (toYaml .Values.dagProcessor.waitForMigrations.env) $ | indent 12 }}
+{{- end }}

Review Comment:
   Try it with gitsync enabled or an extra init container.



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r938062135


##########
chart/templates/dag-processor/dag-processor-deployment.yaml:
##########
@@ -130,6 +130,9 @@ spec:
         {{- if .Values.dagProcessor.extraInitContainers }}
         {{- toYaml .Values.dagProcessor.extraInitContainers | nindent 8 }}
         {{- end }}
+{{- if .Values.dagProcessor.waitForMigrations.env }}
+{{ tpl (toYaml .Values.dagProcessor.waitForMigrations.env) $ | indent 12 }}
+{{- end }}
       containers:
         - name: dag-processor

Review Comment:
   Ah thanks! Added the snippet



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] aliotta commented on pull request #24784: [WIP] added extraEnv to scheduler, webserver, triggerer, flower, workers

Posted by GitBox <gi...@apache.org>.
aliotta commented on PR #24784:
URL: https://github.com/apache/airflow/pull/24784#issuecomment-1172633258

   this PR will need chart tests


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on PR #24784:
URL: https://github.com/apache/airflow/pull/24784#issuecomment-1181500044

   > You should use `env`, not `extraEnv`. `extraEnv` is already used globally for thing based config, so having one a string and another a list is confusing:
   > 
   > https://github.com/apache/airflow/blob/2af19f16a4d94e749bbf6c7c4704e02aac35fc11/chart/values.yaml#L300-L307
   
   @jedcunningham Have updated `extraEnv` to `env` as a property under each container


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r918717859


##########
chart/values.schema.json:
##########
@@ -552,6 +552,11 @@
                                 "IfNotPresent"
                             ],
                             "default": "IfNotPresent"
+                        },
+                        "extraEnv": {
+                            "description": "Add additional env vars to flower.",
+                            "type": "array",
+                            "default": []

Review Comment:
   Have added "items" as standard validation for envVar
   ```
   "items": {
   "$ref": "#/definitions/io.k8s.api.core.v1.EnvVar"
   }
   ```



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r920049348


##########
chart/files/pod-template-file.kubernetes-helm-yaml:
##########
@@ -58,6 +58,9 @@ spec:
           value: LocalExecutor
 {{- include "standard_airflow_environment" . | indent 6}}
 {{- include "custom_airflow_environment" . | indent 6 }}
+{{- if .Values.workers.env }}
+{{ tpl (toYaml .Values.workers.env) $ | indent 12 }}
+{{- end }}

Review Comment:
   @jedcunningham have made the changes to the pod template file here



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r941406450


##########
tests/charts/test_scheduler.py:
##########
@@ -115,6 +115,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[*].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "scheduler": {
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                },
+            },
+            show_only=["templates/scheduler/scheduler-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "executor": "CeleryExecutor",
+                "scheduler": {
+                    "waitForMigrations": {
+                        "enabled": True,

Review Comment:
   done



##########
tests/charts/test_triggerer.py:
##########
@@ -131,6 +131,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[0].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "triggerer": {
+                    "enabled": True,

Review Comment:
   done



##########
tests/charts/test_triggerer.py:
##########
@@ -131,6 +131,39 @@ def test_should_add_extra_volume_and_extra_volume_mount(self):
             "spec.template.spec.containers[0].volumeMounts[0].name", docs[0]
         )
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={
+                "triggerer": {
+                    "enabled": True,
+                    "env": [{"name": "TEST_ENV_1", "value": "test_env_1"}],
+                }
+            },
+            show_only=["templates/triggerer/triggerer-deployment.yaml"],
+        )
+
+        assert {'name': 'TEST_ENV_1', 'value': 'test_env_1'} in jmespath.search(
+            "spec.template.spec.containers[0].env", docs[0]
+        )
+
+    def test_should_add_extraEnvs_to_wait_for_migration_container(self):
+        docs = render_chart(
+            values={
+                "triggerer": {
+                    "enabled": True,
+                    "waitForMigrations": {
+                        "enabled": True,

Review Comment:
   done



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r943734546


##########
chart/templates/dag-processor/dag-processor-deployment.yaml:
##########
@@ -134,6 +134,9 @@ spec:
         {{- if .Values.dagProcessor.extraInitContainers }}
         {{- toYaml .Values.dagProcessor.extraInitContainers | nindent 8 }}
         {{- end }}
+{{- if .Values.dagProcessor.waitForMigrations.env }}
+{{ tpl (toYaml .Values.dagProcessor.waitForMigrations.env) $ | indent 12 }}
+{{- end }}

Review Comment:
   done



##########
tests/charts/test_pod_template_file.py:
##########
@@ -639,6 +639,16 @@ def test_should_add_pod_labels(self):
             "tier": "airflow",
         } == jmespath.search("metadata.labels", docs[0])
 
+    def test_should_add_extraEnvs(self):
+        docs = render_chart(
+            values={"env": [{"name": "TEST_ENV_1", "value": "test_env_1"}]},

Review Comment:
   done



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] rishkarajgi commented on a diff in pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
rishkarajgi commented on code in PR #24784:
URL: https://github.com/apache/airflow/pull/24784#discussion_r947782652


##########
chart/templates/_helpers.yaml:
##########
@@ -782,3 +782,83 @@ Where `.` is the global variables scope and `.Values.workers` the local variable
     {{- end -}}
   {{- end -}}
 {{- end -}}
+{{- define "workers_extra_envs" -}}

Review Comment:
   Done



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] jedcunningham merged pull request #24784: Container specific extra environment variables

Posted by GitBox <gi...@apache.org>.
jedcunningham merged PR #24784:
URL: https://github.com/apache/airflow/pull/24784


-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org