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 2021/12/30 04:52:01 UTC

[GitHub] [airflow] dstandish opened a new pull request #20578: Use KubernetesHook to create api client in KubernetesPodOperator

dstandish opened a new pull request #20578:
URL: https://github.com/apache/airflow/pull/20578


   What I've done here is entirely defer to the KubernetesHook with regard to using airflow conn id vs the three other parameters.
   
   I _think_ this should work. 
   
   But one thing I wonder about are these airflow conf settings in `get_kube_client`:
   
   ```python
       if not in_cluster:
           if cluster_context is None:
               cluster_context = conf.get('kubernetes', 'cluster_context', fallback=None)
           if config_file is None:
               config_file = conf.get('kubernetes', 'config_file', fallback=None)
   
       if conf.getboolean('kubernetes', 'enable_tcp_keepalive'):
           _enable_tcp_keepalive()
   
       if not conf.getboolean('kubernetes', 'verify_ssl'):
           _disable_verify_ssl()
   
       client_conf = _get_kube_config(in_cluster, cluster_context, config_file)
       return _get_client_with_patched_configuration(client_conf)
   ```
   
   It doesn't seem that KubernetesHook lookss at these settings.  Do you think we need to implement either / or logic here?  That is, only use K8s hook if an airflow conn id is provided? And otherwise, continue to use `get_kube_client`?
   
   Thanks
   
   cc @jedcunningham


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

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

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



[GitHub] [airflow] jedcunningham commented on a change in pull request #20578: Use KubernetesHook to create api client in KubernetesPodOperator

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



##########
File path: airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py
##########
@@ -348,15 +351,17 @@ def launcher(self) -> pod_launcher.PodLauncher:
         return pod_launcher.PodLauncher(kube_client=self.client)
 
     @cached_property
-    def client(self) -> CoreV1Api:
-        # todo: use airflow Connection / hook to authenticate to the cluster
-        kwargs: Dict[str, Any] = dict(
+    def k8s_hook(self):

Review comment:
       ```suggestion
       def hook(self):
   ```
   
   nit, for consistency with other operators?

##########
File path: airflow/providers/cncf/kubernetes/operators/kubernetes_pod.py
##########
@@ -348,15 +351,17 @@ def launcher(self) -> pod_launcher.PodLauncher:
         return pod_launcher.PodLauncher(kube_client=self.client)
 
     @cached_property
-    def client(self) -> CoreV1Api:
-        # todo: use airflow Connection / hook to authenticate to the cluster
-        kwargs: Dict[str, Any] = dict(
+    def k8s_hook(self):
+        return KubernetesHook(
+            conn_id=self.kubernetes_conn_id,
             cluster_context=self.cluster_context,
             config_file=self.config_file,
+            in_cluster=self.in_cluster,
         )
-        if self.in_cluster is not None:
-            kwargs.update(in_cluster=self.in_cluster)
-        return kube_client.get_kube_client(**kwargs)
+
+    @cached_property
+    def client(self) -> CoreV1Api:
+        return self.k8s_hook.core_v1_client

Review comment:
       ```suggestion
           return self.hook.core_v1_client
   ```




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

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

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



[GitHub] [airflow] BasPH commented on a change in pull request #20578: Use KubernetesHook to create api client in KubernetesPodOperator

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



##########
File path: airflow/providers/cncf/kubernetes/CHANGELOG.rst
##########
@@ -103,6 +103,12 @@ Other changes in ``pod_launcher.py``:
 
 * Enum-like class ``PodStatus`` is renamed ``PodPhase``, and the values are no longer lower-cased.
 
+Features
+~~~~~~~~
+
+* Can now derive kubernetes creds using airflow connection, through KubernetesHook

Review comment:
       ```suggestion
   * Can now derive Kubernetes credentials from an Airflow connection through the KubernetesHook
   ```




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

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

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



[GitHub] [airflow] github-actions[bot] closed pull request #20578: Use KubernetesHook to create api client in KubernetesPodOperator

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #20578:
URL: https://github.com/apache/airflow/pull/20578


   


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

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

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



[GitHub] [airflow] dstandish commented on pull request #20578: Use KubernetesHook to create api client in KubernetesPodOperator

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


   @jedcunningham @BasPH after some time away from this one I think I see a good way forward.
   
   We can use the `kubernetes` config settings as a "default" for hook behavior.
   
   I.e. we can update the hook so that, it optionally uses these config  values if not overrided by hook params or connection attrs.
   
   ```
       if conf.getboolean('kubernetes', 'enable_tcp_keepalive'):
           _enable_tcp_keepalive()
   
       if not conf.getboolean('kubernetes', 'verify_ssl'):
           _disable_verify_ssl()
   ```
   
   If these two params are the only thing "missing" ( i.e. in the hook logic compared with the get_kube_client logic ), then it should be fully backward compatible.  Then we can  _always_ use the hook and the parameters (context, config, in cluster) can be freely combined with the hook.
   
   ---
   
   Do you have any concerns with using the `kubernetes` config settings as a "default" for hook behavior?  I suppose some could say this integrates provider and core in a way that could potentially be undesirable.  Though others might think it is appropriate to have  the K8s hook take the same defaults as K8s executor.
   
   If we're OK with that, then the question I just need to confirm is, is there anything _else_ that differs between hook and the old get_kube_client
   
   ---
   
   @jedcunningham one issue with deprecating use of  those params is, maybe people want to still use  those params and not a conn id.   Use of those params only (i.e. and not also a conn_id) is supported with the hook.  What I might suggest we do though is add a dict param `hook_params` or `hook_kwargs` and then deprecate those three params _as individual params_ in order to reduce number of params in KPO and be more future proof.
   


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

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

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



[GitHub] [airflow] dstandish commented on pull request #20578: Use KubernetesHook to create api client in KubernetesPodOperator

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


   I will pick this one back up after merge of https://github.com/apache/airflow/pull/20759
   
   Once that is merged it will be easier / cleaner to have the behavior be consistent between hook and "core" kubernetes 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.

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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #20578: Use KubernetesHook to create api client in KubernetesPodOperator

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


   This pull request has been automatically marked as stale because it has not had recent activity. It will be closed in 5 days if no further activity occurs. Thank you for your contributions.


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

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

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