jscheffl commented on code in PR #37058:
URL: https://github.com/apache/airflow/pull/37058#discussion_r1468932566


##########
airflow/io/xcom.py:
##########
@@ -0,0 +1,123 @@
+# 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
+
+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")
+
+
+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)
+
+        url = urlsplit(data)
+        if url.scheme:
+            k = ObjectStoragePath(data)
+
+            if k.is_relative_to(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)
+        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"{run_id}/{task_id}/{str(uuid.uuid4())}"
+                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
+
+    @staticmethod
+    def deserialize_value(
+        result: XCom,
+    ) -> Any:
+        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"), cls=XComDecoder)

Review Comment:
   Two things I am not sure:
   
   1. Do we need to pass the type of compression here - or is this 
auto-detected while reading?
   2. If compression is passed, what happens if the user changed the 
compression configuration after some time. Is the compression type be kept and 
auto-detected?



##########
airflow/io/xcom.py:
##########
@@ -0,0 +1,123 @@
+# 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
+
+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")
+
+
+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)
+
+        url = urlsplit(data)
+        if url.scheme:
+            k = ObjectStoragePath(data)
+
+            if k.is_relative_to(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)
+        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"{run_id}/{task_id}/{str(uuid.uuid4())}"

Review Comment:
   I am not sure but would it make sense to have DAG id also in the path? Then 
if a DAG is deleted, housekeeping might be easier? But you can convinec me of 
anything good as well.
   ```suggestion
                   p = ObjectStoragePath(path) / 
f"{dag_id}/{run_id}/{task_id}/{str(uuid.uuid4())}"
   ```



##########
docs/apache-airflow/core-concepts/xcoms.rst:
##########
@@ -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.
 
+
+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.io.xcom.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 snappy::
+
+      xcom_backend = airflow.io.xcom.XComObjectStoreBackend
+      xcom_objectstorage_path = s3://conn_id@mybucket/key
+      xcom_objectstorage_threshold = 1048576
+      xcom_objectstorage_compression = snappy
+
+.. 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``.

Review Comment:
   Can you add/use an example that is available in Airflow production docker 
per default? Else many users will directly get a bleedy nose if they want to 
test "cool features" broken by default.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@airflow.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to