You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2020/10/20 16:06:04 UTC

[GitHub] [airflow] dimberman opened a new pull request #11693: add python helm testing framework

dimberman opened a new pull request #11693:
URL: https://github.com/apache/airflow/pull/11693


   <!--
   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/master/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 [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] ashb commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r509624100



##########
File path: chart_tests/test_templates/git_sync_scheduler/git-sync-existing-claim.yaml
##########
@@ -0,0 +1,21 @@
+# 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.
+---
+dags:
+ persistence:
+  enabled: true
+  existingClaim: test-claim

Review comment:
       Separate files makes it harder to understand the tests.
   
   Please move these to dicts in the python test files




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] kaxil commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510369064



##########
File path: chart_tests/helm_template_generator.py
##########
@@ -0,0 +1,62 @@
+# 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 subprocess
+import sys
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+import yaml
+from kubernetes.client.api_client import ApiClient
+
+api_client = ApiClient()
+
+
+def render_chart(name, values: Optional[Dict] = None, show_only=None):
+    """
+    Function that renders a helm chart into dictionaries. For helm chart testing only
+
+    :param name:
+    :param values:
+    :param show_only:

Review comment:
       ```suggestion
       :param show_only: only show manifests rendered from the given templates (same as helm template --show-only)
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman merged pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman merged pull request #11693:
URL: https://github.com/apache/airflow/pull/11693


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512181994



##########
File path: .pre-commit-config.yaml
##########
@@ -277,7 +277,7 @@ repos:
         entry: "^\\s*from\\s+\\."
         pass_filenames: true
         files: \.py$
-        exclude: ^tests/
+        exclude: ^tests/|^chart/tests/

Review comment:
       1. Click **File** => **Project structure**
   1. On the left menu, select "**Project settings** => **Modules**
   1. Click **+** >> **Add new module** to a new project module
   1. Select **Python**
   1. **Next**
   1. Type **Chart** in "Module name"" field
   1. **Finish**
   1. Click **Add Content Root** and add `/chart` directory again
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] kaxil commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510365509



##########
File path: .dockerignore
##########
@@ -45,6 +45,7 @@
 # Add tests and kubernetes_tests to context.
 !tests
 !kubernetes_tests
+!chart_tests

Review comment:
       ^^ @dimberman 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] kaxil commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510469448



##########
File path: chart_tests/test_templates/git_sync_scheduler/git-sync-existing-claim.yaml
##########
@@ -0,0 +1,21 @@
+# 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.
+---
+dags:
+ persistence:
+  enabled: true
+  existingClaim: test-claim

Review comment:
       +1

##########
File path: chart_tests/test_templates/git_sync_scheduler/git-sync-existing-claim.yaml
##########
@@ -0,0 +1,21 @@
+# 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.
+---
+dags:
+ persistence:
+  enabled: true
+  existingClaim: test-claim

Review comment:
       +1 to keep it inside Python 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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511213269



##########
File path: scripts/ci/kubernetes/ci_run_helm_testing.sh
##########
@@ -18,7 +18,28 @@
 
 echo "Running helm tests"
 
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
+
+if [[ -f "${BUILD_CACHE_DIR}/.skip_tests" ]]; then
+    echo
+    echo "Skip tests"
+    echo
+    exit
+fi
+
+build_images::prepare_ci_build
+
+build_images::rebuild_ci_image_if_needed
+kind::make_sure_kubernetes_tools_are_installed
+
+
+python -m pip install pytest kubernetes \

Review comment:
       I am concerned that we will be able to achieve a selective selection of tests based on directories, if we do not have a unified setup. Now we will only be able to run the Chart tests if there are changes to Chart only. This will greatly reduce the frequency of running these tests.
   
   The ability to run in Breeze will also facilitate contributions by new contributors. They will be able to run the tests very easily via ``breeze tests -- /chart/tests/``. 
   
   While I'm not sure about the first part, we should definitely support running the tests in Breeze.  This shouldn't be difficult as we just need Helm to have Docker installed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#issuecomment-716051619


   Can you also add docs about running these 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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511642713



##########
File path: chart/tests/conftest.py
##########
@@ -0,0 +1,33 @@
+# 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 os
+import subprocess
+import sys
+
+# We should set these before loading _any_ of the rest of airflow so that the
+# unit test mode config is set as early as possible.
+tests_directory = os.path.dirname(os.path.realpath(__file__))
+
+os.environ["AIRFLOW__CORE__DAGS_FOLDER"] = os.path.join(tests_directory, "dags")
+os.environ["AIRFLOW__CORE__UNIT_TEST_MODE"] = "True"
+os.environ["AWS_DEFAULT_REGION"] = (os.environ.get("AWS_DEFAULT_REGION") or "us-east-1")
+os.environ["CREDENTIALS_DIR"] = (os.environ.get('CREDENTIALS_DIR') or "/files/airflow-breeze-config/keys")
+
+subprocess.check_output(["helm", "repo", "add", "stable",
+                         "https://kubernetes-charts.storage.googleapis.com/"])
+subprocess.check_output(["helm", "dep", "update", sys.path[0]])

Review comment:
       ```suggestion
   import subprocess
   import sys
   
   import pytest
   
   
   @pytest.fixture(autouse=True, scope="session")
   def upgrade_helm():
       """
       Upgrade Helm repo
       """
       subprocess.check_output([
           "helm", "repo", "add", "stable", "https://kubernetes-charts.storage.googleapis.com/"
       ])
       subprocess.check_output(["helm", "dep", "update", sys.path[0]])
   ```
   It is better to avoid executing code at the module level as this results in an bad end user experience. When you run the Pytest command, you have black screen for a long time with no message. After the change is made, the tests are run and fixutres are executed as part of the 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.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r509613561



##########
File path: tests/helm/test_git_sync_scheduler.py
##########
@@ -0,0 +1,136 @@
+# 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 sys
+import unittest
+
+import yaml
+from kubernetes.client import models as k8s
+
+from airflow.utils.helm_template_generator import render_chart, render_k8s_object
+
+OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 22
+
+
+class TestGitSyncScheduler(unittest.TestCase):
+    def setUp(self) -> None:
+        with open(sys.path[0] + "/tests/helm/test_templates/"
+                                "git_sync_scheduler/git-sync-scheduler.yaml") as file:
+            self.basic_object = yaml.load(file)
+        with open(sys.path[0] + "/tests/helm/test_templates/"
+                                "git_sync_scheduler/git-sync-container-spec.yaml") as file:
+            self.object_with_git_container_spec = yaml.load(file)
+        with open(sys.path[0] + "/tests/helm/test_templates/"
+                                "git_sync_scheduler/git-sync-ssh-params.yaml") as file:
+            self.object_with_ssh_params = yaml.load(file)
+        with open(sys.path[0] + "/tests/helm/test_templates/"
+                                "git_sync_scheduler/git-sync-username.yaml") as file:
+            self.object_with_username = yaml.load(file)
+        with open(sys.path[0] + "/tests/helm/test_templates/git_sync_scheduler/"
+                                "git-sync-existing-claim.yaml") as file:
+            self.object_with_existing_claim = yaml.load(file)
+
+    def test_basic(self):
+        res = render_chart("GIT-SYNC", self.basic_object,
+                           show_only=["templates/scheduler/scheduler-deployment.yaml"])
+        dep: k8s.V1Deployment = render_k8s_object(res[0], k8s.V1Deployment)
+        self.assertEqual("dags", dep.spec.template.spec.volumes[1].name)
+
+    def test_git_container_spec(self):
+        res = render_chart("GIT-SYNC", self.object_with_git_container_spec,
+                           show_only=["templates/scheduler/scheduler-deployment.yaml"])
+        dep: k8s.V1Deployment = render_k8s_object(res[0], k8s.V1Deployment)
+        git_sync_container = dep.spec.template.spec.containers[1]
+        self.assertEqual(git_sync_container.image, "test-registry/test-repo:test-tag")
+        self.assertEqual(git_sync_container.name, "git-sync-test")
+        self.assertEqual(git_sync_container.security_context.run_as_user, 65533)
+        env_dict = [e.to_dict() for e in git_sync_container.env]
+        self.assertEqual(env_dict, [
+            {'name': 'GIT_SYNC_REV', 'value': 'HEAD', 'value_from': None},
+            {'name': 'GIT_SYNC_BRANCH', 'value': 'test-branch', 'value_from': None},
+            {'name': 'GIT_SYNC_REPO', 'value': 'https://github.com/apache/airflow.git', 'value_from': None},
+            {'name': 'GIT_SYNC_DEPTH', 'value': '1', 'value_from': None},
+            {'name': 'GIT_SYNC_ROOT', 'value': '/git-root', 'value_from': None},
+            {'name': 'GIT_SYNC_DEST', 'value': 'test-dest', 'value_from': None},
+            {'name': 'GIT_SYNC_ADD_USER', 'value': 'true', 'value_from': None},
+            {'name': 'GIT_SYNC_WAIT', 'value': '66', 'value_from': None},
+            {'name': 'GIT_SYNC_MAX_SYNC_FAILURES', 'value': '70', 'value_from': None}])
+
+        self.assertEqual(git_sync_container.volume_mounts,
+                         [k8s.V1VolumeMount(name="dags", mount_path="/git-root")])
+
+    def test_ssh_params_added(self):
+        res = render_chart("GIT-SYNC", self.object_with_ssh_params,
+                           show_only=["templates/scheduler/scheduler-deployment.yaml"])
+        dep: k8s.V1Deployment = render_k8s_object(res[0], k8s.V1Deployment)
+        git_sync_container = dep.spec.template.spec.containers[1]
+        env_dict = [e.to_dict() for e in git_sync_container.env]
+        self.assertEqual(env_dict, [
+            {'name': 'GIT_SSH_KEY_FILE', 'value': '/etc/git-secret/ssh', 'value_from': None},
+            {'name': 'GIT_SYNC_SSH', 'value': 'true', 'value_from': None},
+            {'name': 'GIT_KNOWN_HOSTS', 'value': 'false', 'value_from': None},
+            {'name': 'GIT_SYNC_REV', 'value': 'HEAD', 'value_from': None},
+            {'name': 'GIT_SYNC_BRANCH', 'value': 'test-branch', 'value_from': None},
+            {'name': 'GIT_SYNC_REPO', 'value': 'https://github.com/apache/airflow.git',
+             'value_from': None},
+            {'name': 'GIT_SYNC_DEPTH', 'value': '1', 'value_from': None},
+            {'name': 'GIT_SYNC_ROOT', 'value': '/git', 'value_from': None},
+            {'name': 'GIT_SYNC_DEST', 'value': 'repo', 'value_from': None},
+            {'name': 'GIT_SYNC_ADD_USER', 'value': 'true', 'value_from': None},
+            {'name': 'GIT_SYNC_WAIT', 'value': '60', 'value_from': None},
+            {'name': 'GIT_SYNC_MAX_SYNC_FAILURES', 'value': '0', 'value_from': None}])
+
+    def test_adds_git_username(self):
+        res = render_chart("GIT-SYNC", self.object_with_username,
+                           show_only=["templates/scheduler/scheduler-deployment.yaml"])
+        dep: k8s.V1Deployment = render_k8s_object(res[0], k8s.V1Deployment)
+        git_sync_container = dep.spec.template.spec.containers[1]
+        env_dict = [e.to_dict() for e in git_sync_container.env]
+        self.assertEqual(env_dict, [
+            {'name': 'GIT_SYNC_USERNAME', 'value': None, 'value_from':
+                {'config_map_key_ref': None,
+                 'field_ref': None,
+                 'resource_field_ref': None,
+                 'secret_key_ref': {'key': 'GIT_SYNC_USERNAME',
+                                    'name': 'user-pass-secret', 'optional': None}}},
+            {'name': 'GIT_SYNC_PASSWORD', 'value': None,
+             'value_from':
+                 {'config_map_key_ref': None,

Review comment:
       @mik-laj enabled black




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#issuecomment-713061415


   [The Workflow run](https://github.com/apache/airflow/actions/runs/318297415) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511642215



##########
File path: Dockerfile.ci
##########
@@ -348,8 +348,19 @@ RUN if command -v airflow; then \
 # Install autocomplete for Kubectl
 RUN echo "source /etc/bash_completion" >> ~/.bashrc
 
+# Install HElm

Review comment:
       ```suggestion
   # Install Helm
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511642810



##########
File path: chart/tests/helm_template_generator.py
##########
@@ -0,0 +1,61 @@
+# 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 subprocess
+import sys
+from tempfile import NamedTemporaryFile
+
+import yaml
+from kubernetes.client.api_client import ApiClient
+
+api_client = ApiClient()
+
+
+def render_chart(name, values, show_only=None):

Review comment:
       ```suggestion
   def render_chart(name = "RELEASE-NAME", values = None, show_only=None):
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] kaxil commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510366148



##########
File path: chart_tests/basic_helm_chart_test.py
##########
@@ -0,0 +1,66 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+
+from .helm_template_generator import render_chart
+
+OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 22
+
+
+class TestBaseChartTest(unittest.TestCase):
+    def test_basic_deployments(self):

Review comment:
       Adding a short description on what this test is actually doing should help future readers understand and  add more tests 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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511222593



##########
File path: scripts/ci/kubernetes/ci_run_helm_testing.sh
##########
@@ -18,7 +18,28 @@
 
 echo "Running helm tests"
 
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
+
+if [[ -f "${BUILD_CACHE_DIR}/.skip_tests" ]]; then
+    echo
+    echo "Skip tests"
+    echo
+    exit
+fi
+
+build_images::prepare_ci_build
+
+build_images::rebuild_ci_image_if_needed
+kind::make_sure_kubernetes_tools_are_installed
+
+
+python -m pip install pytest kubernetes \

Review comment:
       When I did following changes:
   ```diff
   Author: Kamil Breguła <mi...@users.noreply.github.com>
   Date:   Sat Oct 24 03:00:24 2020 +0200
   
       Add Helm to dockerfile
   
   diff --git a/Dockerfile.ci b/Dockerfile.ci
   index e352bedc3..e2528f748 100644
   --- a/Dockerfile.ci
   +++ b/Dockerfile.ci
   @@ -215,6 +215,14 @@ RUN mkdir -p /opt/bats/lib/bats-file \
   
    RUN echo "export PATH=/opt/bats/bin:${PATH}" >> /root/.bashrc
   
   +# Install HElm
   +ARG HELM_VERSION="v3.2.4"
   +
   +RUN SYSTEM=$(uname -s | tr '[:upper:]' '[:lower:]') \
   +    && HELM_URL="https://get.helm.sh/helm-${HELM_VERSION}-${SYSTEM}-amd64.tar.gz" \
   +    && curl --location ${HELM_URL} | tar -xvz -O ${SYSTEM}-amd64/helm > /usr/local/bin/helm \
   +    && chmod +x /usr/local/bin/helm
   +
    # Additional scripts for managing BATS addons
    COPY scripts/docker/load.bash /opt/bats/lib/
    RUN chmod a+x /opt/bats/lib/load.bash
   ```
   
   I was able to run the tests with one command:
   ```
   ./breeze tests  -- chart/tests
   ```
   This is my perfect workflow.  I don't need to know any additional commands to run these 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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511641661



##########
File path: .pre-commit-config.yaml
##########
@@ -153,7 +153,7 @@ repos:
     rev: 20.8b1
     hooks:
       - id: black
-        files: api_connexion/.*\.py|.*providers.*\.py
+        files: api_connexion/.*\.py|.*providers.*\.py|../chart/tests.*\.py

Review comment:
       ```suggestion
           files: api_connexion/.*\.py|.*providers.*\.py|chart/tests/.*\.py
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511643337



##########
File path: scripts/in_container/entrypoint_ci.sh
##########
@@ -242,6 +269,7 @@ else
         "tests/utils"
     )
     WWW_TESTS=("tests/www")
+    HELM_CHART_TESTS=("chart/tests")

Review comment:
       Thank you very much for that. This will make my job much easier.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r509583891



##########
File path: scripts/ci/libraries/_local_mounts.sh
##########
@@ -51,6 +51,7 @@ function local_mounts::generate_local_mounts_list {
         "$prefix"setup.py:/opt/airflow/setup.py:cached
         "$prefix"tests:/opt/airflow/tests:cached
         "$prefix"kubernetes_tests:/opt/airflow/kubernetes_tests:cached
+        "$prefix"chart_tests:/opt/airflow/chart_tests:cached

Review comment:
       I think you should also add /chart/ directory. See: https://github.com/apache/airflow/pull/11694/files




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512156689



##########
File path: TESTING.rst
##########
@@ -196,6 +196,44 @@ Run all Quarantined tests:
 
      ./breeze --test-type Quarantined tests --db-reset
 
+Helm Unit Tests

Review comment:
       In my opinion, it is worth adding a command that runs the tests - `pytest chart/tests`.  All in all, this is the only thing that is important because the rest of the user can copy from other 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.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512170836



##########
File path: chart/tests/git-sync-scheduler_test.yaml
##########
@@ -1,154 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       Yes. I replaced all tests in this file with the python file. I plan to replace the rest of the tests in future PRs and then remove helm-unittest completely.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510775901



##########
File path: .dockerignore
##########
@@ -45,6 +45,7 @@
 # Add tests and kubernetes_tests to context.
 !tests
 !kubernetes_tests
+!chart_tests

Review comment:
       We can also run mypy two times - one for core, one for Helm Chart.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512165649



##########
File path: .pre-commit-config.yaml
##########
@@ -277,7 +277,7 @@ repos:
         entry: "^\\s*from\\s+\\."
         pass_filenames: true
         files: \.py$
-        exclude: ^tests/
+        exclude: ^tests/|^chart/tests/

Review comment:
       In my opinion, this is expected behavior. Instead, we should use the correct imports.
   ````diff
   -from .helm_template_generator import render_chart
   +from tests.helm_template_generator import render_chart
   ````




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512183344



##########
File path: .pre-commit-config.yaml
##########
@@ -277,7 +277,7 @@ repos:
         entry: "^\\s*from\\s+\\."
         pass_filenames: true
         files: \.py$
-        exclude: ^tests/
+        exclude: ^tests/|^chart/tests/

Review comment:
       <img width="1038" alt="Screenshot 2020-10-26 at 19 33 49" src="https://user-images.githubusercontent.com/12058428/97213653-324e0400-17c2-11eb-83f9-544333fbfcb3.png">
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#issuecomment-717640127


   🎉 Thanks!
   
   I will now migrate the rest of the files.
   https://github.com/apache/airflow/pull/11827


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510464290



##########
File path: chart_tests/test_templates/git_sync_scheduler/git-sync-existing-claim.yaml
##########
@@ -0,0 +1,21 @@
+# 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.
+---
+dags:
+ persistence:
+  enabled: true
+  existingClaim: test-claim

Review comment:
       If we want to have the documentation automatically tested, we can save some examples to a YAML file. However, this does not prevent you from trying to keep the full configurations in one place in most cases.
   
   I guess it won't scale because we want to test the Helm Chart in a lot of different configurations. Very soon we will have more configuration files than test files. Each file will probably only be used in one test so that will be difficult to maintain.
   
   On the other hand, if we keep our configurations in a Python file, we can store them locally within the test_ * function. If you open the test, you can see all input parameters and assertions without changing context.  This allows you to review all tests from A to Z much faster.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511198433



##########
File path: .github/workflows/ci.yml
##########
@@ -138,13 +138,32 @@ jobs:
     timeout-minutes: 5
     name: "Checks: Helm tests"
     runs-on: ubuntu-latest
-    needs: [build-info]
+    needs: [build-info, ci-images]
+    env:
+      SKIP: "pylint"
+      MOUNT_LOCAL_SOURCES: "true"
+      PYTHON_MAJOR_MINOR_VERSION: ${{needs.build-info.outputs.defaultPythonVersion}}
     if: >
       needs.build-info.outputs.needs-helm-tests == 'true' &&
       (github.repository == 'apache/airflow' || github.event_name != 'schedule')
     steps:
       - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )"
         uses: actions/checkout@v2
+      - name: "Setup python"

Review comment:
       This seems redundant to me as we already have Python downloaded in the container.
   
   @potiuk Can you help with CI setup?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] kaxil commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510367732



##########
File path: chart_tests/test_git_sync_scheduler.py
##########
@@ -0,0 +1,194 @@
+# 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 sys
+import unittest
+
+import yaml
+from kubernetes.client import models as k8s
+
+from .helm_template_generator import render_chart, render_k8s_object
+
+OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 22
+
+
+class TestGitSyncScheduler(unittest.TestCase):
+    def setUp(self) -> None:
+        with open(
+            sys.path[0] + "/chart_tests/test_templates/" "git_sync_scheduler/git-sync-scheduler.yaml"
+        ) as file:
+            self.basic_object = yaml.load(file)
+        with open(
+            sys.path[0] + "/chart_tests/test_templates/" "git_sync_scheduler/git-sync-container-spec.yaml"
+        ) as file:
+            self.object_with_git_container_spec = yaml.load(file)
+        with open(
+            sys.path[0] + "/chart_tests/test_templates/" "git_sync_scheduler/git-sync-ssh-params.yaml"
+        ) as file:
+            self.object_with_ssh_params = yaml.load(file)
+        with open(
+            sys.path[0] + "/chart_tests/test_templates/" "git_sync_scheduler/git-sync-username.yaml"
+        ) as file:
+            self.object_with_username = yaml.load(file)
+        with open(
+            sys.path[0] + "/chart_tests/test_templates/git_sync_scheduler/" "git-sync-existing-claim.yaml"
+        ) as file:
+            self.object_with_existing_claim = yaml.load(file)

Review comment:
       This looks bad, either use python dicts directly, or create a function that takes a path, and return loaded_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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r508851415



##########
File path: tests/helm/test_git_sync_scheduler.py
##########
@@ -0,0 +1,136 @@
+# 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 sys
+import unittest
+
+import yaml
+from kubernetes.client import models as k8s
+
+from airflow.utils.helm_template_generator import render_chart, render_k8s_object
+
+OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 22
+
+
+class TestGitSyncScheduler(unittest.TestCase):
+    def setUp(self) -> None:
+        with open(sys.path[0] + "/tests/helm/test_templates/"

Review comment:
       I don't see any benefit in keeping this as separate files. It just causes us to switch and look for the related 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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#issuecomment-714793805


   [The Workflow run](https://github.com/apache/airflow/actions/runs/323002198) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] ashb commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510370835



##########
File path: chart_tests/test_templates/git_sync_scheduler/git-sync-existing-claim.yaml
##########
@@ -0,0 +1,21 @@
+# 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.
+---
+dags:
+ persistence:
+  enabled: true
+  existingClaim: test-claim

Review comment:
       That's two hypotheticals - we can do that when we need it.
   
   Right now it will be easier to understand living right along side the test assertions




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511163187



##########
File path: chart/tests/helm_template_generator.py
##########
@@ -0,0 +1,61 @@
+# 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 subprocess
+import sys
+from tempfile import NamedTemporaryFile
+
+import yaml
+from kubernetes.client.api_client import ApiClient
+
+api_client = ApiClient()
+
+
+def render_chart(name, values, show_only=None):
+    """
+    Function that renders a helm chart into dictionaries. For helm chart testing only
+
+    :param name:
+    :param values:
+    :param show_only:
+    :return:
+    """
+    values = values or {}
+    with NamedTemporaryFile() as tmp_file:
+        content = yaml.dump(values)
+        tmp_file.write(content.encode())
+        tmp_file.flush()
+        command = ["helm", "template", name, sys.path[0], '--values', tmp_file.name]
+        if show_only:
+            for i in show_only:
+                command.extend(["--show-only", i])
+        templates = subprocess.check_output(command)
+        k8s_objects = yaml.load_all(templates)
+        k8s_objects = [k8s_object for k8s_object in k8s_objects if k8s_object]  # type: ignore
+        return k8s_objects
+
+
+def render_k8s_object(obj, type_to_render):
+    """
+
+    Function that renders dictionaries into k8s objects. For helm chart testing only.
+
+    :param obj:
+    :param type_to_render:
+    :return:

