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

AIP-58: Add object storage backend for xcom #37058

Merged
merged 20 commits into from
Feb 3, 2024
Merged
Show file tree
Hide file tree
Changes from 13 commits
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
26 changes: 26 additions & 0 deletions airflow/config_templates/config.yml
Original file line number Diff line number Diff line change
Expand Up @@ -375,6 +375,32 @@ core:
type: string
example: "path.to.CustomXCom"
default: "airflow.models.xcom.BaseXCom"
xcom_objectstorage_path:
bolkedebruin marked this conversation as resolved.
Show resolved Hide resolved
bolkedebruin marked this conversation as resolved.
Show resolved Hide resolved
description: |
Path to a location on object storage where XComs can be stored in url format.
version_added: 2.9.0
type: string
example: "s3://conn_id@bucket/path"
default: ""
xcom_objectstorage_threshold:
bolkedebruin marked this conversation as resolved.
Show resolved Hide resolved
description: |
Threshold in bytes for storing XComs in object storage. -1 means always store in the
database. 0 means always store in object storage. Any positive number means
it will be stored in object storage if the size of the value is greater than the threshold.
version_added: 2.9.0
type: integer
example: "1000000"
default: "-1"
xcom_objectstorage_compression:
description: |
Compression algorithm to use when storing XComs in object storage. Supported algorithms
are a.o.: snappy, zip, gzip, bz2, and lzma. If not specified, no compression will be used.
Note that the compression algorithm must be available in the Python installation (e.g.
python-snappy for snappy). Zip, gz, bz2 are available by default.
version_added: 2.9.0
type: string
example: "gz"
default: ""
lazy_load_plugins:
description: |
By default Airflow plugins are lazily-loaded (only loaded when required). Set it to ``False``,
Expand Down
18 changes: 16 additions & 2 deletions airflow/models/xcom.py
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
from typing import TYPE_CHECKING, Any, Generator, Iterable, cast, overload

