From 99d324a1e6bd7a08c53d4be0a23645696249234e Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Mon, 1 Dec 2025 10:10:23 -0800 Subject: [PATCH 01/13] Add support for Python --- amber/operator-requirements.txt | 1 + .../architecture/packaging/output_manager.py | 12 +- .../python/core/models/schema/__init__.py | 2 + .../core/models/schema/attribute_type.py | 6 + .../python/core/models/schema/big_object.py | 96 +++++++ .../main/python/core/models/schema/schema.py | 36 ++- .../core/models/schema/test_big_object.py | 90 +++++++ .../python/core/models/schema/test_schema.py | 63 +++++ .../src/main/python/core/models/test_tuple.py | 99 +++++++ amber/src/main/python/core/models/tuple.py | 12 + .../python/core/storage/document_factory.py | 4 +- .../core/storage/iceberg/iceberg_utils.py | 136 +++++++++- .../storage/iceberg/test_iceberg_document.py | 4 + .../iceberg/test_iceberg_utils_big_object.py | 228 ++++++++++++++++ .../python/core/storage/storage_config.py | 19 +- amber/src/main/python/pytexera/__init__.py | 6 + .../storage/big_object_input_stream.py | 121 +++++++++ .../pytexera/storage/big_object_manager.py | 86 ++++++ .../storage/big_object_output_stream.py | 244 +++++++++++++++++ .../storage/test_big_object_input_stream.py | 240 +++++++++++++++++ .../storage/test_big_object_manager.py | 150 +++++++++++ .../storage/test_big_object_output_stream.py | 254 ++++++++++++++++++ .../main/python/texera_run_python_worker.py | 8 + .../pythonworker/PythonWorkflowWorker.scala | 6 +- build.sbt | 14 +- common/workflow-core/build.sbt | 17 ++ .../apache/texera/amber/util/ArrowUtils.scala | 49 ++-- .../texera/amber/util/ArrowUtilsSpec.scala | 144 +++++++++- 28 files changed, 2103 insertions(+), 44 deletions(-) create mode 100644 amber/src/main/python/core/models/schema/big_object.py create mode 100644 amber/src/main/python/core/models/schema/test_big_object.py create mode 100644 amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py create mode 100644 amber/src/main/python/pytexera/storage/big_object_input_stream.py create mode 100644 amber/src/main/python/pytexera/storage/big_object_manager.py create mode 100644 amber/src/main/python/pytexera/storage/big_object_output_stream.py create mode 100644 amber/src/main/python/pytexera/storage/test_big_object_input_stream.py create mode 100644 amber/src/main/python/pytexera/storage/test_big_object_manager.py create mode 100644 amber/src/main/python/pytexera/storage/test_big_object_output_stream.py diff --git a/amber/operator-requirements.txt b/amber/operator-requirements.txt index ce114ea8415..3328d152f08 100644 --- a/amber/operator-requirements.txt +++ b/amber/operator-requirements.txt @@ -23,3 +23,4 @@ pybase64==1.3.2 torch==2.8.0 scikit-learn==1.5.0 transformers==4.57.3 +boto3==1.40.53 diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index 4003429baf6..3dadcca7186 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -259,10 +259,20 @@ def emit_state( ) def tuple_to_frame(self, tuples: typing.List[Tuple]) -> DataFrame: + from core.models.schema.big_object import BigObject + return DataFrame( frame=Table.from_pydict( { - name: [t[name] for t in tuples] + name: [ + ( + # Convert BigObject objects to URI strings for Arrow serialization + t[name].uri + if isinstance(t[name], BigObject) + else t[name] + ) + for t in tuples + ] for name in self.get_port().get_schema().get_attr_names() }, schema=self.get_port().get_schema().as_arrow_schema(), diff --git a/amber/src/main/python/core/models/schema/__init__.py b/amber/src/main/python/core/models/schema/__init__.py index 7c8b57bb316..94ed63a8cf1 100644 --- a/amber/src/main/python/core/models/schema/__init__.py +++ b/amber/src/main/python/core/models/schema/__init__.py @@ -16,12 +16,14 @@ # under the License. from .attribute_type import AttributeType +from .big_object import BigObject from .field import Field from .schema import Schema __all__ = [ "AttributeType", + "BigObject", "Field", "Schema", ] diff --git a/amber/src/main/python/core/models/schema/attribute_type.py b/amber/src/main/python/core/models/schema/attribute_type.py index 72799b015cc..6adbe1458fd 100644 --- a/amber/src/main/python/core/models/schema/attribute_type.py +++ b/amber/src/main/python/core/models/schema/attribute_type.py @@ -20,6 +20,7 @@ from bidict import bidict from enum import Enum from pyarrow import lib +from core.models.schema.big_object import BigObject class AttributeType(Enum): @@ -37,6 +38,7 @@ class AttributeType(Enum): DOUBLE = 5 TIMESTAMP = 6 BINARY = 7 + BIG_OBJECT = 8 RAW_TYPE_MAPPING = bidict( @@ -48,6 +50,7 @@ class AttributeType(Enum): "BOOLEAN": AttributeType.BOOL, "TIMESTAMP": AttributeType.TIMESTAMP, "BINARY": AttributeType.BINARY, + "BIG_OBJECT": AttributeType.BIG_OBJECT, } ) @@ -59,6 +62,7 @@ class AttributeType(Enum): AttributeType.BOOL: pa.bool_(), AttributeType.BINARY: pa.binary(), AttributeType.TIMESTAMP: pa.timestamp("us"), + AttributeType.BIG_OBJECT: pa.string(), # Serialized as URI string } FROM_ARROW_MAPPING = { @@ -83,6 +87,7 @@ class AttributeType(Enum): AttributeType.BOOL: bool, AttributeType.BINARY: bytes, AttributeType.TIMESTAMP: datetime.datetime, + AttributeType.BIG_OBJECT: BigObject, } FROM_PYOBJECT_MAPPING = { @@ -92,4 +97,5 @@ class AttributeType(Enum): bool: AttributeType.BOOL, bytes: AttributeType.BINARY, datetime.datetime: AttributeType.TIMESTAMP, + BigObject: AttributeType.BIG_OBJECT, } diff --git a/amber/src/main/python/core/models/schema/big_object.py b/amber/src/main/python/core/models/schema/big_object.py new file mode 100644 index 00000000000..128a0a8ade4 --- /dev/null +++ b/amber/src/main/python/core/models/schema/big_object.py @@ -0,0 +1,96 @@ +# 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. + +""" +BigObject represents a reference to a large object stored externally (e.g., S3). +This is a schema type class used throughout the system for handling BIG_OBJECT attribute types. +""" + +from typing import Optional +from urllib.parse import urlparse + + +class BigObject: + """ + BigObject represents a reference to a large object stored in S3. + + Each BigObject is identified by an S3 URI (s3://bucket/path/to/object). + BigObjects are automatically tracked and cleaned up when the workflow execution completes. + + Usage: + from pytexera import BigObject, BigObjectInputStream, BigObjectOutputStream + + # Create a new BigObject for writing + big_object = BigObject() + with BigObjectOutputStream(big_object) as out: + out.write(b"data") + # big_object is now ready to be added to tuples + + # Read from an existing BigObject + with BigObjectInputStream(big_object) as stream: + content = stream.read() + + # Create from existing URI (e.g., from deserialization) + big_object = BigObject("s3://bucket/path/to/object") + """ + + def __init__(self, uri: Optional[str] = None): + """ + Create a BigObject. + + Args: + uri: Optional S3 URI in the format s3://bucket/path/to/object. + If None, creates a new BigObject with a unique S3 URI. + + Raises: + ValueError: If URI is provided but doesn't start with "s3://" + """ + if uri is None: + # Lazy import to avoid circular dependencies + from pytexera.storage.big_object_manager import BigObjectManager + + uri = BigObjectManager.create() + + if not uri.startswith("s3://"): + raise ValueError(f"BigObject URI must start with 's3://', got: {uri}") + + self._uri = uri + + @property + def uri(self) -> str: + """Get the S3 URI of this BigObject.""" + return self._uri + + def get_bucket_name(self) -> str: + """Get the S3 bucket name from the URI.""" + return urlparse(self._uri).netloc + + def get_object_key(self) -> str: + """Get the S3 object key (path) from the URI, without leading slash.""" + return urlparse(self._uri).path.lstrip("/") + + def __str__(self) -> str: + return self._uri + + def __repr__(self) -> str: + return f"BigObject('{self._uri}')" + + def __eq__(self, other) -> bool: + return isinstance(other, BigObject) and self._uri == other._uri + + def __hash__(self) -> int: + return hash(self._uri) diff --git a/amber/src/main/python/core/models/schema/schema.py b/amber/src/main/python/core/models/schema/schema.py index 132ca23884c..9780e526e47 100644 --- a/amber/src/main/python/core/models/schema/schema.py +++ b/amber/src/main/python/core/models/schema/schema.py @@ -81,26 +81,46 @@ def _from_raw_schema(self, raw_schema: Mapping[str, str]) -> None: def _from_arrow_schema(self, arrow_schema: pa.Schema) -> None: """ Resets the Schema by converting a pyarrow.Schema. + Checks field metadata to detect BIG_OBJECT types. :param arrow_schema: a pyarrow.Schema. :return: """ self._name_type_mapping = OrderedDict() for attr_name in arrow_schema.names: - arrow_type = arrow_schema.field(attr_name).type # type: ignore - attr_type = FROM_ARROW_MAPPING[arrow_type.id] + field = arrow_schema.field(attr_name) + + # Check metadata for BIG_OBJECT type (can be stored by either Scala ArrowUtils or Python) + is_big_object = ( + field.metadata and field.metadata.get(b"texera_type") == b"BIG_OBJECT" + ) + + attr_type = ( + AttributeType.BIG_OBJECT + if is_big_object + else FROM_ARROW_MAPPING[field.type.id] + ) + self.add(attr_name, attr_type) def as_arrow_schema(self) -> pa.Schema: """ Creates a new pyarrow.Schema according to the current Schema. + Includes metadata for BIG_OBJECT types to preserve type information. :return: pyarrow.Schema """ - return pa.schema( - [ - pa.field(attr_name, TO_ARROW_MAPPING[attr_type]) - for attr_name, attr_type in self._name_type_mapping.items() - ] - ) + fields = [ + pa.field( + attr_name, + TO_ARROW_MAPPING[attr_type], + metadata=( + {b"texera_type": b"BIG_OBJECT"} + if attr_type == AttributeType.BIG_OBJECT + else None + ), + ) + for attr_name, attr_type in self._name_type_mapping.items() + ] + return pa.schema(fields) def get_attr_names(self) -> List[str]: """ diff --git a/amber/src/main/python/core/models/schema/test_big_object.py b/amber/src/main/python/core/models/schema/test_big_object.py new file mode 100644 index 00000000000..a0f48bf1b7b --- /dev/null +++ b/amber/src/main/python/core/models/schema/test_big_object.py @@ -0,0 +1,90 @@ +# 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. + +import pytest +from unittest.mock import patch, MagicMock +from core.models.schema.big_object import BigObject + + +class TestBigObject: + def test_create_with_uri(self): + """Test creating BigObject with a valid S3 URI.""" + uri = "s3://test-bucket/path/to/object" + big_object = BigObject(uri) + assert big_object.uri == uri + assert str(big_object) == uri + assert repr(big_object) == f"BigObject('{uri}')" + + def test_create_without_uri(self): + """Test creating BigObject without URI (should call BigObjectManager.create).""" + with patch( + "pytexera.storage.big_object_manager.BigObjectManager" + ) as mock_manager: + mock_manager.create.return_value = "s3://bucket/objects/123/uuid" + big_object = BigObject() + assert big_object.uri == "s3://bucket/objects/123/uuid" + mock_manager.create.assert_called_once() + + def test_invalid_uri_raises_value_error(self): + """Test that invalid URI (not starting with s3://) raises ValueError.""" + with pytest.raises(ValueError, match="BigObject URI must start with 's3://'"): + BigObject("http://invalid-uri") + + with pytest.raises(ValueError, match="BigObject URI must start with 's3://'"): + BigObject("invalid-uri") + + def test_get_bucket_name(self): + """Test extracting bucket name from URI.""" + big_object = BigObject("s3://my-bucket/path/to/object") + assert big_object.get_bucket_name() == "my-bucket" + + def test_get_object_key(self): + """Test extracting object key from URI.""" + big_object = BigObject("s3://my-bucket/path/to/object") + assert big_object.get_object_key() == "path/to/object" + + def test_get_object_key_with_leading_slash(self): + """Test extracting object key when URI has leading slash.""" + big_object = BigObject("s3://my-bucket/path/to/object") + # urlparse includes leading slash, but get_object_key removes it + assert big_object.get_object_key() == "path/to/object" + + def test_equality(self): + """Test BigObject equality comparison.""" + uri = "s3://bucket/path" + obj1 = BigObject(uri) + obj2 = BigObject(uri) + obj3 = BigObject("s3://bucket/different") + + assert obj1 == obj2 + assert obj1 != obj3 + assert obj1 != "not a BigObject" + + def test_hash(self): + """Test BigObject hashing.""" + uri = "s3://bucket/path" + obj1 = BigObject(uri) + obj2 = BigObject(uri) + + assert hash(obj1) == hash(obj2) + assert hash(obj1) == hash(uri) + + def test_uri_property(self): + """Test URI property access.""" + uri = "s3://test-bucket/test/path" + big_object = BigObject(uri) + assert big_object.uri == uri diff --git a/amber/src/main/python/core/models/schema/test_schema.py b/amber/src/main/python/core/models/schema/test_schema.py index 2da4e70655f..bd844bad51d 100644 --- a/amber/src/main/python/core/models/schema/test_schema.py +++ b/amber/src/main/python/core/models/schema/test_schema.py @@ -89,3 +89,66 @@ def test_convert_from_raw_schema(self, raw_schema, schema): def test_convert_from_arrow_schema(self, arrow_schema, schema): assert schema == Schema(arrow_schema=arrow_schema) assert schema.as_arrow_schema() == arrow_schema + + def test_big_object_in_raw_schema(self): + """Test creating schema with BIG_OBJECT from raw schema.""" + raw_schema = { + "regular_field": "STRING", + "big_object_field": "BIG_OBJECT", + } + schema = Schema(raw_schema=raw_schema) + assert schema.get_attr_type("regular_field") == AttributeType.STRING + assert schema.get_attr_type("big_object_field") == AttributeType.BIG_OBJECT + + def test_big_object_in_arrow_schema_with_metadata(self): + """Test creating schema with BIG_OBJECT from Arrow schema with metadata.""" + arrow_schema = pa.schema( + [ + pa.field("regular_field", pa.string()), + pa.field( + "big_object_field", + pa.string(), + metadata={b"texera_type": b"BIG_OBJECT"}, + ), + ] + ) + schema = Schema(arrow_schema=arrow_schema) + assert schema.get_attr_type("regular_field") == AttributeType.STRING + assert schema.get_attr_type("big_object_field") == AttributeType.BIG_OBJECT + + def test_big_object_as_arrow_schema_includes_metadata(self): + """Test that BIG_OBJECT fields include metadata in Arrow schema.""" + schema = Schema() + schema.add("regular_field", AttributeType.STRING) + schema.add("big_object_field", AttributeType.BIG_OBJECT) + + arrow_schema = schema.as_arrow_schema() + + # Regular field should have no metadata + regular_field = arrow_schema.field("regular_field") + assert ( + regular_field.metadata is None + or b"texera_type" not in regular_field.metadata + ) + + # BIG_OBJECT field should have metadata + big_object_field = arrow_schema.field("big_object_field") + assert big_object_field.metadata is not None + assert big_object_field.metadata.get(b"texera_type") == b"BIG_OBJECT" + assert big_object_field.type == pa.string() # BIG_OBJECT is stored as string + + def test_round_trip_big_object_schema(self): + """Test round-trip conversion of schema with BIG_OBJECT.""" + original_schema = Schema() + original_schema.add("field1", AttributeType.STRING) + original_schema.add("field2", AttributeType.BIG_OBJECT) + original_schema.add("field3", AttributeType.INT) + + # Convert to Arrow and back + arrow_schema = original_schema.as_arrow_schema() + round_trip_schema = Schema(arrow_schema=arrow_schema) + + assert round_trip_schema == original_schema + assert round_trip_schema.get_attr_type("field1") == AttributeType.STRING + assert round_trip_schema.get_attr_type("field2") == AttributeType.BIG_OBJECT + assert round_trip_schema.get_attr_type("field3") == AttributeType.INT diff --git a/amber/src/main/python/core/models/test_tuple.py b/amber/src/main/python/core/models/test_tuple.py index bfce7bb94fe..20f2ac1c522 100644 --- a/amber/src/main/python/core/models/test_tuple.py +++ b/amber/src/main/python/core/models/test_tuple.py @@ -221,3 +221,102 @@ def test_hash(self): schema, ) assert hash(tuple5) == -2099556631 # calculated with Java + + def test_tuple_with_big_object(self): + """Test tuple with BigObject field.""" + from core.models.schema.big_object import BigObject + + schema = Schema( + raw_schema={ + "regular_field": "STRING", + "big_object_field": "BIG_OBJECT", + } + ) + + big_object = BigObject("s3://test-bucket/path/to/object") + tuple_ = Tuple( + { + "regular_field": "test string", + "big_object_field": big_object, + }, + schema=schema, + ) + + assert tuple_["regular_field"] == "test string" + assert tuple_["big_object_field"] == big_object + assert isinstance(tuple_["big_object_field"], BigObject) + assert tuple_["big_object_field"].uri == "s3://test-bucket/path/to/object" + + def test_tuple_from_arrow_with_big_object(self): + """Test creating tuple from Arrow table with BIG_OBJECT metadata.""" + import pyarrow as pa + from core.models.schema.big_object import BigObject + + # Create Arrow schema with BIG_OBJECT metadata + arrow_schema = pa.schema( + [ + pa.field("regular_field", pa.string()), + pa.field( + "big_object_field", + pa.string(), + metadata={b"texera_type": b"BIG_OBJECT"}, + ), + ] + ) + + # Create Arrow table with URI string for big_object_field + arrow_table = pa.Table.from_pydict( + { + "regular_field": ["test"], + "big_object_field": ["s3://test-bucket/path/to/object"], + }, + schema=arrow_schema, + ) + + # Create tuple from Arrow table + tuple_provider = ArrowTableTupleProvider(arrow_table) + tuples = [ + Tuple({name: field_accessor for name in arrow_table.column_names}) + for field_accessor in tuple_provider + ] + + assert len(tuples) == 1 + tuple_ = tuples[0] + assert tuple_["regular_field"] == "test" + assert isinstance(tuple_["big_object_field"], BigObject) + assert tuple_["big_object_field"].uri == "s3://test-bucket/path/to/object" + + def test_tuple_with_null_big_object(self): + """Test tuple with null BigObject field.""" + import pyarrow as pa + from core.models.schema.big_object import BigObject + + # Create Arrow schema with BIG_OBJECT metadata + arrow_schema = pa.schema( + [ + pa.field( + "big_object_field", + pa.string(), + metadata={b"texera_type": b"BIG_OBJECT"}, + ), + ] + ) + + # Create Arrow table with null value + arrow_table = pa.Table.from_pydict( + { + "big_object_field": [None], + }, + schema=arrow_schema, + ) + + # Create tuple from Arrow table + tuple_provider = ArrowTableTupleProvider(arrow_table) + tuples = [ + Tuple({name: field_accessor for name in arrow_table.column_names}) + for field_accessor in tuple_provider + ] + + assert len(tuples) == 1 + tuple_ = tuples[0] + assert tuple_["big_object_field"] is None diff --git a/amber/src/main/python/core/models/tuple.py b/amber/src/main/python/core/models/tuple.py index e88f08286d7..1c3563aef26 100644 --- a/amber/src/main/python/core/models/tuple.py +++ b/amber/src/main/python/core/models/tuple.py @@ -29,6 +29,7 @@ from typing import Any, List, Iterator, Callable from typing_extensions import Protocol, runtime_checkable +from core.models.schema.big_object import BigObject from .schema.attribute_type import TO_PYOBJECT_MAPPING, AttributeType from .schema.field import Field from .schema.schema import Schema @@ -86,6 +87,7 @@ def field_accessor(field_name: str) -> Field: """ value = self._table.column(field_name).chunks[chunk_idx][tuple_idx].as_py() field_type = self._table.schema.field(field_name).type + field_metadata = self._table.schema.field(field_name).metadata # for binary types, convert pickled objects back. if ( @@ -94,6 +96,16 @@ def field_accessor(field_name: str) -> Field: and value[:6] == b"pickle" ): value = pickle.loads(value[10:]) + + # Convert URI string to BigObject for BIG_OBJECT types + # Metadata is set by Scala ArrowUtils or Python iceberg_utils + elif ( + value is not None + and field_metadata + and field_metadata.get(b"texera_type") == b"BIG_OBJECT" + ): + value = BigObject(value) + return value self._current_idx += 1 diff --git a/amber/src/main/python/core/storage/document_factory.py b/amber/src/main/python/core/storage/document_factory.py index ba15069817e..5e680b30fc8 100644 --- a/amber/src/main/python/core/storage/document_factory.py +++ b/amber/src/main/python/core/storage/document_factory.py @@ -25,6 +25,7 @@ from core.storage.iceberg.iceberg_document import IcebergDocument from core.storage.iceberg.iceberg_utils import ( create_table, + amber_schema_to_iceberg_schema, amber_tuples_to_arrow_table, arrow_table_to_amber_tuples, load_table_metadata, @@ -63,7 +64,8 @@ def create_document(uri: str, schema: Schema) -> VirtualDocument: if resource_type in {VFSResourceType.RESULT}: storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) - iceberg_schema = Schema.as_arrow_schema(schema) + # Convert Amber Schema to Iceberg Schema with BIG_OBJECT field name encoding + iceberg_schema = amber_schema_to_iceberg_schema(schema) create_table( IcebergCatalogInstance.get_instance(), diff --git a/amber/src/main/python/core/storage/iceberg/iceberg_utils.py b/amber/src/main/python/core/storage/iceberg/iceberg_utils.py index 12c841ed71f..2b23e2de82c 100644 --- a/amber/src/main/python/core/storage/iceberg/iceberg_utils.py +++ b/amber/src/main/python/core/storage/iceberg/iceberg_utils.py @@ -25,9 +25,103 @@ from pyiceberg.schema import Schema from pyiceberg.table import Table from typing import Optional, Iterable +from pyiceberg import types as iceberg_types import core +import core.models from core.models import ArrowTableTupleProvider, Tuple +from core.models.schema.attribute_type import AttributeType, TO_ARROW_MAPPING + +# Suffix used to encode BIG_OBJECT fields in Iceberg (must match Scala IcebergUtil) +BIG_OBJECT_FIELD_SUFFIX = "__texera_big_obj_ptr" + +# Type mappings +_ICEBERG_TO_AMBER_TYPE_MAPPING = { + "string": "STRING", + "int": "INT", + "integer": "INT", + "long": "LONG", + "double": "DOUBLE", + "float": "DOUBLE", + "boolean": "BOOL", + "timestamp": "TIMESTAMP", + "binary": "BINARY", +} + +_AMBER_TO_ICEBERG_TYPE_MAPPING = { + AttributeType.STRING: iceberg_types.StringType(), + AttributeType.INT: iceberg_types.IntegerType(), + AttributeType.LONG: iceberg_types.LongType(), + AttributeType.DOUBLE: iceberg_types.DoubleType(), + AttributeType.BOOL: iceberg_types.BooleanType(), + AttributeType.TIMESTAMP: iceberg_types.TimestampType(), + AttributeType.BINARY: iceberg_types.BinaryType(), + AttributeType.BIG_OBJECT: iceberg_types.StringType(), +} + + +def encode_big_object_field_name(field_name: str, attr_type) -> str: + """Encodes BIG_OBJECT field names with suffix for Iceberg storage.""" + if attr_type == AttributeType.BIG_OBJECT: + return f"{field_name}{BIG_OBJECT_FIELD_SUFFIX}" + return field_name + + +def decode_big_object_field_name(field_name: str) -> str: + """Decodes field names by removing BIG_OBJECT suffix if present.""" + if field_name.endswith(BIG_OBJECT_FIELD_SUFFIX): + return field_name[: -len(BIG_OBJECT_FIELD_SUFFIX)] + return field_name + + +def iceberg_schema_to_amber_schema(iceberg_schema: Schema): + """ + Converts PyIceberg Schema to Amber Schema. + Decodes BIG_OBJECT field names and adds Arrow metadata. + """ + arrow_fields = [] + for field in iceberg_schema.fields: + decoded_name = decode_big_object_field_name(field.name) + is_big_object = field.name != decoded_name + + if is_big_object: + attr_type = AttributeType.BIG_OBJECT + else: + iceberg_type_str = str(field.field_type).lower() + attr_type_name = _ICEBERG_TO_AMBER_TYPE_MAPPING.get( + iceberg_type_str, "STRING" + ) + attr_type = getattr(AttributeType, attr_type_name) + + arrow_fields.append( + pa.field( + decoded_name, + TO_ARROW_MAPPING[attr_type], + metadata={b"texera_type": b"BIG_OBJECT"} if is_big_object else None, + ) + ) + + return core.models.Schema(pa.schema(arrow_fields)) + + +def amber_schema_to_iceberg_schema(amber_schema) -> Schema: + """ + Converts Amber Schema to PyIceberg Schema. + Encodes BIG_OBJECT field names with suffix. + """ + fields = [ + iceberg_types.NestedField( + field_id=idx, + name=encode_big_object_field_name(field_name, attr_type), + field_type=_AMBER_TO_ICEBERG_TYPE_MAPPING[attr_type], + required=False, + ) + for idx, (field_name, attr_type) in enumerate( + amber_schema._name_type_mapping.items(), start=1 + ) + ] + + return Schema(*fields) def create_postgres_catalog( @@ -135,27 +229,47 @@ def amber_tuples_to_arrow_table( ) -> pa.Table: """ Converts a list of amber tuples to a pyarrow table for serialization. + Handles BIG_OBJECT field name encoding and serialization. """ - return pa.Table.from_pydict( - { - name: [t[name] for t in tuple_list] - for name in iceberg_schema.as_arrow().names - }, - schema=iceberg_schema.as_arrow(), - ) + from core.models.schema.big_object import BigObject + + tuple_list = list(tuple_list) # Convert to list to allow multiple iterations + data_dict = {} + for encoded_name in iceberg_schema.as_arrow().names: + decoded_name = decode_big_object_field_name(encoded_name) + data_dict[encoded_name] = [ + ( + t[decoded_name].uri + if isinstance(t[decoded_name], BigObject) + else t[decoded_name] + ) + for t in tuple_list + ] + + return pa.Table.from_pydict(data_dict, schema=iceberg_schema.as_arrow()) def arrow_table_to_amber_tuples( iceberg_schema: Schema, arrow_table: pa.Table ) -> Iterable[Tuple]: """ - Converts an arrow table to a list of amber tuples for deserialization. + Converts an arrow table read from Iceberg to Amber tuples. + Properly handles BIG_OBJECT field name decoding and type detection. """ - tuple_provider = ArrowTableTupleProvider(arrow_table) + amber_schema = iceberg_schema_to_amber_schema(iceberg_schema) + arrow_table_with_metadata = pa.Table.from_arrays( + [arrow_table.column(name) for name in arrow_table.column_names], + schema=amber_schema.as_arrow_schema(), + ) + + tuple_provider = ArrowTableTupleProvider(arrow_table_with_metadata) return ( Tuple( - {name: field_accessor for name in arrow_table.column_names}, - schema=core.models.Schema(iceberg_schema.as_arrow()), + { + decode_big_object_field_name(name): field_accessor + for name in arrow_table.column_names + }, + schema=amber_schema, ) for field_accessor in tuple_provider ) diff --git a/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py b/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py index 6d7f7017c62..4eba328345b 100644 --- a/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py +++ b/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py @@ -43,6 +43,10 @@ table_result_namespace="operator-port-result", directory_path="../../../../../../amber/user-resources/workflow-results", commit_batch_size=4096, + s3_endpoint="http://localhost:9000", + s3_region="us-east-1", + s3_auth_username="minioadmin", + s3_auth_password="minioadmin", ) diff --git a/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py b/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py new file mode 100644 index 00000000000..b9ca867f163 --- /dev/null +++ b/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py @@ -0,0 +1,228 @@ +# 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. + +import pytest +import pyarrow as pa +from pyiceberg import types as iceberg_types +from pyiceberg.schema import Schema as IcebergSchema +from core.models import Schema, Tuple +from core.models.schema.attribute_type import AttributeType +from core.models.schema.big_object import BigObject +from core.storage.iceberg.iceberg_utils import ( + encode_big_object_field_name, + decode_big_object_field_name, + iceberg_schema_to_amber_schema, + amber_schema_to_iceberg_schema, + amber_tuples_to_arrow_table, + arrow_table_to_amber_tuples, +) + + +class TestIcebergUtilsBigObject: + def test_encode_big_object_field_name(self): + """Test encoding BIG_OBJECT field names with suffix.""" + assert ( + encode_big_object_field_name("my_field", AttributeType.BIG_OBJECT) + == "my_field__texera_big_obj_ptr" + ) + assert ( + encode_big_object_field_name("my_field", AttributeType.STRING) == "my_field" + ) + + def test_decode_big_object_field_name(self): + """Test decoding BIG_OBJECT field names by removing suffix.""" + assert ( + decode_big_object_field_name("my_field__texera_big_obj_ptr") == "my_field" + ) + assert decode_big_object_field_name("my_field") == "my_field" + assert decode_big_object_field_name("regular_field") == "regular_field" + + def test_amber_schema_to_iceberg_schema_with_big_object(self): + """Test converting Amber schema with BIG_OBJECT to Iceberg schema.""" + amber_schema = Schema() + amber_schema.add("regular_field", AttributeType.STRING) + amber_schema.add("big_object_field", AttributeType.BIG_OBJECT) + amber_schema.add("int_field", AttributeType.INT) + + iceberg_schema = amber_schema_to_iceberg_schema(amber_schema) + + # Check field names are encoded + field_names = [field.name for field in iceberg_schema.fields] + assert "regular_field" in field_names + assert "big_object_field__texera_big_obj_ptr" in field_names + assert "int_field" in field_names + + # Check types + big_object_field = next( + f for f in iceberg_schema.fields if "big_object" in f.name + ) + assert isinstance(big_object_field.field_type, iceberg_types.StringType) + + def test_iceberg_schema_to_amber_schema_with_big_object(self): + """Test converting Iceberg schema with BIG_OBJECT to Amber schema.""" + iceberg_schema = IcebergSchema( + iceberg_types.NestedField( + 1, "regular_field", iceberg_types.StringType(), required=False + ), + iceberg_types.NestedField( + 2, + "big_object_field__texera_big_obj_ptr", + iceberg_types.StringType(), + required=False, + ), + iceberg_types.NestedField( + 3, "int_field", iceberg_types.IntegerType(), required=False + ), + ) + + amber_schema = iceberg_schema_to_amber_schema(iceberg_schema) + + assert amber_schema.get_attr_type("regular_field") == AttributeType.STRING + assert ( + amber_schema.get_attr_type("big_object_field") == AttributeType.BIG_OBJECT + ) + assert amber_schema.get_attr_type("int_field") == AttributeType.INT + + # Check Arrow schema has metadata for BIG_OBJECT + arrow_schema = amber_schema.as_arrow_schema() + big_object_field = arrow_schema.field("big_object_field") + assert big_object_field.metadata is not None + assert big_object_field.metadata.get(b"texera_type") == b"BIG_OBJECT" + + def test_amber_tuples_to_arrow_table_with_big_object(self): + """Test converting Amber tuples with BigObject to Arrow table.""" + amber_schema = Schema() + amber_schema.add("regular_field", AttributeType.STRING) + amber_schema.add("big_object_field", AttributeType.BIG_OBJECT) + + big_object1 = BigObject("s3://bucket/path1") + big_object2 = BigObject("s3://bucket/path2") + + tuples = [ + Tuple( + {"regular_field": "value1", "big_object_field": big_object1}, + schema=amber_schema, + ), + Tuple( + {"regular_field": "value2", "big_object_field": big_object2}, + schema=amber_schema, + ), + ] + + iceberg_schema = amber_schema_to_iceberg_schema(amber_schema) + arrow_table = amber_tuples_to_arrow_table(iceberg_schema, tuples) + + # Check that BigObject values are converted to URI strings + regular_values = arrow_table.column("regular_field").to_pylist() + big_object_values = arrow_table.column( + "big_object_field__texera_big_obj_ptr" + ).to_pylist() + + assert regular_values == ["value1", "value2"] + assert big_object_values == ["s3://bucket/path1", "s3://bucket/path2"] + + def test_arrow_table_to_amber_tuples_with_big_object(self): + """Test converting Arrow table with BIG_OBJECT to Amber tuples.""" + # Create Iceberg schema with encoded field name + iceberg_schema = IcebergSchema( + iceberg_types.NestedField( + 1, "regular_field", iceberg_types.StringType(), required=False + ), + iceberg_types.NestedField( + 2, + "big_object_field__texera_big_obj_ptr", + iceberg_types.StringType(), + required=False, + ), + ) + + # Create Arrow table with URI strings + arrow_table = pa.Table.from_pydict( + { + "regular_field": ["value1", "value2"], + "big_object_field__texera_big_obj_ptr": [ + "s3://bucket/path1", + "s3://bucket/path2", + ], + } + ) + + tuples = list(arrow_table_to_amber_tuples(iceberg_schema, arrow_table)) + + assert len(tuples) == 2 + assert tuples[0]["regular_field"] == "value1" + assert isinstance(tuples[0]["big_object_field"], BigObject) + assert tuples[0]["big_object_field"].uri == "s3://bucket/path1" + + assert tuples[1]["regular_field"] == "value2" + assert isinstance(tuples[1]["big_object_field"], BigObject) + assert tuples[1]["big_object_field"].uri == "s3://bucket/path2" + + def test_round_trip_big_object_tuples(self): + """Test round-trip conversion of tuples with BigObject.""" + amber_schema = Schema() + amber_schema.add("regular_field", AttributeType.STRING) + amber_schema.add("big_object_field", AttributeType.BIG_OBJECT) + + big_object = BigObject("s3://bucket/path/to/object") + original_tuples = [ + Tuple( + {"regular_field": "value1", "big_object_field": big_object}, + schema=amber_schema, + ), + ] + + # Convert to Iceberg and Arrow + iceberg_schema = amber_schema_to_iceberg_schema(amber_schema) + arrow_table = amber_tuples_to_arrow_table(iceberg_schema, original_tuples) + + # Convert back to Amber tuples + retrieved_tuples = list( + arrow_table_to_amber_tuples(iceberg_schema, arrow_table) + ) + + assert len(retrieved_tuples) == 1 + assert retrieved_tuples[0]["regular_field"] == "value1" + assert isinstance(retrieved_tuples[0]["big_object_field"], BigObject) + assert retrieved_tuples[0]["big_object_field"].uri == big_object.uri + + def test_arrow_table_to_amber_tuples_with_null_big_object(self): + """Test converting Arrow table with null BigObject values.""" + iceberg_schema = IcebergSchema( + iceberg_types.NestedField( + 1, "regular_field", iceberg_types.StringType(), required=False + ), + iceberg_types.NestedField( + 2, + "big_object_field__texera_big_obj_ptr", + iceberg_types.StringType(), + required=False, + ), + ) + + arrow_table = pa.Table.from_pydict( + { + "regular_field": ["value1"], + "big_object_field__texera_big_obj_ptr": [None], + } + ) + + tuples = list(arrow_table_to_amber_tuples(iceberg_schema, arrow_table)) + + assert len(tuples) == 1 + assert tuples[0]["regular_field"] == "value1" + assert tuples[0]["big_object_field"] is None diff --git a/amber/src/main/python/core/storage/storage_config.py b/amber/src/main/python/core/storage/storage_config.py index 21c918d5147..1b4e4e229ad 100644 --- a/amber/src/main/python/core/storage/storage_config.py +++ b/amber/src/main/python/core/storage/storage_config.py @@ -32,6 +32,12 @@ class StorageConfig: ICEBERG_FILE_STORAGE_DIRECTORY_PATH = None ICEBERG_TABLE_COMMIT_BATCH_SIZE = None + # S3 configs (for BigObjectManager) + S3_ENDPOINT = None + S3_REGION = None + S3_AUTH_USERNAME = None + S3_AUTH_PASSWORD = None + @classmethod def initialize( cls, @@ -41,10 +47,14 @@ def initialize( table_result_namespace, directory_path, commit_batch_size, + s3_endpoint, + s3_region, + s3_auth_username, + s3_auth_password, ): if cls._initialized: raise RuntimeError( - "Storage config has already been initialized" "and cannot be modified." + "Storage config has already been initialized and cannot be modified." ) cls.ICEBERG_POSTGRES_CATALOG_URI_WITHOUT_SCHEME = postgres_uri_without_scheme @@ -53,6 +63,13 @@ def initialize( cls.ICEBERG_TABLE_RESULT_NAMESPACE = table_result_namespace cls.ICEBERG_FILE_STORAGE_DIRECTORY_PATH = directory_path cls.ICEBERG_TABLE_COMMIT_BATCH_SIZE = int(commit_batch_size) + + # S3 configs + cls.S3_ENDPOINT = s3_endpoint + cls.S3_REGION = s3_region + cls.S3_AUTH_USERNAME = s3_auth_username + cls.S3_AUTH_PASSWORD = s3_auth_password + cls._initialized = True def __new__(cls, *args, **kwargs): diff --git a/amber/src/main/python/pytexera/__init__.py b/amber/src/main/python/pytexera/__init__.py index db78319f490..b2616522e20 100644 --- a/amber/src/main/python/pytexera/__init__.py +++ b/amber/src/main/python/pytexera/__init__.py @@ -21,12 +21,15 @@ from pyamber import * from .storage.dataset_file_document import DatasetFileDocument +from .storage.big_object_input_stream import BigObjectInputStream +from .storage.big_object_output_stream import BigObjectOutputStream from .udf.udf_operator import ( UDFOperatorV2, UDFTableOperator, UDFBatchOperator, UDFSourceOperator, ) +from core.models.schema.big_object import BigObject __all__ = [ "State", @@ -41,6 +44,9 @@ "UDFBatchOperator", "UDFSourceOperator", "DatasetFileDocument", + "BigObject", + "BigObjectInputStream", + "BigObjectOutputStream", # export external tools to be used "overrides", "logger", diff --git a/amber/src/main/python/pytexera/storage/big_object_input_stream.py b/amber/src/main/python/pytexera/storage/big_object_input_stream.py new file mode 100644 index 00000000000..983f5ce91ff --- /dev/null +++ b/amber/src/main/python/pytexera/storage/big_object_input_stream.py @@ -0,0 +1,121 @@ +# 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. + +""" +BigObjectInputStream for reading BigObject data from S3. + +Usage: + with BigObjectInputStream(big_object) as stream: + content = stream.read() +""" + +from typing import BinaryIO, Optional +from functools import wraps +from io import IOBase +from core.models.schema.big_object import BigObject + + +def _require_open(func): + """Decorator to ensure stream is open before reading operations.""" + + @wraps(func) + def wrapper(self, *args, **kwargs): + if self._closed: + raise ValueError("I/O operation on closed stream") + if self._underlying is None: + self._lazy_init() + return func(self, *args, **kwargs) + + return wrapper + + +class BigObjectInputStream(IOBase): + """ + InputStream for reading BigObject data from S3. + + Lazily downloads from S3 on first read. Supports context manager and iteration. + """ + + def __init__(self, big_object: BigObject): + """Initialize stream for reading the given BigObject.""" + super().__init__() + if big_object is None: + raise ValueError("BigObject cannot be None") + self._big_object = big_object + self._underlying: Optional[BinaryIO] = None + self._closed = False + + def _lazy_init(self): + """Download from S3 on first read operation.""" + from pytexera.storage.big_object_manager import BigObjectManager + + s3 = BigObjectManager._get_s3_client() + response = s3.get_object( + Bucket=self._big_object.get_bucket_name(), + Key=self._big_object.get_object_key(), + ) + self._underlying = response["Body"] + + @_require_open + def read(self, n: int = -1) -> bytes: + """Read and return up to n bytes (-1 reads all).""" + return self._underlying.read(n) + + @_require_open + def readline(self, size: int = -1) -> bytes: + """Read and return one line from the stream.""" + return self._underlying.readline(size) + + @_require_open + def readlines(self, hint: int = -1) -> list[bytes]: + """Read and return a list of lines from the stream.""" + return self._underlying.readlines(hint) + + def readable(self) -> bool: + """Return True if the stream can be read from.""" + return not self._closed + + def seekable(self) -> bool: + """Return False - this stream does not support seeking.""" + return False + + @property + def closed(self) -> bool: + """Return True if the stream is closed.""" + return self._closed + + def close(self) -> None: + """Close the stream and release resources.""" + if not self._closed: + self._closed = True + if self._underlying is not None: + self._underlying.close() + + def __enter__(self): + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + self.close() + + def __iter__(self): + return self + + def __next__(self) -> bytes: + line = self.readline() + if not line: + raise StopIteration + return line diff --git a/amber/src/main/python/pytexera/storage/big_object_manager.py b/amber/src/main/python/pytexera/storage/big_object_manager.py new file mode 100644 index 00000000000..577ead00478 --- /dev/null +++ b/amber/src/main/python/pytexera/storage/big_object_manager.py @@ -0,0 +1,86 @@ +# 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. + +""" +Internal BigObject manager for S3 operations. + +Users should not interact with this module directly. Use BigObject() constructor +and BigObjectInputStream/BigObjectOutputStream instead. +""" + +import logging +import time +import uuid +from core.storage.storage_config import StorageConfig + +logger = logging.getLogger(__name__) + + +class BigObjectManager: + """Internal manager for BigObject S3 operations.""" + + _s3_client = None + DEFAULT_BUCKET = "texera-big-objects" + + @classmethod + def _get_s3_client(cls): + """Get or initialize S3 client (lazy initialization, cached).""" + if cls._s3_client is None: + try: + import boto3 + from botocore.config import Config + except ImportError as e: + raise RuntimeError( + "boto3 required. Install with: pip install boto3" + ) from e + + cls._s3_client = boto3.client( + "s3", + endpoint_url=StorageConfig.S3_ENDPOINT, + aws_access_key_id=StorageConfig.S3_AUTH_USERNAME, + aws_secret_access_key=StorageConfig.S3_AUTH_PASSWORD, + region_name=StorageConfig.S3_REGION, + config=Config( + signature_version="s3v4", s3={"addressing_style": "path"} + ), + ) + return cls._s3_client + + @classmethod + def _ensure_bucket_exists(cls, bucket: str): + """Ensure S3 bucket exists, creating it if necessary.""" + s3 = cls._get_s3_client() + try: + s3.head_bucket(Bucket=bucket) + except s3.exceptions.NoSuchBucket: + logger.debug(f"Bucket {bucket} not found, creating it") + s3.create_bucket(Bucket=bucket) + logger.info(f"Created bucket: {bucket}") + + @classmethod + def create(cls) -> str: + """ + Creates a new BigObject reference with a unique S3 URI. + + Returns: + S3 URI string (format: s3://bucket/key) + """ + cls._ensure_bucket_exists(cls.DEFAULT_BUCKET) + timestamp_ms = int(time.time() * 1000) + unique_id = uuid.uuid4() + object_key = f"objects/{timestamp_ms}/{unique_id}" + return f"s3://{cls.DEFAULT_BUCKET}/{object_key}" diff --git a/amber/src/main/python/pytexera/storage/big_object_output_stream.py b/amber/src/main/python/pytexera/storage/big_object_output_stream.py new file mode 100644 index 00000000000..96149915da4 --- /dev/null +++ b/amber/src/main/python/pytexera/storage/big_object_output_stream.py @@ -0,0 +1,244 @@ +# 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. + +""" +BigObjectOutputStream for streaming BigObject data to S3. + +Usage: + from pytexera import BigObject, BigObjectOutputStream + + big_object = BigObject() + with BigObjectOutputStream(big_object) as out: + out.write(b"data") +""" + +from typing import Optional, Union +from io import IOBase +from core.models.schema.big_object import BigObject +from pytexera.storage.big_object_manager import BigObjectManager +import threading +import queue + +# Constants +_CHUNK_SIZE = 64 * 1024 # 64KB +_QUEUE_TIMEOUT = 0.1 + + +class _QueueReader: + """File-like object that reads from a queue.""" + + def __init__(self, q: queue.Queue): + self._queue = q + self._buffer = b"" + self._eof = False + + def read(self, size=-1): + """Read bytes from the queue.""" + if self._eof and not self._buffer: + return b"" + + # Collect chunks until we have enough data or reach EOF + chunks = [self._buffer] if self._buffer else [] + total_size = len(self._buffer) + self._buffer = b"" + needed = size if size != -1 else None + + while not self._eof and (needed is None or total_size < needed): + try: + chunk = self._queue.get(timeout=_QUEUE_TIMEOUT) + if chunk is None: # EOF marker + self._eof = True + break + chunks.append(chunk) + total_size += len(chunk) + except queue.Empty: + continue + + result = b"".join(chunks) + + # If size was specified, split and buffer remainder + if needed is not None and len(result) > needed: + self._buffer = result[needed:] + result = result[:needed] + + return result + + +class BigObjectOutputStream(IOBase): + """ + OutputStream for streaming BigObject data to S3. + + Data is uploaded in the background using multipart upload as you write. + Call close() to complete the upload and ensure all data is persisted. + + This class follows Python's standard I/O interface (io.IOBase). + + Usage: + from pytexera import BigObject, BigObjectOutputStream + + # Create a new BigObject and write to it + big_object = BigObject() + with BigObjectOutputStream(big_object) as out: + out.write(b"Hello, World!") + out.write(b"More data") + # big_object is now ready to be added to tuples + + Note: Not thread-safe. Do not access from multiple threads concurrently. + """ + + def __init__(self, big_object: BigObject): + """ + Initialize a BigObjectOutputStream. + + Args: + big_object: The BigObject reference to write to + + Raises: + ValueError: If big_object is None + """ + super().__init__() + if big_object is None: + raise ValueError("BigObject cannot be None") + + self._big_object = big_object + self._bucket_name = big_object.get_bucket_name() + self._object_key = big_object.get_object_key() + self._closed = False + + # Background upload thread state + self._queue: queue.Queue = queue.Queue(maxsize=_CHUNK_SIZE) + self._upload_exception: Optional[Exception] = None + self._upload_complete = threading.Event() + self._upload_thread: Optional[threading.Thread] = None + self._lock = threading.Lock() + + def write(self, b: Union[bytes, bytearray]) -> int: + """ + Write bytes to the stream. + + Args: + b: Bytes to write + + Returns: + Number of bytes written + + Raises: + ValueError: If stream is closed + IOError: If previous upload failed + """ + if self._closed: + raise ValueError("I/O operation on closed stream") + + # Check if upload has failed + with self._lock: + if self._upload_exception is not None: + raise IOError( + f"Background upload failed: {self._upload_exception}" + ) from self._upload_exception + + # Start upload thread on first write + if self._upload_thread is None: + + def upload_worker(): + try: + BigObjectManager._ensure_bucket_exists(self._bucket_name) + s3 = BigObjectManager._get_s3_client() + reader = _QueueReader(self._queue) + s3.upload_fileobj(reader, self._bucket_name, self._object_key) + except Exception as e: + with self._lock: + self._upload_exception = e + finally: + self._upload_complete.set() + + self._upload_thread = threading.Thread(target=upload_worker, daemon=True) + self._upload_thread.start() + + # Write data in chunks + data = bytes(b) + for offset in range(0, len(data), _CHUNK_SIZE): + self._queue.put(data[offset : offset + _CHUNK_SIZE], block=True) + + return len(data) + + def writable(self) -> bool: + """Return True if the stream can be written to.""" + return not self._closed + + def seekable(self) -> bool: + """Return False - this stream does not support seeking.""" + return False + + @property + def closed(self) -> bool: + """Return True if the stream is closed.""" + return self._closed + + def flush(self) -> None: + """ + Flush the write buffer. + + Note: This doesn't guarantee data is uploaded to S3 yet. + Call close() to ensure upload completion. + """ + # No-op: data is already being consumed by the upload thread + pass + + def close(self) -> None: + """ + Close the stream and complete the S3 upload. + Blocks until upload is complete. Raises IOError if upload failed. + + Raises: + IOError: If upload failed + """ + if self._closed: + return + + self._closed = True + + # Signal EOF to upload thread and wait for completion + if self._upload_thread is not None: + self._queue.put(None, block=True) # EOF marker + self._upload_thread.join() + self._upload_complete.wait() + + # Check for errors and cleanup if needed + with self._lock: + exception = self._upload_exception + + if exception is not None: + self._cleanup_failed_upload() + raise IOError(f"Failed to complete upload: {exception}") from exception + + def _cleanup_failed_upload(self): + """Clean up a failed upload by deleting the S3 object.""" + try: + s3 = BigObjectManager._get_s3_client() + s3.delete_object(Bucket=self._bucket_name, Key=self._object_key) + except Exception: + # Ignore cleanup errors - we're already handling an upload failure + pass + + def __enter__(self): + """Context manager entry.""" + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + """Context manager exit - automatically cleanup.""" + self.close() + return False diff --git a/amber/src/main/python/pytexera/storage/test_big_object_input_stream.py b/amber/src/main/python/pytexera/storage/test_big_object_input_stream.py new file mode 100644 index 00000000000..f15bfd8849a --- /dev/null +++ b/amber/src/main/python/pytexera/storage/test_big_object_input_stream.py @@ -0,0 +1,240 @@ +# 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. + +import pytest +from unittest.mock import patch, MagicMock +from io import BytesIO +from core.models.schema.big_object import BigObject +from pytexera.storage.big_object_input_stream import BigObjectInputStream +from pytexera.storage import big_object_manager + + +class TestBigObjectInputStream: + @pytest.fixture + def big_object(self): + """Create a test BigObject.""" + return BigObject("s3://test-bucket/path/to/object") + + @pytest.fixture + def mock_s3_response(self): + """Create a mock S3 response with a BytesIO body.""" + return {"Body": BytesIO(b"test data content")} + + def test_init_with_valid_big_object(self, big_object): + """Test initialization with a valid BigObject.""" + stream = BigObjectInputStream(big_object) + try: + assert stream._big_object == big_object + assert stream._underlying is None + assert not stream._closed + finally: + stream.close() + + def test_init_with_none_raises_error(self): + """Test that initializing with None raises ValueError.""" + with pytest.raises(ValueError, match="BigObject cannot be None"): + BigObjectInputStream(None) + + def test_lazy_init_downloads_from_s3(self, big_object, mock_s3_response): + """Test that _lazy_init downloads from S3 on first read.""" + with patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client: + mock_s3_client = MagicMock() + mock_s3_client.get_object.return_value = mock_s3_response + mock_get_s3_client.return_value = mock_s3_client + + stream = BigObjectInputStream(big_object) + try: + assert stream._underlying is None # Not initialized yet + + # Trigger lazy init by reading + data = stream.read() + assert data == b"test data content" + assert stream._underlying is not None + + # Verify S3 was called correctly + mock_s3_client.get_object.assert_called_once_with( + Bucket="test-bucket", Key="path/to/object" + ) + finally: + stream.close() + + def test_read_all(self, big_object, mock_s3_response): + """Test reading all data.""" + with patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client: + mock_s3_client = MagicMock() + mock_s3_client.get_object.return_value = mock_s3_response + mock_get_s3_client.return_value = mock_s3_client + + stream = BigObjectInputStream(big_object) + try: + data = stream.read() + assert data == b"test data content" + finally: + stream.close() + + def test_read_partial(self, big_object, mock_s3_response): + """Test reading partial data.""" + with patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client: + mock_s3_client = MagicMock() + mock_s3_client.get_object.return_value = mock_s3_response + mock_get_s3_client.return_value = mock_s3_client + + stream = BigObjectInputStream(big_object) + try: + data = stream.read(4) + assert data == b"test" + finally: + stream.close() + + def test_readline(self, big_object): + """Test reading a line.""" + with patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client: + response = {"Body": BytesIO(b"line1\nline2\nline3")} + mock_s3_client = MagicMock() + mock_s3_client.get_object.return_value = response + mock_get_s3_client.return_value = mock_s3_client + + stream = BigObjectInputStream(big_object) + try: + line = stream.readline() + assert line == b"line1\n" + finally: + stream.close() + + def test_readlines(self, big_object): + """Test reading all lines.""" + with patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client: + response = {"Body": BytesIO(b"line1\nline2\nline3")} + mock_s3_client = MagicMock() + mock_s3_client.get_object.return_value = response + mock_get_s3_client.return_value = mock_s3_client + + stream = BigObjectInputStream(big_object) + try: + lines = stream.readlines() + assert lines == [b"line1\n", b"line2\n", b"line3"] + finally: + stream.close() + + def test_readable(self, big_object): + """Test readable() method.""" + stream = BigObjectInputStream(big_object) + try: + assert stream.readable() is True + + stream.close() + assert stream.readable() is False + finally: + if not stream._closed: + stream.close() + + def test_seekable(self, big_object): + """Test seekable() method (should always return False).""" + stream = BigObjectInputStream(big_object) + try: + assert stream.seekable() is False + finally: + stream.close() + + def test_closed_property(self, big_object): + """Test closed property.""" + stream = BigObjectInputStream(big_object) + try: + assert stream.closed is False + + stream.close() + assert stream.closed is True + finally: + if not stream._closed: + stream.close() + + def test_close(self, big_object, mock_s3_response): + """Test closing the stream.""" + with patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client: + mock_s3_client = MagicMock() + mock_s3_client.get_object.return_value = mock_s3_response + mock_get_s3_client.return_value = mock_s3_client + + stream = BigObjectInputStream(big_object) + stream.read(1) # Trigger lazy init + assert stream._underlying is not None + + stream.close() + assert stream._closed is True + assert stream._underlying.closed + + def test_context_manager(self, big_object, mock_s3_response): + """Test using as context manager.""" + with patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client: + mock_s3_client = MagicMock() + mock_s3_client.get_object.return_value = mock_s3_response + mock_get_s3_client.return_value = mock_s3_client + + with BigObjectInputStream(big_object) as stream: + data = stream.read() + assert data == b"test data content" + assert not stream._closed + + # Stream should be closed after context exit + assert stream._closed + + def test_iteration(self, big_object): + """Test iteration over lines.""" + with patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client: + response = {"Body": BytesIO(b"line1\nline2\nline3")} + mock_s3_client = MagicMock() + mock_s3_client.get_object.return_value = response + mock_get_s3_client.return_value = mock_s3_client + + stream = BigObjectInputStream(big_object) + try: + lines = list(stream) + assert lines == [b"line1\n", b"line2\n", b"line3"] + finally: + stream.close() + + def test_read_after_close_raises_error(self, big_object, mock_s3_response): + """Test that reading after close raises ValueError.""" + with patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client: + mock_s3_client = MagicMock() + mock_s3_client.get_object.return_value = mock_s3_response + mock_get_s3_client.return_value = mock_s3_client + + stream = BigObjectInputStream(big_object) + stream.close() + + with pytest.raises(ValueError, match="I/O operation on closed stream"): + stream.read() + # Stream is already closed, no need to close again diff --git a/amber/src/main/python/pytexera/storage/test_big_object_manager.py b/amber/src/main/python/pytexera/storage/test_big_object_manager.py new file mode 100644 index 00000000000..ec7a8b7eece --- /dev/null +++ b/amber/src/main/python/pytexera/storage/test_big_object_manager.py @@ -0,0 +1,150 @@ +# 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. + +import pytest +from unittest.mock import patch, MagicMock, Mock +from pytexera.storage.big_object_manager import BigObjectManager +from core.storage.storage_config import StorageConfig + + +class TestBigObjectManager: + @pytest.fixture(autouse=True) + def setup_storage_config(self): + """Initialize StorageConfig for tests.""" + if not StorageConfig._initialized: + StorageConfig.initialize( + postgres_uri_without_scheme="localhost:5432/test", + postgres_username="test", + postgres_password="test", + table_result_namespace="test", + directory_path="/tmp/test", + commit_batch_size=1000, + s3_endpoint="http://localhost:9000", + s3_region="us-east-1", + s3_auth_username="minioadmin", + s3_auth_password="minioadmin", + ) + + def test_get_s3_client_initializes_once(self): + """Test that S3 client is initialized and cached.""" + # Reset the client + BigObjectManager._s3_client = None + + with patch("boto3.client") as mock_boto3_client: + mock_client = MagicMock() + mock_boto3_client.return_value = mock_client + + # First call should create client + client1 = BigObjectManager._get_s3_client() + assert client1 == mock_client + assert mock_boto3_client.call_count == 1 + + # Second call should return cached client + client2 = BigObjectManager._get_s3_client() + assert client2 == mock_client + assert mock_boto3_client.call_count == 1 # Still 1, not 2 + + def test_get_s3_client_without_boto3_raises_error(self): + """Test that missing boto3 raises RuntimeError.""" + BigObjectManager._s3_client = None + + import sys + + # Temporarily remove boto3 from sys.modules to simulate it not being installed + boto3_backup = sys.modules.pop("boto3", None) + try: + # Mock the import to raise ImportError + original_import = __import__ + + def mock_import(name, *args, **kwargs): + if name == "boto3": + raise ImportError("No module named boto3") + return original_import(name, *args, **kwargs) + + with patch("builtins.__import__", side_effect=mock_import): + with pytest.raises(RuntimeError, match="boto3 required"): + BigObjectManager._get_s3_client() + finally: + # Restore boto3 if it was there + if boto3_backup is not None: + sys.modules["boto3"] = boto3_backup + + def test_ensure_bucket_exists_when_bucket_exists(self): + """Test that existing bucket doesn't trigger creation.""" + BigObjectManager._s3_client = None + + with patch("boto3.client") as mock_boto3_client: + mock_client = MagicMock() + mock_boto3_client.return_value = mock_client + # head_bucket doesn't raise exception (bucket exists) + mock_client.head_bucket.return_value = None + mock_client.exceptions.NoSuchBucket = type("NoSuchBucket", (Exception,), {}) + + BigObjectManager._ensure_bucket_exists("test-bucket") + mock_client.head_bucket.assert_called_once_with(Bucket="test-bucket") + mock_client.create_bucket.assert_not_called() + + def test_ensure_bucket_exists_creates_bucket_when_missing(self): + """Test that missing bucket triggers creation.""" + BigObjectManager._s3_client = None + + with patch("boto3.client") as mock_boto3_client: + mock_client = MagicMock() + mock_boto3_client.return_value = mock_client + # head_bucket raises NoSuchBucket exception + no_such_bucket = type("NoSuchBucket", (Exception,), {}) + mock_client.exceptions.NoSuchBucket = no_such_bucket + mock_client.head_bucket.side_effect = no_such_bucket() + + BigObjectManager._ensure_bucket_exists("test-bucket") + mock_client.head_bucket.assert_called_once_with(Bucket="test-bucket") + mock_client.create_bucket.assert_called_once_with(Bucket="test-bucket") + + def test_create_generates_unique_uri(self): + """Test that create() generates a unique S3 URI.""" + BigObjectManager._s3_client = None + + with patch("boto3.client") as mock_boto3_client: + mock_client = MagicMock() + mock_boto3_client.return_value = mock_client + mock_client.head_bucket.return_value = None + mock_client.exceptions.NoSuchBucket = type("NoSuchBucket", (Exception,), {}) + + uri = BigObjectManager.create() + + # Check URI format + assert uri.startswith("s3://") + assert uri.startswith(f"s3://{BigObjectManager.DEFAULT_BUCKET}/") + assert "objects/" in uri + + # Verify bucket was checked/created + mock_client.head_bucket.assert_called_once_with( + Bucket=BigObjectManager.DEFAULT_BUCKET + ) + + def test_create_uses_default_bucket(self): + """Test that create() uses the default bucket.""" + BigObjectManager._s3_client = None + + with patch("boto3.client") as mock_boto3_client: + mock_client = MagicMock() + mock_boto3_client.return_value = mock_client + mock_client.head_bucket.return_value = None + mock_client.exceptions.NoSuchBucket = type("NoSuchBucket", (Exception,), {}) + + uri = BigObjectManager.create() + assert BigObjectManager.DEFAULT_BUCKET in uri diff --git a/amber/src/main/python/pytexera/storage/test_big_object_output_stream.py b/amber/src/main/python/pytexera/storage/test_big_object_output_stream.py new file mode 100644 index 00000000000..8e5a8b85850 --- /dev/null +++ b/amber/src/main/python/pytexera/storage/test_big_object_output_stream.py @@ -0,0 +1,254 @@ +# 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. + +import pytest +import time +from unittest.mock import patch, MagicMock, Mock +from core.models.schema.big_object import BigObject +from pytexera.storage.big_object_output_stream import BigObjectOutputStream +from pytexera.storage import big_object_manager + + +class TestBigObjectOutputStream: + @pytest.fixture + def big_object(self): + """Create a test BigObject.""" + return BigObject("s3://test-bucket/path/to/object") + + def test_init_with_valid_big_object(self, big_object): + """Test initialization with a valid BigObject.""" + stream = BigObjectOutputStream(big_object) + assert stream._big_object == big_object + assert stream._bucket_name == "test-bucket" + assert stream._object_key == "path/to/object" + assert not stream._closed + assert stream._upload_thread is None + + def test_init_with_none_raises_error(self): + """Test that initializing with None raises ValueError.""" + with pytest.raises(ValueError, match="BigObject cannot be None"): + BigObjectOutputStream(None) + + def test_write_starts_upload_thread(self, big_object): + """Test that write() starts the upload thread.""" + with ( + patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client, + patch.object( + big_object_manager.BigObjectManager, "_ensure_bucket_exists" + ) as mock_ensure_bucket, + ): + mock_s3 = MagicMock() + mock_get_s3_client.return_value = mock_s3 + mock_ensure_bucket.return_value = None + + stream = BigObjectOutputStream(big_object) + assert stream._upload_thread is None + + stream.write(b"test data") + assert stream._upload_thread is not None + # Thread may have already completed, so just check it was created + assert stream._upload_thread is not None + + # Wait for thread to finish + stream.close() + + def test_write_data(self, big_object): + """Test writing data to the stream.""" + with ( + patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client, + patch.object( + big_object_manager.BigObjectManager, "_ensure_bucket_exists" + ) as mock_ensure_bucket, + ): + mock_s3 = MagicMock() + mock_get_s3_client.return_value = mock_s3 + mock_ensure_bucket.return_value = None + + stream = BigObjectOutputStream(big_object) + bytes_written = stream.write(b"test data") + assert bytes_written == len(b"test data") + + stream.close() + + def test_write_multiple_chunks(self, big_object): + """Test writing multiple chunks of data.""" + with ( + patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client, + patch.object( + big_object_manager.BigObjectManager, "_ensure_bucket_exists" + ) as mock_ensure_bucket, + ): + mock_s3 = MagicMock() + mock_get_s3_client.return_value = mock_s3 + mock_ensure_bucket.return_value = None + + stream = BigObjectOutputStream(big_object) + stream.write(b"chunk1") + stream.write(b"chunk2") + stream.write(b"chunk3") + + stream.close() + + def test_writable(self, big_object): + """Test writable() method.""" + stream = BigObjectOutputStream(big_object) + assert stream.writable() is True + + stream.close() + assert stream.writable() is False + + def test_seekable(self, big_object): + """Test seekable() method (should always return False).""" + stream = BigObjectOutputStream(big_object) + assert stream.seekable() is False + + def test_closed_property(self, big_object): + """Test closed property.""" + stream = BigObjectOutputStream(big_object) + assert stream.closed is False + + stream.close() + assert stream.closed is True + + def test_flush(self, big_object): + """Test flush() method (should be a no-op).""" + stream = BigObjectOutputStream(big_object) + # Should not raise any exception + stream.flush() + + def test_close_completes_upload(self, big_object): + """Test that close() completes the upload.""" + with ( + patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client, + patch.object( + big_object_manager.BigObjectManager, "_ensure_bucket_exists" + ) as mock_ensure_bucket, + ): + mock_s3 = MagicMock() + mock_get_s3_client.return_value = mock_s3 + mock_ensure_bucket.return_value = None + + stream = BigObjectOutputStream(big_object) + stream.write(b"test data") + + # Close should wait for upload to complete + stream.close() + + # Verify upload_fileobj was called + assert mock_s3.upload_fileobj.called + + def test_context_manager(self, big_object): + """Test using as context manager.""" + with ( + patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client, + patch.object( + big_object_manager.BigObjectManager, "_ensure_bucket_exists" + ) as mock_ensure_bucket, + ): + mock_s3 = MagicMock() + mock_get_s3_client.return_value = mock_s3 + mock_ensure_bucket.return_value = None + + with BigObjectOutputStream(big_object) as stream: + stream.write(b"test data") + assert not stream._closed + + # Stream should be closed after context exit + assert stream._closed + + def test_write_after_close_raises_error(self, big_object): + """Test that writing after close raises ValueError.""" + stream = BigObjectOutputStream(big_object) + stream.close() + + with pytest.raises(ValueError, match="I/O operation on closed stream"): + stream.write(b"data") + + def test_close_handles_upload_error(self, big_object): + """Test that close() raises IOError if upload fails.""" + with ( + patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client, + patch.object( + big_object_manager.BigObjectManager, "_ensure_bucket_exists" + ) as mock_ensure_bucket, + ): + mock_s3 = MagicMock() + mock_get_s3_client.return_value = mock_s3 + mock_ensure_bucket.return_value = None + mock_s3.upload_fileobj.side_effect = Exception("Upload failed") + + stream = BigObjectOutputStream(big_object) + stream.write(b"test data") + + with pytest.raises(IOError, match="Failed to complete upload"): + stream.close() + + def test_write_after_upload_error_raises_error(self, big_object): + """Test that writing after upload error raises IOError.""" + with ( + patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client, + patch.object( + big_object_manager.BigObjectManager, "_ensure_bucket_exists" + ) as mock_ensure_bucket, + ): + mock_s3 = MagicMock() + mock_get_s3_client.return_value = mock_s3 + mock_ensure_bucket.return_value = None + mock_s3.upload_fileobj.side_effect = Exception("Upload failed") + + stream = BigObjectOutputStream(big_object) + stream.write(b"test data") + + # Wait a bit for the error to be set + time.sleep(0.1) + + with pytest.raises(IOError, match="Background upload failed"): + stream.write(b"more data") + + def test_multiple_close_calls(self, big_object): + """Test that multiple close() calls are safe.""" + with ( + patch.object( + big_object_manager.BigObjectManager, "_get_s3_client" + ) as mock_get_s3_client, + patch.object( + big_object_manager.BigObjectManager, "_ensure_bucket_exists" + ) as mock_ensure_bucket, + ): + mock_s3 = MagicMock() + mock_get_s3_client.return_value = mock_s3 + mock_ensure_bucket.return_value = None + + stream = BigObjectOutputStream(big_object) + stream.write(b"test data") + stream.close() + # Second close should not raise error + stream.close() diff --git a/amber/src/main/python/texera_run_python_worker.py b/amber/src/main/python/texera_run_python_worker.py index c9594cc2187..3ebf81c201f 100644 --- a/amber/src/main/python/texera_run_python_worker.py +++ b/amber/src/main/python/texera_run_python_worker.py @@ -51,6 +51,10 @@ def init_loguru_logger(stream_log_level) -> None: iceberg_table_namespace, iceberg_file_storage_directory_path, iceberg_table_commit_batch_size, + s3_endpoint, + s3_region, + s3_auth_username, + s3_auth_password, ) = sys.argv init_loguru_logger(logger_level) StorageConfig.initialize( @@ -60,6 +64,10 @@ def init_loguru_logger(stream_log_level) -> None: iceberg_table_namespace, iceberg_file_storage_directory_path, iceberg_table_commit_batch_size, + s3_endpoint, + s3_region, + s3_auth_username, + s3_auth_password, ) # Setting R_HOME environment variable for R-UDF usage diff --git a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala index f32d227fc70..558b99c9b7b 100644 --- a/amber/src/main/scala/org/apache/texera/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala +++ b/amber/src/main/scala/org/apache/texera/amber/engine/architecture/pythonworker/PythonWorkflowWorker.scala @@ -183,7 +183,11 @@ class PythonWorkflowWorker( StorageConfig.icebergPostgresCatalogPassword, StorageConfig.icebergTableResultNamespace, StorageConfig.fileStorageDirectoryPath.toString, - StorageConfig.icebergTableCommitBatchSize.toString + StorageConfig.icebergTableCommitBatchSize.toString, + StorageConfig.s3Endpoint, + StorageConfig.s3Region, + StorageConfig.s3Username, + StorageConfig.s3Password ) ).run(BasicIO.standard(false)) } diff --git a/build.sbt b/build.sbt index 60efe697ce7..dabed3532c9 100644 --- a/build.sbt +++ b/build.sbt @@ -84,7 +84,19 @@ lazy val WorkflowExecutionService = (project in file("amber")) "org.slf4j" % "slf4j-api" % "1.7.26", "org.eclipse.jetty" % "jetty-server" % "9.4.20.v20190813", "org.eclipse.jetty" % "jetty-servlet" % "9.4.20.v20190813", - "org.eclipse.jetty" % "jetty-http" % "9.4.20.v20190813" + "org.eclipse.jetty" % "jetty-http" % "9.4.20.v20190813", + // Netty dependency overrides to ensure compatibility with Arrow 14.0.1 + // Arrow requires Netty 4.1.96.Final to avoid NoSuchFieldError: chunkSize + "io.netty" % "netty-all" % "4.1.96.Final", + "io.netty" % "netty-buffer" % "4.1.96.Final", + "io.netty" % "netty-codec" % "4.1.96.Final", + "io.netty" % "netty-codec-http" % "4.1.96.Final", + "io.netty" % "netty-codec-http2" % "4.1.96.Final", + "io.netty" % "netty-common" % "4.1.96.Final", + "io.netty" % "netty-handler" % "4.1.96.Final", + "io.netty" % "netty-resolver" % "4.1.96.Final", + "io.netty" % "netty-transport" % "4.1.96.Final", + "io.netty" % "netty-transport-native-unix-common" % "4.1.96.Final" ), libraryDependencies ++= Seq( "com.squareup.okhttp3" % "okhttp" % "4.10.0" force () // Force usage of OkHttp 4.10.0 diff --git a/common/workflow-core/build.sbt b/common/workflow-core/build.sbt index ab6b8f27c65..99870f03eb4 100644 --- a/common/workflow-core/build.sbt +++ b/common/workflow-core/build.sbt @@ -115,6 +115,7 @@ libraryDependencies ++= Seq( ///////////////////////////////////////////////////////////////////////////// // Arrow related val arrowVersion = "14.0.1" +val nettyVersion = "4.1.96.Final" val arrowDependencies = Seq( // https://mvnrepository.com/artifact/org.apache.arrow/flight-grpc "org.apache.arrow" % "flight-grpc" % arrowVersion, @@ -124,6 +125,22 @@ val arrowDependencies = Seq( libraryDependencies ++= arrowDependencies +// Netty dependency overrides to ensure compatibility with Arrow +// Arrow 14.0.1 requires Netty 4.1.96.Final for proper memory allocation +// The chunkSize field issue occurs when Netty versions are mismatched +dependencyOverrides ++= Seq( + "io.netty" % "netty-all" % nettyVersion, + "io.netty" % "netty-buffer" % nettyVersion, + "io.netty" % "netty-codec" % nettyVersion, + "io.netty" % "netty-codec-http" % nettyVersion, + "io.netty" % "netty-codec-http2" % nettyVersion, + "io.netty" % "netty-common" % nettyVersion, + "io.netty" % "netty-handler" % nettyVersion, + "io.netty" % "netty-resolver" % nettyVersion, + "io.netty" % "netty-transport" % nettyVersion, + "io.netty" % "netty-transport-native-unix-common" % nettyVersion +) + ///////////////////////////////////////////////////////////////////////////// // Iceberg-related Dependencies ///////////////////////////////////////////////////////////////////////////// diff --git a/common/workflow-core/src/main/scala/org/apache/texera/amber/util/ArrowUtils.scala b/common/workflow-core/src/main/scala/org/apache/texera/amber/util/ArrowUtils.scala index 8e306c12a56..6f7d3b2f9b4 100644 --- a/common/workflow-core/src/main/scala/org/apache/texera/amber/util/ArrowUtils.scala +++ b/common/workflow-core/src/main/scala/org/apache/texera/amber/util/ArrowUtils.scala @@ -26,7 +26,7 @@ import org.apache.arrow.memory.{BufferAllocator, RootAllocator} import org.apache.arrow.vector.types.FloatingPointPrecision import org.apache.arrow.vector.types.TimeUnit.MILLISECOND import org.apache.arrow.vector.types.pojo.ArrowType.PrimitiveType -import org.apache.arrow.vector.types.pojo.{ArrowType, Field} +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType} import org.apache.arrow.vector.{ BigIntVector, BitVector, @@ -73,28 +73,28 @@ object ArrowUtils extends LazyLogging { Tuple .builder(schema) .addSequentially( - vectorSchemaRoot.getFieldVectors.asScala - .map((fieldVector: FieldVector) => { + vectorSchemaRoot.getFieldVectors.asScala.zipWithIndex.map { + case (fieldVector: FieldVector, index: Int) => val value: AnyRef = fieldVector.getObject(rowIndex) try { - val arrowType = fieldVector.getField.getFieldType.getType - val attributeType = toAttributeType(arrowType) + // Use the attribute type from the schema (which includes metadata) + // instead of deriving it from the Arrow type + val attributeType = schema.getAttributes(index).getType AttributeTypeUtils.parseField(value, attributeType) - } catch { case e: Exception => logger.warn("Caught error during parsing Arrow value back to Texera value", e) null } - }) - .toArray + }.toArray ) .build() } /** * Converts an Arrow Schema into Texera Schema. + * Checks field metadata to detect BIG_OBJECT types. * * @param arrowSchema The Arrow Schema to be converted. * @return A Texera Schema. @@ -102,7 +102,12 @@ object ArrowUtils extends LazyLogging { def toTexeraSchema(arrowSchema: org.apache.arrow.vector.types.pojo.Schema): Schema = Schema( arrowSchema.getFields.asScala.map { field => - new Attribute(field.getName, toAttributeType(field.getType)) + val isBigObject = Option(field.getMetadata) + .exists(m => m.containsKey("texera_type") && m.get("texera_type") == "BIG_OBJECT") + + val attributeType = + if (isBigObject) AttributeType.BIG_OBJECT else toAttributeType(field.getType) + new Attribute(field.getName, attributeType) }.toList ) @@ -211,7 +216,7 @@ object ArrowUtils extends LazyLogging { else vector .asInstanceOf[VarCharVector] - .setSafe(index, value.asInstanceOf[String].getBytes(StandardCharsets.UTF_8)) + .setSafe(index, value.toString.getBytes(StandardCharsets.UTF_8)) case _: ArrowType.Binary | _: ArrowType.LargeBinary => if (isNull) vector.asInstanceOf[VarBinaryVector].setNull(index) else @@ -227,19 +232,27 @@ object ArrowUtils extends LazyLogging { /** * Converts an Amber schema into Arrow schema. + * Stores AttributeType in field metadata to preserve BIG_OBJECT type information. * * @param schema The Texera Schema. * @return An Arrow Schema. */ def fromTexeraSchema(schema: Schema): org.apache.arrow.vector.types.pojo.Schema = { - val arrowFields = new util.ArrayList[Field] - - for (amberAttribute <- schema.getAttributes) { - val name = amberAttribute.getName - val field = Field.nullablePrimitive(name, fromAttributeType(amberAttribute.getType)) - arrowFields.add(field) + val arrowFields = schema.getAttributes.map { attribute => + val metadata = if (attribute.getType == AttributeType.BIG_OBJECT) { + val map = new util.HashMap[String, String]() + map.put("texera_type", "BIG_OBJECT") + map + } else null + + new Field( + attribute.getName, + new FieldType(true, fromAttributeType(attribute.getType), null, metadata), + null + ) } - new org.apache.arrow.vector.types.pojo.Schema(arrowFields) + + new org.apache.arrow.vector.types.pojo.Schema(util.Arrays.asList(arrowFields: _*)) } /** @@ -270,7 +283,7 @@ object ArrowUtils extends LazyLogging { case AttributeType.BINARY => new ArrowType.Binary - case AttributeType.STRING | AttributeType.ANY => + case AttributeType.STRING | AttributeType.BIG_OBJECT | AttributeType.ANY => ArrowType.Utf8.INSTANCE case _ => diff --git a/common/workflow-operator/src/test/scala/org/apache/texera/amber/util/ArrowUtilsSpec.scala b/common/workflow-operator/src/test/scala/org/apache/texera/amber/util/ArrowUtilsSpec.scala index edeaf7d4ae7..6464503005d 100644 --- a/common/workflow-operator/src/test/scala/org/apache/texera/amber/util/ArrowUtilsSpec.scala +++ b/common/workflow-operator/src/test/scala/org/apache/texera/amber/util/ArrowUtilsSpec.scala @@ -20,14 +20,15 @@ package org.apache.texera.amber.util import org.apache.texera.amber.core.tuple.AttributeTypeUtils.AttributeTypeException -import org.apache.texera.amber.core.tuple.{AttributeType, Schema, Tuple} +import org.apache.texera.amber.core.tuple.{AttributeType, BigObject, Schema, Tuple} import org.apache.arrow.memory.{BufferAllocator, RootAllocator} import org.apache.arrow.vector.VectorSchemaRoot -import org.apache.arrow.vector.types.pojo.{ArrowType, Field} +import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType} import org.apache.arrow.vector.types.{DateUnit, FloatingPointPrecision, IntervalUnit, TimeUnit} import org.scalatest.flatspec.AnyFlatSpec import java.sql.Timestamp +import java.util import scala.jdk.CollectionConverters.IterableHasAsJava class ArrowUtilsSpec extends AnyFlatSpec { @@ -99,6 +100,9 @@ class ArrowUtilsSpec extends AnyFlatSpec { // but not the other way around. assert(ArrowUtils.fromAttributeType(AttributeType.ANY) == string) + // BIG_OBJECT is converted to ArrowType.Utf8 (same as STRING) + assert(ArrowUtils.fromAttributeType(AttributeType.BIG_OBJECT) == string) + } it should "raise AttributeTypeException when converting unsupported types" in { @@ -239,4 +243,140 @@ class ArrowUtilsSpec extends AnyFlatSpec { } + it should "convert from AttributeType to ArrowType for BIG_OBJECT correctly" in { + // BIG_OBJECT is converted to ArrowType.Utf8 (stored as string) + assert(ArrowUtils.fromAttributeType(AttributeType.BIG_OBJECT) == string) + } + + it should "convert Texera Schema with BIG_OBJECT to Arrow Schema with metadata correctly" in { + val texeraSchemaWithBigObject = Schema() + .add("regular_string", AttributeType.STRING) + .add("big_object_field", AttributeType.BIG_OBJECT) + + val arrowSchema = ArrowUtils.fromTexeraSchema(texeraSchemaWithBigObject) + + // Check that regular string field has no metadata + val regularStringField = arrowSchema.getFields.get(0) + assert(regularStringField.getName == "regular_string") + assert(regularStringField.getType == string) + assert( + regularStringField.getMetadata == null || !regularStringField.getMetadata.containsKey( + "texera_type" + ) + ) + + // Check that BIG_OBJECT field has metadata + val bigObjectField = arrowSchema.getFields.get(1) + assert(bigObjectField.getName == "big_object_field") + assert(bigObjectField.getType == string) // BIG_OBJECT is stored as Utf8 + assert(bigObjectField.getMetadata != null) + assert(bigObjectField.getMetadata.get("texera_type") == "BIG_OBJECT") + } + + it should "convert Arrow Schema with BIG_OBJECT metadata to Texera Schema correctly" in { + // Create Arrow schema with BIG_OBJECT metadata + val bigObjectMetadata = new util.HashMap[String, String]() + bigObjectMetadata.put("texera_type", "BIG_OBJECT") + + val arrowSchemaWithBigObject = new org.apache.arrow.vector.types.pojo.Schema( + Array( + Field.nullablePrimitive("regular_string", string), + new Field( + "big_object_field", + new FieldType(true, string, null, bigObjectMetadata), + null + ) + ).toList.asJava + ) + + val texeraSchema = ArrowUtils.toTexeraSchema(arrowSchemaWithBigObject) + + assert(texeraSchema.getAttribute("regular_string").getName == "regular_string") + assert(texeraSchema.getAttribute("regular_string").getType == AttributeType.STRING) + + assert(texeraSchema.getAttribute("big_object_field").getName == "big_object_field") + assert(texeraSchema.getAttribute("big_object_field").getType == AttributeType.BIG_OBJECT) + } + + it should "set and get Texera Tuple with BIG_OBJECT correctly" in { + val texeraSchemaWithBigObject = Schema() + .add("big_object_field", AttributeType.BIG_OBJECT) + .add("regular_string", AttributeType.STRING) + + val bigObject = new BigObject("s3://test-bucket/path/to/object") + val tuple = Tuple + .builder(texeraSchemaWithBigObject) + .addSequentially( + Array( + bigObject, + "regular string value" + ) + ) + .build() + + val allocator: BufferAllocator = new RootAllocator() + val arrowSchema = ArrowUtils.fromTexeraSchema(texeraSchemaWithBigObject) + val vectorSchemaRoot = VectorSchemaRoot.create(arrowSchema, allocator) + vectorSchemaRoot.allocateNew() + + // Set Tuple into the Vectors + ArrowUtils.appendTexeraTuple(tuple, vectorSchemaRoot) + + // Verify the BIG_OBJECT is stored as string (URI) in Arrow + val storedValue = vectorSchemaRoot.getVector(0).getObject(0) + assert(storedValue.toString == "s3://test-bucket/path/to/object") + + // Get the Tuple from the Vectors + val retrievedTuple = ArrowUtils.getTexeraTuple(0, vectorSchemaRoot) + assert(retrievedTuple.getField[BigObject](0) == bigObject) + assert(retrievedTuple.getField[String](1) == "regular string value") + } + + it should "handle null BIG_OBJECT values correctly" in { + val texeraSchemaWithBigObject = Schema() + .add("big_object_field", AttributeType.BIG_OBJECT) + + val tuple = Tuple + .builder(texeraSchemaWithBigObject) + .addSequentially( + Array( + null.asInstanceOf[BigObject] + ) + ) + .build() + + val allocator: BufferAllocator = new RootAllocator() + val arrowSchema = ArrowUtils.fromTexeraSchema(texeraSchemaWithBigObject) + val vectorSchemaRoot = VectorSchemaRoot.create(arrowSchema, allocator) + vectorSchemaRoot.allocateNew() + + // Set Tuple into the Vectors + ArrowUtils.appendTexeraTuple(tuple, vectorSchemaRoot) + + // Verify null is stored correctly + assert(vectorSchemaRoot.getVector(0).getObject(0) == null) + + // Get the Tuple from the Vectors + val retrievedTuple = ArrowUtils.getTexeraTuple(0, vectorSchemaRoot) + assert(retrievedTuple.getField[BigObject](0) == null) + } + + it should "round-trip BIG_OBJECT schema conversion correctly" in { + val originalSchema = Schema() + .add("field1", AttributeType.STRING) + .add("field2", AttributeType.BIG_OBJECT) + .add("field3", AttributeType.INTEGER) + .add("field4", AttributeType.BIG_OBJECT) + + // Convert to Arrow and back + val arrowSchema = ArrowUtils.fromTexeraSchema(originalSchema) + val roundTripSchema = ArrowUtils.toTexeraSchema(arrowSchema) + + assert(roundTripSchema.getAttribute("field1").getType == AttributeType.STRING) + assert(roundTripSchema.getAttribute("field2").getType == AttributeType.BIG_OBJECT) + assert(roundTripSchema.getAttribute("field3").getType == AttributeType.INTEGER) + assert(roundTripSchema.getAttribute("field4").getType == AttributeType.BIG_OBJECT) + assert(roundTripSchema == originalSchema) + } + } From 7333dae1d03ba695542e9a22ec239abb50b9fa58 Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Mon, 1 Dec 2025 10:34:01 -0800 Subject: [PATCH 02/13] Fix format --- .../python/core/architecture/packaging/output_manager.py | 3 ++- amber/src/main/python/core/models/schema/big_object.py | 6 ++++-- amber/src/main/python/core/models/schema/schema.py | 3 ++- amber/src/main/python/core/models/schema/test_big_object.py | 2 +- amber/src/main/python/core/models/test_tuple.py | 1 - amber/src/main/python/core/storage/document_factory.py | 3 ++- .../core/storage/iceberg/test_iceberg_utils_big_object.py | 1 - .../python/pytexera/storage/big_object_output_stream.py | 2 +- .../main/python/pytexera/storage/test_big_object_manager.py | 2 +- .../pytexera/storage/test_big_object_output_stream.py | 2 +- 10 files changed, 14 insertions(+), 11 deletions(-) diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index 3dadcca7186..2abbb3e4f45 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -266,7 +266,8 @@ def tuple_to_frame(self, tuples: typing.List[Tuple]) -> DataFrame: { name: [ ( - # Convert BigObject objects to URI strings for Arrow serialization + # Convert BigObject objects to URI strings + # for Arrow serialization t[name].uri if isinstance(t[name], BigObject) else t[name] diff --git a/amber/src/main/python/core/models/schema/big_object.py b/amber/src/main/python/core/models/schema/big_object.py index 128a0a8ade4..bb09e66ae2d 100644 --- a/amber/src/main/python/core/models/schema/big_object.py +++ b/amber/src/main/python/core/models/schema/big_object.py @@ -17,7 +17,8 @@ """ BigObject represents a reference to a large object stored externally (e.g., S3). -This is a schema type class used throughout the system for handling BIG_OBJECT attribute types. +This is a schema type class used throughout the system for handling +BIG_OBJECT attribute types. """ from typing import Optional @@ -29,7 +30,8 @@ class BigObject: BigObject represents a reference to a large object stored in S3. Each BigObject is identified by an S3 URI (s3://bucket/path/to/object). - BigObjects are automatically tracked and cleaned up when the workflow execution completes. + BigObjects are automatically tracked and cleaned up when the workflow + execution completes. Usage: from pytexera import BigObject, BigObjectInputStream, BigObjectOutputStream diff --git a/amber/src/main/python/core/models/schema/schema.py b/amber/src/main/python/core/models/schema/schema.py index 9780e526e47..abc85fad2e7 100644 --- a/amber/src/main/python/core/models/schema/schema.py +++ b/amber/src/main/python/core/models/schema/schema.py @@ -89,7 +89,8 @@ def _from_arrow_schema(self, arrow_schema: pa.Schema) -> None: for attr_name in arrow_schema.names: field = arrow_schema.field(attr_name) - # Check metadata for BIG_OBJECT type (can be stored by either Scala ArrowUtils or Python) + # Check metadata for BIG_OBJECT type + # (can be stored by either Scala ArrowUtils or Python) is_big_object = ( field.metadata and field.metadata.get(b"texera_type") == b"BIG_OBJECT" ) diff --git a/amber/src/main/python/core/models/schema/test_big_object.py b/amber/src/main/python/core/models/schema/test_big_object.py index a0f48bf1b7b..495e180f7d1 100644 --- a/amber/src/main/python/core/models/schema/test_big_object.py +++ b/amber/src/main/python/core/models/schema/test_big_object.py @@ -16,7 +16,7 @@ # under the License. import pytest -from unittest.mock import patch, MagicMock +from unittest.mock import patch from core.models.schema.big_object import BigObject diff --git a/amber/src/main/python/core/models/test_tuple.py b/amber/src/main/python/core/models/test_tuple.py index 20f2ac1c522..da271f61bbd 100644 --- a/amber/src/main/python/core/models/test_tuple.py +++ b/amber/src/main/python/core/models/test_tuple.py @@ -289,7 +289,6 @@ def test_tuple_from_arrow_with_big_object(self): def test_tuple_with_null_big_object(self): """Test tuple with null BigObject field.""" import pyarrow as pa - from core.models.schema.big_object import BigObject # Create Arrow schema with BIG_OBJECT metadata arrow_schema = pa.schema( diff --git a/amber/src/main/python/core/storage/document_factory.py b/amber/src/main/python/core/storage/document_factory.py index 5e680b30fc8..4fca15a53e6 100644 --- a/amber/src/main/python/core/storage/document_factory.py +++ b/amber/src/main/python/core/storage/document_factory.py @@ -64,7 +64,8 @@ def create_document(uri: str, schema: Schema) -> VirtualDocument: if resource_type in {VFSResourceType.RESULT}: storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) - # Convert Amber Schema to Iceberg Schema with BIG_OBJECT field name encoding + # Convert Amber Schema to Iceberg Schema with BIG_OBJECT + # field name encoding iceberg_schema = amber_schema_to_iceberg_schema(schema) create_table( diff --git a/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py b/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py index b9ca867f163..f93ff4e2f7d 100644 --- a/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py +++ b/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py @@ -15,7 +15,6 @@ # specific language governing permissions and limitations # under the License. -import pytest import pyarrow as pa from pyiceberg import types as iceberg_types from pyiceberg.schema import Schema as IcebergSchema diff --git a/amber/src/main/python/pytexera/storage/big_object_output_stream.py b/amber/src/main/python/pytexera/storage/big_object_output_stream.py index 96149915da4..f4065ca2ac0 100644 --- a/amber/src/main/python/pytexera/storage/big_object_output_stream.py +++ b/amber/src/main/python/pytexera/storage/big_object_output_stream.py @@ -20,7 +20,7 @@ Usage: from pytexera import BigObject, BigObjectOutputStream - + big_object = BigObject() with BigObjectOutputStream(big_object) as out: out.write(b"data") diff --git a/amber/src/main/python/pytexera/storage/test_big_object_manager.py b/amber/src/main/python/pytexera/storage/test_big_object_manager.py index ec7a8b7eece..f64c68bb6d7 100644 --- a/amber/src/main/python/pytexera/storage/test_big_object_manager.py +++ b/amber/src/main/python/pytexera/storage/test_big_object_manager.py @@ -16,7 +16,7 @@ # under the License. import pytest -from unittest.mock import patch, MagicMock, Mock +from unittest.mock import patch, MagicMock from pytexera.storage.big_object_manager import BigObjectManager from core.storage.storage_config import StorageConfig diff --git a/amber/src/main/python/pytexera/storage/test_big_object_output_stream.py b/amber/src/main/python/pytexera/storage/test_big_object_output_stream.py index 8e5a8b85850..fa7a27cde3a 100644 --- a/amber/src/main/python/pytexera/storage/test_big_object_output_stream.py +++ b/amber/src/main/python/pytexera/storage/test_big_object_output_stream.py @@ -17,7 +17,7 @@ import pytest import time -from unittest.mock import patch, MagicMock, Mock +from unittest.mock import patch, MagicMock from core.models.schema.big_object import BigObject from pytexera.storage.big_object_output_stream import BigObjectOutputStream from pytexera.storage import big_object_manager From 47692b76d425b1b1d1b8d4750bf000ebee0ed18e Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Tue, 2 Dec 2025 17:52:27 -0800 Subject: [PATCH 03/13] Address comments --- .../architecture/packaging/output_manager.py | 13 +-- .../core/models/schema/arrow_schema_utils.py | 106 ++++++++++++++++++ .../main/python/core/models/schema/schema.py | 40 ++----- amber/src/main/python/core/models/tuple.py | 21 ++++ 4 files changed, 136 insertions(+), 44 deletions(-) create mode 100644 amber/src/main/python/core/models/schema/arrow_schema_utils.py diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index 2abbb3e4f45..bf4afbf396f 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -259,21 +259,10 @@ def emit_state( ) def tuple_to_frame(self, tuples: typing.List[Tuple]) -> DataFrame: - from core.models.schema.big_object import BigObject - return DataFrame( frame=Table.from_pydict( { - name: [ - ( - # Convert BigObject objects to URI strings - # for Arrow serialization - t[name].uri - if isinstance(t[name], BigObject) - else t[name] - ) - for t in tuples - ] + name: [t.get_serialized_field(name) for t in tuples] for name in self.get_port().get_schema().get_attr_names() }, schema=self.get_port().get_schema().as_arrow_schema(), diff --git a/amber/src/main/python/core/models/schema/arrow_schema_utils.py b/amber/src/main/python/core/models/schema/arrow_schema_utils.py new file mode 100644 index 00000000000..2884c92c4e2 --- /dev/null +++ b/amber/src/main/python/core/models/schema/arrow_schema_utils.py @@ -0,0 +1,106 @@ +# 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. + +""" +Utilities for converting between Arrow schemas and Amber schemas, +handling BIG_OBJECT metadata preservation. +""" + +import pyarrow as pa +from typing import Mapping + +from core.models.schema.attribute_type import ( + AttributeType, + FROM_ARROW_MAPPING, + TO_ARROW_MAPPING, +) + +# Metadata key used to mark BIG_OBJECT fields in Arrow schemas +TEXERA_TYPE_METADATA_KEY = b"texera_type" +BIG_OBJECT_METADATA_VALUE = b"BIG_OBJECT" + + +def detect_attribute_type_from_arrow_field(field: pa.Field) -> AttributeType: + """ + Detects the AttributeType from an Arrow field, checking metadata for BIG_OBJECT. + + :param field: PyArrow field that may contain metadata + :return: The detected AttributeType + """ + # Check metadata for BIG_OBJECT type + # (can be stored by either Scala ArrowUtils or Python) + is_big_object = ( + field.metadata + and field.metadata.get(TEXERA_TYPE_METADATA_KEY) == BIG_OBJECT_METADATA_VALUE + ) + + if is_big_object: + return AttributeType.BIG_OBJECT + else: + return FROM_ARROW_MAPPING[field.type.id] + + +def create_arrow_field_with_metadata( + attr_name: str, attr_type: AttributeType +) -> pa.Field: + """ + Creates a PyArrow field with appropriate metadata for the given AttributeType. + + :param attr_name: Name of the attribute + :param attr_type: The AttributeType + :return: PyArrow field with metadata if needed + """ + metadata = ( + {TEXERA_TYPE_METADATA_KEY: BIG_OBJECT_METADATA_VALUE} + if attr_type == AttributeType.BIG_OBJECT + else None + ) + + return pa.field(attr_name, TO_ARROW_MAPPING[attr_type], metadata=metadata) + + +def arrow_schema_to_attr_types(arrow_schema: pa.Schema) -> dict[str, AttributeType]: + """ + Converts an Arrow schema to a dictionary of attribute name to AttributeType. + Handles BIG_OBJECT metadata detection. + + :param arrow_schema: PyArrow schema that may contain BIG_OBJECT metadata + :return: Dictionary mapping attribute names to AttributeTypes + """ + attr_types = {} + for attr_name in arrow_schema.names: + field = arrow_schema.field(attr_name) + attr_types[attr_name] = detect_attribute_type_from_arrow_field(field) + return attr_types + + +def attr_types_to_arrow_schema( + attr_types: Mapping[str, AttributeType], +) -> pa.Schema: + """ + Converts a mapping of attribute name to AttributeType into an Arrow schema. + Adds metadata for BIG_OBJECT types. + Preserves the order of attributes from the input mapping. + + :param attr_types: Mapping of attribute names to AttributeTypes (e.g., OrderedDict) + :return: PyArrow schema with metadata for BIG_OBJECT types + """ + fields = [ + create_arrow_field_with_metadata(attr_name, attr_type) + for attr_name, attr_type in attr_types.items() + ] + return pa.schema(fields) diff --git a/amber/src/main/python/core/models/schema/schema.py b/amber/src/main/python/core/models/schema/schema.py index abc85fad2e7..d349807ab79 100644 --- a/amber/src/main/python/core/models/schema/schema.py +++ b/amber/src/main/python/core/models/schema/schema.py @@ -22,8 +22,10 @@ from core.models.schema.attribute_type import ( AttributeType, RAW_TYPE_MAPPING, - FROM_ARROW_MAPPING, - TO_ARROW_MAPPING, +) +from core.models.schema.arrow_schema_utils import ( + arrow_schema_to_attr_types, + attr_types_to_arrow_schema, ) @@ -81,47 +83,21 @@ def _from_raw_schema(self, raw_schema: Mapping[str, str]) -> None: def _from_arrow_schema(self, arrow_schema: pa.Schema) -> None: """ Resets the Schema by converting a pyarrow.Schema. - Checks field metadata to detect BIG_OBJECT types. :param arrow_schema: a pyarrow.Schema. :return: """ self._name_type_mapping = OrderedDict() + attr_types = arrow_schema_to_attr_types(arrow_schema) + # Preserve field order from arrow_schema for attr_name in arrow_schema.names: - field = arrow_schema.field(attr_name) - - # Check metadata for BIG_OBJECT type - # (can be stored by either Scala ArrowUtils or Python) - is_big_object = ( - field.metadata and field.metadata.get(b"texera_type") == b"BIG_OBJECT" - ) - - attr_type = ( - AttributeType.BIG_OBJECT - if is_big_object - else FROM_ARROW_MAPPING[field.type.id] - ) - - self.add(attr_name, attr_type) + self.add(attr_name, attr_types[attr_name]) def as_arrow_schema(self) -> pa.Schema: """ Creates a new pyarrow.Schema according to the current Schema. - Includes metadata for BIG_OBJECT types to preserve type information. :return: pyarrow.Schema """ - fields = [ - pa.field( - attr_name, - TO_ARROW_MAPPING[attr_type], - metadata=( - {b"texera_type": b"BIG_OBJECT"} - if attr_type == AttributeType.BIG_OBJECT - else None - ), - ) - for attr_name, attr_type in self._name_type_mapping.items() - ] - return pa.schema(fields) + return attr_types_to_arrow_schema(self._name_type_mapping) def get_attr_names(self) -> List[str]: """ diff --git a/amber/src/main/python/core/models/tuple.py b/amber/src/main/python/core/models/tuple.py index 1c3563aef26..ff755b15da0 100644 --- a/amber/src/main/python/core/models/tuple.py +++ b/amber/src/main/python/core/models/tuple.py @@ -246,6 +246,27 @@ def as_dict(self) -> "OrderedDict[str, Field]": def as_key_value_pairs(self) -> List[typing.Tuple[str, Field]]: return [(k, v) for k, v in self.as_dict().items()] + def get_serialized_field(self, field_name: str) -> Field: + """ + Get a field value serialized for Arrow table conversion. + For BIG_OBJECT fields, converts BigObject instances to URI strings. + For other fields, returns the value as-is. + + :param field_name: field name + :return: field value (URI string for BIG_OBJECT fields with BigObject values) + """ + value = self[field_name] + + # Convert BigObject to URI string for BIG_OBJECT fields when schema is available + if ( + self._schema is not None + and self._schema.get_attr_type(field_name) == AttributeType.BIG_OBJECT + and isinstance(value, BigObject) + ): + return value.uri + + return value + def get_field_names(self) -> typing.Tuple[str]: return tuple(map(str, self._field_data.keys())) From 6a5cc33310035705fe5af4e451319dea54384d78 Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Mon, 1 Dec 2025 10:10:23 -0800 Subject: [PATCH 04/13] Add support for Python --- .../main/python/core/architecture/packaging/output_manager.py | 2 ++ amber/src/main/python/core/models/schema/schema.py | 2 ++ .../main/python/pytexera/storage/big_object_output_stream.py | 2 +- 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index bf4afbf396f..9daf32fddde 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -259,6 +259,8 @@ def emit_state( ) def tuple_to_frame(self, tuples: typing.List[Tuple]) -> DataFrame: + from core.models.schema.big_object import BigObject + return DataFrame( frame=Table.from_pydict( { diff --git a/amber/src/main/python/core/models/schema/schema.py b/amber/src/main/python/core/models/schema/schema.py index d349807ab79..ee7ca149c50 100644 --- a/amber/src/main/python/core/models/schema/schema.py +++ b/amber/src/main/python/core/models/schema/schema.py @@ -83,6 +83,7 @@ def _from_raw_schema(self, raw_schema: Mapping[str, str]) -> None: def _from_arrow_schema(self, arrow_schema: pa.Schema) -> None: """ Resets the Schema by converting a pyarrow.Schema. + Checks field metadata to detect BIG_OBJECT types. :param arrow_schema: a pyarrow.Schema. :return: """ @@ -95,6 +96,7 @@ def _from_arrow_schema(self, arrow_schema: pa.Schema) -> None: def as_arrow_schema(self) -> pa.Schema: """ Creates a new pyarrow.Schema according to the current Schema. + Includes metadata for BIG_OBJECT types to preserve type information. :return: pyarrow.Schema """ return attr_types_to_arrow_schema(self._name_type_mapping) diff --git a/amber/src/main/python/pytexera/storage/big_object_output_stream.py b/amber/src/main/python/pytexera/storage/big_object_output_stream.py index f4065ca2ac0..96149915da4 100644 --- a/amber/src/main/python/pytexera/storage/big_object_output_stream.py +++ b/amber/src/main/python/pytexera/storage/big_object_output_stream.py @@ -20,7 +20,7 @@ Usage: from pytexera import BigObject, BigObjectOutputStream - + big_object = BigObject() with BigObjectOutputStream(big_object) as out: out.write(b"data") From e42c10dfc2a467046053f6ac7e423a8df7d8ab8d Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Mon, 1 Dec 2025 10:34:01 -0800 Subject: [PATCH 05/13] Fix format --- .../main/python/pytexera/storage/big_object_output_stream.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/amber/src/main/python/pytexera/storage/big_object_output_stream.py b/amber/src/main/python/pytexera/storage/big_object_output_stream.py index 96149915da4..f4065ca2ac0 100644 --- a/amber/src/main/python/pytexera/storage/big_object_output_stream.py +++ b/amber/src/main/python/pytexera/storage/big_object_output_stream.py @@ -20,7 +20,7 @@ Usage: from pytexera import BigObject, BigObjectOutputStream - + big_object = BigObject() with BigObjectOutputStream(big_object) as out: out.write(b"data") From d35253ff526185976af7586299cad725ec2738bc Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Tue, 2 Dec 2025 17:52:27 -0800 Subject: [PATCH 06/13] Address comments --- .../main/python/core/architecture/packaging/output_manager.py | 2 -- amber/src/main/python/core/models/schema/schema.py | 2 -- 2 files changed, 4 deletions(-) diff --git a/amber/src/main/python/core/architecture/packaging/output_manager.py b/amber/src/main/python/core/architecture/packaging/output_manager.py index 9daf32fddde..bf4afbf396f 100644 --- a/amber/src/main/python/core/architecture/packaging/output_manager.py +++ b/amber/src/main/python/core/architecture/packaging/output_manager.py @@ -259,8 +259,6 @@ def emit_state( ) def tuple_to_frame(self, tuples: typing.List[Tuple]) -> DataFrame: - from core.models.schema.big_object import BigObject - return DataFrame( frame=Table.from_pydict( { diff --git a/amber/src/main/python/core/models/schema/schema.py b/amber/src/main/python/core/models/schema/schema.py index ee7ca149c50..d349807ab79 100644 --- a/amber/src/main/python/core/models/schema/schema.py +++ b/amber/src/main/python/core/models/schema/schema.py @@ -83,7 +83,6 @@ def _from_raw_schema(self, raw_schema: Mapping[str, str]) -> None: def _from_arrow_schema(self, arrow_schema: pa.Schema) -> None: """ Resets the Schema by converting a pyarrow.Schema. - Checks field metadata to detect BIG_OBJECT types. :param arrow_schema: a pyarrow.Schema. :return: """ @@ -96,7 +95,6 @@ def _from_arrow_schema(self, arrow_schema: pa.Schema) -> None: def as_arrow_schema(self) -> pa.Schema: """ Creates a new pyarrow.Schema according to the current Schema. - Includes metadata for BIG_OBJECT types to preserve type information. :return: pyarrow.Schema """ return attr_types_to_arrow_schema(self._name_type_mapping) From 830d9837cb287dfbd33d5cc8861ba54c17bc8151 Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Tue, 2 Dec 2025 19:10:59 -0800 Subject: [PATCH 07/13] Rename --- .../python/core/models/schema/__init__.py | 4 +- .../core/models/schema/attribute_type.py | 6 +- .../schema/{big_object.py => large_binary.py} | 40 +++++----- ...est_big_object.py => test_large_binary.py} | 48 ++++++------ .../src/main/python/core/models/test_tuple.py | 14 ++-- amber/src/main/python/core/models/tuple.py | 14 ++-- .../core/storage/iceberg/iceberg_utils.py | 4 +- ....py => test_iceberg_utils_large_binary.py} | 24 +++--- .../python/core/storage/storage_config.py | 2 +- amber/src/main/python/pytexera/__init__.py | 12 +-- ...stream.py => large_binary_input_stream.py} | 20 ++--- ...ect_manager.py => large_binary_manager.py} | 12 +-- ...tream.py => large_binary_output_stream.py} | 38 ++++----- ...m.py => test_large_binary_input_stream.py} | 62 +++++++-------- ...anager.py => test_large_binary_manager.py} | 36 ++++----- ....py => test_large_binary_output_stream.py} | 78 +++++++++---------- 16 files changed, 207 insertions(+), 207 deletions(-) rename amber/src/main/python/core/models/schema/{big_object.py => large_binary.py} (64%) rename amber/src/main/python/core/models/schema/{test_big_object.py => test_large_binary.py} (65%) rename amber/src/main/python/core/storage/iceberg/{test_iceberg_utils_big_object.py => test_iceberg_utils_large_binary.py} (92%) rename amber/src/main/python/pytexera/storage/{big_object_input_stream.py => large_binary_input_stream.py} (85%) rename amber/src/main/python/pytexera/storage/{big_object_manager.py => large_binary_manager.py} (88%) rename amber/src/main/python/pytexera/storage/{big_object_output_stream.py => large_binary_output_stream.py} (86%) rename amber/src/main/python/pytexera/storage/{test_big_object_input_stream.py => test_large_binary_input_stream.py} (80%) rename amber/src/main/python/pytexera/storage/{test_big_object_manager.py => test_large_binary_manager.py} (84%) rename amber/src/main/python/pytexera/storage/{test_big_object_output_stream.py => test_large_binary_output_stream.py} (73%) diff --git a/amber/src/main/python/core/models/schema/__init__.py b/amber/src/main/python/core/models/schema/__init__.py index 94ed63a8cf1..a5a0de386dd 100644 --- a/amber/src/main/python/core/models/schema/__init__.py +++ b/amber/src/main/python/core/models/schema/__init__.py @@ -16,14 +16,14 @@ # under the License. from .attribute_type import AttributeType -from .big_object import BigObject +from .large_binary import largebinary from .field import Field from .schema import Schema __all__ = [ "AttributeType", - "BigObject", + "largebinary", "Field", "Schema", ] diff --git a/amber/src/main/python/core/models/schema/attribute_type.py b/amber/src/main/python/core/models/schema/attribute_type.py index 6adbe1458fd..6f47796732d 100644 --- a/amber/src/main/python/core/models/schema/attribute_type.py +++ b/amber/src/main/python/core/models/schema/attribute_type.py @@ -20,7 +20,7 @@ from bidict import bidict from enum import Enum from pyarrow import lib -from core.models.schema.big_object import BigObject +from core.models.schema.large_binary import largebinary class AttributeType(Enum): @@ -87,7 +87,7 @@ class AttributeType(Enum): AttributeType.BOOL: bool, AttributeType.BINARY: bytes, AttributeType.TIMESTAMP: datetime.datetime, - AttributeType.BIG_OBJECT: BigObject, + AttributeType.BIG_OBJECT: largebinary, } FROM_PYOBJECT_MAPPING = { @@ -97,5 +97,5 @@ class AttributeType(Enum): bool: AttributeType.BOOL, bytes: AttributeType.BINARY, datetime.datetime: AttributeType.TIMESTAMP, - BigObject: AttributeType.BIG_OBJECT, + largebinary: AttributeType.BIG_OBJECT, } diff --git a/amber/src/main/python/core/models/schema/big_object.py b/amber/src/main/python/core/models/schema/large_binary.py similarity index 64% rename from amber/src/main/python/core/models/schema/big_object.py rename to amber/src/main/python/core/models/schema/large_binary.py index bb09e66ae2d..b14a24d3b6e 100644 --- a/amber/src/main/python/core/models/schema/big_object.py +++ b/amber/src/main/python/core/models/schema/large_binary.py @@ -16,7 +16,7 @@ # under the License. """ -BigObject represents a reference to a large object stored externally (e.g., S3). +largebinary represents a reference to a large object stored externally (e.g., S3). This is a schema type class used throughout the system for handling BIG_OBJECT attribute types. """ @@ -25,56 +25,56 @@ from urllib.parse import urlparse -class BigObject: +class largebinary: """ - BigObject represents a reference to a large object stored in S3. + largebinary represents a reference to a large object stored in S3. - Each BigObject is identified by an S3 URI (s3://bucket/path/to/object). - BigObjects are automatically tracked and cleaned up when the workflow + Each largebinary is identified by an S3 URI (s3://bucket/path/to/object). + largebinary objects are automatically tracked and cleaned up when the workflow execution completes. Usage: - from pytexera import BigObject, BigObjectInputStream, BigObjectOutputStream + from pytexera import largebinary, LargeBinaryInputStream, LargeBinaryOutputStream - # Create a new BigObject for writing - big_object = BigObject() - with BigObjectOutputStream(big_object) as out: + # Create a new largebinary for writing + big_object = largebinary() + with LargeBinaryOutputStream(big_object) as out: out.write(b"data") # big_object is now ready to be added to tuples - # Read from an existing BigObject - with BigObjectInputStream(big_object) as stream: + # Read from an existing largebinary + with LargeBinaryInputStream(big_object) as stream: content = stream.read() # Create from existing URI (e.g., from deserialization) - big_object = BigObject("s3://bucket/path/to/object") + big_object = largebinary("s3://bucket/path/to/object") """ def __init__(self, uri: Optional[str] = None): """ - Create a BigObject. + Create a largebinary. Args: uri: Optional S3 URI in the format s3://bucket/path/to/object. - If None, creates a new BigObject with a unique S3 URI. + If None, creates a new largebinary with a unique S3 URI. Raises: ValueError: If URI is provided but doesn't start with "s3://" """ if uri is None: # Lazy import to avoid circular dependencies - from pytexera.storage.big_object_manager import BigObjectManager + from pytexera.storage.large_binary_manager import LargeBinaryManager - uri = BigObjectManager.create() + uri = LargeBinaryManager.create() if not uri.startswith("s3://"): - raise ValueError(f"BigObject URI must start with 's3://', got: {uri}") + raise ValueError(f"largebinary URI must start with 's3://', got: {uri}") self._uri = uri @property def uri(self) -> str: - """Get the S3 URI of this BigObject.""" + """Get the S3 URI of this largebinary.""" return self._uri def get_bucket_name(self) -> str: @@ -89,10 +89,10 @@ def __str__(self) -> str: return self._uri def __repr__(self) -> str: - return f"BigObject('{self._uri}')" + return f"largebinary('{self._uri}')" def __eq__(self, other) -> bool: - return isinstance(other, BigObject) and self._uri == other._uri + return isinstance(other, largebinary) and self._uri == other._uri def __hash__(self) -> int: return hash(self._uri) diff --git a/amber/src/main/python/core/models/schema/test_big_object.py b/amber/src/main/python/core/models/schema/test_large_binary.py similarity index 65% rename from amber/src/main/python/core/models/schema/test_big_object.py rename to amber/src/main/python/core/models/schema/test_large_binary.py index 495e180f7d1..193d6026907 100644 --- a/amber/src/main/python/core/models/schema/test_big_object.py +++ b/amber/src/main/python/core/models/schema/test_large_binary.py @@ -17,68 +17,68 @@ import pytest from unittest.mock import patch -from core.models.schema.big_object import BigObject +from core.models.schema.large_binary import largebinary -class TestBigObject: +class TestLargeBinary: def test_create_with_uri(self): - """Test creating BigObject with a valid S3 URI.""" + """Test creating largebinary with a valid S3 URI.""" uri = "s3://test-bucket/path/to/object" - big_object = BigObject(uri) + big_object = largebinary(uri) assert big_object.uri == uri assert str(big_object) == uri - assert repr(big_object) == f"BigObject('{uri}')" + assert repr(big_object) == f"largebinary('{uri}')" def test_create_without_uri(self): - """Test creating BigObject without URI (should call BigObjectManager.create).""" + """Test creating largebinary without URI (should call LargeBinaryManager.create).""" with patch( - "pytexera.storage.big_object_manager.BigObjectManager" + "pytexera.storage.large_binary_manager.LargeBinaryManager" ) as mock_manager: mock_manager.create.return_value = "s3://bucket/objects/123/uuid" - big_object = BigObject() + big_object = largebinary() assert big_object.uri == "s3://bucket/objects/123/uuid" mock_manager.create.assert_called_once() def test_invalid_uri_raises_value_error(self): """Test that invalid URI (not starting with s3://) raises ValueError.""" - with pytest.raises(ValueError, match="BigObject URI must start with 's3://'"): - BigObject("http://invalid-uri") + with pytest.raises(ValueError, match="largebinary URI must start with 's3://'"): + largebinary("http://invalid-uri") - with pytest.raises(ValueError, match="BigObject URI must start with 's3://'"): - BigObject("invalid-uri") + with pytest.raises(ValueError, match="largebinary URI must start with 's3://'"): + largebinary("invalid-uri") def test_get_bucket_name(self): """Test extracting bucket name from URI.""" - big_object = BigObject("s3://my-bucket/path/to/object") + big_object = largebinary("s3://my-bucket/path/to/object") assert big_object.get_bucket_name() == "my-bucket" def test_get_object_key(self): """Test extracting object key from URI.""" - big_object = BigObject("s3://my-bucket/path/to/object") + big_object = largebinary("s3://my-bucket/path/to/object") assert big_object.get_object_key() == "path/to/object" def test_get_object_key_with_leading_slash(self): """Test extracting object key when URI has leading slash.""" - big_object = BigObject("s3://my-bucket/path/to/object") + big_object = largebinary("s3://my-bucket/path/to/object") # urlparse includes leading slash, but get_object_key removes it assert big_object.get_object_key() == "path/to/object" def test_equality(self): - """Test BigObject equality comparison.""" + """Test largebinary equality comparison.""" uri = "s3://bucket/path" - obj1 = BigObject(uri) - obj2 = BigObject(uri) - obj3 = BigObject("s3://bucket/different") + obj1 = largebinary(uri) + obj2 = largebinary(uri) + obj3 = largebinary("s3://bucket/different") assert obj1 == obj2 assert obj1 != obj3 - assert obj1 != "not a BigObject" + assert obj1 != "not a largebinary" def test_hash(self): - """Test BigObject hashing.""" + """Test largebinary hashing.""" uri = "s3://bucket/path" - obj1 = BigObject(uri) - obj2 = BigObject(uri) + obj1 = largebinary(uri) + obj2 = largebinary(uri) assert hash(obj1) == hash(obj2) assert hash(obj1) == hash(uri) @@ -86,5 +86,5 @@ def test_hash(self): def test_uri_property(self): """Test URI property access.""" uri = "s3://test-bucket/test/path" - big_object = BigObject(uri) + big_object = largebinary(uri) assert big_object.uri == uri diff --git a/amber/src/main/python/core/models/test_tuple.py b/amber/src/main/python/core/models/test_tuple.py index da271f61bbd..5d173d432fd 100644 --- a/amber/src/main/python/core/models/test_tuple.py +++ b/amber/src/main/python/core/models/test_tuple.py @@ -223,8 +223,8 @@ def test_hash(self): assert hash(tuple5) == -2099556631 # calculated with Java def test_tuple_with_big_object(self): - """Test tuple with BigObject field.""" - from core.models.schema.big_object import BigObject + """Test tuple with largebinary field.""" + from core.models.schema.large_binary import largebinary schema = Schema( raw_schema={ @@ -233,7 +233,7 @@ def test_tuple_with_big_object(self): } ) - big_object = BigObject("s3://test-bucket/path/to/object") + big_object = largebinary("s3://test-bucket/path/to/object") tuple_ = Tuple( { "regular_field": "test string", @@ -244,13 +244,13 @@ def test_tuple_with_big_object(self): assert tuple_["regular_field"] == "test string" assert tuple_["big_object_field"] == big_object - assert isinstance(tuple_["big_object_field"], BigObject) + assert isinstance(tuple_["big_object_field"], largebinary) assert tuple_["big_object_field"].uri == "s3://test-bucket/path/to/object" def test_tuple_from_arrow_with_big_object(self): """Test creating tuple from Arrow table with BIG_OBJECT metadata.""" import pyarrow as pa - from core.models.schema.big_object import BigObject + from core.models.schema.large_binary import largebinary # Create Arrow schema with BIG_OBJECT metadata arrow_schema = pa.schema( @@ -283,11 +283,11 @@ def test_tuple_from_arrow_with_big_object(self): assert len(tuples) == 1 tuple_ = tuples[0] assert tuple_["regular_field"] == "test" - assert isinstance(tuple_["big_object_field"], BigObject) + assert isinstance(tuple_["big_object_field"], largebinary) assert tuple_["big_object_field"].uri == "s3://test-bucket/path/to/object" def test_tuple_with_null_big_object(self): - """Test tuple with null BigObject field.""" + """Test tuple with null largebinary field.""" import pyarrow as pa # Create Arrow schema with BIG_OBJECT metadata diff --git a/amber/src/main/python/core/models/tuple.py b/amber/src/main/python/core/models/tuple.py index ff755b15da0..7d770fb61a1 100644 --- a/amber/src/main/python/core/models/tuple.py +++ b/amber/src/main/python/core/models/tuple.py @@ -29,7 +29,7 @@ from typing import Any, List, Iterator, Callable from typing_extensions import Protocol, runtime_checkable -from core.models.schema.big_object import BigObject +from core.models.schema.large_binary import largebinary from .schema.attribute_type import TO_PYOBJECT_MAPPING, AttributeType from .schema.field import Field from .schema.schema import Schema @@ -97,14 +97,14 @@ def field_accessor(field_name: str) -> Field: ): value = pickle.loads(value[10:]) - # Convert URI string to BigObject for BIG_OBJECT types + # Convert URI string to largebinary for BIG_OBJECT types # Metadata is set by Scala ArrowUtils or Python iceberg_utils elif ( value is not None and field_metadata and field_metadata.get(b"texera_type") == b"BIG_OBJECT" ): - value = BigObject(value) + value = largebinary(value) return value @@ -249,19 +249,19 @@ def as_key_value_pairs(self) -> List[typing.Tuple[str, Field]]: def get_serialized_field(self, field_name: str) -> Field: """ Get a field value serialized for Arrow table conversion. - For BIG_OBJECT fields, converts BigObject instances to URI strings. + For BIG_OBJECT fields, converts largebinary instances to URI strings. For other fields, returns the value as-is. :param field_name: field name - :return: field value (URI string for BIG_OBJECT fields with BigObject values) + :return: field value (URI string for BIG_OBJECT fields with largebinary values) """ value = self[field_name] - # Convert BigObject to URI string for BIG_OBJECT fields when schema is available + # Convert largebinary to URI string for BIG_OBJECT fields when schema is available if ( self._schema is not None and self._schema.get_attr_type(field_name) == AttributeType.BIG_OBJECT - and isinstance(value, BigObject) + and isinstance(value, largebinary) ): return value.uri diff --git a/amber/src/main/python/core/storage/iceberg/iceberg_utils.py b/amber/src/main/python/core/storage/iceberg/iceberg_utils.py index 2b23e2de82c..b7e1724e54b 100644 --- a/amber/src/main/python/core/storage/iceberg/iceberg_utils.py +++ b/amber/src/main/python/core/storage/iceberg/iceberg_utils.py @@ -231,7 +231,7 @@ def amber_tuples_to_arrow_table( Converts a list of amber tuples to a pyarrow table for serialization. Handles BIG_OBJECT field name encoding and serialization. """ - from core.models.schema.big_object import BigObject + from core.models.schema.large_binary import largebinary tuple_list = list(tuple_list) # Convert to list to allow multiple iterations data_dict = {} @@ -240,7 +240,7 @@ def amber_tuples_to_arrow_table( data_dict[encoded_name] = [ ( t[decoded_name].uri - if isinstance(t[decoded_name], BigObject) + if isinstance(t[decoded_name], largebinary) else t[decoded_name] ) for t in tuple_list diff --git a/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py b/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py similarity index 92% rename from amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py rename to amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py index f93ff4e2f7d..ec69ebb6712 100644 --- a/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_big_object.py +++ b/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py @@ -20,7 +20,7 @@ from pyiceberg.schema import Schema as IcebergSchema from core.models import Schema, Tuple from core.models.schema.attribute_type import AttributeType -from core.models.schema.big_object import BigObject +from core.models.schema.large_binary import largebinary from core.storage.iceberg.iceberg_utils import ( encode_big_object_field_name, decode_big_object_field_name, @@ -31,7 +31,7 @@ ) -class TestIcebergUtilsBigObject: +class TestIcebergUtilsLargeBinary: def test_encode_big_object_field_name(self): """Test encoding BIG_OBJECT field names with suffix.""" assert ( @@ -103,13 +103,13 @@ def test_iceberg_schema_to_amber_schema_with_big_object(self): assert big_object_field.metadata.get(b"texera_type") == b"BIG_OBJECT" def test_amber_tuples_to_arrow_table_with_big_object(self): - """Test converting Amber tuples with BigObject to Arrow table.""" + """Test converting Amber tuples with largebinary to Arrow table.""" amber_schema = Schema() amber_schema.add("regular_field", AttributeType.STRING) amber_schema.add("big_object_field", AttributeType.BIG_OBJECT) - big_object1 = BigObject("s3://bucket/path1") - big_object2 = BigObject("s3://bucket/path2") + big_object1 = largebinary("s3://bucket/path1") + big_object2 = largebinary("s3://bucket/path2") tuples = [ Tuple( @@ -125,7 +125,7 @@ def test_amber_tuples_to_arrow_table_with_big_object(self): iceberg_schema = amber_schema_to_iceberg_schema(amber_schema) arrow_table = amber_tuples_to_arrow_table(iceberg_schema, tuples) - # Check that BigObject values are converted to URI strings + # Check that largebinary values are converted to URI strings regular_values = arrow_table.column("regular_field").to_pylist() big_object_values = arrow_table.column( "big_object_field__texera_big_obj_ptr" @@ -164,20 +164,20 @@ def test_arrow_table_to_amber_tuples_with_big_object(self): assert len(tuples) == 2 assert tuples[0]["regular_field"] == "value1" - assert isinstance(tuples[0]["big_object_field"], BigObject) + assert isinstance(tuples[0]["big_object_field"], largebinary) assert tuples[0]["big_object_field"].uri == "s3://bucket/path1" assert tuples[1]["regular_field"] == "value2" - assert isinstance(tuples[1]["big_object_field"], BigObject) + assert isinstance(tuples[1]["big_object_field"], largebinary) assert tuples[1]["big_object_field"].uri == "s3://bucket/path2" def test_round_trip_big_object_tuples(self): - """Test round-trip conversion of tuples with BigObject.""" + """Test round-trip conversion of tuples with largebinary.""" amber_schema = Schema() amber_schema.add("regular_field", AttributeType.STRING) amber_schema.add("big_object_field", AttributeType.BIG_OBJECT) - big_object = BigObject("s3://bucket/path/to/object") + big_object = largebinary("s3://bucket/path/to/object") original_tuples = [ Tuple( {"regular_field": "value1", "big_object_field": big_object}, @@ -196,11 +196,11 @@ def test_round_trip_big_object_tuples(self): assert len(retrieved_tuples) == 1 assert retrieved_tuples[0]["regular_field"] == "value1" - assert isinstance(retrieved_tuples[0]["big_object_field"], BigObject) + assert isinstance(retrieved_tuples[0]["big_object_field"], largebinary) assert retrieved_tuples[0]["big_object_field"].uri == big_object.uri def test_arrow_table_to_amber_tuples_with_null_big_object(self): - """Test converting Arrow table with null BigObject values.""" + """Test converting Arrow table with null largebinary values.""" iceberg_schema = IcebergSchema( iceberg_types.NestedField( 1, "regular_field", iceberg_types.StringType(), required=False diff --git a/amber/src/main/python/core/storage/storage_config.py b/amber/src/main/python/core/storage/storage_config.py index 1b4e4e229ad..0d0d07d3284 100644 --- a/amber/src/main/python/core/storage/storage_config.py +++ b/amber/src/main/python/core/storage/storage_config.py @@ -32,7 +32,7 @@ class StorageConfig: ICEBERG_FILE_STORAGE_DIRECTORY_PATH = None ICEBERG_TABLE_COMMIT_BATCH_SIZE = None - # S3 configs (for BigObjectManager) + # S3 configs (for LargeBinaryManager) S3_ENDPOINT = None S3_REGION = None S3_AUTH_USERNAME = None diff --git a/amber/src/main/python/pytexera/__init__.py b/amber/src/main/python/pytexera/__init__.py index b2616522e20..4eec0954b29 100644 --- a/amber/src/main/python/pytexera/__init__.py +++ b/amber/src/main/python/pytexera/__init__.py @@ -21,15 +21,15 @@ from pyamber import * from .storage.dataset_file_document import DatasetFileDocument -from .storage.big_object_input_stream import BigObjectInputStream -from .storage.big_object_output_stream import BigObjectOutputStream +from .storage.large_binary_input_stream import LargeBinaryInputStream +from .storage.large_binary_output_stream import LargeBinaryOutputStream from .udf.udf_operator import ( UDFOperatorV2, UDFTableOperator, UDFBatchOperator, UDFSourceOperator, ) -from core.models.schema.big_object import BigObject +from core.models.schema.large_binary import largebinary __all__ = [ "State", @@ -44,9 +44,9 @@ "UDFBatchOperator", "UDFSourceOperator", "DatasetFileDocument", - "BigObject", - "BigObjectInputStream", - "BigObjectOutputStream", + "largebinary", + "LargeBinaryInputStream", + "LargeBinaryOutputStream", # export external tools to be used "overrides", "logger", diff --git a/amber/src/main/python/pytexera/storage/big_object_input_stream.py b/amber/src/main/python/pytexera/storage/large_binary_input_stream.py similarity index 85% rename from amber/src/main/python/pytexera/storage/big_object_input_stream.py rename to amber/src/main/python/pytexera/storage/large_binary_input_stream.py index 983f5ce91ff..d3522c209a2 100644 --- a/amber/src/main/python/pytexera/storage/big_object_input_stream.py +++ b/amber/src/main/python/pytexera/storage/large_binary_input_stream.py @@ -16,17 +16,17 @@ # under the License. """ -BigObjectInputStream for reading BigObject data from S3. +LargeBinaryInputStream for reading largebinary data from S3. Usage: - with BigObjectInputStream(big_object) as stream: + with LargeBinaryInputStream(big_object) as stream: content = stream.read() """ from typing import BinaryIO, Optional from functools import wraps from io import IOBase -from core.models.schema.big_object import BigObject +from core.models.schema.large_binary import largebinary def _require_open(func): @@ -43,27 +43,27 @@ def wrapper(self, *args, **kwargs): return wrapper -class BigObjectInputStream(IOBase): +class LargeBinaryInputStream(IOBase): """ - InputStream for reading BigObject data from S3. + InputStream for reading largebinary data from S3. Lazily downloads from S3 on first read. Supports context manager and iteration. """ - def __init__(self, big_object: BigObject): - """Initialize stream for reading the given BigObject.""" + def __init__(self, big_object: largebinary): + """Initialize stream for reading the given largebinary.""" super().__init__() if big_object is None: - raise ValueError("BigObject cannot be None") + raise ValueError("largebinary cannot be None") self._big_object = big_object self._underlying: Optional[BinaryIO] = None self._closed = False def _lazy_init(self): """Download from S3 on first read operation.""" - from pytexera.storage.big_object_manager import BigObjectManager + from pytexera.storage.large_binary_manager import LargeBinaryManager - s3 = BigObjectManager._get_s3_client() + s3 = LargeBinaryManager._get_s3_client() response = s3.get_object( Bucket=self._big_object.get_bucket_name(), Key=self._big_object.get_object_key(), diff --git a/amber/src/main/python/pytexera/storage/big_object_manager.py b/amber/src/main/python/pytexera/storage/large_binary_manager.py similarity index 88% rename from amber/src/main/python/pytexera/storage/big_object_manager.py rename to amber/src/main/python/pytexera/storage/large_binary_manager.py index 577ead00478..b219bd486d5 100644 --- a/amber/src/main/python/pytexera/storage/big_object_manager.py +++ b/amber/src/main/python/pytexera/storage/large_binary_manager.py @@ -16,10 +16,10 @@ # under the License. """ -Internal BigObject manager for S3 operations. +Internal largebinary manager for S3 operations. -Users should not interact with this module directly. Use BigObject() constructor -and BigObjectInputStream/BigObjectOutputStream instead. +Users should not interact with this module directly. Use largebinary() constructor +and LargeBinaryInputStream/LargeBinaryOutputStream instead. """ import logging @@ -30,8 +30,8 @@ logger = logging.getLogger(__name__) -class BigObjectManager: - """Internal manager for BigObject S3 operations.""" +class LargeBinaryManager: + """Internal manager for largebinary S3 operations.""" _s3_client = None DEFAULT_BUCKET = "texera-big-objects" @@ -74,7 +74,7 @@ def _ensure_bucket_exists(cls, bucket: str): @classmethod def create(cls) -> str: """ - Creates a new BigObject reference with a unique S3 URI. + Creates a new largebinary reference with a unique S3 URI. Returns: S3 URI string (format: s3://bucket/key) diff --git a/amber/src/main/python/pytexera/storage/big_object_output_stream.py b/amber/src/main/python/pytexera/storage/large_binary_output_stream.py similarity index 86% rename from amber/src/main/python/pytexera/storage/big_object_output_stream.py rename to amber/src/main/python/pytexera/storage/large_binary_output_stream.py index f4065ca2ac0..c80f916e26d 100644 --- a/amber/src/main/python/pytexera/storage/big_object_output_stream.py +++ b/amber/src/main/python/pytexera/storage/large_binary_output_stream.py @@ -16,20 +16,20 @@ # under the License. """ -BigObjectOutputStream for streaming BigObject data to S3. +LargeBinaryOutputStream for streaming largebinary data to S3. Usage: - from pytexera import BigObject, BigObjectOutputStream + from pytexera import largebinary, LargeBinaryOutputStream - big_object = BigObject() - with BigObjectOutputStream(big_object) as out: + big_object = largebinary() + with LargeBinaryOutputStream(big_object) as out: out.write(b"data") """ from typing import Optional, Union from io import IOBase -from core.models.schema.big_object import BigObject -from pytexera.storage.big_object_manager import BigObjectManager +from core.models.schema.large_binary import largebinary +from pytexera.storage.large_binary_manager import LargeBinaryManager import threading import queue @@ -78,9 +78,9 @@ def read(self, size=-1): return result -class BigObjectOutputStream(IOBase): +class LargeBinaryOutputStream(IOBase): """ - OutputStream for streaming BigObject data to S3. + OutputStream for streaming largebinary data to S3. Data is uploaded in the background using multipart upload as you write. Call close() to complete the upload and ensure all data is persisted. @@ -88,11 +88,11 @@ class BigObjectOutputStream(IOBase): This class follows Python's standard I/O interface (io.IOBase). Usage: - from pytexera import BigObject, BigObjectOutputStream + from pytexera import largebinary, LargeBinaryOutputStream - # Create a new BigObject and write to it - big_object = BigObject() - with BigObjectOutputStream(big_object) as out: + # Create a new largebinary and write to it + big_object = largebinary() + with LargeBinaryOutputStream(big_object) as out: out.write(b"Hello, World!") out.write(b"More data") # big_object is now ready to be added to tuples @@ -100,19 +100,19 @@ class BigObjectOutputStream(IOBase): Note: Not thread-safe. Do not access from multiple threads concurrently. """ - def __init__(self, big_object: BigObject): + def __init__(self, big_object: largebinary): """ - Initialize a BigObjectOutputStream. + Initialize a LargeBinaryOutputStream. Args: - big_object: The BigObject reference to write to + big_object: The largebinary reference to write to Raises: ValueError: If big_object is None """ super().__init__() if big_object is None: - raise ValueError("BigObject cannot be None") + raise ValueError("largebinary cannot be None") self._big_object = big_object self._bucket_name = big_object.get_bucket_name() @@ -155,8 +155,8 @@ def write(self, b: Union[bytes, bytearray]) -> int: def upload_worker(): try: - BigObjectManager._ensure_bucket_exists(self._bucket_name) - s3 = BigObjectManager._get_s3_client() + LargeBinaryManager._ensure_bucket_exists(self._bucket_name) + s3 = LargeBinaryManager._get_s3_client() reader = _QueueReader(self._queue) s3.upload_fileobj(reader, self._bucket_name, self._object_key) except Exception as e: @@ -228,7 +228,7 @@ def close(self) -> None: def _cleanup_failed_upload(self): """Clean up a failed upload by deleting the S3 object.""" try: - s3 = BigObjectManager._get_s3_client() + s3 = LargeBinaryManager._get_s3_client() s3.delete_object(Bucket=self._bucket_name, Key=self._object_key) except Exception: # Ignore cleanup errors - we're already handling an upload failure diff --git a/amber/src/main/python/pytexera/storage/test_big_object_input_stream.py b/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py similarity index 80% rename from amber/src/main/python/pytexera/storage/test_big_object_input_stream.py rename to amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py index f15bfd8849a..8648decf2cc 100644 --- a/amber/src/main/python/pytexera/storage/test_big_object_input_stream.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py @@ -18,16 +18,16 @@ import pytest from unittest.mock import patch, MagicMock from io import BytesIO -from core.models.schema.big_object import BigObject -from pytexera.storage.big_object_input_stream import BigObjectInputStream -from pytexera.storage import big_object_manager +from core.models.schema.large_binary import largebinary +from pytexera.storage.large_binary_input_stream import LargeBinaryInputStream +from pytexera.storage import large_binary_manager -class TestBigObjectInputStream: +class TestLargeBinaryInputStream: @pytest.fixture def big_object(self): - """Create a test BigObject.""" - return BigObject("s3://test-bucket/path/to/object") + """Create a test largebinary.""" + return largebinary("s3://test-bucket/path/to/object") @pytest.fixture def mock_s3_response(self): @@ -35,8 +35,8 @@ def mock_s3_response(self): return {"Body": BytesIO(b"test data content")} def test_init_with_valid_big_object(self, big_object): - """Test initialization with a valid BigObject.""" - stream = BigObjectInputStream(big_object) + """Test initialization with a valid largebinary.""" + stream = LargeBinaryInputStream(big_object) try: assert stream._big_object == big_object assert stream._underlying is None @@ -46,19 +46,19 @@ def test_init_with_valid_big_object(self, big_object): def test_init_with_none_raises_error(self): """Test that initializing with None raises ValueError.""" - with pytest.raises(ValueError, match="BigObject cannot be None"): - BigObjectInputStream(None) + with pytest.raises(ValueError, match="largebinary cannot be None"): + LargeBinaryInputStream(None) def test_lazy_init_downloads_from_s3(self, big_object, mock_s3_response): """Test that _lazy_init downloads from S3 on first read.""" with patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) try: assert stream._underlying is None # Not initialized yet @@ -77,13 +77,13 @@ def test_lazy_init_downloads_from_s3(self, big_object, mock_s3_response): def test_read_all(self, big_object, mock_s3_response): """Test reading all data.""" with patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) try: data = stream.read() assert data == b"test data content" @@ -93,13 +93,13 @@ def test_read_all(self, big_object, mock_s3_response): def test_read_partial(self, big_object, mock_s3_response): """Test reading partial data.""" with patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) try: data = stream.read(4) assert data == b"test" @@ -109,14 +109,14 @@ def test_read_partial(self, big_object, mock_s3_response): def test_readline(self, big_object): """Test reading a line.""" with patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client: response = {"Body": BytesIO(b"line1\nline2\nline3")} mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = response mock_get_s3_client.return_value = mock_s3_client - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) try: line = stream.readline() assert line == b"line1\n" @@ -126,14 +126,14 @@ def test_readline(self, big_object): def test_readlines(self, big_object): """Test reading all lines.""" with patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client: response = {"Body": BytesIO(b"line1\nline2\nline3")} mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = response mock_get_s3_client.return_value = mock_s3_client - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) try: lines = stream.readlines() assert lines == [b"line1\n", b"line2\n", b"line3"] @@ -142,7 +142,7 @@ def test_readlines(self, big_object): def test_readable(self, big_object): """Test readable() method.""" - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) try: assert stream.readable() is True @@ -154,7 +154,7 @@ def test_readable(self, big_object): def test_seekable(self, big_object): """Test seekable() method (should always return False).""" - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) try: assert stream.seekable() is False finally: @@ -162,7 +162,7 @@ def test_seekable(self, big_object): def test_closed_property(self, big_object): """Test closed property.""" - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) try: assert stream.closed is False @@ -175,13 +175,13 @@ def test_closed_property(self, big_object): def test_close(self, big_object, mock_s3_response): """Test closing the stream.""" with patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) stream.read(1) # Trigger lazy init assert stream._underlying is not None @@ -192,13 +192,13 @@ def test_close(self, big_object, mock_s3_response): def test_context_manager(self, big_object, mock_s3_response): """Test using as context manager.""" with patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - with BigObjectInputStream(big_object) as stream: + with LargeBinaryInputStream(big_object) as stream: data = stream.read() assert data == b"test data content" assert not stream._closed @@ -209,14 +209,14 @@ def test_context_manager(self, big_object, mock_s3_response): def test_iteration(self, big_object): """Test iteration over lines.""" with patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client: response = {"Body": BytesIO(b"line1\nline2\nline3")} mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = response mock_get_s3_client.return_value = mock_s3_client - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) try: lines = list(stream) assert lines == [b"line1\n", b"line2\n", b"line3"] @@ -226,13 +226,13 @@ def test_iteration(self, big_object): def test_read_after_close_raises_error(self, big_object, mock_s3_response): """Test that reading after close raises ValueError.""" with patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - stream = BigObjectInputStream(big_object) + stream = LargeBinaryInputStream(big_object) stream.close() with pytest.raises(ValueError, match="I/O operation on closed stream"): diff --git a/amber/src/main/python/pytexera/storage/test_big_object_manager.py b/amber/src/main/python/pytexera/storage/test_large_binary_manager.py similarity index 84% rename from amber/src/main/python/pytexera/storage/test_big_object_manager.py rename to amber/src/main/python/pytexera/storage/test_large_binary_manager.py index f64c68bb6d7..8d9e3e223ef 100644 --- a/amber/src/main/python/pytexera/storage/test_big_object_manager.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_manager.py @@ -17,11 +17,11 @@ import pytest from unittest.mock import patch, MagicMock -from pytexera.storage.big_object_manager import BigObjectManager +from pytexera.storage.large_binary_manager import LargeBinaryManager from core.storage.storage_config import StorageConfig -class TestBigObjectManager: +class TestLargeBinaryManager: @pytest.fixture(autouse=True) def setup_storage_config(self): """Initialize StorageConfig for tests.""" @@ -42,25 +42,25 @@ def setup_storage_config(self): def test_get_s3_client_initializes_once(self): """Test that S3 client is initialized and cached.""" # Reset the client - BigObjectManager._s3_client = None + LargeBinaryManager._s3_client = None with patch("boto3.client") as mock_boto3_client: mock_client = MagicMock() mock_boto3_client.return_value = mock_client # First call should create client - client1 = BigObjectManager._get_s3_client() + client1 = LargeBinaryManager._get_s3_client() assert client1 == mock_client assert mock_boto3_client.call_count == 1 # Second call should return cached client - client2 = BigObjectManager._get_s3_client() + client2 = LargeBinaryManager._get_s3_client() assert client2 == mock_client assert mock_boto3_client.call_count == 1 # Still 1, not 2 def test_get_s3_client_without_boto3_raises_error(self): """Test that missing boto3 raises RuntimeError.""" - BigObjectManager._s3_client = None + LargeBinaryManager._s3_client = None import sys @@ -77,7 +77,7 @@ def mock_import(name, *args, **kwargs): with patch("builtins.__import__", side_effect=mock_import): with pytest.raises(RuntimeError, match="boto3 required"): - BigObjectManager._get_s3_client() + LargeBinaryManager._get_s3_client() finally: # Restore boto3 if it was there if boto3_backup is not None: @@ -85,7 +85,7 @@ def mock_import(name, *args, **kwargs): def test_ensure_bucket_exists_when_bucket_exists(self): """Test that existing bucket doesn't trigger creation.""" - BigObjectManager._s3_client = None + LargeBinaryManager._s3_client = None with patch("boto3.client") as mock_boto3_client: mock_client = MagicMock() @@ -94,13 +94,13 @@ def test_ensure_bucket_exists_when_bucket_exists(self): mock_client.head_bucket.return_value = None mock_client.exceptions.NoSuchBucket = type("NoSuchBucket", (Exception,), {}) - BigObjectManager._ensure_bucket_exists("test-bucket") + LargeBinaryManager._ensure_bucket_exists("test-bucket") mock_client.head_bucket.assert_called_once_with(Bucket="test-bucket") mock_client.create_bucket.assert_not_called() def test_ensure_bucket_exists_creates_bucket_when_missing(self): """Test that missing bucket triggers creation.""" - BigObjectManager._s3_client = None + LargeBinaryManager._s3_client = None with patch("boto3.client") as mock_boto3_client: mock_client = MagicMock() @@ -110,13 +110,13 @@ def test_ensure_bucket_exists_creates_bucket_when_missing(self): mock_client.exceptions.NoSuchBucket = no_such_bucket mock_client.head_bucket.side_effect = no_such_bucket() - BigObjectManager._ensure_bucket_exists("test-bucket") + LargeBinaryManager._ensure_bucket_exists("test-bucket") mock_client.head_bucket.assert_called_once_with(Bucket="test-bucket") mock_client.create_bucket.assert_called_once_with(Bucket="test-bucket") def test_create_generates_unique_uri(self): """Test that create() generates a unique S3 URI.""" - BigObjectManager._s3_client = None + LargeBinaryManager._s3_client = None with patch("boto3.client") as mock_boto3_client: mock_client = MagicMock() @@ -124,21 +124,21 @@ def test_create_generates_unique_uri(self): mock_client.head_bucket.return_value = None mock_client.exceptions.NoSuchBucket = type("NoSuchBucket", (Exception,), {}) - uri = BigObjectManager.create() + uri = LargeBinaryManager.create() # Check URI format assert uri.startswith("s3://") - assert uri.startswith(f"s3://{BigObjectManager.DEFAULT_BUCKET}/") + assert uri.startswith(f"s3://{LargeBinaryManager.DEFAULT_BUCKET}/") assert "objects/" in uri # Verify bucket was checked/created mock_client.head_bucket.assert_called_once_with( - Bucket=BigObjectManager.DEFAULT_BUCKET + Bucket=LargeBinaryManager.DEFAULT_BUCKET ) def test_create_uses_default_bucket(self): """Test that create() uses the default bucket.""" - BigObjectManager._s3_client = None + LargeBinaryManager._s3_client = None with patch("boto3.client") as mock_boto3_client: mock_client = MagicMock() @@ -146,5 +146,5 @@ def test_create_uses_default_bucket(self): mock_client.head_bucket.return_value = None mock_client.exceptions.NoSuchBucket = type("NoSuchBucket", (Exception,), {}) - uri = BigObjectManager.create() - assert BigObjectManager.DEFAULT_BUCKET in uri + uri = LargeBinaryManager.create() + assert LargeBinaryManager.DEFAULT_BUCKET in uri diff --git a/amber/src/main/python/pytexera/storage/test_big_object_output_stream.py b/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py similarity index 73% rename from amber/src/main/python/pytexera/storage/test_big_object_output_stream.py rename to amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py index fa7a27cde3a..1793a27a082 100644 --- a/amber/src/main/python/pytexera/storage/test_big_object_output_stream.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py @@ -18,20 +18,20 @@ import pytest import time from unittest.mock import patch, MagicMock -from core.models.schema.big_object import BigObject -from pytexera.storage.big_object_output_stream import BigObjectOutputStream -from pytexera.storage import big_object_manager +from core.models.schema.large_binary import largebinary +from pytexera.storage.large_binary_output_stream import LargeBinaryOutputStream +from pytexera.storage import large_binary_manager -class TestBigObjectOutputStream: +class TestLargeBinaryOutputStream: @pytest.fixture def big_object(self): - """Create a test BigObject.""" - return BigObject("s3://test-bucket/path/to/object") + """Create a test largebinary.""" + return largebinary("s3://test-bucket/path/to/object") def test_init_with_valid_big_object(self, big_object): - """Test initialization with a valid BigObject.""" - stream = BigObjectOutputStream(big_object) + """Test initialization with a valid largebinary.""" + stream = LargeBinaryOutputStream(big_object) assert stream._big_object == big_object assert stream._bucket_name == "test-bucket" assert stream._object_key == "path/to/object" @@ -40,24 +40,24 @@ def test_init_with_valid_big_object(self, big_object): def test_init_with_none_raises_error(self): """Test that initializing with None raises ValueError.""" - with pytest.raises(ValueError, match="BigObject cannot be None"): - BigObjectOutputStream(None) + with pytest.raises(ValueError, match="largebinary cannot be None"): + LargeBinaryOutputStream(None) def test_write_starts_upload_thread(self, big_object): """Test that write() starts the upload thread.""" with ( patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - big_object_manager.BigObjectManager, "_ensure_bucket_exists" + large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) assert stream._upload_thread is None stream.write(b"test data") @@ -72,17 +72,17 @@ def test_write_data(self, big_object): """Test writing data to the stream.""" with ( patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - big_object_manager.BigObjectManager, "_ensure_bucket_exists" + large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) bytes_written = stream.write(b"test data") assert bytes_written == len(b"test data") @@ -92,17 +92,17 @@ def test_write_multiple_chunks(self, big_object): """Test writing multiple chunks of data.""" with ( patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - big_object_manager.BigObjectManager, "_ensure_bucket_exists" + large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) stream.write(b"chunk1") stream.write(b"chunk2") stream.write(b"chunk3") @@ -111,7 +111,7 @@ def test_write_multiple_chunks(self, big_object): def test_writable(self, big_object): """Test writable() method.""" - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) assert stream.writable() is True stream.close() @@ -119,12 +119,12 @@ def test_writable(self, big_object): def test_seekable(self, big_object): """Test seekable() method (should always return False).""" - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) assert stream.seekable() is False def test_closed_property(self, big_object): """Test closed property.""" - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) assert stream.closed is False stream.close() @@ -132,7 +132,7 @@ def test_closed_property(self, big_object): def test_flush(self, big_object): """Test flush() method (should be a no-op).""" - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) # Should not raise any exception stream.flush() @@ -140,17 +140,17 @@ def test_close_completes_upload(self, big_object): """Test that close() completes the upload.""" with ( patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - big_object_manager.BigObjectManager, "_ensure_bucket_exists" + large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) stream.write(b"test data") # Close should wait for upload to complete @@ -163,17 +163,17 @@ def test_context_manager(self, big_object): """Test using as context manager.""" with ( patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - big_object_manager.BigObjectManager, "_ensure_bucket_exists" + large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - with BigObjectOutputStream(big_object) as stream: + with LargeBinaryOutputStream(big_object) as stream: stream.write(b"test data") assert not stream._closed @@ -182,7 +182,7 @@ def test_context_manager(self, big_object): def test_write_after_close_raises_error(self, big_object): """Test that writing after close raises ValueError.""" - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) stream.close() with pytest.raises(ValueError, match="I/O operation on closed stream"): @@ -192,10 +192,10 @@ def test_close_handles_upload_error(self, big_object): """Test that close() raises IOError if upload fails.""" with ( patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - big_object_manager.BigObjectManager, "_ensure_bucket_exists" + large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() @@ -203,7 +203,7 @@ def test_close_handles_upload_error(self, big_object): mock_ensure_bucket.return_value = None mock_s3.upload_fileobj.side_effect = Exception("Upload failed") - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) stream.write(b"test data") with pytest.raises(IOError, match="Failed to complete upload"): @@ -213,10 +213,10 @@ def test_write_after_upload_error_raises_error(self, big_object): """Test that writing after upload error raises IOError.""" with ( patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - big_object_manager.BigObjectManager, "_ensure_bucket_exists" + large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() @@ -224,7 +224,7 @@ def test_write_after_upload_error_raises_error(self, big_object): mock_ensure_bucket.return_value = None mock_s3.upload_fileobj.side_effect = Exception("Upload failed") - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) stream.write(b"test data") # Wait a bit for the error to be set @@ -237,17 +237,17 @@ def test_multiple_close_calls(self, big_object): """Test that multiple close() calls are safe.""" with ( patch.object( - big_object_manager.BigObjectManager, "_get_s3_client" + large_binary_manager.LargeBinaryManager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - big_object_manager.BigObjectManager, "_ensure_bucket_exists" + large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - stream = BigObjectOutputStream(big_object) + stream = LargeBinaryOutputStream(big_object) stream.write(b"test data") stream.close() # Second close should not raise error From ed92f9892285669774a65001fd807a077ea611ca Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Tue, 2 Dec 2025 20:10:04 -0800 Subject: [PATCH 08/13] flake8 --- amber/src/main/python/core/models/schema/test_large_binary.py | 2 +- amber/src/main/python/core/models/tuple.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/amber/src/main/python/core/models/schema/test_large_binary.py b/amber/src/main/python/core/models/schema/test_large_binary.py index 193d6026907..ccea8353986 100644 --- a/amber/src/main/python/core/models/schema/test_large_binary.py +++ b/amber/src/main/python/core/models/schema/test_large_binary.py @@ -30,7 +30,7 @@ def test_create_with_uri(self): assert repr(big_object) == f"largebinary('{uri}')" def test_create_without_uri(self): - """Test creating largebinary without URI (should call LargeBinaryManager.create).""" + """Test creating largebinary without URI (calls LargeBinaryManager.create).""" with patch( "pytexera.storage.large_binary_manager.LargeBinaryManager" ) as mock_manager: diff --git a/amber/src/main/python/core/models/tuple.py b/amber/src/main/python/core/models/tuple.py index 7d770fb61a1..0ab76408c48 100644 --- a/amber/src/main/python/core/models/tuple.py +++ b/amber/src/main/python/core/models/tuple.py @@ -257,7 +257,7 @@ def get_serialized_field(self, field_name: str) -> Field: """ value = self[field_name] - # Convert largebinary to URI string for BIG_OBJECT fields when schema is available + # Convert largebinary to URI string for BIG_OBJECT fields when schema available if ( self._schema is not None and self._schema.get_attr_type(field_name) == AttributeType.BIG_OBJECT From b8e37ae6262e2265ebb4f2d91062b7827d16c6af Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Mon, 8 Dec 2025 17:30:54 -0800 Subject: [PATCH 09/13] Rename to Large Binary --- .../embedded_control_message_manager.py | 6 +- .../architecture/managers/executor_manager.py | 12 +- .../core/models/schema/arrow_schema_utils.py | 30 ++--- .../core/models/schema/attribute_type.py | 10 +- .../python/core/models/schema/large_binary.py | 12 +- .../core/models/schema/test_large_binary.py | 28 ++--- .../python/core/models/schema/test_schema.py | 44 +++---- .../src/main/python/core/models/test_tuple.py | 44 +++---- amber/src/main/python/core/models/tuple.py | 18 +-- .../python/core/storage/document_factory.py | 2 +- .../core/storage/iceberg/iceberg_utils.py | 46 ++++---- .../storage/iceberg/test_iceberg_document.py | 33 +++--- .../test_iceberg_utils_large_binary.py | 111 +++++++++--------- .../storage/large_binary_input_stream.py | 12 +- .../pytexera/storage/large_binary_manager.py | 2 +- .../storage/large_binary_output_stream.py | 24 ++-- .../storage/test_large_binary_input_stream.py | 56 ++++----- .../test_large_binary_output_stream.py | 60 +++++----- .../texera/web/service/WorkflowService.scala | 2 +- .../apache/texera/amber/util/ArrowUtils.scala | 16 +-- .../texera/amber/util/ArrowUtilsSpec.scala | 94 +++++++-------- 21 files changed, 334 insertions(+), 328 deletions(-) diff --git a/amber/src/main/python/core/architecture/managers/embedded_control_message_manager.py b/amber/src/main/python/core/architecture/managers/embedded_control_message_manager.py index 8ba80f28cef..43acb1c4e18 100644 --- a/amber/src/main/python/core/architecture/managers/embedded_control_message_manager.py +++ b/amber/src/main/python/core/architecture/managers/embedded_control_message_manager.py @@ -74,9 +74,9 @@ def is_ecm_aligned( return ecm_completed - def get_channels_within_scope(self, ecm: EmbeddedControlMessage) -> Dict[ - "ChannelIdentity", "Channel" - ].keys: + def get_channels_within_scope( + self, ecm: EmbeddedControlMessage + ) -> Dict["ChannelIdentity", "Channel"].keys: if ecm.scope: upstreams = { channel_id diff --git a/amber/src/main/python/core/architecture/managers/executor_manager.py b/amber/src/main/python/core/architecture/managers/executor_manager.py index 53e5a8903da..fc07ce74f51 100644 --- a/amber/src/main/python/core/architecture/managers/executor_manager.py +++ b/amber/src/main/python/core/architecture/managers/executor_manager.py @@ -139,9 +139,9 @@ class declaration. executor: type(Operator) = self.load_executor_definition(code) self.executor = executor() self.executor.is_source = is_source - assert isinstance(self.executor, SourceOperator) == self.executor.is_source, ( - "Please use SourceOperator API for source operators." - ) + assert ( + isinstance(self.executor, SourceOperator) == self.executor.is_source + ), "Please use SourceOperator API for source operators." def update_executor(self, code: str, is_source: bool) -> None: """ @@ -157,9 +157,9 @@ class declaration. executor: type(Operator) = self.load_executor_definition(code) self.executor = executor() self.executor.is_source = is_source - assert isinstance(self.executor, SourceOperator) == self.executor.is_source, ( - "Please use SourceOperator API for source operators." - ) + assert ( + isinstance(self.executor, SourceOperator) == self.executor.is_source + ), "Please use SourceOperator API for source operators." # overwrite the internal state self.executor.__dict__ = original_internal_state # TODO: diff --git a/amber/src/main/python/core/models/schema/arrow_schema_utils.py b/amber/src/main/python/core/models/schema/arrow_schema_utils.py index 2884c92c4e2..25ac5fa31de 100644 --- a/amber/src/main/python/core/models/schema/arrow_schema_utils.py +++ b/amber/src/main/python/core/models/schema/arrow_schema_utils.py @@ -17,7 +17,7 @@ """ Utilities for converting between Arrow schemas and Amber schemas, -handling BIG_OBJECT metadata preservation. +handling LARGE_BINARY metadata preservation. """ import pyarrow as pa @@ -29,27 +29,27 @@ TO_ARROW_MAPPING, ) -# Metadata key used to mark BIG_OBJECT fields in Arrow schemas +# Metadata key used to mark LARGE_BINARY fields in Arrow schemas TEXERA_TYPE_METADATA_KEY = b"texera_type" -BIG_OBJECT_METADATA_VALUE = b"BIG_OBJECT" +LARGE_BINARY_METADATA_VALUE = b"LARGE_BINARY" def detect_attribute_type_from_arrow_field(field: pa.Field) -> AttributeType: """ - Detects the AttributeType from an Arrow field, checking metadata for BIG_OBJECT. + Detects the AttributeType from an Arrow field, checking metadata for LARGE_BINARY. :param field: PyArrow field that may contain metadata :return: The detected AttributeType """ - # Check metadata for BIG_OBJECT type + # Check metadata for LARGE_BINARY type # (can be stored by either Scala ArrowUtils or Python) - is_big_object = ( + is_large_binary = ( field.metadata - and field.metadata.get(TEXERA_TYPE_METADATA_KEY) == BIG_OBJECT_METADATA_VALUE + and field.metadata.get(TEXERA_TYPE_METADATA_KEY) == LARGE_BINARY_METADATA_VALUE ) - if is_big_object: - return AttributeType.BIG_OBJECT + if is_large_binary: + return AttributeType.LARGE_BINARY else: return FROM_ARROW_MAPPING[field.type.id] @@ -65,8 +65,8 @@ def create_arrow_field_with_metadata( :return: PyArrow field with metadata if needed """ metadata = ( - {TEXERA_TYPE_METADATA_KEY: BIG_OBJECT_METADATA_VALUE} - if attr_type == AttributeType.BIG_OBJECT + {TEXERA_TYPE_METADATA_KEY: LARGE_BINARY_METADATA_VALUE} + if attr_type == AttributeType.LARGE_BINARY else None ) @@ -76,9 +76,9 @@ def create_arrow_field_with_metadata( def arrow_schema_to_attr_types(arrow_schema: pa.Schema) -> dict[str, AttributeType]: """ Converts an Arrow schema to a dictionary of attribute name to AttributeType. - Handles BIG_OBJECT metadata detection. + Handles LARGE_BINARY metadata detection. - :param arrow_schema: PyArrow schema that may contain BIG_OBJECT metadata + :param arrow_schema: PyArrow schema that may contain LARGE_BINARY metadata :return: Dictionary mapping attribute names to AttributeTypes """ attr_types = {} @@ -93,11 +93,11 @@ def attr_types_to_arrow_schema( ) -> pa.Schema: """ Converts a mapping of attribute name to AttributeType into an Arrow schema. - Adds metadata for BIG_OBJECT types. + Adds metadata for LARGE_BINARY types. Preserves the order of attributes from the input mapping. :param attr_types: Mapping of attribute names to AttributeTypes (e.g., OrderedDict) - :return: PyArrow schema with metadata for BIG_OBJECT types + :return: PyArrow schema with metadata for LARGE_BINARY types """ fields = [ create_arrow_field_with_metadata(attr_name, attr_type) diff --git a/amber/src/main/python/core/models/schema/attribute_type.py b/amber/src/main/python/core/models/schema/attribute_type.py index 6f47796732d..8b8011c55ec 100644 --- a/amber/src/main/python/core/models/schema/attribute_type.py +++ b/amber/src/main/python/core/models/schema/attribute_type.py @@ -38,7 +38,7 @@ class AttributeType(Enum): DOUBLE = 5 TIMESTAMP = 6 BINARY = 7 - BIG_OBJECT = 8 + LARGE_BINARY = 8 RAW_TYPE_MAPPING = bidict( @@ -50,7 +50,7 @@ class AttributeType(Enum): "BOOLEAN": AttributeType.BOOL, "TIMESTAMP": AttributeType.TIMESTAMP, "BINARY": AttributeType.BINARY, - "BIG_OBJECT": AttributeType.BIG_OBJECT, + "LARGE_BINARY": AttributeType.LARGE_BINARY, } ) @@ -62,7 +62,7 @@ class AttributeType(Enum): AttributeType.BOOL: pa.bool_(), AttributeType.BINARY: pa.binary(), AttributeType.TIMESTAMP: pa.timestamp("us"), - AttributeType.BIG_OBJECT: pa.string(), # Serialized as URI string + AttributeType.LARGE_BINARY: pa.string(), # Serialized as URI string } FROM_ARROW_MAPPING = { @@ -87,7 +87,7 @@ class AttributeType(Enum): AttributeType.BOOL: bool, AttributeType.BINARY: bytes, AttributeType.TIMESTAMP: datetime.datetime, - AttributeType.BIG_OBJECT: largebinary, + AttributeType.LARGE_BINARY: largebinary, } FROM_PYOBJECT_MAPPING = { @@ -97,5 +97,5 @@ class AttributeType(Enum): bool: AttributeType.BOOL, bytes: AttributeType.BINARY, datetime.datetime: AttributeType.TIMESTAMP, - largebinary: AttributeType.BIG_OBJECT, + largebinary: AttributeType.LARGE_BINARY, } diff --git a/amber/src/main/python/core/models/schema/large_binary.py b/amber/src/main/python/core/models/schema/large_binary.py index b14a24d3b6e..578a3520a86 100644 --- a/amber/src/main/python/core/models/schema/large_binary.py +++ b/amber/src/main/python/core/models/schema/large_binary.py @@ -18,7 +18,7 @@ """ largebinary represents a reference to a large object stored externally (e.g., S3). This is a schema type class used throughout the system for handling -BIG_OBJECT attribute types. +LARGE_BINARY attribute types. """ from typing import Optional @@ -37,17 +37,17 @@ class largebinary: from pytexera import largebinary, LargeBinaryInputStream, LargeBinaryOutputStream # Create a new largebinary for writing - big_object = largebinary() - with LargeBinaryOutputStream(big_object) as out: + large_binary = largebinary() + with LargeBinaryOutputStream(large_binary) as out: out.write(b"data") - # big_object is now ready to be added to tuples + # large_binary is now ready to be added to tuples # Read from an existing largebinary - with LargeBinaryInputStream(big_object) as stream: + with LargeBinaryInputStream(large_binary) as stream: content = stream.read() # Create from existing URI (e.g., from deserialization) - big_object = largebinary("s3://bucket/path/to/object") + large_binary = largebinary("s3://bucket/path/to/object") """ def __init__(self, uri: Optional[str] = None): diff --git a/amber/src/main/python/core/models/schema/test_large_binary.py b/amber/src/main/python/core/models/schema/test_large_binary.py index ccea8353986..d8fc93baa7a 100644 --- a/amber/src/main/python/core/models/schema/test_large_binary.py +++ b/amber/src/main/python/core/models/schema/test_large_binary.py @@ -24,10 +24,10 @@ class TestLargeBinary: def test_create_with_uri(self): """Test creating largebinary with a valid S3 URI.""" uri = "s3://test-bucket/path/to/object" - big_object = largebinary(uri) - assert big_object.uri == uri - assert str(big_object) == uri - assert repr(big_object) == f"largebinary('{uri}')" + large_binary = largebinary(uri) + assert large_binary.uri == uri + assert str(large_binary) == uri + assert repr(large_binary) == f"largebinary('{uri}')" def test_create_without_uri(self): """Test creating largebinary without URI (calls LargeBinaryManager.create).""" @@ -35,8 +35,8 @@ def test_create_without_uri(self): "pytexera.storage.large_binary_manager.LargeBinaryManager" ) as mock_manager: mock_manager.create.return_value = "s3://bucket/objects/123/uuid" - big_object = largebinary() - assert big_object.uri == "s3://bucket/objects/123/uuid" + large_binary = largebinary() + assert large_binary.uri == "s3://bucket/objects/123/uuid" mock_manager.create.assert_called_once() def test_invalid_uri_raises_value_error(self): @@ -49,19 +49,19 @@ def test_invalid_uri_raises_value_error(self): def test_get_bucket_name(self): """Test extracting bucket name from URI.""" - big_object = largebinary("s3://my-bucket/path/to/object") - assert big_object.get_bucket_name() == "my-bucket" + large_binary = largebinary("s3://my-bucket/path/to/object") + assert large_binary.get_bucket_name() == "my-bucket" def test_get_object_key(self): """Test extracting object key from URI.""" - big_object = largebinary("s3://my-bucket/path/to/object") - assert big_object.get_object_key() == "path/to/object" + large_binary = largebinary("s3://my-bucket/path/to/object") + assert large_binary.get_object_key() == "path/to/object" def test_get_object_key_with_leading_slash(self): """Test extracting object key when URI has leading slash.""" - big_object = largebinary("s3://my-bucket/path/to/object") + large_binary = largebinary("s3://my-bucket/path/to/object") # urlparse includes leading slash, but get_object_key removes it - assert big_object.get_object_key() == "path/to/object" + assert large_binary.get_object_key() == "path/to/object" def test_equality(self): """Test largebinary equality comparison.""" @@ -86,5 +86,5 @@ def test_hash(self): def test_uri_property(self): """Test URI property access.""" uri = "s3://test-bucket/test/path" - big_object = largebinary(uri) - assert big_object.uri == uri + large_binary = largebinary(uri) + assert large_binary.uri == uri diff --git a/amber/src/main/python/core/models/schema/test_schema.py b/amber/src/main/python/core/models/schema/test_schema.py index bd844bad51d..60e4c848a54 100644 --- a/amber/src/main/python/core/models/schema/test_schema.py +++ b/amber/src/main/python/core/models/schema/test_schema.py @@ -90,37 +90,37 @@ def test_convert_from_arrow_schema(self, arrow_schema, schema): assert schema == Schema(arrow_schema=arrow_schema) assert schema.as_arrow_schema() == arrow_schema - def test_big_object_in_raw_schema(self): - """Test creating schema with BIG_OBJECT from raw schema.""" + def test_large_binary_in_raw_schema(self): + """Test creating schema with LARGE_BINARY from raw schema.""" raw_schema = { "regular_field": "STRING", - "big_object_field": "BIG_OBJECT", + "large_binary_field": "LARGE_BINARY", } schema = Schema(raw_schema=raw_schema) assert schema.get_attr_type("regular_field") == AttributeType.STRING - assert schema.get_attr_type("big_object_field") == AttributeType.BIG_OBJECT + assert schema.get_attr_type("large_binary_field") == AttributeType.LARGE_BINARY - def test_big_object_in_arrow_schema_with_metadata(self): - """Test creating schema with BIG_OBJECT from Arrow schema with metadata.""" + def test_large_binary_in_arrow_schema_with_metadata(self): + """Test creating schema with LARGE_BINARY from Arrow schema with metadata.""" arrow_schema = pa.schema( [ pa.field("regular_field", pa.string()), pa.field( - "big_object_field", + "large_binary_field", pa.string(), - metadata={b"texera_type": b"BIG_OBJECT"}, + metadata={b"texera_type": b"LARGE_BINARY"}, ), ] ) schema = Schema(arrow_schema=arrow_schema) assert schema.get_attr_type("regular_field") == AttributeType.STRING - assert schema.get_attr_type("big_object_field") == AttributeType.BIG_OBJECT + assert schema.get_attr_type("large_binary_field") == AttributeType.LARGE_BINARY - def test_big_object_as_arrow_schema_includes_metadata(self): - """Test that BIG_OBJECT fields include metadata in Arrow schema.""" + def test_large_binary_as_arrow_schema_includes_metadata(self): + """Test that LARGE_BINARY fields include metadata in Arrow schema.""" schema = Schema() schema.add("regular_field", AttributeType.STRING) - schema.add("big_object_field", AttributeType.BIG_OBJECT) + schema.add("large_binary_field", AttributeType.LARGE_BINARY) arrow_schema = schema.as_arrow_schema() @@ -131,17 +131,19 @@ def test_big_object_as_arrow_schema_includes_metadata(self): or b"texera_type" not in regular_field.metadata ) - # BIG_OBJECT field should have metadata - big_object_field = arrow_schema.field("big_object_field") - assert big_object_field.metadata is not None - assert big_object_field.metadata.get(b"texera_type") == b"BIG_OBJECT" - assert big_object_field.type == pa.string() # BIG_OBJECT is stored as string + # LARGE_BINARY field should have metadata + large_binary_field = arrow_schema.field("large_binary_field") + assert large_binary_field.metadata is not None + assert large_binary_field.metadata.get(b"texera_type") == b"LARGE_BINARY" + assert ( + large_binary_field.type == pa.string() + ) # LARGE_BINARY is stored as string - def test_round_trip_big_object_schema(self): - """Test round-trip conversion of schema with BIG_OBJECT.""" + def test_round_trip_large_binary_schema(self): + """Test round-trip conversion of schema with LARGE_BINARY.""" original_schema = Schema() original_schema.add("field1", AttributeType.STRING) - original_schema.add("field2", AttributeType.BIG_OBJECT) + original_schema.add("field2", AttributeType.LARGE_BINARY) original_schema.add("field3", AttributeType.INT) # Convert to Arrow and back @@ -150,5 +152,5 @@ def test_round_trip_big_object_schema(self): assert round_trip_schema == original_schema assert round_trip_schema.get_attr_type("field1") == AttributeType.STRING - assert round_trip_schema.get_attr_type("field2") == AttributeType.BIG_OBJECT + assert round_trip_schema.get_attr_type("field2") == AttributeType.LARGE_BINARY assert round_trip_schema.get_attr_type("field3") == AttributeType.INT diff --git a/amber/src/main/python/core/models/test_tuple.py b/amber/src/main/python/core/models/test_tuple.py index 5d173d432fd..25d010f80eb 100644 --- a/amber/src/main/python/core/models/test_tuple.py +++ b/amber/src/main/python/core/models/test_tuple.py @@ -222,53 +222,53 @@ def test_hash(self): ) assert hash(tuple5) == -2099556631 # calculated with Java - def test_tuple_with_big_object(self): + def test_tuple_with_large_binary(self): """Test tuple with largebinary field.""" from core.models.schema.large_binary import largebinary schema = Schema( raw_schema={ "regular_field": "STRING", - "big_object_field": "BIG_OBJECT", + "large_binary_field": "LARGE_BINARY", } ) - big_object = largebinary("s3://test-bucket/path/to/object") + large_binary = largebinary("s3://test-bucket/path/to/object") tuple_ = Tuple( { "regular_field": "test string", - "big_object_field": big_object, + "large_binary_field": large_binary, }, schema=schema, ) assert tuple_["regular_field"] == "test string" - assert tuple_["big_object_field"] == big_object - assert isinstance(tuple_["big_object_field"], largebinary) - assert tuple_["big_object_field"].uri == "s3://test-bucket/path/to/object" + assert tuple_["large_binary_field"] == large_binary + assert isinstance(tuple_["large_binary_field"], largebinary) + assert tuple_["large_binary_field"].uri == "s3://test-bucket/path/to/object" - def test_tuple_from_arrow_with_big_object(self): - """Test creating tuple from Arrow table with BIG_OBJECT metadata.""" + def test_tuple_from_arrow_with_large_binary(self): + """Test creating tuple from Arrow table with LARGE_BINARY metadata.""" import pyarrow as pa from core.models.schema.large_binary import largebinary - # Create Arrow schema with BIG_OBJECT metadata + # Create Arrow schema with LARGE_BINARY metadata arrow_schema = pa.schema( [ pa.field("regular_field", pa.string()), pa.field( - "big_object_field", + "large_binary_field", pa.string(), - metadata={b"texera_type": b"BIG_OBJECT"}, + metadata={b"texera_type": b"LARGE_BINARY"}, ), ] ) - # Create Arrow table with URI string for big_object_field + # Create Arrow table with URI string for large_binary_field arrow_table = pa.Table.from_pydict( { "regular_field": ["test"], - "big_object_field": ["s3://test-bucket/path/to/object"], + "large_binary_field": ["s3://test-bucket/path/to/object"], }, schema=arrow_schema, ) @@ -283,20 +283,20 @@ def test_tuple_from_arrow_with_big_object(self): assert len(tuples) == 1 tuple_ = tuples[0] assert tuple_["regular_field"] == "test" - assert isinstance(tuple_["big_object_field"], largebinary) - assert tuple_["big_object_field"].uri == "s3://test-bucket/path/to/object" + assert isinstance(tuple_["large_binary_field"], largebinary) + assert tuple_["large_binary_field"].uri == "s3://test-bucket/path/to/object" - def test_tuple_with_null_big_object(self): + def test_tuple_with_null_large_binary(self): """Test tuple with null largebinary field.""" import pyarrow as pa - # Create Arrow schema with BIG_OBJECT metadata + # Create Arrow schema with LARGE_BINARY metadata arrow_schema = pa.schema( [ pa.field( - "big_object_field", + "large_binary_field", pa.string(), - metadata={b"texera_type": b"BIG_OBJECT"}, + metadata={b"texera_type": b"LARGE_BINARY"}, ), ] ) @@ -304,7 +304,7 @@ def test_tuple_with_null_big_object(self): # Create Arrow table with null value arrow_table = pa.Table.from_pydict( { - "big_object_field": [None], + "large_binary_field": [None], }, schema=arrow_schema, ) @@ -318,4 +318,4 @@ def test_tuple_with_null_big_object(self): assert len(tuples) == 1 tuple_ = tuples[0] - assert tuple_["big_object_field"] is None + assert tuple_["large_binary_field"] is None diff --git a/amber/src/main/python/core/models/tuple.py b/amber/src/main/python/core/models/tuple.py index cbd9678627d..5184daedf56 100644 --- a/amber/src/main/python/core/models/tuple.py +++ b/amber/src/main/python/core/models/tuple.py @@ -97,12 +97,12 @@ def field_accessor(field_name: str) -> Field: ): value = pickle.loads(value[10:]) - # Convert URI string to largebinary for BIG_OBJECT types + # Convert URI string to largebinary for LARGE_BINARY types # Metadata is set by Scala ArrowUtils or Python iceberg_utils elif ( value is not None and field_metadata - and field_metadata.get(b"texera_type") == b"BIG_OBJECT" + and field_metadata.get(b"texera_type") == b"LARGE_BINARY" ): value = largebinary(value) @@ -201,9 +201,9 @@ def __getitem__(self, item: typing.Union[int, str]) -> Field: :param item: field name or field index :return: field value """ - assert isinstance(item, (int, str)), ( - "field can only be retrieved by index or name" - ) + assert isinstance( + item, (int, str) + ), "field can only be retrieved by index or name" if isinstance(item, int): item: str = self.get_field_names()[item] @@ -249,18 +249,18 @@ def as_key_value_pairs(self) -> List[typing.Tuple[str, Field]]: def get_serialized_field(self, field_name: str) -> Field: """ Get a field value serialized for Arrow table conversion. - For BIG_OBJECT fields, converts largebinary instances to URI strings. + For LARGE_BINARY fields, converts largebinary instances to URI strings. For other fields, returns the value as-is. :param field_name: field name - :return: field value (URI string for BIG_OBJECT fields with largebinary values) + :return: field value (URI string for LARGE_BINARY fields with largebinary values) """ value = self[field_name] - # Convert largebinary to URI string for BIG_OBJECT fields when schema available + # Convert largebinary to URI string for LARGE_BINARY fields when schema available if ( self._schema is not None - and self._schema.get_attr_type(field_name) == AttributeType.BIG_OBJECT + and self._schema.get_attr_type(field_name) == AttributeType.LARGE_BINARY and isinstance(value, largebinary) ): return value.uri diff --git a/amber/src/main/python/core/storage/document_factory.py b/amber/src/main/python/core/storage/document_factory.py index 4fca15a53e6..9b686ab66b6 100644 --- a/amber/src/main/python/core/storage/document_factory.py +++ b/amber/src/main/python/core/storage/document_factory.py @@ -64,7 +64,7 @@ def create_document(uri: str, schema: Schema) -> VirtualDocument: if resource_type in {VFSResourceType.RESULT}: storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) - # Convert Amber Schema to Iceberg Schema with BIG_OBJECT + # Convert Amber Schema to Iceberg Schema with LARGE_BINARY # field name encoding iceberg_schema = amber_schema_to_iceberg_schema(schema) diff --git a/amber/src/main/python/core/storage/iceberg/iceberg_utils.py b/amber/src/main/python/core/storage/iceberg/iceberg_utils.py index b7e1724e54b..0a5fca8019d 100644 --- a/amber/src/main/python/core/storage/iceberg/iceberg_utils.py +++ b/amber/src/main/python/core/storage/iceberg/iceberg_utils.py @@ -32,8 +32,8 @@ from core.models import ArrowTableTupleProvider, Tuple from core.models.schema.attribute_type import AttributeType, TO_ARROW_MAPPING -# Suffix used to encode BIG_OBJECT fields in Iceberg (must match Scala IcebergUtil) -BIG_OBJECT_FIELD_SUFFIX = "__texera_big_obj_ptr" +# Suffix used to encode LARGE_BINARY fields in Iceberg (must match Scala IcebergUtil) +LARGE_BINARY_FIELD_SUFFIX = "__texera_large_binary_ptr" # Type mappings _ICEBERG_TO_AMBER_TYPE_MAPPING = { @@ -56,36 +56,36 @@ AttributeType.BOOL: iceberg_types.BooleanType(), AttributeType.TIMESTAMP: iceberg_types.TimestampType(), AttributeType.BINARY: iceberg_types.BinaryType(), - AttributeType.BIG_OBJECT: iceberg_types.StringType(), + AttributeType.LARGE_BINARY: iceberg_types.StringType(), } -def encode_big_object_field_name(field_name: str, attr_type) -> str: - """Encodes BIG_OBJECT field names with suffix for Iceberg storage.""" - if attr_type == AttributeType.BIG_OBJECT: - return f"{field_name}{BIG_OBJECT_FIELD_SUFFIX}" +def encode_large_binary_field_name(field_name: str, attr_type) -> str: + """Encodes LARGE_BINARY field names with suffix for Iceberg storage.""" + if attr_type == AttributeType.LARGE_BINARY: + return f"{field_name}{LARGE_BINARY_FIELD_SUFFIX}" return field_name -def decode_big_object_field_name(field_name: str) -> str: - """Decodes field names by removing BIG_OBJECT suffix if present.""" - if field_name.endswith(BIG_OBJECT_FIELD_SUFFIX): - return field_name[: -len(BIG_OBJECT_FIELD_SUFFIX)] +def decode_large_binary_field_name(field_name: str) -> str: + """Decodes field names by removing LARGE_BINARY suffix if present.""" + if field_name.endswith(LARGE_BINARY_FIELD_SUFFIX): + return field_name[: -len(LARGE_BINARY_FIELD_SUFFIX)] return field_name def iceberg_schema_to_amber_schema(iceberg_schema: Schema): """ Converts PyIceberg Schema to Amber Schema. - Decodes BIG_OBJECT field names and adds Arrow metadata. + Decodes LARGE_BINARY field names and adds Arrow metadata. """ arrow_fields = [] for field in iceberg_schema.fields: - decoded_name = decode_big_object_field_name(field.name) - is_big_object = field.name != decoded_name + decoded_name = decode_large_binary_field_name(field.name) + is_large_binary = field.name != decoded_name - if is_big_object: - attr_type = AttributeType.BIG_OBJECT + if is_large_binary: + attr_type = AttributeType.LARGE_BINARY else: iceberg_type_str = str(field.field_type).lower() attr_type_name = _ICEBERG_TO_AMBER_TYPE_MAPPING.get( @@ -97,7 +97,7 @@ def iceberg_schema_to_amber_schema(iceberg_schema: Schema): pa.field( decoded_name, TO_ARROW_MAPPING[attr_type], - metadata={b"texera_type": b"BIG_OBJECT"} if is_big_object else None, + metadata={b"texera_type": b"LARGE_BINARY"} if is_large_binary else None, ) ) @@ -107,12 +107,12 @@ def iceberg_schema_to_amber_schema(iceberg_schema: Schema): def amber_schema_to_iceberg_schema(amber_schema) -> Schema: """ Converts Amber Schema to PyIceberg Schema. - Encodes BIG_OBJECT field names with suffix. + Encodes LARGE_BINARY field names with suffix. """ fields = [ iceberg_types.NestedField( field_id=idx, - name=encode_big_object_field_name(field_name, attr_type), + name=encode_large_binary_field_name(field_name, attr_type), field_type=_AMBER_TO_ICEBERG_TYPE_MAPPING[attr_type], required=False, ) @@ -229,14 +229,14 @@ def amber_tuples_to_arrow_table( ) -> pa.Table: """ Converts a list of amber tuples to a pyarrow table for serialization. - Handles BIG_OBJECT field name encoding and serialization. + Handles LARGE_BINARY field name encoding and serialization. """ from core.models.schema.large_binary import largebinary tuple_list = list(tuple_list) # Convert to list to allow multiple iterations data_dict = {} for encoded_name in iceberg_schema.as_arrow().names: - decoded_name = decode_big_object_field_name(encoded_name) + decoded_name = decode_large_binary_field_name(encoded_name) data_dict[encoded_name] = [ ( t[decoded_name].uri @@ -254,7 +254,7 @@ def arrow_table_to_amber_tuples( ) -> Iterable[Tuple]: """ Converts an arrow table read from Iceberg to Amber tuples. - Properly handles BIG_OBJECT field name decoding and type detection. + Properly handles LARGE_BINARY field name decoding and type detection. """ amber_schema = iceberg_schema_to_amber_schema(iceberg_schema) arrow_table_with_metadata = pa.Table.from_arrays( @@ -266,7 +266,7 @@ def arrow_table_to_amber_tuples( return ( Tuple( { - decode_big_object_field_name(name): field_accessor + decode_large_binary_field_name(name): field_accessor for name in arrow_table.column_names }, schema=amber_schema, diff --git a/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py b/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py index 34711beb652..10aefaa5fdc 100644 --- a/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py +++ b/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py @@ -207,7 +207,8 @@ def test_concurrent_writes_followed_by_read(self, iceberg_document, sample_items # Create writer's batches item_batches = [ all_items[ - i * batch_size + min(i, remainder) : i * batch_size + i * batch_size + + min(i, remainder) : i * batch_size + min(i, remainder) + batch_size + (1 if i < remainder else 0) @@ -215,9 +216,9 @@ def test_concurrent_writes_followed_by_read(self, iceberg_document, sample_items for i in range(num_writers) ] - assert len(item_batches) == num_writers, ( - f"Expected {num_writers} batches but got {len(item_batches)}" - ) + assert ( + len(item_batches) == num_writers + ), f"Expected {num_writers} batches but got {len(item_batches)}" # Perform concurrent writes def write_batch(batch): @@ -235,9 +236,9 @@ def write_batch(batch): # Read all items back retrieved_items = list(iceberg_document.get()) # Verify that the retrieved items match the original items - assert set(retrieved_items) == set(all_items), ( - "All items should be read correctly after concurrent writes." - ) + assert set(retrieved_items) == set( + all_items + ), "All items should be read correctly after concurrent writes." def test_read_using_range(self, iceberg_document, sample_items): """ @@ -261,14 +262,14 @@ def test_read_using_range(self, iceberg_document, sample_items): item for r in ranges for item in iceberg_document.get_range(r.start, r.stop) ] - assert len(retrieved_items) == len(sample_items), ( - "The number of retrieved items does not match the number of all items." - ) + assert len(retrieved_items) == len( + sample_items + ), "The number of retrieved items does not match the number of all items." # Verify that the retrieved items match the original items - assert set(retrieved_items) == set(sample_items), ( - "All items should be retrieved correctly using ranges." - ) + assert set(retrieved_items) == set( + sample_items + ), "All items should be retrieved correctly using ranges." def test_get_after(self, iceberg_document, sample_items): """ @@ -311,6 +312,6 @@ def test_get_counts(self, iceberg_document, sample_items): writer.put_one(item) writer.close() - assert iceberg_document.get_count() == len(sample_items), ( - "get_count should return the same number as the length of sample_items" - ) + assert iceberg_document.get_count() == len( + sample_items + ), "get_count should return the same number as the length of sample_items" diff --git a/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py b/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py index ec69ebb6712..7859765ae75 100644 --- a/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py +++ b/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py @@ -22,8 +22,8 @@ from core.models.schema.attribute_type import AttributeType from core.models.schema.large_binary import largebinary from core.storage.iceberg.iceberg_utils import ( - encode_big_object_field_name, - decode_big_object_field_name, + encode_large_binary_field_name, + decode_large_binary_field_name, iceberg_schema_to_amber_schema, amber_schema_to_iceberg_schema, amber_tuples_to_arrow_table, @@ -32,29 +32,31 @@ class TestIcebergUtilsLargeBinary: - def test_encode_big_object_field_name(self): - """Test encoding BIG_OBJECT field names with suffix.""" + def test_encode_large_binary_field_name(self): + """Test encoding LARGE_BINARY field names with suffix.""" assert ( - encode_big_object_field_name("my_field", AttributeType.BIG_OBJECT) - == "my_field__texera_big_obj_ptr" + encode_large_binary_field_name("my_field", AttributeType.LARGE_BINARY) + == "my_field__texera_large_binary_ptr" ) assert ( - encode_big_object_field_name("my_field", AttributeType.STRING) == "my_field" + encode_large_binary_field_name("my_field", AttributeType.STRING) + == "my_field" ) - def test_decode_big_object_field_name(self): - """Test decoding BIG_OBJECT field names by removing suffix.""" + def test_decode_large_binary_field_name(self): + """Test decoding LARGE_BINARY field names by removing suffix.""" assert ( - decode_big_object_field_name("my_field__texera_big_obj_ptr") == "my_field" + decode_large_binary_field_name("my_field__texera_large_binary_ptr") + == "my_field" ) - assert decode_big_object_field_name("my_field") == "my_field" - assert decode_big_object_field_name("regular_field") == "regular_field" + assert decode_large_binary_field_name("my_field") == "my_field" + assert decode_large_binary_field_name("regular_field") == "regular_field" - def test_amber_schema_to_iceberg_schema_with_big_object(self): - """Test converting Amber schema with BIG_OBJECT to Iceberg schema.""" + def test_amber_schema_to_iceberg_schema_with_large_binary(self): + """Test converting Amber schema with LARGE_BINARY to Iceberg schema.""" amber_schema = Schema() amber_schema.add("regular_field", AttributeType.STRING) - amber_schema.add("big_object_field", AttributeType.BIG_OBJECT) + amber_schema.add("large_binary_field", AttributeType.LARGE_BINARY) amber_schema.add("int_field", AttributeType.INT) iceberg_schema = amber_schema_to_iceberg_schema(amber_schema) @@ -62,24 +64,24 @@ def test_amber_schema_to_iceberg_schema_with_big_object(self): # Check field names are encoded field_names = [field.name for field in iceberg_schema.fields] assert "regular_field" in field_names - assert "big_object_field__texera_big_obj_ptr" in field_names + assert "large_binary_field__texera_large_binary_ptr" in field_names assert "int_field" in field_names # Check types - big_object_field = next( - f for f in iceberg_schema.fields if "big_object" in f.name + large_binary_field = next( + f for f in iceberg_schema.fields if "large_binary" in f.name ) - assert isinstance(big_object_field.field_type, iceberg_types.StringType) + assert isinstance(large_binary_field.field_type, iceberg_types.StringType) - def test_iceberg_schema_to_amber_schema_with_big_object(self): - """Test converting Iceberg schema with BIG_OBJECT to Amber schema.""" + def test_iceberg_schema_to_amber_schema_with_large_binary(self): + """Test converting Iceberg schema with LARGE_BINARY to Amber schema.""" iceberg_schema = IcebergSchema( iceberg_types.NestedField( 1, "regular_field", iceberg_types.StringType(), required=False ), iceberg_types.NestedField( 2, - "big_object_field__texera_big_obj_ptr", + "large_binary_field__texera_large_binary_ptr", iceberg_types.StringType(), required=False, ), @@ -92,32 +94,33 @@ def test_iceberg_schema_to_amber_schema_with_big_object(self): assert amber_schema.get_attr_type("regular_field") == AttributeType.STRING assert ( - amber_schema.get_attr_type("big_object_field") == AttributeType.BIG_OBJECT + amber_schema.get_attr_type("large_binary_field") + == AttributeType.LARGE_BINARY ) assert amber_schema.get_attr_type("int_field") == AttributeType.INT - # Check Arrow schema has metadata for BIG_OBJECT + # Check Arrow schema has metadata for LARGE_BINARY arrow_schema = amber_schema.as_arrow_schema() - big_object_field = arrow_schema.field("big_object_field") - assert big_object_field.metadata is not None - assert big_object_field.metadata.get(b"texera_type") == b"BIG_OBJECT" + large_binary_field = arrow_schema.field("large_binary_field") + assert large_binary_field.metadata is not None + assert large_binary_field.metadata.get(b"texera_type") == b"LARGE_BINARY" - def test_amber_tuples_to_arrow_table_with_big_object(self): + def test_amber_tuples_to_arrow_table_with_large_binary(self): """Test converting Amber tuples with largebinary to Arrow table.""" amber_schema = Schema() amber_schema.add("regular_field", AttributeType.STRING) - amber_schema.add("big_object_field", AttributeType.BIG_OBJECT) + amber_schema.add("large_binary_field", AttributeType.LARGE_BINARY) - big_object1 = largebinary("s3://bucket/path1") - big_object2 = largebinary("s3://bucket/path2") + large_binary1 = largebinary("s3://bucket/path1") + large_binary2 = largebinary("s3://bucket/path2") tuples = [ Tuple( - {"regular_field": "value1", "big_object_field": big_object1}, + {"regular_field": "value1", "large_binary_field": large_binary1}, schema=amber_schema, ), Tuple( - {"regular_field": "value2", "big_object_field": big_object2}, + {"regular_field": "value2", "large_binary_field": large_binary2}, schema=amber_schema, ), ] @@ -127,15 +130,15 @@ def test_amber_tuples_to_arrow_table_with_big_object(self): # Check that largebinary values are converted to URI strings regular_values = arrow_table.column("regular_field").to_pylist() - big_object_values = arrow_table.column( - "big_object_field__texera_big_obj_ptr" + large_binary_values = arrow_table.column( + "large_binary_field__texera_large_binary_ptr" ).to_pylist() assert regular_values == ["value1", "value2"] - assert big_object_values == ["s3://bucket/path1", "s3://bucket/path2"] + assert large_binary_values == ["s3://bucket/path1", "s3://bucket/path2"] - def test_arrow_table_to_amber_tuples_with_big_object(self): - """Test converting Arrow table with BIG_OBJECT to Amber tuples.""" + def test_arrow_table_to_amber_tuples_with_large_binary(self): + """Test converting Arrow table with LARGE_BINARY to Amber tuples.""" # Create Iceberg schema with encoded field name iceberg_schema = IcebergSchema( iceberg_types.NestedField( @@ -143,7 +146,7 @@ def test_arrow_table_to_amber_tuples_with_big_object(self): ), iceberg_types.NestedField( 2, - "big_object_field__texera_big_obj_ptr", + "large_binary_field__texera_large_binary_ptr", iceberg_types.StringType(), required=False, ), @@ -153,7 +156,7 @@ def test_arrow_table_to_amber_tuples_with_big_object(self): arrow_table = pa.Table.from_pydict( { "regular_field": ["value1", "value2"], - "big_object_field__texera_big_obj_ptr": [ + "large_binary_field__texera_large_binary_ptr": [ "s3://bucket/path1", "s3://bucket/path2", ], @@ -164,23 +167,23 @@ def test_arrow_table_to_amber_tuples_with_big_object(self): assert len(tuples) == 2 assert tuples[0]["regular_field"] == "value1" - assert isinstance(tuples[0]["big_object_field"], largebinary) - assert tuples[0]["big_object_field"].uri == "s3://bucket/path1" + assert isinstance(tuples[0]["large_binary_field"], largebinary) + assert tuples[0]["large_binary_field"].uri == "s3://bucket/path1" assert tuples[1]["regular_field"] == "value2" - assert isinstance(tuples[1]["big_object_field"], largebinary) - assert tuples[1]["big_object_field"].uri == "s3://bucket/path2" + assert isinstance(tuples[1]["large_binary_field"], largebinary) + assert tuples[1]["large_binary_field"].uri == "s3://bucket/path2" - def test_round_trip_big_object_tuples(self): + def test_round_trip_large_binary_tuples(self): """Test round-trip conversion of tuples with largebinary.""" amber_schema = Schema() amber_schema.add("regular_field", AttributeType.STRING) - amber_schema.add("big_object_field", AttributeType.BIG_OBJECT) + amber_schema.add("large_binary_field", AttributeType.LARGE_BINARY) - big_object = largebinary("s3://bucket/path/to/object") + large_binary = largebinary("s3://bucket/path/to/object") original_tuples = [ Tuple( - {"regular_field": "value1", "big_object_field": big_object}, + {"regular_field": "value1", "large_binary_field": large_binary}, schema=amber_schema, ), ] @@ -196,10 +199,10 @@ def test_round_trip_big_object_tuples(self): assert len(retrieved_tuples) == 1 assert retrieved_tuples[0]["regular_field"] == "value1" - assert isinstance(retrieved_tuples[0]["big_object_field"], largebinary) - assert retrieved_tuples[0]["big_object_field"].uri == big_object.uri + assert isinstance(retrieved_tuples[0]["large_binary_field"], largebinary) + assert retrieved_tuples[0]["large_binary_field"].uri == large_binary.uri - def test_arrow_table_to_amber_tuples_with_null_big_object(self): + def test_arrow_table_to_amber_tuples_with_null_large_binary(self): """Test converting Arrow table with null largebinary values.""" iceberg_schema = IcebergSchema( iceberg_types.NestedField( @@ -207,7 +210,7 @@ def test_arrow_table_to_amber_tuples_with_null_big_object(self): ), iceberg_types.NestedField( 2, - "big_object_field__texera_big_obj_ptr", + "large_binary_field__texera_large_binary_ptr", iceberg_types.StringType(), required=False, ), @@ -216,7 +219,7 @@ def test_arrow_table_to_amber_tuples_with_null_big_object(self): arrow_table = pa.Table.from_pydict( { "regular_field": ["value1"], - "big_object_field__texera_big_obj_ptr": [None], + "large_binary_field__texera_large_binary_ptr": [None], } ) @@ -224,4 +227,4 @@ def test_arrow_table_to_amber_tuples_with_null_big_object(self): assert len(tuples) == 1 assert tuples[0]["regular_field"] == "value1" - assert tuples[0]["big_object_field"] is None + assert tuples[0]["large_binary_field"] is None diff --git a/amber/src/main/python/pytexera/storage/large_binary_input_stream.py b/amber/src/main/python/pytexera/storage/large_binary_input_stream.py index d3522c209a2..fb9b82744b5 100644 --- a/amber/src/main/python/pytexera/storage/large_binary_input_stream.py +++ b/amber/src/main/python/pytexera/storage/large_binary_input_stream.py @@ -19,7 +19,7 @@ LargeBinaryInputStream for reading largebinary data from S3. Usage: - with LargeBinaryInputStream(big_object) as stream: + with LargeBinaryInputStream(large_binary) as stream: content = stream.read() """ @@ -50,12 +50,12 @@ class LargeBinaryInputStream(IOBase): Lazily downloads from S3 on first read. Supports context manager and iteration. """ - def __init__(self, big_object: largebinary): + def __init__(self, large_binary: largebinary): """Initialize stream for reading the given largebinary.""" super().__init__() - if big_object is None: + if large_binary is None: raise ValueError("largebinary cannot be None") - self._big_object = big_object + self._large_binary = large_binary self._underlying: Optional[BinaryIO] = None self._closed = False @@ -65,8 +65,8 @@ def _lazy_init(self): s3 = LargeBinaryManager._get_s3_client() response = s3.get_object( - Bucket=self._big_object.get_bucket_name(), - Key=self._big_object.get_object_key(), + Bucket=self._large_binary.get_bucket_name(), + Key=self._large_binary.get_object_key(), ) self._underlying = response["Body"] diff --git a/amber/src/main/python/pytexera/storage/large_binary_manager.py b/amber/src/main/python/pytexera/storage/large_binary_manager.py index b219bd486d5..e53abb5869f 100644 --- a/amber/src/main/python/pytexera/storage/large_binary_manager.py +++ b/amber/src/main/python/pytexera/storage/large_binary_manager.py @@ -34,7 +34,7 @@ class LargeBinaryManager: """Internal manager for largebinary S3 operations.""" _s3_client = None - DEFAULT_BUCKET = "texera-big-objects" + DEFAULT_BUCKET = "texera-large-binaries" @classmethod def _get_s3_client(cls): diff --git a/amber/src/main/python/pytexera/storage/large_binary_output_stream.py b/amber/src/main/python/pytexera/storage/large_binary_output_stream.py index c80f916e26d..e401c6c5bba 100644 --- a/amber/src/main/python/pytexera/storage/large_binary_output_stream.py +++ b/amber/src/main/python/pytexera/storage/large_binary_output_stream.py @@ -21,8 +21,8 @@ Usage: from pytexera import largebinary, LargeBinaryOutputStream - big_object = largebinary() - with LargeBinaryOutputStream(big_object) as out: + large_binary = largebinary() + with LargeBinaryOutputStream(large_binary) as out: out.write(b"data") """ @@ -91,32 +91,32 @@ class LargeBinaryOutputStream(IOBase): from pytexera import largebinary, LargeBinaryOutputStream # Create a new largebinary and write to it - big_object = largebinary() - with LargeBinaryOutputStream(big_object) as out: + large_binary = largebinary() + with LargeBinaryOutputStream(large_binary) as out: out.write(b"Hello, World!") out.write(b"More data") - # big_object is now ready to be added to tuples + # large_binary is now ready to be added to tuples Note: Not thread-safe. Do not access from multiple threads concurrently. """ - def __init__(self, big_object: largebinary): + def __init__(self, large_binary: largebinary): """ Initialize a LargeBinaryOutputStream. Args: - big_object: The largebinary reference to write to + large_binary: The largebinary reference to write to Raises: - ValueError: If big_object is None + ValueError: If large_binary is None """ super().__init__() - if big_object is None: + if large_binary is None: raise ValueError("largebinary cannot be None") - self._big_object = big_object - self._bucket_name = big_object.get_bucket_name() - self._object_key = big_object.get_object_key() + self._large_binary = large_binary + self._bucket_name = large_binary.get_bucket_name() + self._object_key = large_binary.get_object_key() self._closed = False # Background upload thread state diff --git a/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py b/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py index 8648decf2cc..9a1d94b582f 100644 --- a/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py @@ -25,7 +25,7 @@ class TestLargeBinaryInputStream: @pytest.fixture - def big_object(self): + def large_binary(self): """Create a test largebinary.""" return largebinary("s3://test-bucket/path/to/object") @@ -34,11 +34,11 @@ def mock_s3_response(self): """Create a mock S3 response with a BytesIO body.""" return {"Body": BytesIO(b"test data content")} - def test_init_with_valid_big_object(self, big_object): + def test_init_with_valid_large_binary(self, large_binary): """Test initialization with a valid largebinary.""" - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) try: - assert stream._big_object == big_object + assert stream._large_binary == large_binary assert stream._underlying is None assert not stream._closed finally: @@ -49,7 +49,7 @@ def test_init_with_none_raises_error(self): with pytest.raises(ValueError, match="largebinary cannot be None"): LargeBinaryInputStream(None) - def test_lazy_init_downloads_from_s3(self, big_object, mock_s3_response): + def test_lazy_init_downloads_from_s3(self, large_binary, mock_s3_response): """Test that _lazy_init downloads from S3 on first read.""" with patch.object( large_binary_manager.LargeBinaryManager, "_get_s3_client" @@ -58,7 +58,7 @@ def test_lazy_init_downloads_from_s3(self, big_object, mock_s3_response): mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) try: assert stream._underlying is None # Not initialized yet @@ -74,7 +74,7 @@ def test_lazy_init_downloads_from_s3(self, big_object, mock_s3_response): finally: stream.close() - def test_read_all(self, big_object, mock_s3_response): + def test_read_all(self, large_binary, mock_s3_response): """Test reading all data.""" with patch.object( large_binary_manager.LargeBinaryManager, "_get_s3_client" @@ -83,14 +83,14 @@ def test_read_all(self, big_object, mock_s3_response): mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) try: data = stream.read() assert data == b"test data content" finally: stream.close() - def test_read_partial(self, big_object, mock_s3_response): + def test_read_partial(self, large_binary, mock_s3_response): """Test reading partial data.""" with patch.object( large_binary_manager.LargeBinaryManager, "_get_s3_client" @@ -99,14 +99,14 @@ def test_read_partial(self, big_object, mock_s3_response): mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) try: data = stream.read(4) assert data == b"test" finally: stream.close() - def test_readline(self, big_object): + def test_readline(self, large_binary): """Test reading a line.""" with patch.object( large_binary_manager.LargeBinaryManager, "_get_s3_client" @@ -116,14 +116,14 @@ def test_readline(self, big_object): mock_s3_client.get_object.return_value = response mock_get_s3_client.return_value = mock_s3_client - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) try: line = stream.readline() assert line == b"line1\n" finally: stream.close() - def test_readlines(self, big_object): + def test_readlines(self, large_binary): """Test reading all lines.""" with patch.object( large_binary_manager.LargeBinaryManager, "_get_s3_client" @@ -133,16 +133,16 @@ def test_readlines(self, big_object): mock_s3_client.get_object.return_value = response mock_get_s3_client.return_value = mock_s3_client - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) try: lines = stream.readlines() assert lines == [b"line1\n", b"line2\n", b"line3"] finally: stream.close() - def test_readable(self, big_object): + def test_readable(self, large_binary): """Test readable() method.""" - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) try: assert stream.readable() is True @@ -152,17 +152,17 @@ def test_readable(self, big_object): if not stream._closed: stream.close() - def test_seekable(self, big_object): + def test_seekable(self, large_binary): """Test seekable() method (should always return False).""" - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) try: assert stream.seekable() is False finally: stream.close() - def test_closed_property(self, big_object): + def test_closed_property(self, large_binary): """Test closed property.""" - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) try: assert stream.closed is False @@ -172,7 +172,7 @@ def test_closed_property(self, big_object): if not stream._closed: stream.close() - def test_close(self, big_object, mock_s3_response): + def test_close(self, large_binary, mock_s3_response): """Test closing the stream.""" with patch.object( large_binary_manager.LargeBinaryManager, "_get_s3_client" @@ -181,7 +181,7 @@ def test_close(self, big_object, mock_s3_response): mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) stream.read(1) # Trigger lazy init assert stream._underlying is not None @@ -189,7 +189,7 @@ def test_close(self, big_object, mock_s3_response): assert stream._closed is True assert stream._underlying.closed - def test_context_manager(self, big_object, mock_s3_response): + def test_context_manager(self, large_binary, mock_s3_response): """Test using as context manager.""" with patch.object( large_binary_manager.LargeBinaryManager, "_get_s3_client" @@ -198,7 +198,7 @@ def test_context_manager(self, big_object, mock_s3_response): mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - with LargeBinaryInputStream(big_object) as stream: + with LargeBinaryInputStream(large_binary) as stream: data = stream.read() assert data == b"test data content" assert not stream._closed @@ -206,7 +206,7 @@ def test_context_manager(self, big_object, mock_s3_response): # Stream should be closed after context exit assert stream._closed - def test_iteration(self, big_object): + def test_iteration(self, large_binary): """Test iteration over lines.""" with patch.object( large_binary_manager.LargeBinaryManager, "_get_s3_client" @@ -216,14 +216,14 @@ def test_iteration(self, big_object): mock_s3_client.get_object.return_value = response mock_get_s3_client.return_value = mock_s3_client - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) try: lines = list(stream) assert lines == [b"line1\n", b"line2\n", b"line3"] finally: stream.close() - def test_read_after_close_raises_error(self, big_object, mock_s3_response): + def test_read_after_close_raises_error(self, large_binary, mock_s3_response): """Test that reading after close raises ValueError.""" with patch.object( large_binary_manager.LargeBinaryManager, "_get_s3_client" @@ -232,7 +232,7 @@ def test_read_after_close_raises_error(self, big_object, mock_s3_response): mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client - stream = LargeBinaryInputStream(big_object) + stream = LargeBinaryInputStream(large_binary) stream.close() with pytest.raises(ValueError, match="I/O operation on closed stream"): diff --git a/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py b/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py index 1793a27a082..08eefaf3e61 100644 --- a/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py @@ -25,14 +25,14 @@ class TestLargeBinaryOutputStream: @pytest.fixture - def big_object(self): + def large_binary(self): """Create a test largebinary.""" return largebinary("s3://test-bucket/path/to/object") - def test_init_with_valid_big_object(self, big_object): + def test_init_with_valid_large_binary(self, large_binary): """Test initialization with a valid largebinary.""" - stream = LargeBinaryOutputStream(big_object) - assert stream._big_object == big_object + stream = LargeBinaryOutputStream(large_binary) + assert stream._large_binary == large_binary assert stream._bucket_name == "test-bucket" assert stream._object_key == "path/to/object" assert not stream._closed @@ -43,7 +43,7 @@ def test_init_with_none_raises_error(self): with pytest.raises(ValueError, match="largebinary cannot be None"): LargeBinaryOutputStream(None) - def test_write_starts_upload_thread(self, big_object): + def test_write_starts_upload_thread(self, large_binary): """Test that write() starts the upload thread.""" with ( patch.object( @@ -57,7 +57,7 @@ def test_write_starts_upload_thread(self, big_object): mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) assert stream._upload_thread is None stream.write(b"test data") @@ -68,7 +68,7 @@ def test_write_starts_upload_thread(self, big_object): # Wait for thread to finish stream.close() - def test_write_data(self, big_object): + def test_write_data(self, large_binary): """Test writing data to the stream.""" with ( patch.object( @@ -82,13 +82,13 @@ def test_write_data(self, big_object): mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) bytes_written = stream.write(b"test data") assert bytes_written == len(b"test data") stream.close() - def test_write_multiple_chunks(self, big_object): + def test_write_multiple_chunks(self, large_binary): """Test writing multiple chunks of data.""" with ( patch.object( @@ -102,41 +102,41 @@ def test_write_multiple_chunks(self, big_object): mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) stream.write(b"chunk1") stream.write(b"chunk2") stream.write(b"chunk3") stream.close() - def test_writable(self, big_object): + def test_writable(self, large_binary): """Test writable() method.""" - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) assert stream.writable() is True stream.close() assert stream.writable() is False - def test_seekable(self, big_object): + def test_seekable(self, large_binary): """Test seekable() method (should always return False).""" - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) assert stream.seekable() is False - def test_closed_property(self, big_object): + def test_closed_property(self, large_binary): """Test closed property.""" - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) assert stream.closed is False stream.close() assert stream.closed is True - def test_flush(self, big_object): + def test_flush(self, large_binary): """Test flush() method (should be a no-op).""" - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) # Should not raise any exception stream.flush() - def test_close_completes_upload(self, big_object): + def test_close_completes_upload(self, large_binary): """Test that close() completes the upload.""" with ( patch.object( @@ -150,7 +150,7 @@ def test_close_completes_upload(self, big_object): mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) stream.write(b"test data") # Close should wait for upload to complete @@ -159,7 +159,7 @@ def test_close_completes_upload(self, big_object): # Verify upload_fileobj was called assert mock_s3.upload_fileobj.called - def test_context_manager(self, big_object): + def test_context_manager(self, large_binary): """Test using as context manager.""" with ( patch.object( @@ -173,22 +173,22 @@ def test_context_manager(self, big_object): mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - with LargeBinaryOutputStream(big_object) as stream: + with LargeBinaryOutputStream(large_binary) as stream: stream.write(b"test data") assert not stream._closed # Stream should be closed after context exit assert stream._closed - def test_write_after_close_raises_error(self, big_object): + def test_write_after_close_raises_error(self, large_binary): """Test that writing after close raises ValueError.""" - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) stream.close() with pytest.raises(ValueError, match="I/O operation on closed stream"): stream.write(b"data") - def test_close_handles_upload_error(self, big_object): + def test_close_handles_upload_error(self, large_binary): """Test that close() raises IOError if upload fails.""" with ( patch.object( @@ -203,13 +203,13 @@ def test_close_handles_upload_error(self, big_object): mock_ensure_bucket.return_value = None mock_s3.upload_fileobj.side_effect = Exception("Upload failed") - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) stream.write(b"test data") with pytest.raises(IOError, match="Failed to complete upload"): stream.close() - def test_write_after_upload_error_raises_error(self, big_object): + def test_write_after_upload_error_raises_error(self, large_binary): """Test that writing after upload error raises IOError.""" with ( patch.object( @@ -224,7 +224,7 @@ def test_write_after_upload_error_raises_error(self, big_object): mock_ensure_bucket.return_value = None mock_s3.upload_fileobj.side_effect = Exception("Upload failed") - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) stream.write(b"test data") # Wait a bit for the error to be set @@ -233,7 +233,7 @@ def test_write_after_upload_error_raises_error(self, big_object): with pytest.raises(IOError, match="Background upload failed"): stream.write(b"more data") - def test_multiple_close_calls(self, big_object): + def test_multiple_close_calls(self, large_binary): """Test that multiple close() calls are safe.""" with ( patch.object( @@ -247,7 +247,7 @@ def test_multiple_close_calls(self, big_object): mock_get_s3_client.return_value = mock_s3 mock_ensure_bucket.return_value = None - stream = LargeBinaryOutputStream(big_object) + stream = LargeBinaryOutputStream(large_binary) stream.write(b"test data") stream.close() # Second close should not raise error diff --git a/amber/src/main/scala/org/apache/texera/web/service/WorkflowService.scala b/amber/src/main/scala/org/apache/texera/web/service/WorkflowService.scala index ee7cbe55443..aa593cdcc65 100644 --- a/amber/src/main/scala/org/apache/texera/web/service/WorkflowService.scala +++ b/amber/src/main/scala/org/apache/texera/web/service/WorkflowService.scala @@ -311,7 +311,7 @@ class WorkflowService( * 2. Clears URI references from the execution registry * 3. Safely clears all result and console message documents * 4. Expires Iceberg snapshots for runtime statistics - * 5. Deletes big objects from MinIO + * 5. Deletes large binaries from MinIO * * @param eid The execution identity to clean up resources for */ diff --git a/common/workflow-core/src/main/scala/org/apache/texera/amber/util/ArrowUtils.scala b/common/workflow-core/src/main/scala/org/apache/texera/amber/util/ArrowUtils.scala index 6f7d3b2f9b4..626047fda21 100644 --- a/common/workflow-core/src/main/scala/org/apache/texera/amber/util/ArrowUtils.scala +++ b/common/workflow-core/src/main/scala/org/apache/texera/amber/util/ArrowUtils.scala @@ -94,7 +94,7 @@ object ArrowUtils extends LazyLogging { /** * Converts an Arrow Schema into Texera Schema. - * Checks field metadata to detect BIG_OBJECT types. + * Checks field metadata to detect LARGE_BINARY types. * * @param arrowSchema The Arrow Schema to be converted. * @return A Texera Schema. @@ -102,11 +102,11 @@ object ArrowUtils extends LazyLogging { def toTexeraSchema(arrowSchema: org.apache.arrow.vector.types.pojo.Schema): Schema = Schema( arrowSchema.getFields.asScala.map { field => - val isBigObject = Option(field.getMetadata) - .exists(m => m.containsKey("texera_type") && m.get("texera_type") == "BIG_OBJECT") + val isLargeBinary = Option(field.getMetadata) + .exists(m => m.containsKey("texera_type") && m.get("texera_type") == "LARGE_BINARY") val attributeType = - if (isBigObject) AttributeType.BIG_OBJECT else toAttributeType(field.getType) + if (isLargeBinary) AttributeType.LARGE_BINARY else toAttributeType(field.getType) new Attribute(field.getName, attributeType) }.toList ) @@ -232,16 +232,16 @@ object ArrowUtils extends LazyLogging { /** * Converts an Amber schema into Arrow schema. - * Stores AttributeType in field metadata to preserve BIG_OBJECT type information. + * Stores AttributeType in field metadata to preserve LARGE_BINARY type information. * * @param schema The Texera Schema. * @return An Arrow Schema. */ def fromTexeraSchema(schema: Schema): org.apache.arrow.vector.types.pojo.Schema = { val arrowFields = schema.getAttributes.map { attribute => - val metadata = if (attribute.getType == AttributeType.BIG_OBJECT) { + val metadata = if (attribute.getType == AttributeType.LARGE_BINARY) { val map = new util.HashMap[String, String]() - map.put("texera_type", "BIG_OBJECT") + map.put("texera_type", "LARGE_BINARY") map } else null @@ -283,7 +283,7 @@ object ArrowUtils extends LazyLogging { case AttributeType.BINARY => new ArrowType.Binary - case AttributeType.STRING | AttributeType.BIG_OBJECT | AttributeType.ANY => + case AttributeType.STRING | AttributeType.LARGE_BINARY | AttributeType.ANY => ArrowType.Utf8.INSTANCE case _ => diff --git a/common/workflow-operator/src/test/scala/org/apache/texera/amber/util/ArrowUtilsSpec.scala b/common/workflow-operator/src/test/scala/org/apache/texera/amber/util/ArrowUtilsSpec.scala index 6464503005d..6a97df8ca3e 100644 --- a/common/workflow-operator/src/test/scala/org/apache/texera/amber/util/ArrowUtilsSpec.scala +++ b/common/workflow-operator/src/test/scala/org/apache/texera/amber/util/ArrowUtilsSpec.scala @@ -20,7 +20,7 @@ package org.apache.texera.amber.util import org.apache.texera.amber.core.tuple.AttributeTypeUtils.AttributeTypeException -import org.apache.texera.amber.core.tuple.{AttributeType, BigObject, Schema, Tuple} +import org.apache.texera.amber.core.tuple.{AttributeType, LargeBinary, Schema, Tuple} import org.apache.arrow.memory.{BufferAllocator, RootAllocator} import org.apache.arrow.vector.VectorSchemaRoot import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType} @@ -100,8 +100,8 @@ class ArrowUtilsSpec extends AnyFlatSpec { // but not the other way around. assert(ArrowUtils.fromAttributeType(AttributeType.ANY) == string) - // BIG_OBJECT is converted to ArrowType.Utf8 (same as STRING) - assert(ArrowUtils.fromAttributeType(AttributeType.BIG_OBJECT) == string) + // LARGE_BINARY is converted to ArrowType.Utf8 (same as STRING) + assert(ArrowUtils.fromAttributeType(AttributeType.LARGE_BINARY) == string) } @@ -243,17 +243,17 @@ class ArrowUtilsSpec extends AnyFlatSpec { } - it should "convert from AttributeType to ArrowType for BIG_OBJECT correctly" in { - // BIG_OBJECT is converted to ArrowType.Utf8 (stored as string) - assert(ArrowUtils.fromAttributeType(AttributeType.BIG_OBJECT) == string) + it should "convert from AttributeType to ArrowType for LARGE_BINARY correctly" in { + // LARGE_BINARY is converted to ArrowType.Utf8 (stored as string) + assert(ArrowUtils.fromAttributeType(AttributeType.LARGE_BINARY) == string) } - it should "convert Texera Schema with BIG_OBJECT to Arrow Schema with metadata correctly" in { - val texeraSchemaWithBigObject = Schema() + it should "convert Texera Schema with LARGE_BINARY to Arrow Schema with metadata correctly" in { + val texeraSchemaWithLargeBinary = Schema() .add("regular_string", AttributeType.STRING) - .add("big_object_field", AttributeType.BIG_OBJECT) + .add("large_binary_field", AttributeType.LARGE_BINARY) - val arrowSchema = ArrowUtils.fromTexeraSchema(texeraSchemaWithBigObject) + val arrowSchema = ArrowUtils.fromTexeraSchema(texeraSchemaWithLargeBinary) // Check that regular string field has no metadata val regularStringField = arrowSchema.getFields.get(0) @@ -265,88 +265,88 @@ class ArrowUtilsSpec extends AnyFlatSpec { ) ) - // Check that BIG_OBJECT field has metadata - val bigObjectField = arrowSchema.getFields.get(1) - assert(bigObjectField.getName == "big_object_field") - assert(bigObjectField.getType == string) // BIG_OBJECT is stored as Utf8 - assert(bigObjectField.getMetadata != null) - assert(bigObjectField.getMetadata.get("texera_type") == "BIG_OBJECT") + // Check that LARGE_BINARY field has metadata + val largeBinaryField = arrowSchema.getFields.get(1) + assert(largeBinaryField.getName == "large_binary_field") + assert(largeBinaryField.getType == string) // LARGE_BINARY is stored as Utf8 + assert(largeBinaryField.getMetadata != null) + assert(largeBinaryField.getMetadata.get("texera_type") == "LARGE_BINARY") } - it should "convert Arrow Schema with BIG_OBJECT metadata to Texera Schema correctly" in { - // Create Arrow schema with BIG_OBJECT metadata - val bigObjectMetadata = new util.HashMap[String, String]() - bigObjectMetadata.put("texera_type", "BIG_OBJECT") + it should "convert Arrow Schema with LARGE_BINARY metadata to Texera Schema correctly" in { + // Create Arrow schema with LARGE_BINARY metadata + val largeBinaryMetadata = new util.HashMap[String, String]() + largeBinaryMetadata.put("texera_type", "LARGE_BINARY") - val arrowSchemaWithBigObject = new org.apache.arrow.vector.types.pojo.Schema( + val arrowSchemaWithLargeBinary = new org.apache.arrow.vector.types.pojo.Schema( Array( Field.nullablePrimitive("regular_string", string), new Field( - "big_object_field", - new FieldType(true, string, null, bigObjectMetadata), + "large_binary_field", + new FieldType(true, string, null, largeBinaryMetadata), null ) ).toList.asJava ) - val texeraSchema = ArrowUtils.toTexeraSchema(arrowSchemaWithBigObject) + val texeraSchema = ArrowUtils.toTexeraSchema(arrowSchemaWithLargeBinary) assert(texeraSchema.getAttribute("regular_string").getName == "regular_string") assert(texeraSchema.getAttribute("regular_string").getType == AttributeType.STRING) - assert(texeraSchema.getAttribute("big_object_field").getName == "big_object_field") - assert(texeraSchema.getAttribute("big_object_field").getType == AttributeType.BIG_OBJECT) + assert(texeraSchema.getAttribute("large_binary_field").getName == "large_binary_field") + assert(texeraSchema.getAttribute("large_binary_field").getType == AttributeType.LARGE_BINARY) } - it should "set and get Texera Tuple with BIG_OBJECT correctly" in { - val texeraSchemaWithBigObject = Schema() - .add("big_object_field", AttributeType.BIG_OBJECT) + it should "set and get Texera Tuple with LARGE_BINARY correctly" in { + val texeraSchemaWithLargeBinary = Schema() + .add("large_binary_field", AttributeType.LARGE_BINARY) .add("regular_string", AttributeType.STRING) - val bigObject = new BigObject("s3://test-bucket/path/to/object") + val largeBinary = new LargeBinary("s3://test-bucket/path/to/object") val tuple = Tuple - .builder(texeraSchemaWithBigObject) + .builder(texeraSchemaWithLargeBinary) .addSequentially( Array( - bigObject, + largeBinary, "regular string value" ) ) .build() val allocator: BufferAllocator = new RootAllocator() - val arrowSchema = ArrowUtils.fromTexeraSchema(texeraSchemaWithBigObject) + val arrowSchema = ArrowUtils.fromTexeraSchema(texeraSchemaWithLargeBinary) val vectorSchemaRoot = VectorSchemaRoot.create(arrowSchema, allocator) vectorSchemaRoot.allocateNew() // Set Tuple into the Vectors ArrowUtils.appendTexeraTuple(tuple, vectorSchemaRoot) - // Verify the BIG_OBJECT is stored as string (URI) in Arrow + // Verify the LARGE_BINARY is stored as string (URI) in Arrow val storedValue = vectorSchemaRoot.getVector(0).getObject(0) assert(storedValue.toString == "s3://test-bucket/path/to/object") // Get the Tuple from the Vectors val retrievedTuple = ArrowUtils.getTexeraTuple(0, vectorSchemaRoot) - assert(retrievedTuple.getField[BigObject](0) == bigObject) + assert(retrievedTuple.getField[LargeBinary](0) == largeBinary) assert(retrievedTuple.getField[String](1) == "regular string value") } - it should "handle null BIG_OBJECT values correctly" in { - val texeraSchemaWithBigObject = Schema() - .add("big_object_field", AttributeType.BIG_OBJECT) + it should "handle null LARGE_BINARY values correctly" in { + val texeraSchemaWithLargeBinary = Schema() + .add("large_binary_field", AttributeType.LARGE_BINARY) val tuple = Tuple - .builder(texeraSchemaWithBigObject) + .builder(texeraSchemaWithLargeBinary) .addSequentially( Array( - null.asInstanceOf[BigObject] + null.asInstanceOf[LargeBinary] ) ) .build() val allocator: BufferAllocator = new RootAllocator() - val arrowSchema = ArrowUtils.fromTexeraSchema(texeraSchemaWithBigObject) + val arrowSchema = ArrowUtils.fromTexeraSchema(texeraSchemaWithLargeBinary) val vectorSchemaRoot = VectorSchemaRoot.create(arrowSchema, allocator) vectorSchemaRoot.allocateNew() @@ -358,24 +358,24 @@ class ArrowUtilsSpec extends AnyFlatSpec { // Get the Tuple from the Vectors val retrievedTuple = ArrowUtils.getTexeraTuple(0, vectorSchemaRoot) - assert(retrievedTuple.getField[BigObject](0) == null) + assert(retrievedTuple.getField[LargeBinary](0) == null) } - it should "round-trip BIG_OBJECT schema conversion correctly" in { + it should "round-trip LARGE_BINARY schema conversion correctly" in { val originalSchema = Schema() .add("field1", AttributeType.STRING) - .add("field2", AttributeType.BIG_OBJECT) + .add("field2", AttributeType.LARGE_BINARY) .add("field3", AttributeType.INTEGER) - .add("field4", AttributeType.BIG_OBJECT) + .add("field4", AttributeType.LARGE_BINARY) // Convert to Arrow and back val arrowSchema = ArrowUtils.fromTexeraSchema(originalSchema) val roundTripSchema = ArrowUtils.toTexeraSchema(arrowSchema) assert(roundTripSchema.getAttribute("field1").getType == AttributeType.STRING) - assert(roundTripSchema.getAttribute("field2").getType == AttributeType.BIG_OBJECT) + assert(roundTripSchema.getAttribute("field2").getType == AttributeType.LARGE_BINARY) assert(roundTripSchema.getAttribute("field3").getType == AttributeType.INTEGER) - assert(roundTripSchema.getAttribute("field4").getType == AttributeType.BIG_OBJECT) + assert(roundTripSchema.getAttribute("field4").getType == AttributeType.LARGE_BINARY) assert(roundTripSchema == originalSchema) } From eb4e1c0d504483189143e60dde7063672f8d21fd Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Mon, 8 Dec 2025 17:45:55 -0800 Subject: [PATCH 10/13] Format files --- .../embedded_control_message_manager.py | 6 ++-- .../architecture/managers/executor_manager.py | 12 +++---- amber/src/main/python/core/models/tuple.py | 6 ++-- .../storage/iceberg/test_iceberg_document.py | 33 +++++++++---------- 4 files changed, 28 insertions(+), 29 deletions(-) diff --git a/amber/src/main/python/core/architecture/managers/embedded_control_message_manager.py b/amber/src/main/python/core/architecture/managers/embedded_control_message_manager.py index 43acb1c4e18..8ba80f28cef 100644 --- a/amber/src/main/python/core/architecture/managers/embedded_control_message_manager.py +++ b/amber/src/main/python/core/architecture/managers/embedded_control_message_manager.py @@ -74,9 +74,9 @@ def is_ecm_aligned( return ecm_completed - def get_channels_within_scope( - self, ecm: EmbeddedControlMessage - ) -> Dict["ChannelIdentity", "Channel"].keys: + def get_channels_within_scope(self, ecm: EmbeddedControlMessage) -> Dict[ + "ChannelIdentity", "Channel" + ].keys: if ecm.scope: upstreams = { channel_id diff --git a/amber/src/main/python/core/architecture/managers/executor_manager.py b/amber/src/main/python/core/architecture/managers/executor_manager.py index fc07ce74f51..53e5a8903da 100644 --- a/amber/src/main/python/core/architecture/managers/executor_manager.py +++ b/amber/src/main/python/core/architecture/managers/executor_manager.py @@ -139,9 +139,9 @@ class declaration. executor: type(Operator) = self.load_executor_definition(code) self.executor = executor() self.executor.is_source = is_source - assert ( - isinstance(self.executor, SourceOperator) == self.executor.is_source - ), "Please use SourceOperator API for source operators." + assert isinstance(self.executor, SourceOperator) == self.executor.is_source, ( + "Please use SourceOperator API for source operators." + ) def update_executor(self, code: str, is_source: bool) -> None: """ @@ -157,9 +157,9 @@ class declaration. executor: type(Operator) = self.load_executor_definition(code) self.executor = executor() self.executor.is_source = is_source - assert ( - isinstance(self.executor, SourceOperator) == self.executor.is_source - ), "Please use SourceOperator API for source operators." + assert isinstance(self.executor, SourceOperator) == self.executor.is_source, ( + "Please use SourceOperator API for source operators." + ) # overwrite the internal state self.executor.__dict__ = original_internal_state # TODO: diff --git a/amber/src/main/python/core/models/tuple.py b/amber/src/main/python/core/models/tuple.py index 5184daedf56..b510b28f149 100644 --- a/amber/src/main/python/core/models/tuple.py +++ b/amber/src/main/python/core/models/tuple.py @@ -201,9 +201,9 @@ def __getitem__(self, item: typing.Union[int, str]) -> Field: :param item: field name or field index :return: field value """ - assert isinstance( - item, (int, str) - ), "field can only be retrieved by index or name" + assert isinstance(item, (int, str)), ( + "field can only be retrieved by index or name" + ) if isinstance(item, int): item: str = self.get_field_names()[item] diff --git a/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py b/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py index 10aefaa5fdc..34711beb652 100644 --- a/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py +++ b/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py @@ -207,8 +207,7 @@ def test_concurrent_writes_followed_by_read(self, iceberg_document, sample_items # Create writer's batches item_batches = [ all_items[ - i * batch_size - + min(i, remainder) : i * batch_size + i * batch_size + min(i, remainder) : i * batch_size + min(i, remainder) + batch_size + (1 if i < remainder else 0) @@ -216,9 +215,9 @@ def test_concurrent_writes_followed_by_read(self, iceberg_document, sample_items for i in range(num_writers) ] - assert ( - len(item_batches) == num_writers - ), f"Expected {num_writers} batches but got {len(item_batches)}" + assert len(item_batches) == num_writers, ( + f"Expected {num_writers} batches but got {len(item_batches)}" + ) # Perform concurrent writes def write_batch(batch): @@ -236,9 +235,9 @@ def write_batch(batch): # Read all items back retrieved_items = list(iceberg_document.get()) # Verify that the retrieved items match the original items - assert set(retrieved_items) == set( - all_items - ), "All items should be read correctly after concurrent writes." + assert set(retrieved_items) == set(all_items), ( + "All items should be read correctly after concurrent writes." + ) def test_read_using_range(self, iceberg_document, sample_items): """ @@ -262,14 +261,14 @@ def test_read_using_range(self, iceberg_document, sample_items): item for r in ranges for item in iceberg_document.get_range(r.start, r.stop) ] - assert len(retrieved_items) == len( - sample_items - ), "The number of retrieved items does not match the number of all items." + assert len(retrieved_items) == len(sample_items), ( + "The number of retrieved items does not match the number of all items." + ) # Verify that the retrieved items match the original items - assert set(retrieved_items) == set( - sample_items - ), "All items should be retrieved correctly using ranges." + assert set(retrieved_items) == set(sample_items), ( + "All items should be retrieved correctly using ranges." + ) def test_get_after(self, iceberg_document, sample_items): """ @@ -312,6 +311,6 @@ def test_get_counts(self, iceberg_document, sample_items): writer.put_one(item) writer.close() - assert iceberg_document.get_count() == len( - sample_items - ), "get_count should return the same number as the length of sample_items" + assert iceberg_document.get_count() == len(sample_items), ( + "get_count should return the same number as the length of sample_items" + ) From a0a4d1c50baf1bbb21f5df941765a9bdc4e864a5 Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Fri, 12 Dec 2025 19:06:58 -0800 Subject: [PATCH 11/13] Address comments --- amber/src/main/python/core/models/__init__.py | 3 +- .../python/core/models/schema/__init__.py | 2 +- .../core/models/schema/attribute_type.py | 2 +- .../src/main/python/core/models/test_tuple.py | 4 +- amber/src/main/python/core/models/tuple.py | 2 +- .../main/python/core/models/type/__init__.py | 21 ++++ .../models/{schema => type}/large_binary.py | 5 +- .../{schema => type}/test_large_binary.py | 13 +-- .../core/storage/iceberg/iceberg_utils.py | 2 +- .../test_iceberg_utils_large_binary.py | 2 +- .../python/core/storage/storage_config.py | 2 +- amber/src/main/python/pytexera/__init__.py | 2 +- .../storage/large_binary_input_stream.py | 6 +- .../pytexera/storage/large_binary_manager.py | 96 +++++++++---------- .../storage/large_binary_output_stream.py | 10 +- .../storage/test_large_binary_input_stream.py | 20 ++-- .../storage/test_large_binary_manager.py | 34 +++---- .../test_large_binary_output_stream.py | 34 +++---- 18 files changed, 140 insertions(+), 120 deletions(-) create mode 100644 amber/src/main/python/core/models/type/__init__.py rename amber/src/main/python/core/models/{schema => type}/large_binary.py (96%) rename amber/src/main/python/core/models/{schema => type}/test_large_binary.py (89%) diff --git a/amber/src/main/python/core/models/__init__.py b/amber/src/main/python/core/models/__init__.py index 9011c1db887..d24fe0a277d 100644 --- a/amber/src/main/python/core/models/__init__.py +++ b/amber/src/main/python/core/models/__init__.py @@ -15,6 +15,7 @@ # specific language governing permissions and limitations # under the License. +import builtins from inspect import Traceback from typing import NamedTuple @@ -36,7 +37,7 @@ class ExceptionInfo(NamedTuple): - exc: type + exc: builtins.type value: Exception tb: Traceback diff --git a/amber/src/main/python/core/models/schema/__init__.py b/amber/src/main/python/core/models/schema/__init__.py index a5a0de386dd..306fe4e1d4e 100644 --- a/amber/src/main/python/core/models/schema/__init__.py +++ b/amber/src/main/python/core/models/schema/__init__.py @@ -16,7 +16,7 @@ # under the License. from .attribute_type import AttributeType -from .large_binary import largebinary +from core.models.type.large_binary import largebinary from .field import Field from .schema import Schema diff --git a/amber/src/main/python/core/models/schema/attribute_type.py b/amber/src/main/python/core/models/schema/attribute_type.py index 8b8011c55ec..24d0745f41e 100644 --- a/amber/src/main/python/core/models/schema/attribute_type.py +++ b/amber/src/main/python/core/models/schema/attribute_type.py @@ -20,7 +20,7 @@ from bidict import bidict from enum import Enum from pyarrow import lib -from core.models.schema.large_binary import largebinary +from core.models.type.large_binary import largebinary class AttributeType(Enum): diff --git a/amber/src/main/python/core/models/test_tuple.py b/amber/src/main/python/core/models/test_tuple.py index 25d010f80eb..efb4fdf5c71 100644 --- a/amber/src/main/python/core/models/test_tuple.py +++ b/amber/src/main/python/core/models/test_tuple.py @@ -224,7 +224,7 @@ def test_hash(self): def test_tuple_with_large_binary(self): """Test tuple with largebinary field.""" - from core.models.schema.large_binary import largebinary + from core.models.type.large_binary import largebinary schema = Schema( raw_schema={ @@ -250,7 +250,7 @@ def test_tuple_with_large_binary(self): def test_tuple_from_arrow_with_large_binary(self): """Test creating tuple from Arrow table with LARGE_BINARY metadata.""" import pyarrow as pa - from core.models.schema.large_binary import largebinary + from core.models.type.large_binary import largebinary # Create Arrow schema with LARGE_BINARY metadata arrow_schema = pa.schema( diff --git a/amber/src/main/python/core/models/tuple.py b/amber/src/main/python/core/models/tuple.py index b510b28f149..916301406f5 100644 --- a/amber/src/main/python/core/models/tuple.py +++ b/amber/src/main/python/core/models/tuple.py @@ -29,7 +29,7 @@ from typing import Any, List, Iterator, Callable from typing_extensions import Protocol, runtime_checkable -from core.models.schema.large_binary import largebinary +from core.models.type.large_binary import largebinary from .schema.attribute_type import TO_PYOBJECT_MAPPING, AttributeType from .schema.field import Field from .schema.schema import Schema diff --git a/amber/src/main/python/core/models/type/__init__.py b/amber/src/main/python/core/models/type/__init__.py new file mode 100644 index 00000000000..b6456eefbe5 --- /dev/null +++ b/amber/src/main/python/core/models/type/__init__.py @@ -0,0 +1,21 @@ +# 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 .large_binary import largebinary + +__all__ = ["largebinary"] + diff --git a/amber/src/main/python/core/models/schema/large_binary.py b/amber/src/main/python/core/models/type/large_binary.py similarity index 96% rename from amber/src/main/python/core/models/schema/large_binary.py rename to amber/src/main/python/core/models/type/large_binary.py index 578a3520a86..855727d1f8f 100644 --- a/amber/src/main/python/core/models/schema/large_binary.py +++ b/amber/src/main/python/core/models/type/large_binary.py @@ -63,9 +63,9 @@ def __init__(self, uri: Optional[str] = None): """ if uri is None: # Lazy import to avoid circular dependencies - from pytexera.storage.large_binary_manager import LargeBinaryManager + from pytexera.storage import large_binary_manager - uri = LargeBinaryManager.create() + uri = large_binary_manager.create() if not uri.startswith("s3://"): raise ValueError(f"largebinary URI must start with 's3://', got: {uri}") @@ -96,3 +96,4 @@ def __eq__(self, other) -> bool: def __hash__(self) -> int: return hash(self._uri) + diff --git a/amber/src/main/python/core/models/schema/test_large_binary.py b/amber/src/main/python/core/models/type/test_large_binary.py similarity index 89% rename from amber/src/main/python/core/models/schema/test_large_binary.py rename to amber/src/main/python/core/models/type/test_large_binary.py index d8fc93baa7a..5f416f0bbd3 100644 --- a/amber/src/main/python/core/models/schema/test_large_binary.py +++ b/amber/src/main/python/core/models/type/test_large_binary.py @@ -17,7 +17,7 @@ import pytest from unittest.mock import patch -from core.models.schema.large_binary import largebinary +from core.models.type.large_binary import largebinary class TestLargeBinary: @@ -30,14 +30,14 @@ def test_create_with_uri(self): assert repr(large_binary) == f"largebinary('{uri}')" def test_create_without_uri(self): - """Test creating largebinary without URI (calls LargeBinaryManager.create).""" + """Test creating largebinary without URI (calls large_binary_manager.create).""" with patch( - "pytexera.storage.large_binary_manager.LargeBinaryManager" - ) as mock_manager: - mock_manager.create.return_value = "s3://bucket/objects/123/uuid" + "pytexera.storage.large_binary_manager.create" + ) as mock_create: + mock_create.return_value = "s3://bucket/objects/123/uuid" large_binary = largebinary() assert large_binary.uri == "s3://bucket/objects/123/uuid" - mock_manager.create.assert_called_once() + mock_create.assert_called_once() def test_invalid_uri_raises_value_error(self): """Test that invalid URI (not starting with s3://) raises ValueError.""" @@ -88,3 +88,4 @@ def test_uri_property(self): uri = "s3://test-bucket/test/path" large_binary = largebinary(uri) assert large_binary.uri == uri + diff --git a/amber/src/main/python/core/storage/iceberg/iceberg_utils.py b/amber/src/main/python/core/storage/iceberg/iceberg_utils.py index 0a5fca8019d..9e17b2e0e82 100644 --- a/amber/src/main/python/core/storage/iceberg/iceberg_utils.py +++ b/amber/src/main/python/core/storage/iceberg/iceberg_utils.py @@ -231,7 +231,7 @@ def amber_tuples_to_arrow_table( Converts a list of amber tuples to a pyarrow table for serialization. Handles LARGE_BINARY field name encoding and serialization. """ - from core.models.schema.large_binary import largebinary + from core.models.type.large_binary import largebinary tuple_list = list(tuple_list) # Convert to list to allow multiple iterations data_dict = {} diff --git a/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py b/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py index 7859765ae75..c601d4f7190 100644 --- a/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py +++ b/amber/src/main/python/core/storage/iceberg/test_iceberg_utils_large_binary.py @@ -20,7 +20,7 @@ from pyiceberg.schema import Schema as IcebergSchema from core.models import Schema, Tuple from core.models.schema.attribute_type import AttributeType -from core.models.schema.large_binary import largebinary +from core.models.type.large_binary import largebinary from core.storage.iceberg.iceberg_utils import ( encode_large_binary_field_name, decode_large_binary_field_name, diff --git a/amber/src/main/python/core/storage/storage_config.py b/amber/src/main/python/core/storage/storage_config.py index 0d0d07d3284..c55495ea14c 100644 --- a/amber/src/main/python/core/storage/storage_config.py +++ b/amber/src/main/python/core/storage/storage_config.py @@ -32,7 +32,7 @@ class StorageConfig: ICEBERG_FILE_STORAGE_DIRECTORY_PATH = None ICEBERG_TABLE_COMMIT_BATCH_SIZE = None - # S3 configs (for LargeBinaryManager) + # S3 configs (for large_binary_manager module) S3_ENDPOINT = None S3_REGION = None S3_AUTH_USERNAME = None diff --git a/amber/src/main/python/pytexera/__init__.py b/amber/src/main/python/pytexera/__init__.py index 4eec0954b29..e40d1a43fe0 100644 --- a/amber/src/main/python/pytexera/__init__.py +++ b/amber/src/main/python/pytexera/__init__.py @@ -29,7 +29,7 @@ UDFBatchOperator, UDFSourceOperator, ) -from core.models.schema.large_binary import largebinary +from core.models.type.large_binary import largebinary __all__ = [ "State", diff --git a/amber/src/main/python/pytexera/storage/large_binary_input_stream.py b/amber/src/main/python/pytexera/storage/large_binary_input_stream.py index fb9b82744b5..8e7d8640403 100644 --- a/amber/src/main/python/pytexera/storage/large_binary_input_stream.py +++ b/amber/src/main/python/pytexera/storage/large_binary_input_stream.py @@ -26,7 +26,7 @@ from typing import BinaryIO, Optional from functools import wraps from io import IOBase -from core.models.schema.large_binary import largebinary +from core.models.type.large_binary import largebinary def _require_open(func): @@ -61,9 +61,9 @@ def __init__(self, large_binary: largebinary): def _lazy_init(self): """Download from S3 on first read operation.""" - from pytexera.storage.large_binary_manager import LargeBinaryManager + from pytexera.storage import large_binary_manager - s3 = LargeBinaryManager._get_s3_client() + s3 = large_binary_manager._get_s3_client() response = s3.get_object( Bucket=self._large_binary.get_bucket_name(), Key=self._large_binary.get_object_key(), diff --git a/amber/src/main/python/pytexera/storage/large_binary_manager.py b/amber/src/main/python/pytexera/storage/large_binary_manager.py index e53abb5869f..e9e14d68c4d 100644 --- a/amber/src/main/python/pytexera/storage/large_binary_manager.py +++ b/amber/src/main/python/pytexera/storage/large_binary_manager.py @@ -22,65 +22,61 @@ and LargeBinaryInputStream/LargeBinaryOutputStream instead. """ -import logging import time import uuid +from loguru import logger from core.storage.storage_config import StorageConfig -logger = logging.getLogger(__name__) +# Module-level state +_s3_client = None +DEFAULT_BUCKET = "texera-large-binaries" -class LargeBinaryManager: - """Internal manager for largebinary S3 operations.""" +def _get_s3_client(): + """Get or initialize S3 client (lazy initialization, cached).""" + global _s3_client + if _s3_client is None: + try: + import boto3 + from botocore.config import Config + except ImportError as e: + raise RuntimeError( + "boto3 required. Install with: pip install boto3" + ) from e - _s3_client = None - DEFAULT_BUCKET = "texera-large-binaries" + _s3_client = boto3.client( + "s3", + endpoint_url=StorageConfig.S3_ENDPOINT, + aws_access_key_id=StorageConfig.S3_AUTH_USERNAME, + aws_secret_access_key=StorageConfig.S3_AUTH_PASSWORD, + region_name=StorageConfig.S3_REGION, + config=Config( + signature_version="s3v4", s3={"addressing_style": "path"} + ), + ) + return _s3_client - @classmethod - def _get_s3_client(cls): - """Get or initialize S3 client (lazy initialization, cached).""" - if cls._s3_client is None: - try: - import boto3 - from botocore.config import Config - except ImportError as e: - raise RuntimeError( - "boto3 required. Install with: pip install boto3" - ) from e - cls._s3_client = boto3.client( - "s3", - endpoint_url=StorageConfig.S3_ENDPOINT, - aws_access_key_id=StorageConfig.S3_AUTH_USERNAME, - aws_secret_access_key=StorageConfig.S3_AUTH_PASSWORD, - region_name=StorageConfig.S3_REGION, - config=Config( - signature_version="s3v4", s3={"addressing_style": "path"} - ), - ) - return cls._s3_client +def _ensure_bucket_exists(bucket: str): + """Ensure S3 bucket exists, creating it if necessary.""" + s3 = _get_s3_client() + try: + s3.head_bucket(Bucket=bucket) + except s3.exceptions.NoSuchBucket: + logger.debug(f"Bucket {bucket} not found, creating it") + s3.create_bucket(Bucket=bucket) + logger.info(f"Created bucket: {bucket}") - @classmethod - def _ensure_bucket_exists(cls, bucket: str): - """Ensure S3 bucket exists, creating it if necessary.""" - s3 = cls._get_s3_client() - try: - s3.head_bucket(Bucket=bucket) - except s3.exceptions.NoSuchBucket: - logger.debug(f"Bucket {bucket} not found, creating it") - s3.create_bucket(Bucket=bucket) - logger.info(f"Created bucket: {bucket}") - @classmethod - def create(cls) -> str: - """ - Creates a new largebinary reference with a unique S3 URI. +def create() -> str: + """ + Creates a new largebinary reference with a unique S3 URI. - Returns: - S3 URI string (format: s3://bucket/key) - """ - cls._ensure_bucket_exists(cls.DEFAULT_BUCKET) - timestamp_ms = int(time.time() * 1000) - unique_id = uuid.uuid4() - object_key = f"objects/{timestamp_ms}/{unique_id}" - return f"s3://{cls.DEFAULT_BUCKET}/{object_key}" + Returns: + S3 URI string (format: s3://bucket/key) + """ + _ensure_bucket_exists(DEFAULT_BUCKET) + timestamp_ms = int(time.time() * 1000) + unique_id = uuid.uuid4() + object_key = f"objects/{timestamp_ms}/{unique_id}" + return f"s3://{DEFAULT_BUCKET}/{object_key}" diff --git a/amber/src/main/python/pytexera/storage/large_binary_output_stream.py b/amber/src/main/python/pytexera/storage/large_binary_output_stream.py index e401c6c5bba..af4f1a275c2 100644 --- a/amber/src/main/python/pytexera/storage/large_binary_output_stream.py +++ b/amber/src/main/python/pytexera/storage/large_binary_output_stream.py @@ -28,8 +28,8 @@ from typing import Optional, Union from io import IOBase -from core.models.schema.large_binary import largebinary -from pytexera.storage.large_binary_manager import LargeBinaryManager +from core.models.type.large_binary import largebinary +from pytexera.storage import large_binary_manager import threading import queue @@ -155,8 +155,8 @@ def write(self, b: Union[bytes, bytearray]) -> int: def upload_worker(): try: - LargeBinaryManager._ensure_bucket_exists(self._bucket_name) - s3 = LargeBinaryManager._get_s3_client() + large_binary_manager._ensure_bucket_exists(self._bucket_name) + s3 = large_binary_manager._get_s3_client() reader = _QueueReader(self._queue) s3.upload_fileobj(reader, self._bucket_name, self._object_key) except Exception as e: @@ -228,7 +228,7 @@ def close(self) -> None: def _cleanup_failed_upload(self): """Clean up a failed upload by deleting the S3 object.""" try: - s3 = LargeBinaryManager._get_s3_client() + s3 = large_binary_manager._get_s3_client() s3.delete_object(Bucket=self._bucket_name, Key=self._object_key) except Exception: # Ignore cleanup errors - we're already handling an upload failure diff --git a/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py b/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py index 9a1d94b582f..f7f657d4137 100644 --- a/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py @@ -18,7 +18,7 @@ import pytest from unittest.mock import patch, MagicMock from io import BytesIO -from core.models.schema.large_binary import largebinary +from core.models.type.large_binary import largebinary from pytexera.storage.large_binary_input_stream import LargeBinaryInputStream from pytexera.storage import large_binary_manager @@ -52,7 +52,7 @@ def test_init_with_none_raises_error(self): def test_lazy_init_downloads_from_s3(self, large_binary, mock_s3_response): """Test that _lazy_init downloads from S3 on first read.""" with patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response @@ -77,7 +77,7 @@ def test_lazy_init_downloads_from_s3(self, large_binary, mock_s3_response): def test_read_all(self, large_binary, mock_s3_response): """Test reading all data.""" with patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response @@ -93,7 +93,7 @@ def test_read_all(self, large_binary, mock_s3_response): def test_read_partial(self, large_binary, mock_s3_response): """Test reading partial data.""" with patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response @@ -109,7 +109,7 @@ def test_read_partial(self, large_binary, mock_s3_response): def test_readline(self, large_binary): """Test reading a line.""" with patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client: response = {"Body": BytesIO(b"line1\nline2\nline3")} mock_s3_client = MagicMock() @@ -126,7 +126,7 @@ def test_readline(self, large_binary): def test_readlines(self, large_binary): """Test reading all lines.""" with patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client: response = {"Body": BytesIO(b"line1\nline2\nline3")} mock_s3_client = MagicMock() @@ -175,7 +175,7 @@ def test_closed_property(self, large_binary): def test_close(self, large_binary, mock_s3_response): """Test closing the stream.""" with patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response @@ -192,7 +192,7 @@ def test_close(self, large_binary, mock_s3_response): def test_context_manager(self, large_binary, mock_s3_response): """Test using as context manager.""" with patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response @@ -209,7 +209,7 @@ def test_context_manager(self, large_binary, mock_s3_response): def test_iteration(self, large_binary): """Test iteration over lines.""" with patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client: response = {"Body": BytesIO(b"line1\nline2\nline3")} mock_s3_client = MagicMock() @@ -226,7 +226,7 @@ def test_iteration(self, large_binary): def test_read_after_close_raises_error(self, large_binary, mock_s3_response): """Test that reading after close raises ValueError.""" with patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response diff --git a/amber/src/main/python/pytexera/storage/test_large_binary_manager.py b/amber/src/main/python/pytexera/storage/test_large_binary_manager.py index 8d9e3e223ef..a657f244f38 100644 --- a/amber/src/main/python/pytexera/storage/test_large_binary_manager.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_manager.py @@ -17,7 +17,7 @@ import pytest from unittest.mock import patch, MagicMock -from pytexera.storage.large_binary_manager import LargeBinaryManager +from pytexera.storage import large_binary_manager from core.storage.storage_config import StorageConfig @@ -42,25 +42,25 @@ def setup_storage_config(self): def test_get_s3_client_initializes_once(self): """Test that S3 client is initialized and cached.""" # Reset the client - LargeBinaryManager._s3_client = None + large_binary_manager._s3_client = None with patch("boto3.client") as mock_boto3_client: mock_client = MagicMock() mock_boto3_client.return_value = mock_client # First call should create client - client1 = LargeBinaryManager._get_s3_client() + client1 = large_binary_manager._get_s3_client() assert client1 == mock_client assert mock_boto3_client.call_count == 1 # Second call should return cached client - client2 = LargeBinaryManager._get_s3_client() + client2 = large_binary_manager._get_s3_client() assert client2 == mock_client assert mock_boto3_client.call_count == 1 # Still 1, not 2 def test_get_s3_client_without_boto3_raises_error(self): """Test that missing boto3 raises RuntimeError.""" - LargeBinaryManager._s3_client = None + large_binary_manager._s3_client = None import sys @@ -77,7 +77,7 @@ def mock_import(name, *args, **kwargs): with patch("builtins.__import__", side_effect=mock_import): with pytest.raises(RuntimeError, match="boto3 required"): - LargeBinaryManager._get_s3_client() + large_binary_manager._get_s3_client() finally: # Restore boto3 if it was there if boto3_backup is not None: @@ -85,7 +85,7 @@ def mock_import(name, *args, **kwargs): def test_ensure_bucket_exists_when_bucket_exists(self): """Test that existing bucket doesn't trigger creation.""" - LargeBinaryManager._s3_client = None + large_binary_manager._s3_client = None with patch("boto3.client") as mock_boto3_client: mock_client = MagicMock() @@ -94,13 +94,13 @@ def test_ensure_bucket_exists_when_bucket_exists(self): mock_client.head_bucket.return_value = None mock_client.exceptions.NoSuchBucket = type("NoSuchBucket", (Exception,), {}) - LargeBinaryManager._ensure_bucket_exists("test-bucket") + large_binary_manager._ensure_bucket_exists("test-bucket") mock_client.head_bucket.assert_called_once_with(Bucket="test-bucket") mock_client.create_bucket.assert_not_called() def test_ensure_bucket_exists_creates_bucket_when_missing(self): """Test that missing bucket triggers creation.""" - LargeBinaryManager._s3_client = None + large_binary_manager._s3_client = None with patch("boto3.client") as mock_boto3_client: mock_client = MagicMock() @@ -110,13 +110,13 @@ def test_ensure_bucket_exists_creates_bucket_when_missing(self): mock_client.exceptions.NoSuchBucket = no_such_bucket mock_client.head_bucket.side_effect = no_such_bucket() - LargeBinaryManager._ensure_bucket_exists("test-bucket") + large_binary_manager._ensure_bucket_exists("test-bucket") mock_client.head_bucket.assert_called_once_with(Bucket="test-bucket") mock_client.create_bucket.assert_called_once_with(Bucket="test-bucket") def test_create_generates_unique_uri(self): """Test that create() generates a unique S3 URI.""" - LargeBinaryManager._s3_client = None + large_binary_manager._s3_client = None with patch("boto3.client") as mock_boto3_client: mock_client = MagicMock() @@ -124,21 +124,21 @@ def test_create_generates_unique_uri(self): mock_client.head_bucket.return_value = None mock_client.exceptions.NoSuchBucket = type("NoSuchBucket", (Exception,), {}) - uri = LargeBinaryManager.create() + uri = large_binary_manager.create() # Check URI format assert uri.startswith("s3://") - assert uri.startswith(f"s3://{LargeBinaryManager.DEFAULT_BUCKET}/") + assert uri.startswith(f"s3://{large_binary_manager.DEFAULT_BUCKET}/") assert "objects/" in uri # Verify bucket was checked/created mock_client.head_bucket.assert_called_once_with( - Bucket=LargeBinaryManager.DEFAULT_BUCKET + Bucket=large_binary_manager.DEFAULT_BUCKET ) def test_create_uses_default_bucket(self): """Test that create() uses the default bucket.""" - LargeBinaryManager._s3_client = None + large_binary_manager._s3_client = None with patch("boto3.client") as mock_boto3_client: mock_client = MagicMock() @@ -146,5 +146,5 @@ def test_create_uses_default_bucket(self): mock_client.head_bucket.return_value = None mock_client.exceptions.NoSuchBucket = type("NoSuchBucket", (Exception,), {}) - uri = LargeBinaryManager.create() - assert LargeBinaryManager.DEFAULT_BUCKET in uri + uri = large_binary_manager.create() + assert large_binary_manager.DEFAULT_BUCKET in uri diff --git a/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py b/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py index 08eefaf3e61..50bdfe49efd 100644 --- a/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py @@ -18,7 +18,7 @@ import pytest import time from unittest.mock import patch, MagicMock -from core.models.schema.large_binary import largebinary +from core.models.type.large_binary import largebinary from pytexera.storage.large_binary_output_stream import LargeBinaryOutputStream from pytexera.storage import large_binary_manager @@ -47,10 +47,10 @@ def test_write_starts_upload_thread(self, large_binary): """Test that write() starts the upload thread.""" with ( patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" + large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() @@ -72,10 +72,10 @@ def test_write_data(self, large_binary): """Test writing data to the stream.""" with ( patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" + large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() @@ -92,10 +92,10 @@ def test_write_multiple_chunks(self, large_binary): """Test writing multiple chunks of data.""" with ( patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" + large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() @@ -140,10 +140,10 @@ def test_close_completes_upload(self, large_binary): """Test that close() completes the upload.""" with ( patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" + large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() @@ -163,10 +163,10 @@ def test_context_manager(self, large_binary): """Test using as context manager.""" with ( patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" + large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() @@ -192,10 +192,10 @@ def test_close_handles_upload_error(self, large_binary): """Test that close() raises IOError if upload fails.""" with ( patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" + large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() @@ -213,10 +213,10 @@ def test_write_after_upload_error_raises_error(self, large_binary): """Test that writing after upload error raises IOError.""" with ( patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" + large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() @@ -237,10 +237,10 @@ def test_multiple_close_calls(self, large_binary): """Test that multiple close() calls are safe.""" with ( patch.object( - large_binary_manager.LargeBinaryManager, "_get_s3_client" + large_binary_manager, "_get_s3_client" ) as mock_get_s3_client, patch.object( - large_binary_manager.LargeBinaryManager, "_ensure_bucket_exists" + large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, ): mock_s3 = MagicMock() From 3ed7fb63b120eca4705d5dc55694a5146b1d4010 Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Fri, 12 Dec 2025 19:11:11 -0800 Subject: [PATCH 12/13] Address comments --- .../core/models/schema/arrow_schema_utils.py | 51 +------------ .../models/schema/attribute_type_utils.py | 73 +++++++++++++++++++ 2 files changed, 77 insertions(+), 47 deletions(-) create mode 100644 amber/src/main/python/core/models/schema/attribute_type_utils.py diff --git a/amber/src/main/python/core/models/schema/arrow_schema_utils.py b/amber/src/main/python/core/models/schema/arrow_schema_utils.py index 25ac5fa31de..527e0095e65 100644 --- a/amber/src/main/python/core/models/schema/arrow_schema_utils.py +++ b/amber/src/main/python/core/models/schema/arrow_schema_utils.py @@ -23,55 +23,12 @@ import pyarrow as pa from typing import Mapping -from core.models.schema.attribute_type import ( - AttributeType, - FROM_ARROW_MAPPING, - TO_ARROW_MAPPING, +from core.models.schema.attribute_type import AttributeType +from core.models.schema.attribute_type_utils import ( + detect_attribute_type_from_arrow_field, + create_arrow_field_with_metadata, ) -# Metadata key used to mark LARGE_BINARY fields in Arrow schemas -TEXERA_TYPE_METADATA_KEY = b"texera_type" -LARGE_BINARY_METADATA_VALUE = b"LARGE_BINARY" - - -def detect_attribute_type_from_arrow_field(field: pa.Field) -> AttributeType: - """ - Detects the AttributeType from an Arrow field, checking metadata for LARGE_BINARY. - - :param field: PyArrow field that may contain metadata - :return: The detected AttributeType - """ - # Check metadata for LARGE_BINARY type - # (can be stored by either Scala ArrowUtils or Python) - is_large_binary = ( - field.metadata - and field.metadata.get(TEXERA_TYPE_METADATA_KEY) == LARGE_BINARY_METADATA_VALUE - ) - - if is_large_binary: - return AttributeType.LARGE_BINARY - else: - return FROM_ARROW_MAPPING[field.type.id] - - -def create_arrow_field_with_metadata( - attr_name: str, attr_type: AttributeType -) -> pa.Field: - """ - Creates a PyArrow field with appropriate metadata for the given AttributeType. - - :param attr_name: Name of the attribute - :param attr_type: The AttributeType - :return: PyArrow field with metadata if needed - """ - metadata = ( - {TEXERA_TYPE_METADATA_KEY: LARGE_BINARY_METADATA_VALUE} - if attr_type == AttributeType.LARGE_BINARY - else None - ) - - return pa.field(attr_name, TO_ARROW_MAPPING[attr_type], metadata=metadata) - def arrow_schema_to_attr_types(arrow_schema: pa.Schema) -> dict[str, AttributeType]: """ diff --git a/amber/src/main/python/core/models/schema/attribute_type_utils.py b/amber/src/main/python/core/models/schema/attribute_type_utils.py new file mode 100644 index 00000000000..5a4b2930368 --- /dev/null +++ b/amber/src/main/python/core/models/schema/attribute_type_utils.py @@ -0,0 +1,73 @@ +# 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. + +""" +Utilities for converting between AttributeTypes and Arrow field types, +handling LARGE_BINARY metadata preservation. +""" + +import pyarrow as pa + +from core.models.schema.attribute_type import ( + AttributeType, + FROM_ARROW_MAPPING, + TO_ARROW_MAPPING, +) + +# Metadata key used to mark LARGE_BINARY fields in Arrow schemas +TEXERA_TYPE_METADATA_KEY = b"texera_type" +LARGE_BINARY_METADATA_VALUE = b"LARGE_BINARY" + + +def detect_attribute_type_from_arrow_field(field: pa.Field) -> AttributeType: + """ + Detects the AttributeType from an Arrow field, checking metadata for LARGE_BINARY. + + :param field: PyArrow field that may contain metadata + :return: The detected AttributeType + """ + # Check metadata for LARGE_BINARY type + # (can be stored by either Scala ArrowUtils or Python) + is_large_binary = ( + field.metadata + and field.metadata.get(TEXERA_TYPE_METADATA_KEY) == LARGE_BINARY_METADATA_VALUE + ) + + if is_large_binary: + return AttributeType.LARGE_BINARY + else: + return FROM_ARROW_MAPPING[field.type.id] + + +def create_arrow_field_with_metadata( + attr_name: str, attr_type: AttributeType +) -> pa.Field: + """ + Creates a PyArrow field with appropriate metadata for the given AttributeType. + + :param attr_name: Name of the attribute + :param attr_type: The AttributeType + :return: PyArrow field with metadata if needed + """ + metadata = ( + {TEXERA_TYPE_METADATA_KEY: LARGE_BINARY_METADATA_VALUE} + if attr_type == AttributeType.LARGE_BINARY + else None + ) + + return pa.field(attr_name, TO_ARROW_MAPPING[attr_type], metadata=metadata) + From 3a64dd1eea8df2928e0efb81abf40e57f0510736 Mon Sep 17 00:00:00 2001 From: Kunwoo Park Date: Fri, 12 Dec 2025 19:23:15 -0800 Subject: [PATCH 13/13] Format files --- .../models/schema/attribute_type_utils.py | 1 - .../main/python/core/models/type/__init__.py | 1 - .../python/core/models/type/large_binary.py | 1 - .../core/models/type/test_large_binary.py | 5 +-- .../pytexera/storage/large_binary_manager.py | 8 ++--- .../storage/test_large_binary_input_stream.py | 36 +++++-------------- .../test_large_binary_output_stream.py | 32 +++++------------ 7 files changed, 20 insertions(+), 64 deletions(-) diff --git a/amber/src/main/python/core/models/schema/attribute_type_utils.py b/amber/src/main/python/core/models/schema/attribute_type_utils.py index 5a4b2930368..3918fdfc342 100644 --- a/amber/src/main/python/core/models/schema/attribute_type_utils.py +++ b/amber/src/main/python/core/models/schema/attribute_type_utils.py @@ -70,4 +70,3 @@ def create_arrow_field_with_metadata( ) return pa.field(attr_name, TO_ARROW_MAPPING[attr_type], metadata=metadata) - diff --git a/amber/src/main/python/core/models/type/__init__.py b/amber/src/main/python/core/models/type/__init__.py index b6456eefbe5..41344433aab 100644 --- a/amber/src/main/python/core/models/type/__init__.py +++ b/amber/src/main/python/core/models/type/__init__.py @@ -18,4 +18,3 @@ from .large_binary import largebinary __all__ = ["largebinary"] - diff --git a/amber/src/main/python/core/models/type/large_binary.py b/amber/src/main/python/core/models/type/large_binary.py index 855727d1f8f..581a688912b 100644 --- a/amber/src/main/python/core/models/type/large_binary.py +++ b/amber/src/main/python/core/models/type/large_binary.py @@ -96,4 +96,3 @@ def __eq__(self, other) -> bool: def __hash__(self) -> int: return hash(self._uri) - diff --git a/amber/src/main/python/core/models/type/test_large_binary.py b/amber/src/main/python/core/models/type/test_large_binary.py index 5f416f0bbd3..36310e1dd53 100644 --- a/amber/src/main/python/core/models/type/test_large_binary.py +++ b/amber/src/main/python/core/models/type/test_large_binary.py @@ -31,9 +31,7 @@ def test_create_with_uri(self): def test_create_without_uri(self): """Test creating largebinary without URI (calls large_binary_manager.create).""" - with patch( - "pytexera.storage.large_binary_manager.create" - ) as mock_create: + with patch("pytexera.storage.large_binary_manager.create") as mock_create: mock_create.return_value = "s3://bucket/objects/123/uuid" large_binary = largebinary() assert large_binary.uri == "s3://bucket/objects/123/uuid" @@ -88,4 +86,3 @@ def test_uri_property(self): uri = "s3://test-bucket/test/path" large_binary = largebinary(uri) assert large_binary.uri == uri - diff --git a/amber/src/main/python/pytexera/storage/large_binary_manager.py b/amber/src/main/python/pytexera/storage/large_binary_manager.py index e9e14d68c4d..e061eac6228 100644 --- a/amber/src/main/python/pytexera/storage/large_binary_manager.py +++ b/amber/src/main/python/pytexera/storage/large_binary_manager.py @@ -40,9 +40,7 @@ def _get_s3_client(): import boto3 from botocore.config import Config except ImportError as e: - raise RuntimeError( - "boto3 required. Install with: pip install boto3" - ) from e + raise RuntimeError("boto3 required. Install with: pip install boto3") from e _s3_client = boto3.client( "s3", @@ -50,9 +48,7 @@ def _get_s3_client(): aws_access_key_id=StorageConfig.S3_AUTH_USERNAME, aws_secret_access_key=StorageConfig.S3_AUTH_PASSWORD, region_name=StorageConfig.S3_REGION, - config=Config( - signature_version="s3v4", s3={"addressing_style": "path"} - ), + config=Config(signature_version="s3v4", s3={"addressing_style": "path"}), ) return _s3_client diff --git a/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py b/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py index f7f657d4137..85bdbd13fa1 100644 --- a/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_input_stream.py @@ -51,9 +51,7 @@ def test_init_with_none_raises_error(self): def test_lazy_init_downloads_from_s3(self, large_binary, mock_s3_response): """Test that _lazy_init downloads from S3 on first read.""" - with patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client: + with patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client @@ -76,9 +74,7 @@ def test_lazy_init_downloads_from_s3(self, large_binary, mock_s3_response): def test_read_all(self, large_binary, mock_s3_response): """Test reading all data.""" - with patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client: + with patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client @@ -92,9 +88,7 @@ def test_read_all(self, large_binary, mock_s3_response): def test_read_partial(self, large_binary, mock_s3_response): """Test reading partial data.""" - with patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client: + with patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client @@ -108,9 +102,7 @@ def test_read_partial(self, large_binary, mock_s3_response): def test_readline(self, large_binary): """Test reading a line.""" - with patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client: + with patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client: response = {"Body": BytesIO(b"line1\nline2\nline3")} mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = response @@ -125,9 +117,7 @@ def test_readline(self, large_binary): def test_readlines(self, large_binary): """Test reading all lines.""" - with patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client: + with patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client: response = {"Body": BytesIO(b"line1\nline2\nline3")} mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = response @@ -174,9 +164,7 @@ def test_closed_property(self, large_binary): def test_close(self, large_binary, mock_s3_response): """Test closing the stream.""" - with patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client: + with patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client @@ -191,9 +179,7 @@ def test_close(self, large_binary, mock_s3_response): def test_context_manager(self, large_binary, mock_s3_response): """Test using as context manager.""" - with patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client: + with patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client @@ -208,9 +194,7 @@ def test_context_manager(self, large_binary, mock_s3_response): def test_iteration(self, large_binary): """Test iteration over lines.""" - with patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client: + with patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client: response = {"Body": BytesIO(b"line1\nline2\nline3")} mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = response @@ -225,9 +209,7 @@ def test_iteration(self, large_binary): def test_read_after_close_raises_error(self, large_binary, mock_s3_response): """Test that reading after close raises ValueError.""" - with patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client: + with patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client: mock_s3_client = MagicMock() mock_s3_client.get_object.return_value = mock_s3_response mock_get_s3_client.return_value = mock_s3_client diff --git a/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py b/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py index 50bdfe49efd..7ebcc9b4cfd 100644 --- a/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py +++ b/amber/src/main/python/pytexera/storage/test_large_binary_output_stream.py @@ -46,9 +46,7 @@ def test_init_with_none_raises_error(self): def test_write_starts_upload_thread(self, large_binary): """Test that write() starts the upload thread.""" with ( - patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client, + patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client, patch.object( large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, @@ -71,9 +69,7 @@ def test_write_starts_upload_thread(self, large_binary): def test_write_data(self, large_binary): """Test writing data to the stream.""" with ( - patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client, + patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client, patch.object( large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, @@ -91,9 +87,7 @@ def test_write_data(self, large_binary): def test_write_multiple_chunks(self, large_binary): """Test writing multiple chunks of data.""" with ( - patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client, + patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client, patch.object( large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, @@ -139,9 +133,7 @@ def test_flush(self, large_binary): def test_close_completes_upload(self, large_binary): """Test that close() completes the upload.""" with ( - patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client, + patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client, patch.object( large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, @@ -162,9 +154,7 @@ def test_close_completes_upload(self, large_binary): def test_context_manager(self, large_binary): """Test using as context manager.""" with ( - patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client, + patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client, patch.object( large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, @@ -191,9 +181,7 @@ def test_write_after_close_raises_error(self, large_binary): def test_close_handles_upload_error(self, large_binary): """Test that close() raises IOError if upload fails.""" with ( - patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client, + patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client, patch.object( large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, @@ -212,9 +200,7 @@ def test_close_handles_upload_error(self, large_binary): def test_write_after_upload_error_raises_error(self, large_binary): """Test that writing after upload error raises IOError.""" with ( - patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client, + patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client, patch.object( large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket, @@ -236,9 +222,7 @@ def test_write_after_upload_error_raises_error(self, large_binary): def test_multiple_close_calls(self, large_binary): """Test that multiple close() calls are safe.""" with ( - patch.object( - large_binary_manager, "_get_s3_client" - ) as mock_get_s3_client, + patch.object(large_binary_manager, "_get_s3_client") as mock_get_s3_client, patch.object( large_binary_manager, "_ensure_bucket_exists" ) as mock_ensure_bucket,