Review comment:
       ```suggestion
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511203127



##########
File path: scripts/ci/kubernetes/ci_run_helm_testing.sh
##########
@@ -18,7 +18,28 @@
 
 echo "Running helm tests"
 
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
+
+if [[ -f "${BUILD_CACHE_DIR}/.skip_tests" ]]; then
+    echo
+    echo "Skip tests"
+    echo
+    exit
+fi
+
+build_images::prepare_ci_build
+
+build_images::rebuild_ci_image_if_needed
+kind::make_sure_kubernetes_tools_are_installed
+
+
+python -m pip install pytest kubernetes \

Review comment:
       @mik-laj Is this necessary? I'm pretty heavily basing this on how we currently do kubernetes testing https://github.com/apache/airflow/blob/master/scripts/ci/kubernetes/ci_run_kubernetes_tests.sh This seems like a pretty one-off situation in a similar fashion (doesn't really require anything from the airflow CI and in reality besides the k8s library the testing environment should be very basic).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] kaxil commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510366640



##########
File path: chart_tests/helm_template_generator.py
##########
@@ -0,0 +1,62 @@
+# 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 subprocess
+import sys
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+import yaml
+from kubernetes.client.api_client import ApiClient
+
+api_client = ApiClient()
+
+
+def render_chart(name, values: Optional[Dict] = None, show_only=None):
+    """
+    Function that renders a helm chart into dictionaries. For helm chart testing only
+
+    :param name:
+    :param values:
+    :param show_only:
+    :return:
+    """
+    values = values or {}
+    with NamedTemporaryFile() as tmp_file:
+        content = yaml.dump(values)
+        tmp_file.write(content.encode())
+        tmp_file.flush()
+        command = ["helm", "template", name, sys.path[0] + '/chart/', '--values', tmp_file.name]
+        if show_only:
+            for i in show_only:
+                command.extend(["--show-only", i])
+        templates = subprocess.check_output(command)
+        k8s_objects = yaml.load_all(templates)
+        k8s_objects = [k8s_object for k8s_object in k8s_objects if k8s_object]  # type: ignore
+        return k8s_objects
+
+
+def render_k8s_object(obj, type_to_render):
+    """
+

Review comment:
       ```suggestion
   ```

##########
File path: chart_tests/helm_template_generator.py
##########
@@ -0,0 +1,62 @@
+# 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 subprocess
+import sys
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+import yaml
+from kubernetes.client.api_client import ApiClient
+
+api_client = ApiClient()
+
+
+def render_chart(name, values: Optional[Dict] = None, show_only=None):
+    """
+    Function that renders a helm chart into dictionaries. For helm chart testing only
+
+    :param name:
+    :param values:
+    :param show_only:
+    :return:
+    """
+    values = values or {}
+    with NamedTemporaryFile() as tmp_file:
+        content = yaml.dump(values)
+        tmp_file.write(content.encode())
+        tmp_file.flush()
+        command = ["helm", "template", name, sys.path[0] + '/chart/', '--values', tmp_file.name]
+        if show_only:
+            for i in show_only:
+                command.extend(["--show-only", i])
+        templates = subprocess.check_output(command)
+        k8s_objects = yaml.load_all(templates)
+        k8s_objects = [k8s_object for k8s_object in k8s_objects if k8s_object]  # type: ignore
+        return k8s_objects
+
+
+def render_k8s_object(obj, type_to_render):
+    """
+
+    Function that renders dictionaries into k8s objects. For helm chart testing only.
+
+    :param obj:
+    :param type_to_render:
+    :return:

Review comment:
       Same 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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511642251



##########
File path: Dockerfile.ci
##########
@@ -348,8 +348,19 @@ RUN if command -v airflow; then \
 # Install autocomplete for Kubectl
 RUN echo "source /etc/bash_completion" >> ~/.bashrc
 
+# Install HElm
+ARG HELM_VERSION="v3.2.4"
+
+RUN SYSTEM=$(uname -s | tr '[:upper:]' '[:lower:]') \
+    && HELM_URL="https://get.helm.sh/helm-${HELM_VERSION}-${SYSTEM}-amd64.tar.gz" \
+    && curl --location "${HELM_URL}" | tar -xvz -O "${SYSTEM}"-amd64/helm > /usr/local/bin/helm \
+    && chmod +x /usr/local/bin/helm
+
 WORKDIR ${AIRFLOW_SOURCES}
 
+RUN helm repo add stable https://kubernetes-charts.storage.googleapis.com/

Review comment:
       Is it needed?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#issuecomment-716861094






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510568399



##########
File path: chart_tests/test_templates/git_sync_scheduler/git-sync-existing-claim.yaml
##########
@@ -0,0 +1,21 @@
+# 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.
+---
+dags:
+ persistence:
+  enabled: true
+  existingClaim: test-claim

Review comment:
       @ashb @mik-laj @kaxil  I moved the configurations into the python files, but I think there could be value here in keeping them as yaml strings as it gives a better 1:1 comparison to an actual launch of the helm chart. Would like feedback though.

##########
File path: .dockerignore
##########
@@ -45,6 +45,7 @@
 # Add tests and kubernetes_tests to context.
 !tests
 !kubernetes_tests
+!chart_tests

Review comment:
       moved to 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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512176444



##########
File path: chart/tests/git-sync-scheduler_test.yaml
##########
@@ -1,154 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       Ok. Thanks.
   
   I've already converted the rest of the files.
   https://github.com/apache/airflow/pull/11827




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#issuecomment-715624688


   [The Workflow run](https://github.com/apache/airflow/actions/runs/325087956) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511642736



##########
File path: chart/tests/git-sync-scheduler_test.yaml
##########
@@ -1,154 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       Is it expected?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r509553551



##########
File path: tests/helm/__init__.py
##########
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       @mik-laj I created a `chart_tests` directory as many parts of breeze assume we don't have the chart in the docker image.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511642916



##########
File path: scripts/ci/testing/ci_run_airflow_testing.sh
##########
@@ -60,11 +60,12 @@ function run_airflow_testing_in_docker() {
             echo "Skip creating kerberos network"
         fi
         docker-compose --log-level INFO \
-          -f "${SCRIPTS_CI_DIR}/docker-compose/base.yml" \
-          -f "${SCRIPTS_CI_DIR}/docker-compose/backend-${BACKEND}.yml" \
-          "${INTEGRATIONS[@]}" \
-          "${DOCKER_COMPOSE_LOCAL[@]}" \
-             run airflow "${@}"
+            -f "${SCRIPTS_CI_DIR}/docker-compose/base.yml" \

Review comment:
       Is it expected?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512169932



##########
File path: .pre-commit-config.yaml
##########
@@ -277,7 +277,7 @@ repos:
         entry: "^\\s*from\\s+\\."
         pass_filenames: true
         files: \.py$
-        exclude: ^tests/
+        exclude: ^tests/|^chart/tests/

Review comment:
       <img width="786" alt="Screen Shot 2020-10-26 at 11 10 42 AM" src="https://user-images.githubusercontent.com/2644098/97211278-0a3bb200-177c-11eb-9ea9-dad29c2d8fac.png">
   That works on the docker-based tests, but is shows an error on PyCharm which might make development difficult. Any idea how to fix that? I think pycharm usually assumes root.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r509401206



##########
File path: tests/helm/test_git_sync_scheduler.py
##########
@@ -0,0 +1,136 @@
+# 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 sys
+import unittest
+
+import yaml
+from kubernetes.client import models as k8s
+
+from airflow.utils.helm_template_generator import render_chart, render_k8s_object
+
+OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 22
+
+
+class TestGitSyncScheduler(unittest.TestCase):
+    def setUp(self) -> None:
+        with open(sys.path[0] + "/tests/helm/test_templates/"

Review comment:
       Do you mean separate files for different test scenarios or separate files for different yamls?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511162977



##########
File path: chart/tests/helm_template_generator.py
##########
@@ -0,0 +1,61 @@
+# 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 subprocess
+import sys
+from tempfile import NamedTemporaryFile
+
+import yaml
+from kubernetes.client.api_client import ApiClient
+
+api_client = ApiClient()
+
+
+def render_chart(name, values, show_only=None):
+    """
+    Function that renders a helm chart into dictionaries. For helm chart testing only
+
+    :param name:
+    :param values:
+    :param show_only:
+    :return:

Review comment:
       ```suggestion
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512162293



