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 2023/01/06 23:33:11 UTC

[GitHub] [airflow] jedcunningham opened a new pull request, #28776: Only patch single label when adopting pod

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

   When KubernetesExecutor adopts pods, it was patching the pod with the pod it retrieved from the k8s api, while just updating a single label. Normally this works just fine, but there are cases where the pod you pull from the k8s api can't be used as-is when patching - it results in a 422 `Forbidden: pod updates may not change fields other than ...`.
   
   Instead we now just pass the single label we need to update to patch, allowing us to avoid accidentally "updating" other fields.
   
   Closes #24015


-- 
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 a diff in pull request #28776: Only patch single label when adopting pod

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


##########
airflow/executors/kubernetes_executor.py:
##########
@@ -778,26 +778,28 @@ def adopt_launched_task(
             assert self.scheduler_job_id
 
         self.log.info("attempting to adopt pod %s", pod.metadata.name)
-        pod.metadata.labels["airflow-worker"] = pod_generator.make_safe_label_value(self.scheduler_job_id)
         pod_id = annotations_to_key(pod.metadata.annotations)
         if pod_id not in pod_ids:
             self.log.error("attempting to adopt taskinstance which was not specified by database: %s", pod_id)
             return
 
+        new_worker_id_label = pod_generator.make_safe_label_value(self.scheduler_job_id)
         try:
             kube_client.patch_namespaced_pod(
                 name=pod.metadata.name,
                 namespace=pod.metadata.namespace,
-                body=PodGenerator.serialize_pod(pod),
+                body={"metadata": {"labels": {"airflow-worker": new_worker_id_label}}},
             )
-            pod_ids.pop(pod_id)
-            self.running.add(pod_id)
         except ApiException as e:
             self.log.info("Failed to adopt pod %s. Reason: %s", pod.metadata.name, e)
+            return
+
+        pod_ids.pop(pod_id)

Review Comment:
   ```suggestion
           del pod_ids[pod_id]
   ```



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

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

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


[GitHub] [airflow] jedcunningham commented on a diff in pull request #28776: Only patch single label when adopting pod

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


##########
tests/executors/test_kubernetes_executor.py:
##########
@@ -718,20 +718,42 @@ def test_adopt_launched_task(self, mock_kube_client):
         pod_ids = {ti_key: {}}
 
         executor.adopt_launched_task(mock_kube_client, pod=pod, pod_ids=pod_ids)
-        assert mock_kube_client.patch_namespaced_pod.call_args[1] == {
-            "body": {
-                "metadata": {
-                    "labels": {"airflow-worker": "modified"},
-                    "annotations": annotations,
-                    "name": "foo",
-                }
-            },
-            "name": "foo",
-            "namespace": None,
-        }
+        mock_kube_client.patch_namespaced_pod.assert_called_once_with(
+            body={"metadata": {"labels": {"airflow-worker": "modified"}}},
+            name="foo",
+            namespace=None,
+        )
         assert pod_ids == {}
         assert executor.running == {ti_key}
 
+    @mock.patch("airflow.executors.kubernetes_executor.get_kube_client")
+    def test_adopt_completed_pods(self, mock_kube_client):

Review Comment:
   The problem is we'd end up with `test_adopt_completed_pods_only_those_from_other_schedulers_and_only_patch_label`. The one for `adopt_launched_task` would be even worse. Let me come up with something.



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

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

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


[GitHub] [airflow] jedcunningham merged pull request #28776: Only patch single label when adopting pod

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


-- 
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] potiuk commented on a diff in pull request #28776: Only patch single label when adopting pod

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


##########
tests/executors/test_kubernetes_executor.py:
##########
@@ -718,20 +718,42 @@ def test_adopt_launched_task(self, mock_kube_client):
         pod_ids = {ti_key: {}}
 
         executor.adopt_launched_task(mock_kube_client, pod=pod, pod_ids=pod_ids)
-        assert mock_kube_client.patch_namespaced_pod.call_args[1] == {
-            "body": {
-                "metadata": {
-                    "labels": {"airflow-worker": "modified"},
-                    "annotations": annotations,
-                    "name": "foo",
-                }
-            },
-            "name": "foo",
-            "namespace": None,
-        }
+        mock_kube_client.patch_namespaced_pod.assert_called_once_with(
+            body={"metadata": {"labels": {"airflow-worker": "modified"}}},
+            name="foo",
+            namespace=None,
+        )
         assert pod_ids == {}
         assert executor.running == {ti_key}
 
+    @mock.patch("airflow.executors.kubernetes_executor.get_kube_client")
+    def test_adopt_completed_pods(self, mock_kube_client):

Review Comment:
   I am rather opting for very long and very descriptive test method names:
   
   `test_adopted_pod_only_update_label_and_nothing_else` 
   
   (I am not joking).
   
   



-- 
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] potiuk commented on a diff in pull request #28776: Only patch single label when adopting pod

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


##########
tests/executors/test_kubernetes_executor.py:
##########
@@ -718,20 +718,42 @@ def test_adopt_launched_task(self, mock_kube_client):
         pod_ids = {ti_key: {}}
 
         executor.adopt_launched_task(mock_kube_client, pod=pod, pod_ids=pod_ids)
-        assert mock_kube_client.patch_namespaced_pod.call_args[1] == {
-            "body": {
-                "metadata": {
-                    "labels": {"airflow-worker": "modified"},
-                    "annotations": annotations,
-                    "name": "foo",
-                }
-            },
-            "name": "foo",
-            "namespace": None,
-        }
+        mock_kube_client.patch_namespaced_pod.assert_called_once_with(
+            body={"metadata": {"labels": {"airflow-worker": "modified"}}},
+            name="foo",
+            namespace=None,
+        )
         assert pod_ids == {}
         assert executor.running == {ti_key}
 
+    @mock.patch("airflow.executors.kubernetes_executor.get_kube_client")
+    def test_adopt_completed_pods(self, mock_kube_client):

Review Comment:
   I am rather opting for veruy long and very descriptive test method names:
   
   `test_adopted_pod_only_update_label_and_nothing_else` 
   
   (I am not joking).
   
   



-- 
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 a diff in pull request #28776: Only patch single label when adopting pod

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


##########
tests/executors/test_kubernetes_executor.py:
##########
@@ -718,20 +718,42 @@ def test_adopt_launched_task(self, mock_kube_client):
         pod_ids = {ti_key: {}}
 
         executor.adopt_launched_task(mock_kube_client, pod=pod, pod_ids=pod_ids)
-        assert mock_kube_client.patch_namespaced_pod.call_args[1] == {
-            "body": {
-                "metadata": {
-                    "labels": {"airflow-worker": "modified"},
-                    "annotations": annotations,
-                    "name": "foo",
-                }
-            },
-            "name": "foo",
-            "namespace": None,
-        }
+        mock_kube_client.patch_namespaced_pod.assert_called_once_with(
+            body={"metadata": {"labels": {"airflow-worker": "modified"}}},
+            name="foo",
+            namespace=None,
+        )
         assert pod_ids == {}
         assert executor.running == {ti_key}
 
+    @mock.patch("airflow.executors.kubernetes_executor.get_kube_client")
+    def test_adopt_completed_pods(self, mock_kube_client):

Review Comment:
   Could add docstring saying what this is testing 



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

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

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