diff --git a/airflow/configuration.py b/airflow/configuration.py index e120b26012200..b9bc0aa4ca933 100644 --- a/airflow/configuration.py +++ b/airflow/configuration.py @@ -32,7 +32,7 @@ # Ignored Mypy on configparser because it thinks the configparser module has no _UNSET attribute from configparser import _UNSET, ConfigParser, NoOptionError, NoSectionError # type: ignore from json.decoder import JSONDecodeError -from typing import Any, Dict, List, Optional, Union +from typing import Any, Dict, List, Optional, Tuple, Union from airflow.exceptions import AirflowConfigException from airflow.secrets import DEFAULT_SECRETS_SEARCH_PATH, BaseSecretsBackend @@ -47,6 +47,15 @@ warnings.filterwarnings(action='default', category=DeprecationWarning, module='airflow') warnings.filterwarnings(action='default', category=PendingDeprecationWarning, module='airflow') +_SQLITE3_VERSION_PATTERN = re.compile(r"(?P^\d+(?:\.\d+)*)\D?.*$") + + +def _parse_sqlite_version(s: str) -> Tuple[int, ...]: + match = _SQLITE3_VERSION_PATTERN.match(s) + if match is None: + return () + return tuple(int(p) for p in match.group("version").split(".")) + def expand_env_var(env_var): """ @@ -267,15 +276,15 @@ def _validate_config_dependencies(self): raise AirflowConfigException(f"error: cannot use sqlite with the {self.get('core', 'executor')}") if is_sqlite: import sqlite3 - from distutils.version import StrictVersion from airflow.utils.docs import get_docs_url # Some of the features in storing rendered fields require sqlite version >= 3.15.0 - min_sqlite_version = '3.15.0' - if StrictVersion(sqlite3.sqlite_version) < StrictVersion(min_sqlite_version): + min_sqlite_version = (3, 15, 0) + if _parse_sqlite_version(sqlite3.sqlite_version) < min_sqlite_version: + min_sqlite_version_str = ".".join(str(s) for s in min_sqlite_version) raise AirflowConfigException( - f"error: sqlite C library version too old (< {min_sqlite_version}). " + f"error: sqlite C library version too old (< {min_sqlite_version_str}). " f"See {get_docs_url('howto/set-up-database.html#setting-up-a-sqlite-database')}" ) diff --git a/airflow/operators/sql.py b/airflow/operators/sql.py index 3eacc1dc1574f..f03442fa6ee37 100644 --- a/airflow/operators/sql.py +++ b/airflow/operators/sql.py @@ -15,7 +15,6 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. -from distutils.util import strtobool from typing import Any, Dict, Iterable, List, Mapping, Optional, SupportsAbs, Union from airflow.compat.functools import cached_property @@ -25,6 +24,19 @@ from airflow.models import BaseOperator, SkipMixin +def parse_boolean(val: str) -> Union[str, bool]: + """Try to parse a string into boolean. + + Raises ValueError if the input is not a valid true- or false-like string value. + """ + val = val.lower() + if val in ('y', 'yes', 't', 'true', 'on', '1'): + return True + if val in ('n', 'no', 'f', 'false', 'off', '0'): + return False + raise ValueError(f"{val!r} is not a boolean-like string value") + + class BaseSQLOperator(BaseOperator): """ This is a base class for generic SQL Operator to get a DB Hook @@ -538,7 +550,7 @@ def execute(self, context: Dict): follow_branch = self.follow_task_ids_if_true elif isinstance(query_result, str): # return result is not Boolean, try to convert from String to Boolean - if bool(strtobool(query_result)): + if parse_boolean(query_result): follow_branch = self.follow_task_ids_if_true elif isinstance(query_result, int): if bool(query_result): diff --git a/airflow/providers/jenkins/hooks/jenkins.py b/airflow/providers/jenkins/hooks/jenkins.py index 534e8390f58df..55782b7b0da43 100644 --- a/airflow/providers/jenkins/hooks/jenkins.py +++ b/airflow/providers/jenkins/hooks/jenkins.py @@ -17,11 +17,10 @@ # under the License. # -from distutils.util import strtobool - import jenkins from airflow.hooks.base import BaseHook +from airflow.utils.strings import to_boolean class JenkinsHook(BaseHook): @@ -38,11 +37,7 @@ def __init__(self, conn_id: str = default_conn_name) -> None: self.connection = connection connection_prefix = 'http' # connection.extra contains info about using https (true) or http (false) - if connection.extra is None or connection.extra == '': - connection.extra = 'false' - # set a default value to connection.extra - # to avoid rising ValueError in strtobool - if strtobool(connection.extra): + if to_boolean(connection.extra): connection_prefix = 'https' url = f'{connection_prefix}://{connection.host}:{connection.port}' self.log.info('Trying to connect to %s', url) diff --git a/airflow/providers/tableau/hooks/tableau.py b/airflow/providers/tableau/hooks/tableau.py index 8ec18ec86ca7e..1b9e41fc50c5f 100644 --- a/airflow/providers/tableau/hooks/tableau.py +++ b/airflow/providers/tableau/hooks/tableau.py @@ -16,9 +16,8 @@ # under the License. import time import warnings -from distutils.util import strtobool from enum import Enum -from typing import Any, Optional +from typing import Any, Optional, Union from tableauserverclient import Pager, PersonalAccessTokenAuth, Server, TableauAuth from tableauserverclient.server import Auth @@ -27,6 +26,19 @@ from airflow.hooks.base import BaseHook +def parse_boolean(val: str) -> Union[str, bool]: + """Try to parse a string into boolean. + + The string is returned as-is if it does not look like a boolean value. + """ + val = val.lower() + if val in ('y', 'yes', 't', 'true', 'on', '1'): + return True + if val in ('n', 'no', 'f', 'false', 'off', '0'): + return False + return val + + class TableauJobFailedException(AirflowException): """An exception that indicates that a Job failed to complete.""" @@ -71,12 +83,9 @@ def __init__(self, site_id: Optional[str] = None, tableau_conn_id: str = default self.conn = self.get_connection(self.tableau_conn_id) self.site_id = site_id or self.conn.extra_dejson.get('site_id', '') self.server = Server(self.conn.host) - verify = self.conn.extra_dejson.get('verify', True) + verify: Any = self.conn.extra_dejson.get('verify', True) if isinstance(verify, str): - try: - verify = bool(strtobool(verify)) - except ValueError: - pass + verify = parse_boolean(verify) self.server.add_http_options( options_dict={'verify': verify, 'cert': self.conn.extra_dejson.get('cert', None)} ) diff --git a/airflow/utils/strings.py b/airflow/utils/strings.py index c1823ff919a5f..72c7fa687baf0 100644 --- a/airflow/utils/strings.py +++ b/airflow/utils/strings.py @@ -18,6 +18,7 @@ import string from random import choice +from typing import Optional def get_random_string(length=8, choices=string.ascii_letters + string.digits): @@ -25,6 +26,13 @@ def get_random_string(length=8, choices=string.ascii_letters + string.digits): return ''.join(choice(choices) for _ in range(length)) -def to_boolean(astring): - """Convert a string to a boolean""" - return False if astring is None else astring.lower() in ['true', 't', 'y', 'yes', '1'] +TRUE_LIKE_VALUES = {"on", "t", "true", "y", "yes", "1"} + + +def to_boolean(astring: Optional[str]) -> bool: + """Convert a string to a boolean.""" + if astring is None: + return False + if astring.lower() in TRUE_LIKE_VALUES: + return True + return False diff --git a/dev/provider_packages/remove_old_releases.py b/dev/provider_packages/remove_old_releases.py index 2c8c62d6feb7a..63fa73d311e78 100644 --- a/dev/provider_packages/remove_old_releases.py +++ b/dev/provider_packages/remove_old_releases.py @@ -24,19 +24,21 @@ """ import argparse import glob +import operator import os import subprocess from collections import defaultdict -from distutils.version import LooseVersion from typing import Dict, List, NamedTuple +from packaging.version import Version + class VersionedFile(NamedTuple): base: str version: str suffix: str type: str - comparable_version: LooseVersion + comparable_version: Version def split_version_and_suffix(file_name: str, suffix: str) -> VersionedFile: @@ -47,7 +49,7 @@ def split_version_and_suffix(file_name: str, suffix: str) -> VersionedFile: version=version, suffix=suffix, type=no_version_file + "-" + suffix, - comparable_version=LooseVersion(version), + comparable_version=Version(version), ) @@ -60,7 +62,7 @@ def process_all_files(directory: str, suffix: str, execute: bool): package_types_dicts[versioned_file.type].append(versioned_file) for package_types in package_types_dicts.values(): - package_types.sort(key=lambda x: x.comparable_version) + package_types.sort(key=operator.attrgetter("comparable_version")) for package_types in package_types_dicts.values(): if len(package_types) == 1: diff --git a/docs/exts/sphinx_script_update.py b/docs/exts/sphinx_script_update.py index 2fa3b527ae5bb..1926727d4b100 100644 --- a/docs/exts/sphinx_script_update.py +++ b/docs/exts/sphinx_script_update.py @@ -17,9 +17,9 @@ import hashlib import json import os +import shutil import sys import tempfile -from distutils.file_util import copy_file from functools import lru_cache from typing import Dict @@ -30,6 +30,11 @@ log = logging.getLogger(__name__) +def _copy_file(src: str, dst: str) -> None: + log.info("Copying %s -> %s", src, dst) + shutil.copy2(src, dst, follow_symlinks=False) + + def _gethash(string: str): hash_object = hashlib.sha256(string.encode()) return hash_object.hexdigest() @@ -107,7 +112,7 @@ def build_finished(app, exception): output_filename = "script.js" cache_filepath = fetch_and_cache(script_url, output_filename) - copy_file(cache_filepath, os.path.join(app.builder.outdir, '_static', "redoc.js")) + _copy_file(cache_filepath, os.path.join(app.builder.outdir, '_static', "redoc.js")) def setup(app): diff --git a/setup.py b/setup.py index c3014fd2a4e52..659b8818f652a 100644 --- a/setup.py +++ b/setup.py @@ -23,7 +23,6 @@ import sys import unittest from copy import deepcopy -from distutils import log from os.path import dirname, relpath from textwrap import wrap from typing import Dict, List @@ -32,6 +31,10 @@ from setuptools.command.develop import develop as develop_orig from setuptools.command.install import install as install_orig +# Setuptools patches this import to point to a vendored copy instead of the +# stdlib, which is deprecated in Python 3.10 and will be removed in 3.12. +from distutils import log # isort: skip + # Controls whether providers are installed from packages or directly from sources # It is turned on by default in case of development environments such as Breeze # And it is particularly useful when you add a new provider and there is no