##########
File path: TESTING.rst
##########
@@ -196,6 +196,44 @@ Run all Quarantined tests:
 
      ./breeze --test-type Quarantined tests --db-reset
 
+Helm Unit Tests

Review comment:
       @mik-laj added




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#issuecomment-713840380


   [The Workflow run](https://github.com/apache/airflow/actions/runs/320623917) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] kaxil commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510365286



##########
File path: chart_tests/helm_template_generator.py
##########
@@ -0,0 +1,62 @@
+# 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 subprocess
+import sys
+from tempfile import NamedTemporaryFile
+from typing import Dict, Optional
+
+import yaml
+from kubernetes.client.api_client import ApiClient
+
+api_client = ApiClient()
+
+
+def render_chart(name, values: Optional[Dict] = None, show_only=None):
+    """
+    Function that renders a helm chart into dictionaries. For helm chart testing only
+
+    :param name:
+    :param values:
+    :param show_only:
+    :return:

Review comment:
       Either we should all the descriptions or remove these 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.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512157333



##########
File path: TESTING.rst
##########
@@ -196,6 +196,44 @@ Run all Quarantined tests:
 
      ./breeze --test-type Quarantined tests --db-reset
 
+Helm Unit Tests

Review comment:
       Oof forgot htat line :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512169424



##########
File path: .pre-commit-config.yaml
##########
@@ -277,7 +277,7 @@ repos:
         entry: "^\\s*from\\s+\\."
         pass_filenames: true
         files: \.py$
-        exclude: ^tests/
+        exclude: ^tests/|^chart/tests/

Review comment:
       <img width="678" alt="Screen Shot 2020-10-19 at 5 29 24 PM" src="https://user-images.githubusercontent.com/2644098/97211212-ef693d80-177b-11eb-8b74-0cab89306949.png">
   That seems to work on the tests but any idea how to get it to register in pycharm? want to make sure that's not too difficult.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512170502



##########
File path: chart/requirements.lock
##########
@@ -1,6 +1,6 @@
 dependencies:
 - name: postgresql
-  repository: https://kubernetes-charts.storage.googleapis.com
+  repository: https://kubernetes-charts.storage.googleapis.com/
   version: 6.3.12
-digest: sha256:58d88cf56e78b2380091e9e16cc6ccf58b88b3abe4a1886dd47cd9faef5309af
-generated: "2020-06-21T19:11:53.498134738+02:00"
+digest: sha256:e8d53453861c590e6ae176331634c9268a11cf894be17ed580fa2b347101be97

Review comment:
       I think this is because we recently upgraded postgres versions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511641707



##########
File path: .pre-commit-config.yaml
##########
@@ -201,7 +201,7 @@ repos:
         args:
           - --convention=pep257
           - --add-ignore=D100,D102,D104,D105,D107,D205,D400,D401
-        exclude: ^tests/.*\.py$|^scripts/.*\.py$|^dev|^provider_packages|^kubernetes_tests|.*example_dags/.*
+        exclude: ^tests/.*\.py$|^scripts/.*\.py$|^dev|^pro.*ages|^.*_tests|.*example_dags/.*|^chart/.*\.py

Review comment:
       ```suggestion
           exclude: ^tests/.*\.py$|^scripts/.*\.py$|^dev|^provider_packages|^kubernetes_tests|.*example_dags/.*|^chart/.*\.py
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511641661



##########
File path: .pre-commit-config.yaml
##########
@@ -153,7 +153,7 @@ repos:
     rev: 20.8b1
     hooks:
       - id: black
-        files: api_connexion/.*\.py|.*providers.*\.py
+        files: api_connexion/.*\.py|.*providers.*\.py|../chart/tests.*\.py

Review comment:
       ```suggestion
           files: api_connexion/.*\.py|.*providers.*\.py|^chart/tests/.*\.py
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511642166



##########
File path: .pre-commit-config.yaml
##########
@@ -451,6 +451,12 @@ repos:
         language: system
         entry: "./scripts/ci/pre_commit/pre_commit_mypy.sh"
         files: \.py$
+        exclude: ^dev|^provider_packages|^chart
+      - id: mypy-helm
+        name: Run mypy for helm chart tests

Review comment:
       ```suggestion
         - id: mypy
           name: Run mypy for helm chart tests
   ```
   pre-commit is smart enough to allow duplicate IDs with different file filter configurations. See:  https://github.com/apache/airflow/blob/872b1566a11cb73297e657ff325161721b296574/.pre-commit-config.yaml#L29-L139
   On the other hand, it makes it easier to run a single check, because you don't need to know all the IDs, but you just need to know 1 ID and the directory you want to check.
   ```
   git ls-files | grep "^chart" | xargs pre-commit run mypy --files
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#issuecomment-713882972


   [The Workflow run](https://github.com/apache/airflow/actions/runs/320741472) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511223301



##########
File path: scripts/ci/kubernetes/ci_run_helm_testing.sh
##########
@@ -18,7 +18,28 @@
 
 echo "Running helm tests"
 
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
+
+if [[ -f "${BUILD_CACHE_DIR}/.skip_tests" ]]; then
+    echo
+    echo "Skip tests"
+    echo
+    exit
+fi
+
+build_images::prepare_ci_build
+
+build_images::rebuild_ci_image_if_needed
+kind::make_sure_kubernetes_tools_are_installed
+
+
+python -m pip install pytest kubernetes \

Review comment:
       <img width="1440" alt="Screenshot 2020-10-24 at 03 04 02" src="https://user-images.githubusercontent.com/12058428/97064367-9c3d9200-15a5-11eb-82d1-797f447bf926.png">
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r508845350



##########
File path: airflow/utils/helm_template_generator.py
##########
@@ -0,0 +1,62 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       This file should not be in `/airflow/` as it is only used in 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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r508852272



##########
File path: tests/helm/test_git_sync_scheduler.py
##########
@@ -0,0 +1,136 @@
+# 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 sys
+import unittest
+
+import yaml
+from kubernetes.client import models as k8s
+
+from airflow.utils.helm_template_generator import render_chart, render_k8s_object
+
+OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 22
+
+
+class TestGitSyncScheduler(unittest.TestCase):
+    def setUp(self) -> None:
+        with open(sys.path[0] + "/tests/helm/test_templates/"
+                                "git_sync_scheduler/git-sync-scheduler.yaml") as file:
+            self.basic_object = yaml.load(file)
+        with open(sys.path[0] + "/tests/helm/test_templates/"
+                                "git_sync_scheduler/git-sync-container-spec.yaml") as file:
+            self.object_with_git_container_spec = yaml.load(file)
+        with open(sys.path[0] + "/tests/helm/test_templates/"
+                                "git_sync_scheduler/git-sync-ssh-params.yaml") as file:
+            self.object_with_ssh_params = yaml.load(file)
+        with open(sys.path[0] + "/tests/helm/test_templates/"
+                                "git_sync_scheduler/git-sync-username.yaml") as file:
+            self.object_with_username = yaml.load(file)
+        with open(sys.path[0] + "/tests/helm/test_templates/git_sync_scheduler/"
+                                "git-sync-existing-claim.yaml") as file:
+            self.object_with_existing_claim = yaml.load(file)
+
+    def test_basic(self):
+        res = render_chart("GIT-SYNC", self.basic_object,
+                           show_only=["templates/scheduler/scheduler-deployment.yaml"])
+        dep: k8s.V1Deployment = render_k8s_object(res[0], k8s.V1Deployment)
+        self.assertEqual("dags", dep.spec.template.spec.volumes[1].name)
+
+    def test_git_container_spec(self):
+        res = render_chart("GIT-SYNC", self.object_with_git_container_spec,
+                           show_only=["templates/scheduler/scheduler-deployment.yaml"])
+        dep: k8s.V1Deployment = render_k8s_object(res[0], k8s.V1Deployment)
+        git_sync_container = dep.spec.template.spec.containers[1]
+        self.assertEqual(git_sync_container.image, "test-registry/test-repo:test-tag")
+        self.assertEqual(git_sync_container.name, "git-sync-test")
+        self.assertEqual(git_sync_container.security_context.run_as_user, 65533)
+        env_dict = [e.to_dict() for e in git_sync_container.env]
+        self.assertEqual(env_dict, [
+            {'name': 'GIT_SYNC_REV', 'value': 'HEAD', 'value_from': None},
+            {'name': 'GIT_SYNC_BRANCH', 'value': 'test-branch', 'value_from': None},
+            {'name': 'GIT_SYNC_REPO', 'value': 'https://github.com/apache/airflow.git', 'value_from': None},
+            {'name': 'GIT_SYNC_DEPTH', 'value': '1', 'value_from': None},
+            {'name': 'GIT_SYNC_ROOT', 'value': '/git-root', 'value_from': None},
+            {'name': 'GIT_SYNC_DEST', 'value': 'test-dest', 'value_from': None},
+            {'name': 'GIT_SYNC_ADD_USER', 'value': 'true', 'value_from': None},
+            {'name': 'GIT_SYNC_WAIT', 'value': '66', 'value_from': None},
+            {'name': 'GIT_SYNC_MAX_SYNC_FAILURES', 'value': '70', 'value_from': None}])
+
+        self.assertEqual(git_sync_container.volume_mounts,
+                         [k8s.V1VolumeMount(name="dags", mount_path="/git-root")])
+
+    def test_ssh_params_added(self):
+        res = render_chart("GIT-SYNC", self.object_with_ssh_params,
+                           show_only=["templates/scheduler/scheduler-deployment.yaml"])
+        dep: k8s.V1Deployment = render_k8s_object(res[0], k8s.V1Deployment)
+        git_sync_container = dep.spec.template.spec.containers[1]
+        env_dict = [e.to_dict() for e in git_sync_container.env]
+        self.assertEqual(env_dict, [
+            {'name': 'GIT_SSH_KEY_FILE', 'value': '/etc/git-secret/ssh', 'value_from': None},
+            {'name': 'GIT_SYNC_SSH', 'value': 'true', 'value_from': None},
+            {'name': 'GIT_KNOWN_HOSTS', 'value': 'false', 'value_from': None},
+            {'name': 'GIT_SYNC_REV', 'value': 'HEAD', 'value_from': None},
+            {'name': 'GIT_SYNC_BRANCH', 'value': 'test-branch', 'value_from': None},
+            {'name': 'GIT_SYNC_REPO', 'value': 'https://github.com/apache/airflow.git',
+             'value_from': None},
+            {'name': 'GIT_SYNC_DEPTH', 'value': '1', 'value_from': None},
+            {'name': 'GIT_SYNC_ROOT', 'value': '/git', 'value_from': None},
+            {'name': 'GIT_SYNC_DEST', 'value': 'repo', 'value_from': None},
+            {'name': 'GIT_SYNC_ADD_USER', 'value': 'true', 'value_from': None},
+            {'name': 'GIT_SYNC_WAIT', 'value': '60', 'value_from': None},
+            {'name': 'GIT_SYNC_MAX_SYNC_FAILURES', 'value': '0', 'value_from': None}])
+
+    def test_adds_git_username(self):
+        res = render_chart("GIT-SYNC", self.object_with_username,
+                           show_only=["templates/scheduler/scheduler-deployment.yaml"])
+        dep: k8s.V1Deployment = render_k8s_object(res[0], k8s.V1Deployment)
+        git_sync_container = dep.spec.template.spec.containers[1]
+        env_dict = [e.to_dict() for e in git_sync_container.env]
+        self.assertEqual(env_dict, [
+            {'name': 'GIT_SYNC_USERNAME', 'value': None, 'value_from':
+                {'config_map_key_ref': None,
+                 'field_ref': None,
+                 'resource_field_ref': None,
+                 'secret_key_ref': {'key': 'GIT_SYNC_USERNAME',
+                                    'name': 'user-pass-secret', 'optional': None}}},
+            {'name': 'GIT_SYNC_PASSWORD', 'value': None,
+             'value_from':
+                 {'config_map_key_ref': None,

Review comment:
       Can you enable black for this directory? I think this could significantly improve the readability of the tests. We try to enable black for the new code, eg airflow.providers, airflow.api_connexion.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511163980



##########
File path: scripts/ci/kubernetes/ci_run_helm_testing.sh
##########
@@ -20,6 +20,8 @@ echo "Running helm tests"
 
 chart_directory="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )/../../../chart/"
 
+pytest chart/tests

Review comment:
       I am not sure this is the correct way to run these tests.  I think we should use Docker to ensure a uniform development environment.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511500739



##########
File path: scripts/ci/kubernetes/ci_run_helm_testing.sh
##########
@@ -18,7 +18,28 @@
 
 echo "Running helm tests"
 
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
+
+if [[ -f "${BUILD_CACHE_DIR}/.skip_tests" ]]; then
+    echo
+    echo "Skip tests"
+    echo
+    exit
+fi
+
+build_images::prepare_ci_build
+
+build_images::rebuild_ci_image_if_needed
+kind::make_sure_kubernetes_tools_are_installed
+
+
+python -m pip install pytest kubernetes \

Review comment:
       @mik-laj great suggestion. I have integrated it into the breeze environment so now users can run that command (and the CI will use a similar path to all other python docker-based testing).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511641707



##########
File path: .pre-commit-config.yaml
##########
@@ -201,7 +201,7 @@ repos:
         args:
           - --convention=pep257
           - --add-ignore=D100,D102,D104,D105,D107,D205,D400,D401
-        exclude: ^tests/.*\.py$|^scripts/.*\.py$|^dev|^provider_packages|^kubernetes_tests|.*example_dags/.*
+        exclude: ^tests/.*\.py$|^scripts/.*\.py$|^dev|^pro.*ages|^.*_tests|.*example_dags/.*|^chart/.*\.py

Review comment:
       ```suggestion
           exclude: |
             (?x)
             ^tests/.*\.py$|
             ^scripts/.*\.py$|
             ^dev|
             ^provider_packages|
             ^kubernetes_tests|
             .*example_dags/.*|
             ^chart/.*\.py$
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#issuecomment-715556344


   [The Workflow run](https://github.com/apache/airflow/actions/runs/324831734) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511643293



##########
File path: scripts/in_container/entrypoint_ci.sh
##########
@@ -170,7 +170,33 @@ set -u
 export RESULT_LOG_FILE="/files/test_result.xml"
 
 if [[ "${GITHUB_ACTIONS}" == "true" ]]; then
-    EXTRA_PYTEST_ARGS=(
+    if [[ "${TEST_TYPE}" == "Helm" ]]; then
+        EXTRA_PYTEST_ARGS=(

Review comment:
       Can you extract common args? I know that Bash and boards aren't your best friends, but I have an example for you that shows how it can be done in Bash. 
   https://github.com/apache/airflow/blob/872b1566a11cb73297e657ff325161721b296574/scripts/in_container/run_cli_tool.sh#L72-L84
   https://github.com/apache/airflow/blob/872b1566a11cb73297e657ff325161721b296574/scripts/in_container/run_cli_tool.sh#L88-L91
   https://github.com/apache/airflow/blob/872b1566a11cb73297e657ff325161721b296574/scripts/in_container/run_cli_tool.sh#L102
   https://github.com/apache/airflow/blob/872b1566a11cb73297e657ff325161721b296574/scripts/in_container/run_cli_tool.sh#L138
   https://github.com/apache/airflow/blob/872b1566a11cb73297e657ff325161721b296574/scripts/in_container/run_cli_tool.sh#L141-L143
   https://github.com/apache/airflow/blob/872b1566a11cb73297e657ff325161721b296574/scripts/in_container/run_cli_tool.sh#L146




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511642303



##########
File path: chart/requirements.lock
##########
@@ -1,6 +1,6 @@
 dependencies:
 - name: postgresql
-  repository: https://kubernetes-charts.storage.googleapis.com
+  repository: https://kubernetes-charts.storage.googleapis.com/
   version: 6.3.12
-digest: sha256:58d88cf56e78b2380091e9e16cc6ccf58b88b3abe4a1886dd47cd9faef5309af
-generated: "2020-06-21T19:11:53.498134738+02:00"
+digest: sha256:e8d53453861c590e6ae176331634c9268a11cf894be17ed580fa2b347101be97

Review comment:
       Is is expected?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511528599



##########
File path: .github/workflows/ci.yml
##########
@@ -424,6 +428,52 @@ jobs:
           name: airflow-provider-readmes
           path: "./files/airflow-readme-*"
 
+  helm-python-tests:
+    timeout-minutes: 5
+    name: "Python unit tests for helm chart"
+    runs-on: ubuntu-latest
+    needs: [build-info, ci-images]
+    env:
+      MOUNT_LOCAL_SOURCES: "true"
+      RUN_TESTS: true
+      TEST_TYPE: "Helm"
+      BACKEND: "None"

Review comment:
       You can also use SQLite. That shouldn't cause any problems in our case. This is also the default database used in Breeze.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#issuecomment-715612401


   [The Workflow run](https://github.com/apache/airflow/actions/runs/325032049) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks$,^Build docs$,^Spell check docs$,^Backport packages$,^Checks: Helm tests$,^Test OpenAPI*.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511180388



##########
File path: scripts/ci/kubernetes/ci_run_helm_testing.sh
##########
@@ -18,7 +18,28 @@
 
 echo "Running helm tests"
 
+. "$( dirname "${BASH_SOURCE[0]}" )/../libraries/_script_init.sh"
+
+if [[ -f "${BUILD_CACHE_DIR}/.skip_tests" ]]; then
+    echo
+    echo "Skip tests"
+    echo
+    exit
+fi
+
+build_images::prepare_ci_build
+
+build_images::rebuild_ci_image_if_needed
+kind::make_sure_kubernetes_tools_are_installed
+
+
+python -m pip install pytest kubernetes \

Review comment:
       Can you look here? https://github.com/apache/airflow/blob/master/scripts/in_container/entrypoint_ci.sh#L232-L277
   It seems to me that you should generalize the test run environment.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] ashb commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
ashb commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r509621798



##########
File path: .dockerignore
##########
@@ -45,6 +45,7 @@
 # Add tests and kubernetes_tests to context.
 !tests
 !kubernetes_tests
+!chart_tests

Review comment:
       Anything stopping us using chart/tests like we had before, just with python files?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510571196



##########
File path: .dockerignore
##########
@@ -45,6 +45,7 @@
 # Add tests and kubernetes_tests to context.
 !tests
 !kubernetes_tests
+!chart_tests

Review comment:
       @ashb @kaxil moving the files to `/chart` caused mypy to fail
   
   ```
   Run mypy..........................................................................................................................Failed
   - hook id: mypy
   - exit code: 2
   
   tests/__init__.py: error: Duplicate module named 'tests' (also at
   'chart/tests/__init__.py')
   tests/__init__.py: error: Are you missing an __init__.py?
   Found 2 errors in 1 file (checked 2459 source files)
   ```
   
   So I guess our options are either create a `chart_tests` directory, change the name of the testing directory within the chart, or add an __init__.py  to the chart directory (unless y'all have another route we can take). Between those three choices I'd rather the `chart_tests` because I don't like the idea of putting an __init__.py into a helm chart for testing purposes. WDYT?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511664620



##########
File path: .pre-commit-config.yaml
##########
@@ -277,7 +277,7 @@ repos:
         entry: "^\\s*from\\s+\\."
         pass_filenames: true
         files: \.py$
-        exclude: ^tests/
+        exclude: ^tests/|^chart/tests/

Review comment:
       It is needed? We can solve pylint problems with a trick similar to mypy.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511115327



##########
File path: chart_tests/test_templates/git_sync_scheduler/git-sync-existing-claim.yaml
##########
@@ -0,0 +1,21 @@
+# 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.
+---
+dags:
+ persistence:
+  enabled: true
+  existingClaim: test-claim

Review comment:
       It looks good to me now.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511664620



##########
File path: .pre-commit-config.yaml
##########
@@ -277,7 +277,7 @@ repos:
         entry: "^\\s*from\\s+\\."
         pass_filenames: true
         files: \.py$
-        exclude: ^tests/
+        exclude: ^tests/|^chart/tests/

Review comment:
       It is needed?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510362017



##########
File path: chart_tests/test_templates/git_sync_scheduler/git-sync-existing-claim.yaml
##########
@@ -0,0 +1,21 @@
+# 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.
+---
+dags:
+ persistence:
+  enabled: true
+  existingClaim: test-claim

Review comment:
       @ashb I think that keeping these as yaml files will make sense as this scales (imagine if we need to test 20 seperate configurations for one scenario, we'd have to maintain 20 dictionaries within a single python file that would be really verbose). It also could be used in documentation later.
   
   @kaxil WDYT?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] dimberman commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
dimberman commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r512162781



##########
File path: .pre-commit-config.yaml
##########
@@ -277,7 +277,7 @@ repos:
         entry: "^\\s*from\\s+\\."
         pass_filenames: true
         files: \.py$
-        exclude: ^tests/
+        exclude: ^tests/|^chart/tests/

Review comment:
       I think this one is needed as if it's not excluded it will try to parse those files




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r511609078



##########
File path: .github/workflows/ci.yml
##########
@@ -424,6 +424,52 @@ jobs:
           name: airflow-provider-readmes
           path: "./files/airflow-readme-*"
 
+  helm-python-tests:
+    timeout-minutes: 5
+    name: "Python unit tests for helm chart"
+    runs-on: ubuntu-latest
+    needs: [build-info, ci-images]
+    env:
+      MOUNT_LOCAL_SOURCES: "true"
+      RUN_TESTS: true
+      TEST_TYPES: "Helm"
+      BACKEND: "sqlite"
+      PYTHON_MAJOR_MINOR_VERSION: ${{needs.build-info.outputs.defaultPythonVersion}}
+    if: >
+      needs.build-info.outputs.run-tests == 'true' &&

Review comment:
       ```suggestion
         needs.build-info.outputs.needs-helm-tests == 'true' &&
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] kaxil commented on a change in pull request #11693: Add Python Helm testing framework

Posted by GitBox <gi...@apache.org>.
kaxil commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r510366148



##########
File path: chart_tests/basic_helm_chart_test.py
##########
@@ -0,0 +1,66 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import unittest
+
+from .helm_template_generator import render_chart
+
+OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 22
+
+
+class TestBaseChartTest(unittest.TestCase):
+    def test_basic_deployments(self):

Review comment:
       Adding a short description on what this test is actually doing should help future readers to add more tests 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.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r509579685



##########
File path: tests/helm/test_git_sync_scheduler.py
##########
@@ -0,0 +1,136 @@
+# 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 sys
+import unittest
+
+import yaml
+from kubernetes.client import models as k8s
+
+from airflow.utils.helm_template_generator import render_chart, render_k8s_object
+
+OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 22
+
+
+class TestGitSyncScheduler(unittest.TestCase):
+    def setUp(self) -> None:
+        with open(sys.path[0] + "/tests/helm/test_templates/"

Review comment:
       Separate files at all. Parameters can be passed as a dictionary in Python files.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #11693: add python helm testing framework

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #11693:
URL: https://github.com/apache/airflow/pull/11693#discussion_r508850541



##########
File path: tests/helm/__init__.py
##########
@@ -0,0 +1,16 @@
+# Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       I think these files should be move to [`charts/tests/`](https://github.com/apache/airflow/tree/master/chart/tests) directory. It is very confusing that the tests are in this directory as this directory mainly contains tests for Airflow. The new contributor will have trouble finding these 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.

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