Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Bugfix: Fix overriding pod_template_file in KubernetesExecutor #15197

Merged
merged 4 commits into from
Apr 5, 2021
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion airflow/executors/kubernetes_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -496,7 +496,7 @@ def execute_async(
return

if executor_config:
pod_template_file = executor_config.get("pod_template_override", None)
pod_template_file = executor_config.get("pod_template_file", None)
else:
pod_template_file = None
if not self.task_queue:
Expand Down
2 changes: 1 addition & 1 deletion docs/apache-airflow/executor/kubernetes.rst
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ name ``base`` and a second container containing your desired sidecar.
:end-before: [END task_with_sidecar]

You can also create custom ``pod_template_file`` on a per-task basis so that you can recycle the same base values between multiple tasks.
This will replace the default ``pod_template_file`` named in the airflow.cfg and then override that template using the ``pod_override_spec``.
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

pod_override_spec never existed

This will replace the default ``pod_template_file`` named in the airflow.cfg and then override that template using the ``pod_override``.

Here is an example of a task with both features:

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
# 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.
---
kind: Pod
apiVersion: v1
metadata:
name: dummy-name-dont-delete
namespace: dummy-name-dont-delete
labels:
mylabel: foo
spec:
containers:
- name: base
image: dummy-name-dont-delete
securityContext:
runAsUser: 50000
fsGroup: 50000
imagePullSecrets:
- name: airflow-registry
schedulerName: default-scheduler
75 changes: 75 additions & 0 deletions tests/executors/test_kubernetes_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,15 @@
# specific language governing permissions and limitations
# under the License.
#
import pathlib
import random
import re
import string
import unittest
from datetime import datetime
from unittest import mock

import pytest
from kubernetes.client import models as k8s
from urllib3 import HTTPResponse

Expand Down Expand Up @@ -215,6 +217,79 @@ def test_invalid_executor_config(self, mock_get_kube_client, mock_kubernetes_job

assert list(executor.event_buffer.values())[0][1] == "Invalid executor_config passed"

@pytest.mark.execution_timeout(10)
@unittest.skipIf(AirflowKubernetesScheduler is None, 'kubernetes python package is not installed')
@mock.patch('airflow.kubernetes.pod_launcher.PodLauncher.run_pod_async')
@mock.patch('airflow.executors.kubernetes_executor.get_kube_client')
def test_pod_template_file_override_in_executor_config(self, mock_get_kube_client, mock_run_pod_async):
current_folder = pathlib.Path(__file__).parent.absolute()
template_file = str(
(current_folder / "kubernetes_executor_template_files" / "basic_template.yaml").absolute()
)

mock_kube_client = mock.patch('kubernetes.client.CoreV1Api', autospec=True)
mock_get_kube_client.return_value = mock_kube_client

with conf_vars({('kubernetes', 'pod_template_file'): ''}):
executor = self.kubernetes_executor
executor.start()

assert executor.event_buffer == {}
assert executor.task_queue.empty()

executor.execute_async(
key=('dag', 'task', datetime.utcnow(), 1),
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

May be redundant, but to be on more safe side, we may want to assert the execution date here as well, rather than just having mock.ANY.

We have have something like below here

            execution_date = datetime.utcnow()
            executor.execute_async(
                key=('dag', 'task', execution_date, 1),
                ... ...

Then in mock_run_pod_async.assert_called_once_with(), have

annotations={
    'dag_id': 'dag',
    'execution_date': execution_date.isoformat(),
    ... ...
}

and

labels={
    'airflow-worker': '5',
    'airflow_version': mock.ANY,
    'dag_id': 'dag',
    'execution_date': pod_generator.datetime_to_label_safe_datestring(execution_date),
    ... ...
}

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done in 855d401

queue=None,
command=['airflow', 'tasks', 'run', 'true', 'some_parameter'],
executor_config={
"pod_template_file": template_file,
"pod_override": k8s.V1Pod(metadata=k8s.V1ObjectMeta(labels={"release": "stable"})),
},
)

assert not executor.task_queue.empty()
task = executor.task_queue.get_nowait()
_, _, expected_executor_config, expected_pod_template_file = task

# Test that the correct values have been put to queue
assert expected_executor_config.metadata.labels == {'release': 'stable'}
assert expected_pod_template_file == template_file

self.kubernetes_executor.kube_scheduler.run_next(task)
mock_run_pod_async.assert_called_once_with(
k8s.V1Pod(
api_version="v1",
kind="Pod",
metadata=k8s.V1ObjectMeta(
name=mock.ANY,
namespace="default",
annotations={
'dag_id': 'dag',
'execution_date': mock.ANY,
'task_id': 'task',
'try_number': '1',
},
labels={
'airflow-worker': '5',
'airflow_version': mock.ANY,
'dag_id': 'dag',
'execution_date': mock.ANY,
'kubernetes_executor': 'True',
'mylabel': 'foo',
'release': 'stable',
'task_id': 'task',
'try_number': '1',
},
),
spec=k8s.V1PodSpec(
containers=mock.ANY,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Possible to make the assertion more granular? Container spec is worth testing carefully, e.g. the image used.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was in two minds with that -- I added it back in 855d401

image_pull_secrets=[k8s.V1LocalObjectReference(name='airflow-registry')],
scheduler_name='default-scheduler',
security_context=k8s.V1PodSecurityContext(fs_group=50000, run_as_user=50000),
),
)
)

@mock.patch('airflow.executors.kubernetes_executor.KubernetesJobWatcher')
@mock.patch('airflow.executors.kubernetes_executor.get_kube_client')
def test_change_state_running(self, mock_get_kube_client, mock_kubernetes_job_watcher):
Expand Down