import attr
from deprecated import deprecated
from sqlalchemy import (
Column,
ForeignKeyConstraint,
Expand Down Expand Up @@ -368,6 +369,7 @@ def get_one(
@staticmethod
@provide_session
@internal_api_call
@deprecated
def get_one(
execution_date: datetime.datetime | None = None,
key: str | None = None,
Expand Down Expand Up @@ -418,7 +420,7 @@ def get_one(

result = query.with_entities(BaseXCom.value).first()
if result:
return BaseXCom.deserialize_value(result)
return XCom.deserialize_value(result)
return None

@overload
Expand Down Expand Up @@ -556,9 +558,15 @@ def delete(cls, xcoms: XCom | Iterable[XCom], session: Session) -> None:
for xcom in xcoms:
if not isinstance(xcom, XCom):
raise TypeError(f"Expected XCom; received {xcom.__class__.__name__}")
XCom.purge(xcom, session)
session.delete(xcom)
session.commit()

@staticmethod
def purge(xcom: XCom, session: Session) -> None:
"""Purge an XCom entry from underlying storage implementations."""
pass

@overload
@staticmethod
@internal_api_call
Expand Down Expand Up @@ -641,7 +649,13 @@ def clear(
query = session.query(BaseXCom).filter_by(dag_id=dag_id, task_id=task_id, run_id=run_id)
if map_index is not None:
query = query.filter_by(map_index=map_index)
query.delete()

for xcom in query:
# print(f"Clearing XCOM {xcom} with value {xcom.value}")
XCom.purge(xcom, session)
session.delete(xcom)

session.commit()
bolkedebruin marked this conversation as resolved.
Show resolved Hide resolved

@staticmethod
def serialize_value(
Expand Down
7 changes: 7 additions & 0 deletions airflow/provider.yaml.schema.json
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,13 @@
"type": "string"
}
},
"xcom": {
"type": "array",
"description": "XCom module names",
"items": {
"type": "string"
}
},
"transfers": {
"type": "array",
"items": {
Expand Down
3 changes: 3 additions & 0 deletions airflow/providers/common/io/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -43,3 +43,6 @@ operators:
- integration-name: Common IO
python-modules:
- airflow.providers.common.io.operators.file_transfer

xcom:
- airflow.providers.common.io.xcom.backend
37 changes: 37 additions & 0 deletions airflow/providers/common/io/xcom/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
# 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.
from __future__ import annotations

import packaging.version

__all__ = ["__version__"]
bolkedebruin marked this conversation as resolved.
Show resolved Hide resolved

__version__ = "1.3.0"


try:
from airflow import __version__ as airflow_version
except ImportError:
from airflow.version import version as airflow_version

if packaging.version.parse(packaging.version.parse(airflow_version).base_version) < packaging.version.parse(
"2.9.0"
):
raise RuntimeError(
f"The package `apache-airflow-providers-common-io:{__version__}` needs Apache Airflow 2.9.0+"
bolkedebruin marked this conversation as resolved.
Show resolved Hide resolved
f"for XCom Object Storage support."
)
164 changes: 164 additions & 0 deletions airflow/providers/common/io/xcom/backend.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,164 @@
# 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.
from __future__ import annotations

import json
import uuid
from typing import TYPE_CHECKING, Any, TypeVar
from urllib.parse import urlsplit

import fsspec.utils

from airflow.configuration import conf
from airflow.io.path import ObjectStoragePath
from airflow.models.xcom import BaseXCom
from airflow.utils.json import XComDecoder, XComEncoder

if TYPE_CHECKING:
from sqlalchemy.orm import Session

from airflow.models import XCom

T = TypeVar("T")


def _is_relative_to(o: ObjectStoragePath, other: ObjectStoragePath) -> bool:
"""This is a port of the pathlib.Path.is_relative_to method. It is not available in python 3.8."""
if hasattr(o, "is_relative_to"):
return o.is_relative_to(other)

try:
o.relative_to(other)
return True
except ValueError:
return False


def _get_compression_suffix(compression: str) -> str:
"""This returns the compression suffix for the given compression.

:raises ValueError: if the compression is not supported
"""
for suffix, c in fsspec.utils.compressions.items():
if c == compression:
return suffix

raise ValueError(f"Compression {compression} is not supported. Make sure it is installed.")


class XComObjectStoreBackend(BaseXCom):
"""XCom backend that stores data in an object store or database depending on the size of the data.

If the value is larger than the configured threshold, it will be stored in an object store.
Otherwise, it will be stored in the database. If it is stored in an object store, the path
to the object in the store will be returned and saved in the database (by BaseXCom). Otherwise, the value
itself will be returned and thus saved in the database.
"""

@staticmethod
def _get_key(data: str) -> str:
"""This gets the key from the url and normalizes it to be relative to the configured path.

:raises ValueError: if the key is not relative to the configured path
:raises TypeError: if the url is not a valid url or cannot be split
"""
path = conf.get("core", "xcom_objectstore_path", fallback="")
p = ObjectStoragePath(path)

try:
url = urlsplit(data)
except AttributeError:
raise TypeError(f"Not a valid url: {data}")

if url.scheme:
k = ObjectStoragePath(data)

if _is_relative_to(k, p) is False:
raise ValueError(f"Invalid key: {data}")
else:
return data.replace(path, "", 1).lstrip("/")

raise ValueError(f"Not a valid url: {data}")

@staticmethod
def serialize_value(
value: T,
*,
key: str | None = None,
task_id: str | None = None,
dag_id: str | None = None,
run_id: str | None = None,
map_index: int | None = None,
) -> bytes | str:
s_val = json.dumps(value, cls=XComEncoder).encode("utf-8")
path = conf.get("core", "xcom_objectstore_path", fallback="")
compression = conf.get("core", "xcom_objectstore_compression", fallback=None)

if compression:
suffix = "." + _get_compression_suffix(compression)
else:
suffix = ""

threshold = conf.getint("core", "xcom_objectstore_threshold", fallback=-1)

if path and -1 < threshold < len(s_val):
# safeguard against collisions
while True:
p = ObjectStoragePath(path) / f"{dag_id}/{run_id}/{task_id}/{str(uuid.uuid4())}{suffix}"
if not p.exists():
break

if not p.parent.exists():
p.parent.mkdir(parents=True, exist_ok=True)

with p.open("wb", compression=compression) as f:
f.write(s_val)

return BaseXCom.serialize_value(str(p))
else:
return s_val
bolkedebruin marked this conversation as resolved.
Show resolved Hide resolved

@staticmethod
def deserialize_value(
result: XCom,
) -> Any:
"""Deserializes the value from the database or object storage.

Compression is inferred from the file extension.
"""
data = BaseXCom.deserialize_value(result)
path = conf.get("core", "xcom_objectstore_path", fallback="")

try:
p = ObjectStoragePath(path) / XComObjectStoreBackend._get_key(data)
return json.load(p.open("rb", compression="infer"), cls=XComDecoder)
except TypeError:
return data
except ValueError:
return data

@staticmethod
def purge(xcom: XCom, session: Session) -> None:
path = conf.get("core", "xcom_objectstore_path", fallback="")
if isinstance(xcom.value, str):
try:
p = ObjectStoragePath(path) / XComObjectStoreBackend._get_key(xcom.value)
p.unlink(missing_ok=True)
except TypeError:
pass
except ValueError:
pass
25 changes: 25 additions & 0 deletions docs/apache-airflow/core-concepts/xcoms.rst
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,31 @@ XComs are a relative of :doc:`variables`, with the main difference being that XC

If the first task run is not succeeded then on every retry task XComs will be cleared to make the task run idempotent.


bolkedebruin marked this conversation as resolved.
Show resolved Hide resolved
Object Storage XCom Backend
---------------------------

The default XCom backend is the :class:`~airflow.models.xcom.BaseXCom` class, which stores XComs in the Airflow database. This is fine for small values, but can be problematic for large values, or for large numbers of XComs.

To enable storing XComs in an object store, you can set the ``xcom_backend`` configuration option to ``airflow.providers.common.io.xcom.backend.XComObjectStoreBackend``. You will also need to set ``xcom_objectstorage_path`` to the desired location. The connection
id is obtained from the user part of the url the you will provide, e.g. ``xcom_objectstorage_path = s3://conn_id@mybucket/key``. Furthermore, ``xcom_objectstorage_threshold`` is required
to be something larger than -1. Any object smaller than the threshold in bytes will be stored in the database and anything larger will be be
put in object storage. This will allow a hybrid setup. If an xcom is stored on object storage a reference will be
saved in the database. Finally, you can set ``xcom_objectstorage_compression`` to fsspec supported compression methods like ``zip`` or ``snappy`` to
compress the data before storing it in object storage.

So for example the following configuration will store anything above 1MB in S3 and will compress it using gzip::

xcom_backend = airflow.providers.common.io.xcom.backend.XComObjectStoreBackend
xcom_objectstorage_path = s3://conn_id@mybucket/key
xcom_objectstorage_threshold = 1048576
xcom_objectstorage_compression = gzip

.. note::

Compression requires the support for it is installed in your python environment. For example, to use ``snappy`` compression, you need to install ``python-snappy``. Zip, gzip and bz2 work out of the box.


Custom XCom Backends
--------------------

Expand Down
1 change: 1 addition & 0 deletions docs/spelling_wordlist.txt
Original file line number Diff line number Diff line change
Expand Up @@ -1079,6 +1079,7 @@ Oauth
oauth
Oauthlib
objectORfile
objectstorage
observability
od
odbc
Expand Down
4 changes: 3 additions & 1 deletion tests/models/test_xcom.py
Original file line number Diff line number Diff line change
Expand Up @@ -569,7 +569,8 @@ def setup_for_xcom_clear(self, task_instance, push_simple_json_xcom):
push_simple_json_xcom(ti=task_instance, key="xcom_1", value={"key": "value"})

@pytest.mark.usefixtures("setup_for_xcom_clear")
def test_xcom_clear(self, session, task_instance):
@mock.patch("airflow.models.xcom.XCom.purge")
def test_xcom_clear(self, mock_purge, session, task_instance):
assert session.query(XCom).count() == 1
XCom.clear(
dag_id=task_instance.dag_id,
Expand All @@ -578,6 +579,7 @@ def test_xcom_clear(self, session, task_instance):
session=session,
)
assert session.query(XCom).count() == 0
assert mock_purge.call_count == 1

@pytest.mark.usefixtures("setup_for_xcom_clear")
def test_xcom_clear_with_execution_date(self, session, task_instance):
Expand Down
16 changes: 16 additions & 0 deletions tests/providers/common/io/xcom/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.
Loading
Loading