You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by ka...@apache.org on 2020/11/27 19:50:39 UTC

[airflow] branch v1-10-test updated (f20d315 -> 64d84cb)

This is an automated email from the ASF dual-hosted git repository.

kaxilnaik pushed a change to branch v1-10-test
in repository https://gitbox.apache.org/repos/asf/airflow.git.


 discard f20d315  Typo Fix: Deprecated config force_log_out_after was not used (#12661)
    omit 74e0247  Fix issue with empty Resources in executor_config (#12633)
    omit 55cafbb  Actually run against the version of the DB we select in the matrix. (#12591)
     add 847820f  Add BaseOperatorMetaclassRule (#12629)
     add 2a7944d  BugFix: Tasks with depends_on_past or task_concurrency are stuck (#12663)
     new ca53e70  Actually run against the version of the DB we select in the matrix. (#12591)
     new 540e70b  Fix issue with empty Resources in executor_config (#12633)
     new 64d84cb  Typo Fix: Deprecated config force_log_out_after was not used (#12661)

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (f20d315)
            \
             N -- N -- N   refs/heads/v1-10-test (64d84cb)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 airflow/models/dagrun.py                           |  2 +-
 .../rules/custom_operator_metaclass_rule.py        | 55 ++++++++++++
 tests/jobs/test_scheduler_job.py                   | 98 ++++++++++++++++++++++
 .../rules/test_custom_operator_metaclass_rule.py   | 55 ++++++++++++
 4 files changed, 209 insertions(+), 1 deletion(-)
 create mode 100644 airflow/upgrade/rules/custom_operator_metaclass_rule.py
 create mode 100644 tests/upgrade/rules/test_custom_operator_metaclass_rule.py


[airflow] 02/03: Fix issue with empty Resources in executor_config (#12633)

Posted by ka...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kaxilnaik pushed a commit to branch v1-10-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 540e70b050da80b73570fd02dc3721f27b585ace
Author: Daniel Imberman <da...@gmail.com>
AuthorDate: Fri Nov 27 05:52:49 2020 -0800

    Fix issue with empty Resources in executor_config (#12633)
    
    Fixes an issue where if a user specifies a request but not a limit in
    resources for the executor_config, backwards compat can not parse
    
    (cherry picked from commit 84eecf94bab1a8c66b5161f03c6631448fb4850e)
---
 airflow/contrib/kubernetes/pod.py   |   4 +-
 tests/kubernetes/models/test_pod.py | 240 ++++++++++++++++++++++--------------
 2 files changed, 152 insertions(+), 92 deletions(-)

diff --git a/airflow/contrib/kubernetes/pod.py b/airflow/contrib/kubernetes/pod.py
index d1f30a8..7e38147 100644
--- a/airflow/contrib/kubernetes/pod.py
+++ b/airflow/contrib/kubernetes/pod.py
@@ -250,8 +250,8 @@ def _extract_ports(ports):
 
 def _extract_resources(resources):
     if isinstance(resources, k8s.V1ResourceRequirements):
-        requests = resources.requests
-        limits = resources.limits
+        requests = resources.requests or {}
+        limits = resources.limits or {}
         return Resources(
             request_memory=requests.get('memory', None),
             request_cpu=requests.get('cpu', None),
diff --git a/tests/kubernetes/models/test_pod.py b/tests/kubernetes/models/test_pod.py
index f8df28a..6939597 100644
--- a/tests/kubernetes/models/test_pod.py
+++ b/tests/kubernetes/models/test_pod.py
@@ -19,63 +19,88 @@ from tests.compat import mock
 from kubernetes.client import ApiClient
 import kubernetes.client.models as k8s
 from airflow.kubernetes.pod import Port
+from airflow.kubernetes.pod import Resources
+from airflow.contrib.kubernetes.pod import _extract_resources
 from airflow.kubernetes.pod_generator import PodGenerator
 from airflow.kubernetes.k8s_model import append_to_pod
 
 
 class TestPod(unittest.TestCase):
+    def test_extract_resources(self):
+        res = _extract_resources(k8s.V1ResourceRequirements())
+        self.assertEqual(
+            res.to_k8s_client_obj().to_dict(), Resources().to_k8s_client_obj().to_dict()
+        )
+        res = _extract_resources(k8s.V1ResourceRequirements(limits={"memory": "1G"}))
+        self.assertEqual(
+            res.to_k8s_client_obj().to_dict(),
+            Resources(limit_memory="1G").to_k8s_client_obj().to_dict(),
+        )
+        res = _extract_resources(k8s.V1ResourceRequirements(requests={"memory": "1G"}))
+        self.assertEqual(
+            res.to_k8s_client_obj().to_dict(),
+            Resources(request_memory="1G").to_k8s_client_obj().to_dict(),
+        )
+        res = _extract_resources(
+            k8s.V1ResourceRequirements(
+                limits={"memory": "1G"}, requests={"memory": "1G"}
+            )
+        )
+        self.assertEqual(
+            res.to_k8s_client_obj().to_dict(),
+            Resources(limit_memory="1G", request_memory="1G")
+            .to_k8s_client_obj()
+            .to_dict(),
+        )
 
     def test_port_to_k8s_client_obj(self):
-        port = Port('http', 80)
+        port = Port("http", 80)
         self.assertEqual(
             port.to_k8s_client_obj(),
-            k8s.V1ContainerPort(
-                name='http',
-                container_port=80
-            )
+            k8s.V1ContainerPort(name="http", container_port=80),
         )
 
-    @mock.patch('uuid.uuid4')
+    @mock.patch("uuid.uuid4")
     def test_port_attach_to_pod(self, mock_uuid):
         import uuid
-        static_uuid = uuid.UUID('cf4a56d2-8101-4217-b027-2af6216feb48')
+
+        static_uuid = uuid.UUID("cf4a56d2-8101-4217-b027-2af6216feb48")
         mock_uuid.return_value = static_uuid
-        pod = PodGenerator(image='airflow-worker:latest', name='base').gen_pod()
-        ports = [
-            Port('https', 443),
-            Port('http', 80)
-        ]
+        pod = PodGenerator(image="airflow-worker:latest", name="base").gen_pod()
+        ports = [Port("https", 443), Port("http", 80)]
         k8s_client = ApiClient()
         result = append_to_pod(pod, ports)
         result = k8s_client.sanitize_for_serialization(result)
-        self.assertEqual({
-            'apiVersion': 'v1',
-            'kind': 'Pod',
-            'metadata': {'name': 'base-' + static_uuid.hex},
-            'spec': {
-                'containers': [{
-                    'args': [],
-                    'command': [],
-                    'env': [],
-                    'envFrom': [],
-                    'image': 'airflow-worker:latest',
-                    'name': 'base',
-                    'ports': [{
-                        'name': 'https',
-                        'containerPort': 443
-                    }, {
-                        'name': 'http',
-                        'containerPort': 80
-                    }],
-                    'volumeMounts': [],
-                }],
-                'hostNetwork': False,
-                'imagePullSecrets': [],
-                'volumes': []
-            }
-        }, result)
+        self.assertEqual(
+            {
+                "apiVersion": "v1",
+                "kind": "Pod",
+                "metadata": {"name": "base-" + static_uuid.hex},
+                "spec": {
+                    "containers": [
+                        {
+                            "args": [],
+                            "command": [],
+                            "env": [],
+                            "envFrom": [],
+                            "image": "airflow-worker:latest",
+                            "name": "base",
+                            "ports": [
+                                {"name": "https", "containerPort": 443},
+                                {"name": "http", "containerPort": 80},
+                            ],
+                            "volumeMounts": [],
+                        }
+                    ],
+                    "hostNetwork": False,
+                    "imagePullSecrets": [],
+                    "volumes": [],
+                },
+            },
+            result,
+        )
 
-    @mock.patch('uuid.uuid4')
+    @mock.patch("uuid.uuid4")
     def test_to_v1_pod(self, mock_uuid):
         from airflow.contrib.kubernetes.pod import Pod as DeprecatedPod
         from airflow.kubernetes.volume import Volume
@@ -83,7 +108,8 @@ class TestPod(unittest.TestCase):
         from airflow.kubernetes.secret import Secret
         from airflow.kubernetes.pod import Resources
         import uuid
-        static_uuid = uuid.UUID('cf4a56d2-8101-4217-b027-2af6216feb48')
+
+        static_uuid = uuid.UUID("cf4a56d2-8101-4217-b027-2af6216feb48")
         mock_uuid.return_value = static_uuid
 
         pod = DeprecatedPod(
@@ -94,24 +120,26 @@ class TestPod(unittest.TestCase):
             envs={"test_key": "test_value"},
             cmds=["airflow"],
             resources=Resources(
-                request_memory="1G",
-                request_cpu="100Mi",
-                limit_gpu="100G"
+                request_memory="1G", request_cpu="100Mi", limit_gpu="100G"
             ),
             init_containers=k8s.V1Container(
                 name="test-container",
-                volume_mounts=k8s.V1VolumeMount(mount_path="/foo/bar", name="init-volume-secret")
+                volume_mounts=k8s.V1VolumeMount(
+                    mount_path="/foo/bar", name="init-volume-secret"
+                ),
             ),
             volumes=[
                 Volume(name="foo", configs={}),
-                {"name": "bar", 'secret': {'secretName': 'volume-secret'}}
+                {"name": "bar", "secret": {"secretName": "volume-secret"}},
             ],
             secrets=[
                 Secret("volume", None, "init-volume-secret"),
-                Secret('env', "AIRFLOW_SECRET", 'secret_name', "airflow_config"),
-                Secret("volume", "/opt/airflow", "volume-secret", "secret-key")
+                Secret("env", "AIRFLOW_SECRET", "secret_name", "airflow_config"),
+                Secret("volume", "/opt/airflow", "volume-secret", "secret-key"),
+            ],
+            volume_mounts=[
+                VolumeMount(name="foo", mount_path="/mnt", sub_path="/", read_only=True)
             ],
-            volume_mounts=[VolumeMount(name="foo", mount_path="/mnt", sub_path="/", read_only=True)]
         )
 
         k8s_client = ApiClient()
@@ -119,47 +147,79 @@ class TestPod(unittest.TestCase):
         result = pod.to_v1_kubernetes_pod()
         result = k8s_client.sanitize_for_serialization(result)
 
-        expected = \
-            {'metadata': {'annotations': {},
-                          'labels': {},
-                          'name': 'bar',
-                          'namespace': 'baz'},
-             'spec': {'affinity': {},
-                      'containers': [{'args': [],
-                                      'command': ['airflow'],
-                                      'env': [{'name': 'test_key', 'value': 'test_value'},
-                                              {'name': 'AIRFLOW_SECRET',
-                                               'valueFrom': {'secretKeyRef': {'key': 'airflow_config',
-                                                                              'name': 'secret_name'}}}],
-                                      'envFrom': [],
-                                      'image': 'foo',
-                                      'imagePullPolicy': 'Never',
-                                      'name': 'base',
-                                      'resources': {'limits': {'nvidia.com/gpu': '100G'},
-                                                    'requests': {'cpu': '100Mi',
-                                                                 'memory': '1G'}},
-                                      'volumeMounts': [{'mountPath': '/mnt',
-                                                        'name': 'foo',
-                                                        'readOnly': True,
-                                                        'subPath': '/'},
-                                                       {'mountPath': '/opt/airflow',
-                                                        'name': 'secretvol' + str(static_uuid),
-                                                        'readOnly': True}]}],
-                      'hostNetwork': False,
-                      'imagePullSecrets': [],
-                      'initContainers': {'name': 'test-container',
-                                         'volumeMounts': {'mountPath': '/foo/bar',
-                                                          'name': 'init-volume-secret'}},
-                      'nodeSelector': {},
-                      'securityContext': {},
-                      'tolerations': [],
-                      'volumes': [{'name': 'foo'},
-                                  {'name': 'bar',
-                                   'secret': {'secretName': 'volume-secret'}},
-                                  {'name': 'secretvol' + str(static_uuid),
-                                   'secret': {'secretName': 'init-volume-secret'}},
-                                  {'name': 'secretvol' + str(static_uuid),
-                                   'secret': {'secretName': 'volume-secret'}}
-                                  ]}}
+        expected = {
+            "metadata": {
+                "annotations": {},
+                "labels": {},
+                "name": "bar",
+                "namespace": "baz",
+            },
+            "spec": {
+                "affinity": {},
+                "containers": [
+                    {
+                        "args": [],
+                        "command": ["airflow"],
+                        "env": [
+                            {"name": "test_key", "value": "test_value"},
+                            {
+                                "name": "AIRFLOW_SECRET",
+                                "valueFrom": {
+                                    "secretKeyRef": {
+                                        "key": "airflow_config",
+                                        "name": "secret_name",
+                                    }
+                                },
+                            },
+                        ],
+                        "envFrom": [],
+                        "image": "foo",
+                        "imagePullPolicy": "Never",
+                        "name": "base",
+                        "resources": {
+                            "limits": {"nvidia.com/gpu": "100G"},
+                            "requests": {"cpu": "100Mi", "memory": "1G"},
+                        },
+                        "volumeMounts": [
+                            {
+                                "mountPath": "/mnt",
+                                "name": "foo",
+                                "readOnly": True,
+                                "subPath": "/",
+                            },
+                            {
+                                "mountPath": "/opt/airflow",
+                                "name": "secretvol" + str(static_uuid),
+                                "readOnly": True,
+                            },
+                        ],
+                    }
+                ],
+                "hostNetwork": False,
+                "imagePullSecrets": [],
+                "initContainers": {
+                    "name": "test-container",
+                    "volumeMounts": {
+                        "mountPath": "/foo/bar",
+                        "name": "init-volume-secret",
+                    },
+                },
+                "nodeSelector": {},
+                "securityContext": {},
+                "tolerations": [],
+                "volumes": [
+                    {"name": "foo"},
+                    {"name": "bar", "secret": {"secretName": "volume-secret"}},
+                    {
+                        "name": "secretvol" + str(static_uuid),
+                        "secret": {"secretName": "init-volume-secret"},
+                    },
+                    {
+                        "name": "secretvol" + str(static_uuid),
+                        "secret": {"secretName": "volume-secret"},
+                    },
+                ],
+            },
+        }
         self.maxDiff = None
         self.assertEqual(expected, result)


[airflow] 03/03: Typo Fix: Deprecated config force_log_out_after was not used (#12661)

Posted by ka...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kaxilnaik pushed a commit to branch v1-10-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit 64d84cbea4a3aeb83579ff6c2909d3d10be744e6
Author: Kaxil Naik <ka...@gmail.com>
AuthorDate: Fri Nov 27 17:36:10 2020 +0000

    Typo Fix: Deprecated config force_log_out_after was not used (#12661)
    
    `force_logout_after` should be `force_log_out_after` in the code
    section https://github.com/apache/airflow/blob/master/airflow/settings.py#L372-L381.
    
    As `force_log_out_after` is actually used and written in
    https://github.com/apache/airflow/blob/c5700a56bb3b9a5b872bda0fe0d3de82b0128bdf/UPDATING.md#unify-user-session-lifetime-configuration.
    
    (cherry picked from commit 531e00660af0cc7729792ef08559edd84c6c46ab)
---
 airflow/settings.py | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/airflow/settings.py b/airflow/settings.py
index 2aedb72..c708d90 100644
--- a/airflow/settings.py
+++ b/airflow/settings.py
@@ -386,7 +386,7 @@ def get_session_lifetime_config():
     session_lifetime_minutes = conf.get('webserver', 'session_lifetime_minutes', fallback=None)
     session_lifetime_days = conf.get('webserver', 'session_lifetime_days', fallback=None)
     uses_deprecated_lifetime_configs = session_lifetime_days or conf.get(
-        'webserver', 'force_logout_after', fallback=None
+        'webserver', 'force_log_out_after', fallback=None
     )
 
     minutes_per_day = 24 * 60
@@ -395,7 +395,7 @@ def get_session_lifetime_config():
         warnings.warn(
             '`session_lifetime_days` option from `[webserver]` section has been '
             'renamed to `session_lifetime_minutes`. The new option allows to configure '
-            'session lifetime in minutes. The `force_logout_after` option has been removed '
+            'session lifetime in minutes. The `force_log_out_after` option has been removed '
             'from `[webserver]` section. Please update your configuration.',
             category=DeprecationWarning,
         )


[airflow] 01/03: Actually run against the version of the DB we select in the matrix. (#12591)

Posted by ka...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

kaxilnaik pushed a commit to branch v1-10-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit ca53e70c2aff5e7ab3eef53a3fc9e813b989fb8a
Author: Ash Berlin-Taylor <as...@firemirror.com>
AuthorDate: Wed Nov 25 20:17:10 2020 +0000

    Actually run against the version of the DB we select in the matrix. (#12591)
    
    Due to a bug in Breeze initialization code, we were always running
    against Postgres 9.6 and MySQL 5.7, even when the matrix selected
    something else.
    
    (We were overwriting the POSTGRES_VERSION and MYSQL_VERSION environment
    variables in initialization code)
    
    (cherry picked from commit 54adda50c69b17ffc997348b8151c6196ecf5045)
---
 scripts/ci/libraries/_initialization.sh | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/scripts/ci/libraries/_initialization.sh b/scripts/ci/libraries/_initialization.sh
index 396bded..5890834 100644
--- a/scripts/ci/libraries/_initialization.sh
+++ b/scripts/ci/libraries/_initialization.sh
@@ -102,10 +102,10 @@ function initialization::initialize_base_variables() {
     export CURRENT_MYSQL_VERSIONS
 
     # Default Postgres versions
-    export POSTGRES_VERSION=${CURRENT_POSTGRES_VERSIONS[0]}
+    export POSTGRES_VERSION=${POSTGRES_VERSION:=${CURRENT_POSTGRES_VERSIONS[0]}}
 
     # Default MySQL versions
-    export MYSQL_VERSION=${CURRENT_MYSQL_VERSIONS[0]}
+    export MYSQL_VERSION=${MYSQL_VERSION:=${CURRENT_MYSQL_VERSIONS[0]}}
 
     # If set to true, the database will be reset at entry. Works for Postgres and MySQL
     export DB_RESET=${DB_RESET:="false"}