Skip to content

Commit 00af5c0

Browse files
westonkleladkal
andauthored
Replace urlparse with urlsplit (#27389)
* Replace urlparse with urlsplit in s3 files Co-authored-by: eladkal <45845474+eladkal@users.noreply.github.com>
1 parent 93699a3 commit 00af5c0

File tree

27 files changed

+64
-69
lines changed

27 files changed

+64
-69
lines changed

β€Žairflow/cli/commands/connection_command.py

Lines changed: 4 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@
2424
import warnings
2525
from pathlib import Path
2626
from typing import Any
27-
from urllib.parse import urlparse, urlunparse
27+
from urllib.parse import urlsplit, urlunsplit
2828

2929
from sqlalchemy.orm import exc
3030

@@ -133,12 +133,12 @@ def _is_stdout(fileio: io.TextIOWrapper) -> bool:
133133

134134
def _valid_uri(uri: str) -> bool:
135135
"""Check if a URI is valid, by checking if both scheme and netloc are available"""
136-
uri_parts = urlparse(uri)
136+
uri_parts = urlsplit(uri)
137137
return uri_parts.scheme != "" and uri_parts.netloc != ""
138138

139139

140140
@cache
141-
def _get_connection_types():
141+
def _get_connection_types() -> list[str]:
142142
"""Returns connection types available."""
143143
_connection_types = ["fs", "mesos_framework-id", "email", "generic"]
144144
providers_manager = ProvidersManager()
@@ -148,10 +148,6 @@ def _get_connection_types():
148148
return _connection_types
149149

150150

151-
def _valid_conn_type(conn_type: str) -> bool:
152-
return conn_type in _get_connection_types()
153-
154-
155151
def connections_export(args):
156152
"""Exports all connections to a file"""
157153
file_formats = [".yaml", ".json", ".env"]
@@ -269,15 +265,14 @@ def connections_add(args):
269265
msg = msg.format(
270266
conn_id=new_conn.conn_id,
271267
uri=args.conn_uri
272-
or urlunparse(
268+
or urlunsplit(
273269
(
274270
new_conn.conn_type,
275271
f"{new_conn.login or ''}:{'******' if new_conn.password else ''}"
276272
f"@{new_conn.host or ''}:{new_conn.port or ''}",
277273
new_conn.schema or "",
278274
"",
279275
"",
280-
"",
281276
)
282277
),
283278
)

β€Žairflow/config_templates/airflow_local_settings.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@
2121
import os
2222
from pathlib import Path
2323
from typing import Any
24-
from urllib.parse import urlparse
24+
from urllib.parse import urlsplit
2525

2626
from airflow.configuration import conf
2727
from airflow.exceptions import AirflowException
@@ -221,7 +221,7 @@
221221

222222
DEFAULT_LOGGING_CONFIG["handlers"].update(S3_REMOTE_HANDLERS)
223223
elif REMOTE_BASE_LOG_FOLDER.startswith("cloudwatch://"):
224-
url_parts = urlparse(REMOTE_BASE_LOG_FOLDER)
224+
url_parts = urlsplit(REMOTE_BASE_LOG_FOLDER)
225225
CLOUDWATCH_REMOTE_HANDLERS: dict[str, dict[str, str | None]] = {
226226
"task": {
227227
"class": "airflow.providers.amazon.aws.log.cloudwatch_task_handler.CloudwatchTaskHandler",
@@ -264,7 +264,7 @@
264264
elif REMOTE_BASE_LOG_FOLDER.startswith("stackdriver://"):
265265
key_path = conf.get_mandatory_value("logging", "GOOGLE_KEY_PATH", fallback=None)
266266
# stackdriver:///airflow-tasks => airflow-tasks
267-
log_name = urlparse(REMOTE_BASE_LOG_FOLDER).path[1:]
267+
log_name = urlsplit(REMOTE_BASE_LOG_FOLDER).path[1:]
268268
STACKDRIVER_REMOTE_HANDLERS = {
269269
"task": {
270270
"class": "airflow.providers.google.cloud.log.stackdriver_task_handler.StackdriverTaskHandler",

β€Žairflow/configuration.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@
3737
from json.decoder import JSONDecodeError
3838
from re import Pattern
3939
from typing import IO, Any, Dict, Iterable, Tuple, Union
40-
from urllib.parse import urlparse
40+
from urllib.parse import urlsplit
4141

4242
from typing_extensions import overload
4343

@@ -403,7 +403,7 @@ def _upgrade_postgres_metastore_conn(self):
403403
old_value = self.get(section, key)
404404
bad_schemes = ["postgres+psycopg2", "postgres"]
405405
good_scheme = "postgresql"
406-
parsed = urlparse(old_value)
406+
parsed = urlsplit(old_value)
407407
if parsed.scheme in bad_schemes:
408408
warnings.warn(
409409
f"Bad scheme in Airflow configuration core > sql_alchemy_conn: `{parsed.scheme}`. "

β€Žairflow/datasets/__init__.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717
from __future__ import annotations
1818

1919
from typing import Any
20-
from urllib.parse import urlparse
20+
from urllib.parse import urlsplit
2121

2222
import attr
2323

@@ -37,6 +37,6 @@ def _check_uri(self, attr, uri: str):
3737
uri.encode("ascii")
3838
except UnicodeEncodeError:
3939
raise ValueError(f"{attr.name!r} must be ascii")
40-
parsed = urlparse(uri)
40+
parsed = urlsplit(uri)
4141
if parsed.scheme and parsed.scheme.lower() == "airflow":
4242
raise ValueError(f"{attr.name!r} scheme `airflow` is reserved")

β€Žairflow/models/connection.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@
2121
import logging
2222
import warnings
2323
from json import JSONDecodeError
24-
from urllib.parse import parse_qsl, quote, unquote, urlencode, urlparse
24+
from urllib.parse import parse_qsl, quote, unquote, urlencode, urlsplit
2525

2626
from sqlalchemy import Boolean, Column, Integer, String, Text
2727
from sqlalchemy.ext.declarative import declared_attr
@@ -188,7 +188,7 @@ def _normalize_conn_type(conn_type):
188188
return conn_type
189189

190190
def _parse_from_uri(self, uri: str):
191-
uri_parts = urlparse(uri)
191+
uri_parts = urlsplit(uri)
192192
conn_type = uri_parts.scheme
193193
self.conn_type = self._normalize_conn_type(conn_type)
194194
self.host = _parse_netloc_to_hostname(uri_parts)

β€Žairflow/models/dataset.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717
# under the License.
1818
from __future__ import annotations
1919

20-
from urllib.parse import urlparse
20+
from urllib.parse import urlsplit
2121

2222
import sqlalchemy_jsonfield
2323
from sqlalchemy import (
@@ -83,7 +83,7 @@ def __init__(self, uri: str, **kwargs):
8383
uri.encode("ascii")
8484
except UnicodeEncodeError:
8585
raise ValueError("URI must be ascii")
86-
parsed = urlparse(uri)
86+
parsed = urlsplit(uri)
8787
if parsed.scheme and parsed.scheme.lower() == "airflow":
8888
raise ValueError("Scheme `airflow` is reserved.")
8989
super().__init__(uri=uri, **kwargs)

β€Žairflow/providers/alibaba/cloud/hooks/oss.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@
2020
from functools import wraps
2121
from inspect import signature
2222
from typing import TYPE_CHECKING, Callable, TypeVar, cast
23-
from urllib.parse import urlparse
23+
from urllib.parse import urlsplit
2424

2525
import oss2
2626
from oss2.exceptions import ClientError
@@ -108,7 +108,7 @@ def parse_oss_url(ossurl: str) -> tuple:
108108
:param ossurl: The OSS Url to parse.
109109
:return: the parsed bucket name and key
110110
"""
111-
parsed_url = urlparse(ossurl)
111+
parsed_url = urlsplit(ossurl)
112112

113113
if not parsed_url.netloc:
114114
raise AirflowException(f'Please provide a bucket_name instead of "{ossurl}"')

β€Žairflow/providers/alibaba/cloud/sensors/oss_key.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@
1818
from __future__ import annotations
1919

2020
from typing import TYPE_CHECKING, Sequence
21-
from urllib.parse import urlparse
21+
from urllib.parse import urlsplit
2222

2323
from airflow.compat.functools import cached_property
2424
from airflow.exceptions import AirflowException
@@ -69,13 +69,13 @@ def poke(self, context: Context):
6969
@returns True if the object exists, False otherwise
7070
"""
7171
if self.bucket_name is None:
72-
parsed_url = urlparse(self.bucket_key)
72+
parsed_url = urlsplit(self.bucket_key)
7373
if parsed_url.netloc == "":
7474
raise AirflowException("If key is a relative path from root, please provide a bucket_name")
7575
self.bucket_name = parsed_url.netloc
7676
self.bucket_key = parsed_url.path.lstrip("/")
7777
else:
78-
parsed_url = urlparse(self.bucket_key)
78+
parsed_url = urlsplit(self.bucket_key)
7979
if parsed_url.scheme != "" or parsed_url.netloc != "":
8080
raise AirflowException(
8181
"If bucket_name is provided, bucket_key"

β€Žairflow/providers/amazon/aws/hooks/s3.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@
3131
from pathlib import Path
3232
from tempfile import NamedTemporaryFile, gettempdir
3333
from typing import Any, Callable, TypeVar, cast
34-
from urllib.parse import urlparse
34+
from urllib.parse import urlsplit
3535
from uuid import uuid4
3636

3737
from boto3.s3.transfer import S3Transfer, TransferConfig
@@ -153,7 +153,7 @@ def parse_s3_url(s3url: str) -> tuple[str, str]:
153153
"""
154154
format = s3url.split("//")
155155
if format[0].lower() == "s3:":
156-
parsed_url = urlparse(s3url)
156+
parsed_url = urlsplit(s3url)
157157
if not parsed_url.netloc:
158158
raise AirflowException(f'Please provide a bucket name using a valid format: "{s3url}"')
159159

@@ -190,7 +190,7 @@ def get_s3_bucket_key(
190190
if bucket is None:
191191
return S3Hook.parse_s3_url(key)
192192

193-
parsed_url = urlparse(key)
193+
parsed_url = urlsplit(key)
194194
if parsed_url.scheme != "" or parsed_url.netloc != "":
195195
raise TypeError(
196196
f"If `{bucket_param_name}` is provided, {key_param_name} should be a relative path "

β€Žairflow/providers/amazon/aws/transfers/s3_to_sftp.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@
2020
import warnings
2121
from tempfile import NamedTemporaryFile
2222
from typing import TYPE_CHECKING, Sequence
23-
from urllib.parse import urlparse
23+
from urllib.parse import urlsplit
2424

2525
from airflow.models import BaseOperator
2626
from airflow.providers.amazon.aws.hooks.s3 import S3Hook
@@ -80,7 +80,7 @@ def __init__(
8080
@staticmethod
8181
def get_s3_key(s3_key: str) -> str:
8282
"""This parses the correct format for S3 keys regardless of how the S3 url is passed."""
83-
parsed_s3_key = urlparse(s3_key)
83+
parsed_s3_key = urlsplit(s3_key)
8484
return parsed_s3_key.path.lstrip("/")
8585

8686
def execute(self, context: Context) -> None:

0 commit comments

Comments
 (0)