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/11/16 17:37:23 UTC

[GitHub] [airflow] dimberman opened a new pull request #12384: Make K8sPodOperator backwards compatible

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


   <!--
   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] kaxil commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: kubernetes_tests/test_kubernetes_pod_operator_backcompat.py
##########
@@ -0,0 +1,948 @@
+# pylint: disable=unused-argument
+# 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.
+

Review comment:
       Ok looks like you are importinv VolumeMounts, Volume etc from the old paths to make sure it works

##########
File path: kubernetes_tests/test_kubernetes_pod_operator_backcompat.py
##########
@@ -0,0 +1,948 @@
+# pylint: disable=unused-argument
+# 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.
+

Review comment:
       Ok looks like you are importing VolumeMounts, Volume etc from the old paths to make sure it works




----------------------------------------------------------------
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 pull request #12384: Make K8sPodOperator backwards compatible

Posted by GitBox <gi...@apache.org>.
ashb commented on pull request #12384:
URL: https://github.com/apache/airflow/pull/12384#issuecomment-728337484


   Just discussed with Daniel: We're going to add (back) a `airflow/kubernetes/volume_mount.py` to "core" airflow that contains something like:
   
   ```python
   from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
   
   warnings.warn(
       "This module is deprecated. Please use `kubernetes.client.models.V1Volume`.",
       DeprecationWarning,
       stacklevel=2,
   )
   ```
   
   We may need to wrap that import in a `warnings.catch_warnings, otherwise the warning will be reported from the wrong context (we ideally want the file:lineno of the warning to be the import line in the user's DAG):
   
   ```python
   import warnings
   
   with warnings.catch_warnings():
       from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
   
   warnings.warn(
       "This module is deprecated. Please use `kubernetes.client.models.V1Volume`.",
       DeprecationWarning,
       stacklevel=2,
   )
   ```
   
   This means the old imports continue to work, but the bulk of the compat code lives in the provider.


----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")
+

Review comment:
       @dimberman: Not broken, but can be improved :) 
   
   This commit: https://github.com/apache/airflow/commit/8d0be15ae4e4c1cd51d61ccd71fe0a8bef7f0ffe removes the backcompat/pod.py and other files from backports as they are not needed there. Except that 
   
   `from airflow.kubernetes.pod_runtime_info_env import PodRuntimeInfoEnv` in `kubernetes_pod.py` still points to backcompat because this file is excluded from the rename above.
   
   The "perfect" fix is to:
   1) Also replace the import in `./kubernetes_pod.py`. Can be likely done with this without affecting the other imports in ./kubernetes.pod:
   
   ```
           self.qry.select_package("airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info")\
                 .rename("airflow.kubernetes.pod_runtime_info")
   ```
   
   2) remove the backcompat files as in https://github.com/apache/airflow/commit/8d0be15ae4e4c1cd51d61ccd71fe0a8bef7f0ffe
   
   
   




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/368524888) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider 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] dimberman merged pull request #12384: Make K8sPodOperator backwards compatible

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


   


----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: kubernetes_tests/test_kubernetes_pod_operator_backcompat.py
##########
@@ -0,0 +1,948 @@
+# pylint: disable=unused-argument
+# 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.
+

Review comment:
       How is this file testing anything differently than `kubernetes_tests/test_kubernetes_pod_operator.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] ashb commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       ~What is this `rename()` achieving then? What dist does/should it appear in?~




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/volume_mount.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.
+"""Classes for interacting with Kubernetes API"""
+
+import warnings
+
+from kubernetes.client import models as k8s
+
+warnings.warn(
+    "This module is deprecated. Please use `kubernetes.client.models.V1Volume`.",

Review comment:
       @ashb  we can actually just change the import path using bowler for backport_providers. I was asked by @kaxil and @potiuk to set this up exclusively in the provider package. This will allow us to upgrade these as needed without releasing new airflow 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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       From this respect good @ashb ! the backport provider package only contains airflow.providers files. We do not add airflow/kubernetes.
   
   Just double-checked and installed both airflow 1.10.12 and backport cncf,kubernetes from pcakges and seem to be all ok.
   
   This is what you get in 'backwards_compat_converters.py in the backport package:
   
   ```
   from typing import List
   
   from kubernetes.client import models as k8s
   
   from airflow.exceptions import AirflowException
   from airflow.kubernetes.pod import Port, Resources
   from airflow.kubernetes.pod_runtime_info_env import PodRuntimeInfoEnv
   from airflow.kubernetes.volume import Volume
   from airflow.kubernetes.volume_mount import VolumeMount
   
   
   def _convert_kube_model_object(obj, old_class, new_class):
       convert_op = getattr(obj, "to_k8s_client_obj", None)
       if callable(convert_op):
           return obj.to_k8s_client_obj()
   
   
   ```




----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       But we should not I believe. This import should be changed in the backport provider as well.




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

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



[GitHub] [airflow] ashb commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/volume_mount.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.
+"""Classes for interacting with Kubernetes API"""
+
+import warnings
+
+from kubernetes.client import models as k8s
+
+warnings.warn(
+    "This module is deprecated. Please use `kubernetes.client.models.V1Volume`.",

Review comment:
       See https://github.com/apache/airflow/pull/12384#discussion_r524534519




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume
+    @param volume:
+    @return: k8s.V1Volume
+    """
+    return _convert_kube_model_object(volume, Volume, k8s.V1Volume)
+
+
+def convert_volume_mount(volume_mount) -> k8s.V1VolumeMount:
+    """
+    Converts an airflow VolumeMount object into a k8s.V1VolumeMount
+    @param volume_mount:
+    @return: k8s.V1VolumeMount

Review comment:
       No, black doesn't look at doc comments.
   
   pydocstyle might.




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       (I think that's cos Dan didn't tick the "allow commits from maintainers?" Or maybe that only works for PRs from personal repos -- I had the same trying to push to a PolideaInternal PR you opened)




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py
##########
@@ -208,11 +221,15 @@ def __init__(  # pylint: disable=too-many-arguments,too-many-locals
         self.arguments = arguments or []
         self.labels = labels or {}
         self.startup_timeout_seconds = startup_timeout_seconds
-        self.env_vars = env_vars or []
+        self.env_vars = convert_env_vars(env_vars) if env_vars else []
+        if pod_runtime_info_envs:
+            self.env_vars.extend([convert_pod_runtime_info_env(p) for p in pod_runtime_info_envs])
         self.env_from = env_from or []
-        self.ports = ports or []
-        self.volume_mounts = volume_mounts or []
-        self.volumes = volumes or []
+        if configmaps:
+            self.env_from.extend([convert_configmap(c) for c in configmaps])
+        self.ports = [convert_port(p) for p in ports] if ports else []
+        self.volume_mounts = [convert_volume_mount(v) for v in volume_mounts] if volume_mounts else []
+        self.volumes = [convert_volume(volume) for volume in volumes] if volumes else []

Review comment:
       These won't issue deprecation warnings, but that's okay because the imports will, right?




----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       > This brings up an interesting point. What is the behaviour when upgrading from 1.10.x with backport provider to 2.0.0 with normal provider. Yes pip will complain about version conflicts, but that still isn't an error (especially given many people will be on an older pip still)
   
   This will work fine. Backport packages need airflow ~1.10. PIP will not allow to upgrade airflow without removing backports.




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: kubernetes_tests/test_kubernetes_pod_operator_backcompat.py
##########
@@ -0,0 +1,948 @@
+# pylint: disable=unused-argument
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import json
+import logging
+import sys
+import textwrap
+import unittest
+from unittest import mock
+from unittest.mock import patch
+
+import kubernetes.client.models as k8s
+import pendulum
+from kubernetes.client.api_client import ApiClient
+from kubernetes.client.rest import ApiException
+
+from airflow.exceptions import AirflowException
+from airflow.kubernetes import kube_client
+from airflow.kubernetes.pod import Port
+from airflow.kubernetes.pod_generator import PodDefaults
+from airflow.kubernetes.pod_launcher import PodLauncher
+from airflow.kubernetes.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.kubernetes.secret import Secret
+from airflow.kubernetes.volume import Volume
+from airflow.kubernetes.volume_mount import VolumeMount
+from airflow.models import DAG, TaskInstance
+from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator
+from airflow.utils import timezone
+from airflow.version import version as airflow_version
+
+
+# noinspection DuplicatedCode
+def create_context(task):
+    dag = DAG(dag_id="dag")
+    tzinfo = pendulum.timezone("Europe/Amsterdam")
+    execution_date = timezone.datetime(2016, 1, 1, 1, 0, 0, tzinfo=tzinfo)
+    task_instance = TaskInstance(task=task, execution_date=execution_date)
+    return {
+        "dag": dag,
+        "ts": execution_date.isoformat(),
+        "task": task,
+        "ti": task_instance,
+    }
+
+
+# noinspection DuplicatedCode,PyUnusedLocal
+class TestKubernetesPodOperatorSystem(unittest.TestCase):
+    def get_current_task_name(self):
+        # reverse test name to make pod name unique (it has limited length)
+        return "_" + unittest.TestCase.id(self).replace(".", "_")[::-1]
+
+    def setUp(self):
+        self.maxDiff = None  # pylint: disable=invalid-name
+        self.api_client = ApiClient()
+        self.expected_pod = {
+            'apiVersion': 'v1',
+            'kind': 'Pod',
+            'metadata': {
+                'namespace': 'default',
+                'name': mock.ANY,
+                'annotations': {},
+                'labels': {
+                    'foo': 'bar',
+                    'kubernetes_pod_operator': 'True',
+                    'airflow_version': airflow_version.replace('+', '-'),
+                    'execution_date': '2016-01-01T0100000100-a2f50a31f',
+                    'dag_id': 'dag',
+                    'task_id': 'task',
+                    'try_number': '1',
+                },
+            },
+            'spec': {
+                'affinity': {},
+                'containers': [
+                    {
+                        'image': 'ubuntu:16.04',
+                        'args': ["echo 10"],
+                        'command': ["bash", "-cx"],
+                        'env': [],
+                        'envFrom': [],
+                        'resources': {},
+                        'name': 'base',
+                        'ports': [],
+                        'volumeMounts': [],
+                    }
+                ],
+                'hostNetwork': False,
+                'imagePullSecrets': [],
+                'initContainers': [],
+                'nodeSelector': {},
+                'restartPolicy': 'Never',
+                'securityContext': {},
+                'serviceAccountName': 'default',
+                'tolerations': [],
+                'volumes': [],
+            },
+        }
+
+    def tearDown(self):
+        client = kube_client.get_kube_client(in_cluster=False)
+        client.delete_collection_namespaced_pod(namespace="default")
+
+    def create_context(self, task):
+        dag = DAG(dag_id="dag")
+        tzinfo = pendulum.timezone("Europe/Amsterdam")
+        execution_date = timezone.datetime(2016, 1, 1, 1, 0, 0, tzinfo=tzinfo)
+        task_instance = TaskInstance(task=task, execution_date=execution_date)
+        return {
+            "dag": dag,
+            "ts": execution_date.isoformat(),
+            "task": task,
+            "ti": task_instance,
+        }
+
+    def test_do_xcom_push_defaults_false(self):
+        k = KubernetesPodOperator(
+            namespace='default',
+            image="ubuntu:16.04",
+            cmds=["bash", "-cx"],
+            arguments=["echo 10"],
+            labels={"foo": "bar"},
+            name="test",
+            task_id="task",
+            in_cluster=False,
+            do_xcom_push=False,
+        )
+        self.assertFalse(k.do_xcom_push)
+
+    @mock.patch("airflow.kubernetes.pod_launcher.PodLauncher.start_pod")
+    @mock.patch("airflow.kubernetes.pod_launcher.PodLauncher.monitor_pod")
+    @mock.patch("airflow.kubernetes.kube_client.get_kube_client")
+    def test_config_path(self, client_mock, monitor_mock, start_mock):  # pylint: disable=unused-argument

Review comment:
       This test does not test any old behaviour, does it?
   
   i.e. it is just dupe of same test (`test_config_path`) in tests/providers/cncf/kubernetes/operators/test_kubernetes_pod.py
   
   Can we remove all such tests that are not strictly checking the old behaviour too please?




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume

Review comment:
       ```suggestion
       """
       Converts an airflow Volume object into a k8s.V1Volume
       
   ```
   
   Need a blank line between description and params




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume
+    @param volume:
+    @return: k8s.V1Volume
+    """
+    return _convert_kube_model_object(volume, Volume, k8s.V1Volume)
+
+
+def convert_volume_mount(volume_mount) -> k8s.V1VolumeMount:
+    """
+    Converts an airflow VolumeMount object into a k8s.V1VolumeMount
+    @param volume_mount:
+    @return: k8s.V1VolumeMount

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] potiuk commented on pull request #12384: Make K8sPodOperator backwards compatible

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #12384:
URL: https://github.com/apache/airflow/pull/12384#issuecomment-728782677


   Love it! 


----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/volume_mount.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.
+"""Classes for interacting with Kubernetes API"""
+
+import warnings
+
+from kubernetes.client import models as k8s
+
+warnings.warn(
+    "This module is deprecated. Please use `kubernetes.client.models.V1Volume`.",

Review comment:
       This needs to live as `from airflow.kubernetes.volume_mount import VolumeMount` to be a back-compat layer.
   
   Same for all the others.




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: kubernetes_tests/test_kubernetes_pod_operator_backcompat.py
##########
@@ -0,0 +1,948 @@
+# pylint: disable=unused-argument
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import json
+import logging
+import sys
+import textwrap
+import unittest
+from unittest import mock
+from unittest.mock import patch
+
+import kubernetes.client.models as k8s
+import pendulum
+from kubernetes.client.api_client import ApiClient
+from kubernetes.client.rest import ApiException
+
+from airflow.exceptions import AirflowException
+from airflow.kubernetes import kube_client
+from airflow.kubernetes.pod import Port
+from airflow.kubernetes.pod_generator import PodDefaults
+from airflow.kubernetes.pod_launcher import PodLauncher
+from airflow.kubernetes.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.kubernetes.secret import Secret
+from airflow.kubernetes.volume import Volume
+from airflow.kubernetes.volume_mount import VolumeMount
+from airflow.models import DAG, TaskInstance
+from airflow.providers.cncf.kubernetes.operators.kubernetes_pod import KubernetesPodOperator
+from airflow.utils import timezone
+from airflow.version import version as airflow_version
+
+
+# noinspection DuplicatedCode
+def create_context(task):
+    dag = DAG(dag_id="dag")
+    tzinfo = pendulum.timezone("Europe/Amsterdam")
+    execution_date = timezone.datetime(2016, 1, 1, 1, 0, 0, tzinfo=tzinfo)
+    task_instance = TaskInstance(task=task, execution_date=execution_date)
+    return {
+        "dag": dag,
+        "ts": execution_date.isoformat(),
+        "task": task,
+        "ti": task_instance,
+    }
+
+
+# noinspection DuplicatedCode,PyUnusedLocal
+class TestKubernetesPodOperatorSystem(unittest.TestCase):
+    def get_current_task_name(self):
+        # reverse test name to make pod name unique (it has limited length)
+        return "_" + unittest.TestCase.id(self).replace(".", "_")[::-1]
+
+    def setUp(self):
+        self.maxDiff = None  # pylint: disable=invalid-name
+        self.api_client = ApiClient()
+        self.expected_pod = {
+            'apiVersion': 'v1',
+            'kind': 'Pod',
+            'metadata': {
+                'namespace': 'default',
+                'name': mock.ANY,
+                'annotations': {},
+                'labels': {
+                    'foo': 'bar',
+                    'kubernetes_pod_operator': 'True',
+                    'airflow_version': airflow_version.replace('+', '-'),
+                    'execution_date': '2016-01-01T0100000100-a2f50a31f',
+                    'dag_id': 'dag',
+                    'task_id': 'task',
+                    'try_number': '1',
+                },
+            },
+            'spec': {
+                'affinity': {},
+                'containers': [
+                    {
+                        'image': 'ubuntu:16.04',
+                        'args': ["echo 10"],
+                        'command': ["bash", "-cx"],
+                        'env': [],
+                        'envFrom': [],
+                        'resources': {},
+                        'name': 'base',
+                        'ports': [],
+                        'volumeMounts': [],
+                    }
+                ],
+                'hostNetwork': False,
+                'imagePullSecrets': [],
+                'initContainers': [],
+                'nodeSelector': {},
+                'restartPolicy': 'Never',
+                'securityContext': {},
+                'serviceAccountName': 'default',
+                'tolerations': [],
+                'volumes': [],
+            },
+        }
+
+    def tearDown(self):
+        client = kube_client.get_kube_client(in_cluster=False)
+        client.delete_collection_namespaced_pod(namespace="default")
+
+    def create_context(self, task):
+        dag = DAG(dag_id="dag")
+        tzinfo = pendulum.timezone("Europe/Amsterdam")
+        execution_date = timezone.datetime(2016, 1, 1, 1, 0, 0, tzinfo=tzinfo)
+        task_instance = TaskInstance(task=task, execution_date=execution_date)
+        return {
+            "dag": dag,
+            "ts": execution_date.isoformat(),
+            "task": task,
+            "ti": task_instance,
+        }
+
+    def test_do_xcom_push_defaults_false(self):
+        k = KubernetesPodOperator(
+            namespace='default',
+            image="ubuntu:16.04",
+            cmds=["bash", "-cx"],
+            arguments=["echo 10"],
+            labels={"foo": "bar"},
+            name="test",
+            task_id="task",
+            in_cluster=False,
+            do_xcom_push=False,
+        )
+        self.assertFalse(k.do_xcom_push)
+
+    @mock.patch("airflow.kubernetes.pod_launcher.PodLauncher.start_pod")
+    @mock.patch("airflow.kubernetes.pod_launcher.PodLauncher.monitor_pod")
+    @mock.patch("airflow.kubernetes.kube_client.get_kube_client")
+    def test_config_path(self, client_mock, monitor_mock, start_mock):  # pylint: disable=unused-argument
+        from airflow.utils.state import State

Review comment:
       Can we move this import at top level -- since looks like it is used in more than 1 test




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume
+    @param volume:
+    @return: k8s.V1Volume

Review comment:
       ```suggestion
       :param volume:
       :return: k8s.V1Volume
   ```
   
   Empty description for Volume




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")
+

Review comment:
       Does select_module just operate on that one module, not sub modules under it?




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       @potiuk Turns out we need those files:
   
   ```
     File "/usr/local/lib/python3.6/site-packages/airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py", line 39, in <module>
       from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
   ModuleNotFoundError: No module named 'airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env'
   ```




----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       One small thing that we could do - maybe also remove the backcompat classes. They seem to not be removed during the refactoring and they are not needed in the backport at all:
   
   ![image](https://user-images.githubusercontent.com/595491/99374141-91370280-28c2-11eb-8bc8-53b85bbeb601.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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       Yeah. They should be removed after rename. However Interesting thing is that the package installed correctly (prepare backport providers job in CI does it).




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/volume.py
##########
@@ -0,0 +1,64 @@
+# 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.
+"""This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.datasync`."""
+
+import warnings
+
+from kubernetes.client import models as k8s
+
+warnings.warn(
+    "This module is deprecated. Please use `kubernetes.client.models.V1Volume`.",
+    DeprecationWarning,
+    stacklevel=2,
+)
+
+
+class Volume:
+    """Backward compatible Volume"""
+
+    def __init__(self, name, configs):
+        """Adds Kubernetes Volume to pod. allows pod to access features like ConfigMaps
+        and Persistent Volumes
+        :param name: the name of the volume mount

Review comment:
       ```suggestion
   
           :param name: the name of the volume mount
   ```




----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       We should simply add a step in UPDATING to MD to make sure they remove backports simply (so that they are not tempted to force installation even with version conflict.




----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       For example backport cncf.kubernetes:
   
   ```
          install_requires=[
               'apache-airflow>=1.10.12, <2.0.0',
               'cryptography>=2.0.0',
               'kubernetes>=3.0.0, <12.0.0',
           ],
   ```




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/367154652) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider 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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/pod.py
##########
@@ -0,0 +1,110 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""Classes for interacting with Kubernetes API"""
+
+from kubernetes.client import models as k8s
+
+
+class Resources:
+    """backwards compat for Resources"""
+
+    __slots__ = (
+        'request_memory',
+        'request_cpu',
+        'limit_memory',
+        'limit_cpu',
+        'limit_gpu',
+        'request_ephemeral_storage',
+        'limit_ephemeral_storage',
+    )
+
+    """
+    :param request_memory: requested memory
+    :type request_memory: str
+    :param request_cpu: requested CPU number
+    :type request_cpu: float | str
+    :param request_ephemeral_storage: requested ephemeral storage
+    :type request_ephemeral_storage: str
+    :param limit_memory: limit for memory usage
+    :type limit_memory: str
+    :param limit_cpu: Limit for CPU used
+    :type limit_cpu: float | str
+    :param limit_gpu: Limits for GPU used
+    :type limit_gpu: int
+    :param limit_ephemeral_storage: Limit for ephemeral storage
+    :type limit_ephemeral_storage: float | str
+    """
+
+    def __init__(
+        self,
+        request_memory=None,
+        request_cpu=None,
+        request_ephemeral_storage=None,
+        limit_memory=None,
+        limit_cpu=None,
+        limit_gpu=None,
+        limit_ephemeral_storage=None,
+    ):
+        self.request_memory = request_memory
+        self.request_cpu = request_cpu
+        self.request_ephemeral_storage = request_ephemeral_storage
+        self.limit_memory = limit_memory
+        self.limit_cpu = limit_cpu
+        self.limit_gpu = limit_gpu
+        self.limit_ephemeral_storage = limit_ephemeral_storage
+
+    def to_k8s_client_obj(self):
+        """
+        Converts to k8s object.
+
+        @rtype: object
+        """
+        limits_raw = {
+            'cpu': self.limit_cpu,
+            'memory': self.limit_memory,
+            'nvidia.com/gpu': self.limit_gpu,
+            'ephemeral-storage': self.limit_ephemeral_storage,
+        }
+        requests_raw = {
+            'cpu': self.request_cpu,
+            'memory': self.request_memory,
+            'ephemeral-storage': self.request_ephemeral_storage,
+        }
+
+        limits = {k: v for k, v in limits_raw.items() if v}
+        requests = {k: v for k, v in requests_raw.items() if v}
+        resource_req = k8s.V1ResourceRequirements(limits=limits, requests=requests)
+        return resource_req
+
+
+class Port:
+    """POD port"""
+
+    __slots__ = ('name', 'container_port')
+
+    def __init__(self, name=None, container_port=None):
+        """Creates port"""
+        self.name = name
+        self.container_port = container_port
+
+    def to_k8s_client_obj(self):
+        """
+        Converts to k8s object.
+
+        @rtype: object

Review comment:
       ```suggestion
           :rtype: object
   ```




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/366619313) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider 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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       Yeah, pip doesn't really care if two distributions install the same file -- it'll just happily overwrite the existing file.
   
   Which is fine right until you come to uninstall things.
   
   This brings up an interesting point. What is the behaviour when upgrading from 1.10.x with backport provider to 2.0.0 with normal provider. Yes pip will complain about version conflicts, but that still isn't an error (especially given many people will be on an older pip still)




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/pod_runtime_info_env.py
##########
@@ -0,0 +1,43 @@
+# 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.
+"""Classes for interacting with Kubernetes API"""
+
+
+class PodRuntimeInfoEnv:
+    """Defines Pod runtime information as environment variable"""
+
+    def __init__(self, name, field_path):
+        """
+        Adds Kubernetes pod runtime information as environment variables such as namespace, pod IP, pod name.
+        Full list of options can be found in kubernetes documentation.
+
+        :param name: the name of the environment variable
+        :type: name: str
+        :param field_path: path to pod runtime info. Ex: metadata.namespace | status.podIP
+        :type: field_path: str
+        """
+        self.name = name
+        self.field_path = field_path
+
+    def to_k8s_client_obj(self):
+        """:return: kubernetes.client.models.V1EnvVar"""

Review comment:
       ```suggestion
       def to_k8s_client_obj(self):
           """Converts to k8s object.
           
           :return: kubernetes.client.models.V1EnvVar
           """
   ```




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -142,6 +142,7 @@ def rename_deprecated_modules(self) -> None:
             ("airflow.operators.bash", "airflow.operators.bash_operator"),
             ("airflow.operators.python", "airflow.operators.python_operator"),
             ("airflow.utils.session", "airflow.utils.db"),
+            ("airflow.providers.cncf.kubernetes.backcompat", "airflow.kubernetes"),

Review comment:
       Oh - I just saw this -- that'll help us for backport, but won't help for anyone trying to use it on 2.0.0 (i.e. upgrade then fix errors approach)




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume
+    @param volume:
+    @return: k8s.V1Volume
+    """
+    return _convert_kube_model_object(volume, Volume, k8s.V1Volume)
+
+
+def convert_volume_mount(volume_mount) -> k8s.V1VolumeMount:
+    """
+    Converts an airflow VolumeMount object into a k8s.V1VolumeMount
+    @param volume_mount:
+    @return: k8s.V1VolumeMount
+    """
+    return _convert_kube_model_object(volume_mount, VolumeMount, k8s.V1VolumeMount)
+
+
+def convert_resources(resources) -> k8s.V1ResourceRequirements:
+    """
+    Converts an airflow Resources object into a k8s.V1ResourceRequirements
+    @param resources:

Review comment:
       same here

##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume
+    @param volume:
+    @return: k8s.V1Volume
+    """
+    return _convert_kube_model_object(volume, Volume, k8s.V1Volume)
+
+
+def convert_volume_mount(volume_mount) -> k8s.V1VolumeMount:
+    """
+    Converts an airflow VolumeMount object into a k8s.V1VolumeMount
+    @param volume_mount:
+    @return: k8s.V1VolumeMount
+    """
+    return _convert_kube_model_object(volume_mount, VolumeMount, k8s.V1VolumeMount)
+
+
+def convert_resources(resources) -> k8s.V1ResourceRequirements:
+    """
+    Converts an airflow Resources object into a k8s.V1ResourceRequirements
+    @param resources:
+    @return: k8s.V1ResourceRequirements
+    """
+    if isinstance(resources, dict):
+        resources = Resources(**resources)
+    return _convert_kube_model_object(resources, Resources, k8s.V1ResourceRequirements)
+
+
+def convert_port(port) -> k8s.V1ContainerPort:
+    """
+    Converts an airflow Port object into a k8s.V1ContainerPort
+    @param port:

Review comment:
       same here

##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume
+    @param volume:
+    @return: k8s.V1Volume
+    """
+    return _convert_kube_model_object(volume, Volume, k8s.V1Volume)
+
+
+def convert_volume_mount(volume_mount) -> k8s.V1VolumeMount:
+    """
+    Converts an airflow VolumeMount object into a k8s.V1VolumeMount
+    @param volume_mount:
+    @return: k8s.V1VolumeMount
+    """
+    return _convert_kube_model_object(volume_mount, VolumeMount, k8s.V1VolumeMount)
+
+
+def convert_resources(resources) -> k8s.V1ResourceRequirements:
+    """
+    Converts an airflow Resources object into a k8s.V1ResourceRequirements
+    @param resources:
+    @return: k8s.V1ResourceRequirements
+    """
+    if isinstance(resources, dict):
+        resources = Resources(**resources)
+    return _convert_kube_model_object(resources, Resources, k8s.V1ResourceRequirements)
+
+
+def convert_port(port) -> k8s.V1ContainerPort:
+    """
+    Converts an airflow Port object into a k8s.V1ContainerPort
+    @param port:
+    @return: k8s.V1ContainerPort
+    """
+    return _convert_kube_model_object(port, Port, k8s.V1ContainerPort)
+
+
+def convert_env_vars(env_vars) -> List[k8s.V1EnvVar]:
+    """
+    Converts a dictionary into a list of env_vars
+    @param env_vars:

Review comment:
       same here

##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume
+    @param volume:
+    @return: k8s.V1Volume
+    """
+    return _convert_kube_model_object(volume, Volume, k8s.V1Volume)
+
+
+def convert_volume_mount(volume_mount) -> k8s.V1VolumeMount:
+    """
+    Converts an airflow VolumeMount object into a k8s.V1VolumeMount
+    @param volume_mount:
+    @return: k8s.V1VolumeMount
+    """
+    return _convert_kube_model_object(volume_mount, VolumeMount, k8s.V1VolumeMount)
+
+
+def convert_resources(resources) -> k8s.V1ResourceRequirements:
+    """
+    Converts an airflow Resources object into a k8s.V1ResourceRequirements
+    @param resources:
+    @return: k8s.V1ResourceRequirements
+    """
+    if isinstance(resources, dict):
+        resources = Resources(**resources)
+    return _convert_kube_model_object(resources, Resources, k8s.V1ResourceRequirements)
+
+
+def convert_port(port) -> k8s.V1ContainerPort:
+    """
+    Converts an airflow Port object into a k8s.V1ContainerPort
+    @param port:
+    @return: k8s.V1ContainerPort
+    """
+    return _convert_kube_model_object(port, Port, k8s.V1ContainerPort)
+
+
+def convert_env_vars(env_vars) -> List[k8s.V1EnvVar]:
+    """
+    Converts a dictionary into a list of env_vars
+    @param env_vars:
+    @return:
+    """
+    if isinstance(env_vars, dict):
+        res = []
+        for k, v in env_vars.items():
+            res.append(k8s.V1EnvVar(name=k, value=v))
+        return res
+    elif isinstance(env_vars, list):
+        return env_vars
+    else:
+        raise AirflowException(f"Expected dict or list, got {type(env_vars)}")
+
+
+def convert_pod_runtime_info_env(pod_runtime_info_envs) -> k8s.V1EnvVar:
+    """
+    Converts a PodRuntimeInfoEnv into an k8s.V1EnvVar
+    @param pod_runtime_info_envs:

Review comment:
       same here

##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume
+    @param volume:
+    @return: k8s.V1Volume
+    """
+    return _convert_kube_model_object(volume, Volume, k8s.V1Volume)
+
+
+def convert_volume_mount(volume_mount) -> k8s.V1VolumeMount:
+    """
+    Converts an airflow VolumeMount object into a k8s.V1VolumeMount
+    @param volume_mount:
+    @return: k8s.V1VolumeMount
+    """
+    return _convert_kube_model_object(volume_mount, VolumeMount, k8s.V1VolumeMount)
+
+
+def convert_resources(resources) -> k8s.V1ResourceRequirements:
+    """
+    Converts an airflow Resources object into a k8s.V1ResourceRequirements
+    @param resources:
+    @return: k8s.V1ResourceRequirements
+    """
+    if isinstance(resources, dict):
+        resources = Resources(**resources)
+    return _convert_kube_model_object(resources, Resources, k8s.V1ResourceRequirements)
+
+
+def convert_port(port) -> k8s.V1ContainerPort:
+    """
+    Converts an airflow Port object into a k8s.V1ContainerPort
+    @param port:
+    @return: k8s.V1ContainerPort
+    """
+    return _convert_kube_model_object(port, Port, k8s.V1ContainerPort)
+
+
+def convert_env_vars(env_vars) -> List[k8s.V1EnvVar]:
+    """
+    Converts a dictionary into a list of env_vars
+    @param env_vars:
+    @return:
+    """
+    if isinstance(env_vars, dict):
+        res = []
+        for k, v in env_vars.items():
+            res.append(k8s.V1EnvVar(name=k, value=v))
+        return res
+    elif isinstance(env_vars, list):
+        return env_vars
+    else:
+        raise AirflowException(f"Expected dict or list, got {type(env_vars)}")
+
+
+def convert_pod_runtime_info_env(pod_runtime_info_envs) -> k8s.V1EnvVar:
+    """
+    Converts a PodRuntimeInfoEnv into an k8s.V1EnvVar
+    @param pod_runtime_info_envs:
+    @return:
+    """
+    return _convert_kube_model_object(pod_runtime_info_envs, PodRuntimeInfoEnv, k8s.V1EnvVar)
+
+
+def convert_image_pull_secrets(image_pull_secrets) -> List[k8s.V1LocalObjectReference]:
+    """
+    Converts a PodRuntimeInfoEnv into an k8s.V1EnvVar
+    @param convert_image_pull_secrets:
+    @return:

Review comment:
       same here

##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume
+    @param volume:
+    @return: k8s.V1Volume
+    """
+    return _convert_kube_model_object(volume, Volume, k8s.V1Volume)
+
+
+def convert_volume_mount(volume_mount) -> k8s.V1VolumeMount:
+    """
+    Converts an airflow VolumeMount object into a k8s.V1VolumeMount
+    @param volume_mount:
+    @return: k8s.V1VolumeMount
+    """
+    return _convert_kube_model_object(volume_mount, VolumeMount, k8s.V1VolumeMount)
+
+
+def convert_resources(resources) -> k8s.V1ResourceRequirements:
+    """
+    Converts an airflow Resources object into a k8s.V1ResourceRequirements
+    @param resources:
+    @return: k8s.V1ResourceRequirements
+    """
+    if isinstance(resources, dict):
+        resources = Resources(**resources)
+    return _convert_kube_model_object(resources, Resources, k8s.V1ResourceRequirements)
+
+
+def convert_port(port) -> k8s.V1ContainerPort:
+    """
+    Converts an airflow Port object into a k8s.V1ContainerPort
+    @param port:
+    @return: k8s.V1ContainerPort
+    """
+    return _convert_kube_model_object(port, Port, k8s.V1ContainerPort)
+
+
+def convert_env_vars(env_vars) -> List[k8s.V1EnvVar]:
+    """
+    Converts a dictionary into a list of env_vars
+    @param env_vars:
+    @return:
+    """
+    if isinstance(env_vars, dict):
+        res = []
+        for k, v in env_vars.items():
+            res.append(k8s.V1EnvVar(name=k, value=v))
+        return res
+    elif isinstance(env_vars, list):
+        return env_vars
+    else:
+        raise AirflowException(f"Expected dict or list, got {type(env_vars)}")
+
+
+def convert_pod_runtime_info_env(pod_runtime_info_envs) -> k8s.V1EnvVar:
+    """
+    Converts a PodRuntimeInfoEnv into an k8s.V1EnvVar
+    @param pod_runtime_info_envs:
+    @return:
+    """
+    return _convert_kube_model_object(pod_runtime_info_envs, PodRuntimeInfoEnv, k8s.V1EnvVar)
+
+
+def convert_image_pull_secrets(image_pull_secrets) -> List[k8s.V1LocalObjectReference]:
+    """
+    Converts a PodRuntimeInfoEnv into an k8s.V1EnvVar
+    @param convert_image_pull_secrets:
+    @return:
+    """
+    if isinstance(image_pull_secrets, str):
+        secrets = image_pull_secrets.split(",")
+        return [k8s.V1LocalObjectReference(name=secret) for secret in secrets]
+    else:
+        return image_pull_secrets
+
+
+def convert_configmap(configmaps) -> k8s.V1EnvFromSource:
+    """
+    Converts a str into an k8s.V1EnvFromSource
+    @param configmaps:

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] potiuk commented on pull request #12384: Make K8sPodOperator backwards compatible

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #12384:
URL: https://github.com/apache/airflow/pull/12384#issuecomment-729489123


   Love it! 


----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       @dimberman ^^ have you seen this discussion ? 




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/volume_mount.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.
+"""Classes for interacting with Kubernetes API"""
+
+import warnings
+
+from kubernetes.client import models as k8s
+
+warnings.warn(
+    "This module is deprecated. Please use `kubernetes.client.models.V1Volume`.",

Review comment:
       Not the deprecation message, but this file. Calling it airflow.providers.cncf.kubernetes.backcompat.volume_mount won't help users.




----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")
+

Review comment:
       There are none under it :). It's the full package name not the module. But likely it should be select_package indeed :)




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/366700085) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider 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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       Waaaaait -- just noticed that 1.10.x still has an `airflow/kubernetes/pod.py` etc. What is this going to do?




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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


   The PR needs to run all tests because it modifies core of Airflow! Please rebase it to latest master or ask committer to re-run it!


----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/366569440) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider 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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       the kubernetes_pod.py was excluded from the previous rename, but pod_runtime_info can be changed in all classes.




----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       @ashb @dimberman Fixup here (I could not push it to astronomer's repo) : https://github.com/apache/airflow/pull/12392/commits/8d0be15ae4e4c1cd51d61ccd71fe0a8bef7f0ffe




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/volume.py
##########
@@ -0,0 +1,64 @@
+# 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.
+"""This module is deprecated. Please use `airflow.providers.amazon.aws.hooks.datasync`."""

Review comment:
       Copy/paste error: `datasync`




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/pod_runtime_info_env.py
##########
@@ -0,0 +1,43 @@
+# 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.
+"""Classes for interacting with Kubernetes API"""
+
+
+class PodRuntimeInfoEnv:
+    """Defines Pod runtime information as environment variable"""
+
+    def __init__(self, name, field_path):
+        """
+        Adds Kubernetes pod runtime information as environment variables such as namespace, pod IP, pod name.
+        Full list of options can be found in kubernetes documentation.
+
+        :param name: the name of the environment variable
+        :type: name: str
+        :param field_path: path to pod runtime info. Ex: metadata.namespace | status.podIP
+        :type: field_path: str
+        """
+        self.name = name
+        self.field_path = field_path
+
+    def to_k8s_client_obj(self):
+        """:return: kubernetes.client.models.V1EnvVar"""
+        import kubernetes.client.models as k8s

Review comment:
       Does this import need to be here anymore?




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/368526750) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider 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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py
##########
@@ -208,11 +221,15 @@ def __init__(  # pylint: disable=too-many-arguments,too-many-locals
         self.arguments = arguments or []
         self.labels = labels or {}
         self.startup_timeout_seconds = startup_timeout_seconds
-        self.env_vars = env_vars or []
+        self.env_vars = convert_env_vars(env_vars) if env_vars else []
+        if pod_runtime_info_envs:
+            self.env_vars.extend([convert_pod_runtime_info_env(p) for p in pod_runtime_info_envs])
         self.env_from = env_from or []
-        self.ports = ports or []
-        self.volume_mounts = volume_mounts or []
-        self.volumes = volumes or []
+        if configmaps:
+            self.env_from.extend([convert_configmap(c) for c in configmaps])
+        self.ports = [convert_port(p) for p in ports] if ports else []
+        self.volume_mounts = [convert_volume_mount(v) for v in volume_mounts] if volume_mounts else []
+        self.volumes = [convert_volume(volume) for volume in volumes] if volumes else []

Review comment:
       Right




----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")
+

Review comment:
       ```suggestion
           self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info")\
                 .rename("airflow.kubernetes.pod_runtime_info")
   ```
   
   That + restoring class removal should fix it.
   




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       Force-pushed back to the previous commit.




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       > @ashb @dimberman Fixup here (I could not push it to astronomer's repo) : 8d0be15
   
   Pushed this as https://github.com/apache/airflow/pull/12384/commits/d5d9e8b51b0ae534c8ac3f69a341c415ec8d3fb0




----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       We should simply add a step in UPDATING to 2.0 to make sure they remove backports simply (so that they are not tempted to force installation even with version conflict.




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")
+

Review comment:
       ohhh ok. I think I get it. 
   
   So basically the idea here is that we want to delete the backcompat files in 1.10 since we're not using them, though we were running into a bowler issue because we weren't changing one import in `kubernetes_pod.py` (this was because we needed to keep the original import for the converters.  So adding that extra query fixes the `pod_runtime_info_env` import and allows us to delete 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] github-actions[bot] commented on pull request #12384: Make K8sPodOperator backwards compatible

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/368725451) is cancelling this PR. It has some failed jobs matching ^Pylint$,^Static checks,^Build docs$,^Spell check docs$,^Backport packages$,^Provider 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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       What is this `rename()` achieving then? What dist does/should it appear in?




----------------------------------------------------------------
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] potiuk commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")

Review comment:
       So the proper fix will be to remove the classes from backcompat and add rename for that one as well.




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

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



[GitHub] [airflow] dimberman commented on a change in pull request #12384: Make K8sPodOperator backwards compatible

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



##########
File path: dev/provider_packages/refactor_provider_packages.py
##########
@@ -146,6 +146,13 @@ def rename_deprecated_modules(self) -> None:
         for new, old in changes:
             self.qry.select_module(new).rename(old)
 
+        def is_not_k8spodop(node: LN, capture: Capture, filename: Filename) -> bool:
+            return not filename.endswith("/kubernetes_pod.py")
+
+        self.qry.select_module("airflow.providers.cncf.kubernetes.backcompat").filter(
+            callback=is_not_k8spodop
+        ).rename("airflow.kubernetes")
+

Review comment:
       @potiuk @ashb wait can someone please break this down for me? I'm still confused as to what is broken.




----------------------------------------------------------------
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 #12384: Make K8sPodOperator backwards compatible

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



##########
File path: airflow/providers/cncf/kubernetes/backcompat/backwards_compat_converters.py
##########
@@ -0,0 +1,123 @@
+# 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.
+"""Executes task in a Kubernetes POD"""
+
+from typing import List
+
+from kubernetes.client import models as k8s
+
+from airflow.exceptions import AirflowException
+from airflow.providers.cncf.kubernetes.backcompat.pod import Port, Resources
+from airflow.providers.cncf.kubernetes.backcompat.pod_runtime_info_env import PodRuntimeInfoEnv
+from airflow.providers.cncf.kubernetes.backcompat.volume import Volume
+from airflow.providers.cncf.kubernetes.backcompat.volume_mount import VolumeMount
+
+
+def _convert_kube_model_object(obj, old_class, new_class):
+    convert_op = getattr(obj, "to_k8s_client_obj", None)
+    if callable(convert_op):
+        return obj.to_k8s_client_obj()
+    elif isinstance(obj, new_class):
+        return obj
+    else:
+        raise AirflowException(f"Expected {old_class} or {new_class}, got {type(obj)}")
+
+
+def convert_volume(volume) -> k8s.V1Volume:
+    """
+    Converts an airflow Volume object into a k8s.V1Volume
+    @param volume:
+    @return: k8s.V1Volume
+    """
+    return _convert_kube_model_object(volume, Volume, k8s.V1Volume)
+
+
+def convert_volume_mount(volume_mount) -> k8s.V1VolumeMount:
+    """
+    Converts an airflow VolumeMount object into a k8s.V1VolumeMount
+    @param volume_mount:
+    @return: k8s.V1VolumeMount

Review comment:
       Is there a setting on black we can create for this or some form of a static check that automatically fixes this?




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