From 7bc6fb4765746a43d7821d4673b36819413a226e Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Sun, 19 Jan 2025 17:48:33 -0800 Subject: [PATCH 01/29] Working version. --- core/amber/requirements.txt | 6 +- .../core/models/schema/attribute_type.py | 2 +- .../src/main/python/core/storage/__init__.py | 0 .../core/storage/iceberg_catalog_instance.py | 48 ++++ .../python/core/storage/iceberg_document.py | 216 +++++++++++++++++ .../core/storage/iceberg_table_writer.py | 113 +++++++++ .../main/python/core/storage/iceberg_utils.py | 92 ++++++++ .../python/core/storage/model/__init__.py | 0 .../storage/model/buffered_item_writer.py | 56 +++++ .../model/readonly_virtual_document.py | 85 +++++++ .../core/storage/model/virtual_document.py | 78 ++++++ .../core/storage/test_iceberg_document.py | 222 ++++++++++++++++++ 12 files changed, 916 insertions(+), 2 deletions(-) create mode 100644 core/amber/src/main/python/core/storage/__init__.py create mode 100644 core/amber/src/main/python/core/storage/iceberg_catalog_instance.py create mode 100644 core/amber/src/main/python/core/storage/iceberg_document.py create mode 100644 core/amber/src/main/python/core/storage/iceberg_table_writer.py create mode 100644 core/amber/src/main/python/core/storage/iceberg_utils.py create mode 100644 core/amber/src/main/python/core/storage/model/__init__.py create mode 100644 core/amber/src/main/python/core/storage/model/buffered_item_writer.py create mode 100644 core/amber/src/main/python/core/storage/model/readonly_virtual_document.py create mode 100644 core/amber/src/main/python/core/storage/model/virtual_document.py create mode 100644 core/amber/src/main/python/core/storage/test_iceberg_document.py diff --git a/core/amber/requirements.txt b/core/amber/requirements.txt index cd79a6157d4..ca0f39adb24 100644 --- a/core/amber/requirements.txt +++ b/core/amber/requirements.txt @@ -26,4 +26,8 @@ bidict==0.22.0 cached_property==1.5.2 psutil==5.9.0 transformers==4.44.2 -tzlocal==2.1 \ No newline at end of file +tzlocal==2.1 +pyiceberg==0.8.1 +grpclib==0.4.7 +packaging==24.1 +readerwriterlock==1.0.9 \ No newline at end of file diff --git a/core/amber/src/main/python/core/models/schema/attribute_type.py b/core/amber/src/main/python/core/models/schema/attribute_type.py index f12a3b92f5b..3ea433a7d15 100644 --- a/core/amber/src/main/python/core/models/schema/attribute_type.py +++ b/core/amber/src/main/python/core/models/schema/attribute_type.py @@ -42,7 +42,7 @@ class AttributeType(Enum): AttributeType.DOUBLE: pa.float64(), AttributeType.BOOL: pa.bool_(), AttributeType.BINARY: pa.binary(), - AttributeType.TIMESTAMP: pa.timestamp("ms", tz="UTC"), + AttributeType.TIMESTAMP: pa.timestamp("us"), } FROM_ARROW_MAPPING = { diff --git a/core/amber/src/main/python/core/storage/__init__.py b/core/amber/src/main/python/core/storage/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py new file mode 100644 index 00000000000..4e7ec3ebc04 --- /dev/null +++ b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py @@ -0,0 +1,48 @@ +from typing import Optional + +from pyiceberg.catalog import Catalog +from pyiceberg.catalog.sql import SqlCatalog + +warehouse_path = "/Users/xzliu/IdeaProjects/texera/core/amber/user-resources/workflow-results" + + +class IcebergCatalogInstance: + """ + IcebergCatalogInstance is a singleton that manages the Iceberg catalog instance. + - Provides a single shared catalog for all Iceberg table-related operations. + - Lazily initializes the catalog on first access. + - Supports replacing the catalog instance for testing or reconfiguration. + """ + _instance: Optional[Catalog] = None + + @classmethod + def get_instance(cls): + """ + Retrieves the singleton Iceberg catalog instance. + - If the catalog is not initialized, it is lazily created using the configured properties. + :return: the Iceberg catalog instance. + """ + if cls._instance is None: + cls._instance = SqlCatalog( + "texera_iceberg", + **{ + "uri": "postgresql+psycopg2://iceberg_py_test:test@localhost/test_iceberg_catalog_jan16", + "warehouse": f"file://{warehouse_path}", + "init_catalog_tables": "true" + } + ) + return cls._instance + + @classmethod + def replace_instance(cls, catalog: Catalog): + """ + Replaces the existing Iceberg catalog instance. + - This method is useful for testing or dynamically updating the catalog. + :param catalog: the new Iceberg catalog instance to replace the current one. + """ + cls._instance = catalog + + +if __name__ == '__main__': + ins = IcebergCatalogInstance.get_instance() + print(ins) diff --git a/core/amber/src/main/python/core/storage/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg_document.py new file mode 100644 index 00000000000..ad72eeb3980 --- /dev/null +++ b/core/amber/src/main/python/core/storage/iceberg_document.py @@ -0,0 +1,216 @@ +from threading import RLock +from typing import Iterator, Optional +from typing import TypeVar +from urllib.parse import ParseResult, urlparse + +from pyiceberg.catalog import Catalog +from pyiceberg.schema import Schema +from pyiceberg.table import Table, FileScanTask +from readerwriterlock import rwlock + +import core.models +from core.models import Tuple +from core.storage.iceberg_catalog_instance import IcebergCatalogInstance +from core.storage.iceberg_utils import create_table, load_table_metadata, read_data_file_as_iterator +from core.storage.model.virtual_document import VirtualDocument +from core.storage.iceberg_table_writer import IcebergTableWriter + +# Define a type variable +T = TypeVar('T') + + +class IcebergDocument(VirtualDocument[T]): + """ + IcebergDocument is used to read and write a set of T as an Iceberg table. + It provides iterator-based read methods and supports multiple writers to write to the same table. + + - On construction, the table will be created if it does not exist. + - If the table exists, it will be overridden. + + :param table_namespace: Namespace of the table. + :param table_name: Name of the table. + :param table_schema: Schema of the table. + """ + + def __init__( + self, + table_namespace: str, + table_name: str, + table_schema: Schema, + catalog: Optional[Catalog] = None + ): + self.table_namespace = table_namespace + self.table_name = table_name + self.table_schema = table_schema + + self.lock = rwlock.RWLockFair() + self.catalog = catalog or self._load_catalog() + + # Create or override the table during initialization + create_table( + self.catalog, + self.table_namespace, + self.table_name, + self.table_schema, + override_if_exists=True + ) + + def _load_catalog(self) -> Catalog: + """Load the Iceberg catalog.""" + # Implement catalog loading logic here, e.g., load from configuration + return IcebergCatalogInstance.get_instance() + + def get_uri(self) -> ParseResult: + """Returns the URI of the table location.""" + table = load_table_metadata(self.catalog, self.table_namespace, self.table_name) + if not table: + raise Exception(f"table {self.table_namespace}.{self.table_name} doesn't exist.") + return urlparse(table.location()) + + def clear(self): + """Deletes the table and clears its contents.""" + with self.lock.gen_wlock(): + table_identifier = f"{self.table_namespace}.{self.table_name}" + if self.catalog.table_exists(table_identifier): + self.catalog.drop_table(table_identifier) + + def get(self) -> Iterator[T]: + """Get an iterator for reading all records from the table.""" + return self._get_using_file_sequence_order(0, None) + + def get_range(self, from_index: int, until: int) -> Iterator[T]: + """Get records within a specified range [from, until).""" + return self._get_using_file_sequence_order(from_index, until) + + def get_after(self, offset: int) -> Iterator[T]: + """Get records starting after a specified offset.""" + return self._get_using_file_sequence_order(offset, None) + + def get_count(self) -> int: + """Get the total count of records in the table.""" + table = load_table_metadata(self.catalog, self.table_namespace, self.table_name) + if not table: + return 0 + return sum(f.file.record_count for f in table.scan().plan_files()) + + def writer(self, writer_identifier: str): + """Creates a BufferedItemWriter for writing data to the table.""" + return IcebergTableWriter( + writer_identifier=writer_identifier, + catalog=self.catalog, + table_namespace=self.table_namespace, + table_name=self.table_name, + table_schema=self.table_schema + ) + + def _get_using_file_sequence_order( + self, from_index: int, until: Optional[int] + ) -> Iterator[T]: + """Utility to get records within a specified range.""" + with self.lock.gen_rlock(): + iterator = IcebergIterator( + from_index, + until, + self.catalog, + self.table_namespace, + self.table_name, + self.table_schema + ) + return iterator + + +class IcebergIterator(Iterator[T]): + def __init__(self, from_index, until, catalog, table_namespace, table_name, table_schema): + self.from_index = from_index + self.until = until + self.catalog = catalog + self.table_namespace = table_namespace + self.table_name = table_name + self.table_schema = table_schema + self.lock = RLock() + self.num_of_skipped_records = 0 + self.num_of_returned_records = 0 + self.total_records_to_return = self.until - self.from_index if until else float("inf") + self.current_record_iterator = iter([]) + self.table = self._load_table_metadata() + self.usable_file_iterator = self._seek_to_usable_file() + + def _load_table_metadata(self) -> Optional[Table]: + """Load table metadata.""" + return load_table_metadata(self.catalog, self.table_namespace, self.table_name) + + def _seek_to_usable_file(self) -> Iterator[FileScanTask]: + """Find usable file scan tasks starting from the specified record index.""" + with self.lock: + if self.num_of_skipped_records > self.from_index: + raise RuntimeError("seek operation should not be called") + + # Refresh table snapshots + if not self.table: + self.table = self._load_table_metadata() + + if self.table: + try: + self.table.refresh() + # Retrieve the entries from the table + entries = self.table.inspect.entries() + + # Convert to a Pandas DataFrame for easy manipulation + entries_df = entries.to_pandas() + + # Sort by file_sequence_number + file_sequence_map = { + row["data_file"]["file_path"]: row["file_sequence_number"] + for _, row in entries_df.iterrows() + } + + # Retrieve and sort the file scan tasks by file sequence number + file_scan_tasks = list(self.table.scan().plan_files()) + sorted_file_scan_tasks = sorted( + file_scan_tasks, + key=lambda task: file_sequence_map.get(task.file.file_path, float('inf')) # Use float('inf') for missing files + ) + # Skip records in files before the `from_index` + for task in sorted_file_scan_tasks: + record_count = task.file.record_count + if self.num_of_skipped_records + record_count <= self.from_index: + self.num_of_skipped_records += record_count + continue + yield task + except Exception: + return iter([]) + else: + return iter([]) + + def __iter__(self) -> Iterator[T]: + return self + + def __next__(self) -> T: + if self.num_of_returned_records >= self.total_records_to_return: + raise StopIteration("No more records available") + + while True: + try: + record = Tuple(next(self.current_record_iterator), schema=core.models.Schema(self.table_schema)) + self.num_of_returned_records += 1 + return record + except StopIteration: + # current_record_iterator is exhausted, need to go to the next file + try: + next_file = next(self.usable_file_iterator) + self.current_record_iterator = read_data_file_as_iterator(next_file, self.table) + # Skip records within the file if necessary + records_to_skip_in_file = self.from_index - self.num_of_skipped_records + if records_to_skip_in_file > 0: + self.current_record_iterator = self._skip_records( + self.current_record_iterator, + records_to_skip_in_file + ) + self.num_of_skipped_records += records_to_skip_in_file + except StopIteration: + # no more files left in this table + raise StopIteration("No more records available") + + @staticmethod + def _skip_records(iterator, count): + return iter(list(iterator)[count:]) diff --git a/core/amber/src/main/python/core/storage/iceberg_table_writer.py b/core/amber/src/main/python/core/storage/iceberg_table_writer.py new file mode 100644 index 00000000000..ce870bc889f --- /dev/null +++ b/core/amber/src/main/python/core/storage/iceberg_table_writer.py @@ -0,0 +1,113 @@ +from pathlib import Path + +from tenacity import retry, wait_exponential, stop_after_attempt +from typing import List, TypeVar + +import pyarrow as pa +from pyiceberg.catalog import Catalog +from pyiceberg.table import Table + +from core.storage.model.buffered_item_writer import BufferedItemWriter + +# Define a type variable for the data type T +T = TypeVar('T') + + +class IcebergTableWriter(BufferedItemWriter[T]): + """ + IcebergTableWriter writes data to the given Iceberg table in an append-only way. + - Each time the buffer is flushed, a new data file is created with a unique name. + - The `writer_identifier` is used to prefix the created files. + - Iceberg data files are immutable once created. So each flush will create a distinct file. + + **Thread Safety**: This writer is NOT thread-safe, so only one thread should call this writer. + + :param writer_identifier: A unique identifier used to prefix the created files. + :param catalog: The Iceberg catalog to manage table metadata. + :param table_namespace: The namespace of the Iceberg table. + :param table_name: The name of the Iceberg table. + :param table_schema: The schema of the Iceberg table. + :param buffer_size: The maximum size of the buffer before flushing. + """ + + def __init__( + self, + writer_identifier: str, + catalog: Catalog, + table_namespace: str, + table_name: str, + table_schema: pa.Schema, + buffer_size: int = 4096 # Default buffer size TODO: move to config + ): + self.writer_identifier = writer_identifier + self.catalog = catalog + self.table_namespace = table_namespace + self.table_name = table_name + self.table_schema = table_schema + self.buffer_size = buffer_size + + # Internal state + self.buffer: List[T] = [] + self.filename_idx = 0 + self.record_id = 0 + + # Load the Iceberg table + self.table: Table = self.catalog.load_table(f"{self.table_namespace}.{self.table_name}") + + @property + def buffer_size(self) -> int: + return self._buffer_size + + def open(self) -> None: + """Open the writer and clear the buffer.""" + self.buffer.clear() + + def put_one(self, item: T) -> None: + """Add a single item to the buffer.""" + self.buffer.append(item) + if len(self.buffer) >= self.buffer_size: + self.flush_buffer() + + def remove_one(self, item: T) -> None: + """Remove a single item from the buffer.""" + self.buffer.remove(item) + + def flush_buffer(self) -> None: + """Flush the current buffer to a new Iceberg data file.""" + if not self.buffer: + return + df = pa.Table.from_pydict( + { + name: [t[name] for t in self.buffer] + for name in self.table_schema.names + }, + schema=self.table_schema, + ) + + def append_to_table_with_retry(pa_df: pa.Table) -> None: + """Appends a pyarrow dataframe to the table in the catalog using tenacity exponential backoff.""" + + @retry( + wait=wait_exponential(multiplier=1, min=4, max=32), + stop=stop_after_attempt(10), + reraise=True + ) + def append_with_retry(): + # table = catalog.load_table(table_name) # <---- If a process appends between this line ... + self.table.refresh() + self.table.append(pa_df) # <----- and this line, then Tenacity will retry. + self.filename_idx+=1 + + append_with_retry() + + append_to_table_with_retry(df) + self.buffer.clear() + + def close(self) -> None: + """Close the writer, ensuring any remaining buffered items are flushed.""" + if self.buffer: + self.flush_buffer() + + @buffer_size.setter + def buffer_size(self, value): + self._buffer_size = value diff --git a/core/amber/src/main/python/core/storage/iceberg_utils.py b/core/amber/src/main/python/core/storage/iceberg_utils.py new file mode 100644 index 00000000000..7e11d170416 --- /dev/null +++ b/core/amber/src/main/python/core/storage/iceberg_utils.py @@ -0,0 +1,92 @@ +import pyarrow as pa +import pyiceberg.table +from pyiceberg.catalog import Catalog +from pyiceberg.expressions import AlwaysTrue +from pyiceberg.manifest import DataFile +from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC +from pyiceberg.schema import Schema +from pyiceberg.table import Table +from typing import Optional, Iterator +from pyiceberg.io.pyarrow import ArrowScan + +from pyiceberg.typedef import Record + +from core.models import ArrowTableTupleProvider, Tuple + + +def create_table( + catalog: Catalog, + table_namespace: str, + table_name: str, + table_schema: Schema, + override_if_exists: bool = False +) -> Table: + """ + Creates a new Iceberg table with the specified schema and properties. + - Drops the existing table if `override_if_exists` is true and the table already exists. + - Creates an unpartitioned table with custom commit retry properties. + + :param catalog: The Iceberg catalog to manage the table. + :param table_namespace: The namespace of the table. + :param table_name: The name of the table. + :param table_schema: The schema of the table. + :param override_if_exists: Whether to drop and recreate the table if it exists. + :return: The created Iceberg table. + """ + table_properties = { + "commit.num-retries": str(5), # Replace with your configuration values + "commit.max-retry-wait-ms": str(5000), # Example retry wait time + "commit.min-retry-wait-ms": str(100) # Example retry wait time + } + + identifier = f"{table_namespace}.{table_name}" + + if catalog.table_exists(identifier) and override_if_exists: + catalog.drop_table(identifier) + + catalog.create_namespace_if_not_exists(table_namespace) + + table = catalog.create_table( + identifier=identifier, + schema=table_schema, + partition_spec=UNPARTITIONED_PARTITION_SPEC, + properties=table_properties + ) + + return table + + +def load_table_metadata( + catalog: Catalog, + table_namespace: str, + table_name: str +) -> Optional[Table]: + """ + Loads metadata for an existing Iceberg table. + - Returns the table if it exists and is successfully loaded. + - Returns None if the table does not exist or cannot be loaded. + + :param catalog: The Iceberg catalog to load the table from. + :param table_namespace: The namespace of the table. + :param table_name: The name of the table. + :return: The table if found, or None if not found. + """ + identifier = f"{table_namespace}.{table_name}" # Construct table identifier + try: + return catalog.load_table(identifier) + except Exception: + return None + + +def read_data_file_as_iterator(planfile: pyiceberg.table.FileScanTask, iceberg_table: pyiceberg.table.Table) -> Iterator[Record]: + """Reads a data file and returns an iterator over its records.""" + arrow_table: pa.Table = ArrowScan( + iceberg_table.metadata, iceberg_table.io, iceberg_table.schema(), AlwaysTrue(), True + ).to_table([planfile]) + tuple_provider = ArrowTableTupleProvider(arrow_table) + tuples = [ + Tuple({name: field_accessor for name in arrow_table.column_names}) + for field_accessor in tuple_provider + ] + for tuples in tuples: + yield tuples diff --git a/core/amber/src/main/python/core/storage/model/__init__.py b/core/amber/src/main/python/core/storage/model/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/amber/src/main/python/core/storage/model/buffered_item_writer.py b/core/amber/src/main/python/core/storage/model/buffered_item_writer.py new file mode 100644 index 00000000000..17f0b54a3ba --- /dev/null +++ b/core/amber/src/main/python/core/storage/model/buffered_item_writer.py @@ -0,0 +1,56 @@ +from abc import ABC, abstractmethod +from typing import Generic, TypeVar + +# Define a type variable +T = TypeVar('T') + + +class BufferedItemWriter(ABC, Generic[T]): + """ + BufferedItemWriter provides an interface for writing items to a buffer and performing I/O operations. + The items are buffered before being written to the underlying storage to optimize performance. + + :param T: the type of data items to be written. + """ + + @property + @abstractmethod + def buffer_size(self) -> int: + """ + The size of the buffer. + :return: the buffer size. + """ + pass + + @abstractmethod + def open(self) -> None: + """ + Open the writer, initializing any necessary resources. + This method should be called before any write operations. + """ + pass + + @abstractmethod + def close(self) -> None: + """ + Close the writer, flushing any remaining items in the buffer + to the underlying storage and releasing any held resources. + """ + pass + + @abstractmethod + def put_one(self, item: T) -> None: + """ + Put one item into the buffer. If the buffer is full, it should be flushed to the underlying storage. + :param item: the data item to be written. + """ + pass + + @abstractmethod + def remove_one(self, item: T) -> None: + """ + Remove one item from the buffer. If the item is not found in the buffer, an appropriate action should be taken, + such as throwing an exception or ignoring the request. + :param item: the data item to be removed. + """ + pass diff --git a/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py b/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py new file mode 100644 index 00000000000..a2f4074cb6d --- /dev/null +++ b/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py @@ -0,0 +1,85 @@ +from abc import ABC, abstractmethod +from typing import Generic, TypeVar, Iterator +from urllib.parse import ParseResult # Python's URI equivalent +import io # For InputStream equivalent +import os # For File equivalent + +# Define a type variable +T = TypeVar('T') + + +class ReadonlyVirtualDocument(ABC, Generic[T]): + """ + ReadonlyVirtualDocument provides an abstraction for read operations over a single resource. + This class can be implemented by resources that only need to support read-related functionality. + + :param T: the type of data that can be accessed via an index. + """ + + @abstractmethod + def get_uri(self) -> ParseResult: + """ + Get the URI of the corresponding document. + :return: the URI of the document + """ + pass + + @abstractmethod + def get_item(self, i: int) -> T: + """ + Find the ith item and return. + :param i: index starting from 0 + :return: data item of type T + """ + pass + + @abstractmethod + def get(self) -> Iterator[T]: + """ + Get an iterator that iterates over all indexed items. + :return: an iterator that returns data items of type T + """ + pass + + @abstractmethod + def get_range(self, from_index: int, until: int) -> Iterator[T]: + """ + Get an iterator of a sequence starting from index `from_index`, until index `until`. + :param from_index: the starting index (inclusive) + :param until: the ending index (exclusive) + :return: an iterator that returns data items of type T + """ + pass + + @abstractmethod + def get_after(self, offset: int) -> Iterator[T]: + """ + Get an iterator of all items after the specified index `offset`. + :param offset: the starting index (exclusive) + :return: an iterator that returns data items of type T + """ + pass + + @abstractmethod + def get_count(self) -> int: + """ + Get the count of items in the document. + :return: the count of items + """ + pass + + @abstractmethod + def as_input_stream(self) -> io.IOBase: + """ + Read the document as an input stream. + :return: the input stream + """ + pass + + @abstractmethod + def as_file(self) -> os.PathLike: + """ + Read or materialize the document as a file. + :return: the file object + """ + pass diff --git a/core/amber/src/main/python/core/storage/model/virtual_document.py b/core/amber/src/main/python/core/storage/model/virtual_document.py new file mode 100644 index 00000000000..4918466f8fe --- /dev/null +++ b/core/amber/src/main/python/core/storage/model/virtual_document.py @@ -0,0 +1,78 @@ +import io +import os +from abc import ABC, abstractmethod +from urllib.parse import ParseResult + +from core.storage.model.buffered_item_writer import BufferedItemWriter +from core.storage.model.readonly_virtual_document import ReadonlyVirtualDocument +from overrides import overrides +from typing import TypeVar, Iterator + +# Define a type variable +T = TypeVar('T') + + +class VirtualDocument(ReadonlyVirtualDocument[T], ABC): + """ + VirtualDocument provides the abstraction of performing read/write/copy/delete operations over a single resource. + Note that all methods have a default implementation. This is because one document implementation may not be able + to reasonably support all methods. + + :param T: the type of data that can use index to read and write. + """ + + @abstractmethod + def get_uri(self) -> ParseResult: + """ + Get the URI of the corresponding document. + :return: the URI of the document + """ + pass + + @overrides + def get_item(self, i: int) -> T: + raise NotImplementedError("get_item method is not implemented") + + @overrides + def get(self) -> Iterator[T]: + raise NotImplementedError("get method is not implemented") + + @overrides + def get_range(self, from_index: int, until: int) -> Iterator[T]: + raise NotImplementedError("get_range method is not implemented") + + @overrides + def get_after(self, offset: int) -> Iterator[T]: + raise NotImplementedError("get_after method is not implemented") + + @overrides + def get_count(self) -> int: + raise NotImplementedError("get_count method is not implemented") + + def set_item(self, i: int, item: T) -> None: + raise NotImplementedError("set_item method is not implemented") + + def writer(self, writer_identifier: str) -> "BufferedItemWriter[T]": + raise NotImplementedError("writer method is not implemented") + + def append(self, item: T) -> None: + raise NotImplementedError("append method is not implemented") + + def append_items(self, items: Iterator[T]) -> None: + raise NotImplementedError("append_items method is not implemented") + + def append_stream(self, input_stream: io.IOBase) -> None: + raise NotImplementedError("append_stream method is not implemented") + + def as_input_stream(self) -> io.IOBase: + raise NotImplementedError("as_input_stream method is not implemented") + + def as_file(self) -> os.PathLike: + raise NotImplementedError("as_file method is not implemented") + + @abstractmethod + def clear(self) -> None: + """ + Physically remove the current document. + """ + pass diff --git a/core/amber/src/main/python/core/storage/test_iceberg_document.py b/core/amber/src/main/python/core/storage/test_iceberg_document.py new file mode 100644 index 00000000000..5d75eca9388 --- /dev/null +++ b/core/amber/src/main/python/core/storage/test_iceberg_document.py @@ -0,0 +1,222 @@ +import datetime +import random +import uuid +from concurrent.futures import as_completed +from concurrent.futures.thread import ThreadPoolExecutor + +import pytest + +from core.models import Schema, Tuple +from core.storage.iceberg_document import IcebergDocument + + +# Function to generate random binary data +def generate_random_binary(size): + return bytearray(random.getrandbits(8) for _ in range(size)) + + +class TestIcebergDocument: + + @pytest.fixture + def amber_schema(self): + return Schema( + raw_schema={ + "col-string": "STRING", + "col-int": "INTEGER", + "col-bool": "BOOLEAN", + "col-long": "LONG", + "col-double": "DOUBLE", + "col-timestamp": "TIMESTAMP", + "col-binary": "BINARY", + } + ) + + @pytest.fixture + def iceberg_document(self, amber_schema): + operator_uuid = str(uuid.uuid4()).replace("-", "") + uri = f"wid_0_eid_0_opid_test_table_{operator_uuid}_pid_0_E_result" + arrow_schema = amber_schema.as_arrow_schema() + return IcebergDocument("operator-result", uri, arrow_schema) + + @pytest.fixture + def sample_items(self, amber_schema) -> [Tuple]: + base_tuples = [ + Tuple( + { + "col-string": "Hello World", + "col-int": 42, + "col-bool": True, + "col-long": 1123213213213, + "col-double": 214214.9969346, + "col-timestamp": datetime.datetime.now(), + "col-binary": b"hello", + }, + schema=amber_schema + ), + Tuple( + { + "col-string": "", + "col-int": -1, + "col-bool": False, + "col-long": -98765432109876, + "col-double": -0.001, + "col-timestamp": datetime.datetime.fromtimestamp(100000000), + "col-binary": bytearray([255, 0, 0, 64]), + }, + schema=amber_schema + ), + Tuple( + { + "col-string": "Special Characters: \n\t\r", + "col-int": 2147483647, + "col-bool": True, + "col-long": 9223372036854775807, + "col-double": 1.7976931348623157E308, + "col-timestamp": datetime.datetime.fromtimestamp(1234567890), + "col-binary": bytearray([1, 2, 3, 4, 5]), + }, + schema=amber_schema + ), + ] + + # Generate additional tuples + additional_tuples = [ + Tuple( + { + "col-string": None if i % 7 == 0 else f"Generated String {i}", + "col-int": None if i % 5 == 0 else i, + "col-bool": None if i % 6 == 0 else i % 2 == 0, + "col-long": None if i % 4 == 0 else i * 1000000, + "col-double": None if i % 3 == 0 else i * 0.12345, + "col-timestamp": None if i % 8 == 0 else datetime.datetime.fromtimestamp( + datetime.datetime.now().timestamp() + i), + "col-binary": None if i % 9 == 0 else generate_random_binary(10), + }, + schema=amber_schema, + ) + for i in range(1, 20001) + ] + + return base_tuples + additional_tuples + + def test_basic_read_and_write( + self, + iceberg_document, + sample_items + ): + writer = iceberg_document.writer(str(uuid.uuid4())) + writer.open() + for item in sample_items: + writer.put_one(item) + writer.close() + retrieved_items = list(iceberg_document.get()) + assert (sample_items == retrieved_items) + + def test_clear_document(self, iceberg_document, sample_items): + writer = iceberg_document.writer(str(uuid.uuid4())) + writer.open() + for item in sample_items: + writer.put_one(item) + writer.close() + assert (len(list(iceberg_document.get())) > 0) + + iceberg_document.clear() + assert (len(list(iceberg_document.get())) == 0) + + def test_handle_empty_read(self, iceberg_document): + retrieved_items = list(iceberg_document.get()) + assert (retrieved_items == []) + + def test_concurrent_writes_followed_by_read(self, iceberg_document, sample_items): + all_items = sample_items + num_writers = 5 + # Calculate the batch size and the remainder + batch_size = len(all_items) // num_writers + remainder = len(all_items) % num_writers + # Create writer's batches + item_batches = [ + all_items[i * batch_size + min(i, remainder): i * batch_size + min(i, remainder) + batch_size + ( + 1 if i < remainder else 0)] + for i in range(num_writers) + ] + + assert (len(item_batches) == num_writers, f"Expected {num_writers} batches but got {len(item_batches)}") + + # Perform concurrent writes + def write_batch(batch): + writer = iceberg_document.writer(str(uuid.uuid4())) + writer.open() + for item in batch: + writer.put_one(item) + writer.close() + + with ThreadPoolExecutor(max_workers=num_writers) as executor: + futures = [executor.submit(write_batch, batch) for batch in item_batches] + for future in as_completed(futures): + future.result() # Wait for each future to complete + + # 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.") + + def test_read_using_range(self, iceberg_document, sample_items): + writer = iceberg_document.writer(str(uuid.uuid4())) + writer.open() + for item in sample_items: + writer.put_one(item) + writer.close() + # Read all items using ranges + batch_size = 1500 + # Generate ranges + ranges = [range(i, min(i + batch_size, len(sample_items))) for i in range(0, len(sample_items), batch_size)] + + # Retrieve items using ranges + retrieved_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.") + + # Verify that the retrieved items match the original items + assert (set(retrieved_items) == set(sample_items), "All items should be retrieved correctly using ranges.") + + def test_get_after(self, iceberg_document, sample_items): + writer = iceberg_document.writer(str(uuid.uuid4())) + writer.open() + for item in sample_items: + writer.put_one(item) + writer.close() + # Test get_after for various offsets + offsets = [0, len(sample_items) // 2, len(sample_items) - 1] + for offset in offsets: + if offset < len(sample_items): + expected_items = sample_items[offset:] + else: + expected_items = [] + + retrieved_items = list(iceberg_document.get_after(offset)) + assert (retrieved_items == expected_items, ( + f"get_after({offset}) did not return the expected items. " + f"Expected: {expected_items}, Got: {retrieved_items}" + )) + + # Test get_after for an offset beyond the range + invalid_offset = len(sample_items) + retrieved_items = list(iceberg_document.get_after(invalid_offset)) + assert (not retrieved_items, ( + f"get_after({invalid_offset}) should return an empty list, but got: {retrieved_items}" + )) + + def test_get_counts(self, iceberg_document, sample_items): + writer = iceberg_document.writer(str(uuid.uuid4())) + writer.open() + for item in sample_items: + writer.put_one(item) + writer.close() + + assert(iceberg_document.get_count() == len(sample_items), "get_count should return the same number with " + "allItems") From 8d2493a280a4f7ceed5e63d9027b78a336d5ad47 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Mon, 20 Jan 2025 12:43:25 -0800 Subject: [PATCH 02/29] Refactoring of serde. --- core/amber/requirements.txt | 1 - .../python/core/storage/iceberg_document.py | 23 +++++++++++----- .../core/storage/iceberg_table_writer.py | 20 +++++--------- .../main/python/core/storage/iceberg_utils.py | 27 ++++++++++++++----- .../core/storage/test_iceberg_document.py | 9 ++++++- 5 files changed, 51 insertions(+), 29 deletions(-) diff --git a/core/amber/requirements.txt b/core/amber/requirements.txt index ca0f39adb24..f515135d3d2 100644 --- a/core/amber/requirements.txt +++ b/core/amber/requirements.txt @@ -29,5 +29,4 @@ transformers==4.44.2 tzlocal==2.1 pyiceberg==0.8.1 grpclib==0.4.7 -packaging==24.1 readerwriterlock==1.0.9 \ No newline at end of file diff --git a/core/amber/src/main/python/core/storage/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg_document.py index ad72eeb3980..84e4b962f7b 100644 --- a/core/amber/src/main/python/core/storage/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg_document.py @@ -1,17 +1,18 @@ from threading import RLock -from typing import Iterator, Optional +from typing import Iterator, Optional, Callable, Iterable, List from typing import TypeVar from urllib.parse import ParseResult, urlparse from pyiceberg.catalog import Catalog from pyiceberg.schema import Schema from pyiceberg.table import Table, FileScanTask +import pyarrow as pa from readerwriterlock import rwlock import core.models from core.models import Tuple from core.storage.iceberg_catalog_instance import IcebergCatalogInstance -from core.storage.iceberg_utils import create_table, load_table_metadata, read_data_file_as_iterator +from core.storage.iceberg_utils import create_table, load_table_metadata, read_data_file_as_arrow_table from core.storage.model.virtual_document import VirtualDocument from core.storage.iceberg_table_writer import IcebergTableWriter @@ -37,11 +38,15 @@ def __init__( table_namespace: str, table_name: str, table_schema: Schema, + serde: Callable[[Schema, Iterator[T]], pa.Table], + deserde: Callable[[Schema, pa.Table], Iterator[T]], catalog: Optional[Catalog] = None ): self.table_namespace = table_namespace self.table_name = table_name self.table_schema = table_schema + self.serde = serde + self.deserde = deserde self.lock = rwlock.RWLockFair() self.catalog = catalog or self._load_catalog() @@ -100,7 +105,8 @@ def writer(self, writer_identifier: str): catalog=self.catalog, table_namespace=self.table_namespace, table_name=self.table_name, - table_schema=self.table_schema + table_schema=self.table_schema, + serde=self.serde ) def _get_using_file_sequence_order( @@ -114,19 +120,21 @@ def _get_using_file_sequence_order( self.catalog, self.table_namespace, self.table_name, - self.table_schema + self.table_schema, + self.deserde ) return iterator class IcebergIterator(Iterator[T]): - def __init__(self, from_index, until, catalog, table_namespace, table_name, table_schema): + def __init__(self, from_index, until, catalog, table_namespace, table_name, table_schema, deserde): self.from_index = from_index self.until = until self.catalog = catalog self.table_namespace = table_namespace self.table_name = table_name self.table_schema = table_schema + self.deserde = deserde self.lock = RLock() self.num_of_skipped_records = 0 self.num_of_returned_records = 0 @@ -191,14 +199,15 @@ def __next__(self) -> T: while True: try: - record = Tuple(next(self.current_record_iterator), schema=core.models.Schema(self.table_schema)) + record = next(self.current_record_iterator) self.num_of_returned_records += 1 return record except StopIteration: # current_record_iterator is exhausted, need to go to the next file try: next_file = next(self.usable_file_iterator) - self.current_record_iterator = read_data_file_as_iterator(next_file, self.table) + arrow_table = read_data_file_as_arrow_table(next_file, self.table) + self.current_record_iterator = self.deserde(self.table_schema, arrow_table) # Skip records within the file if necessary records_to_skip_in_file = self.from_index - self.num_of_skipped_records if records_to_skip_in_file > 0: diff --git a/core/amber/src/main/python/core/storage/iceberg_table_writer.py b/core/amber/src/main/python/core/storage/iceberg_table_writer.py index ce870bc889f..a661820e8f9 100644 --- a/core/amber/src/main/python/core/storage/iceberg_table_writer.py +++ b/core/amber/src/main/python/core/storage/iceberg_table_writer.py @@ -1,11 +1,9 @@ -from pathlib import Path - -from tenacity import retry, wait_exponential, stop_after_attempt -from typing import List, TypeVar - import pyarrow as pa from pyiceberg.catalog import Catalog +from pyiceberg.schema import Schema from pyiceberg.table import Table +from tenacity import retry, wait_exponential, stop_after_attempt +from typing import List, TypeVar, Callable, Iterator, Iterable from core.storage.model.buffered_item_writer import BufferedItemWriter @@ -37,6 +35,7 @@ def __init__( table_namespace: str, table_name: str, table_schema: pa.Schema, + serde: Callable[[Schema, Iterator[T]], pa.Table], buffer_size: int = 4096 # Default buffer size TODO: move to config ): self.writer_identifier = writer_identifier @@ -44,6 +43,7 @@ def __init__( self.table_namespace = table_namespace self.table_name = table_name self.table_schema = table_schema + self.serde = serde self.buffer_size = buffer_size # Internal state @@ -76,13 +76,7 @@ def flush_buffer(self) -> None: """Flush the current buffer to a new Iceberg data file.""" if not self.buffer: return - df = pa.Table.from_pydict( - { - name: [t[name] for t in self.buffer] - for name in self.table_schema.names - }, - schema=self.table_schema, - ) + df = self.serde(self.table_schema, self.buffer) def append_to_table_with_retry(pa_df: pa.Table) -> None: """Appends a pyarrow dataframe to the table in the catalog using tenacity exponential backoff.""" @@ -96,7 +90,7 @@ def append_with_retry(): # table = catalog.load_table(table_name) # <---- If a process appends between this line ... self.table.refresh() self.table.append(pa_df) # <----- and this line, then Tenacity will retry. - self.filename_idx+=1 + self.filename_idx += 1 append_with_retry() diff --git a/core/amber/src/main/python/core/storage/iceberg_utils.py b/core/amber/src/main/python/core/storage/iceberg_utils.py index 7e11d170416..187bc93ad54 100644 --- a/core/amber/src/main/python/core/storage/iceberg_utils.py +++ b/core/amber/src/main/python/core/storage/iceberg_utils.py @@ -2,15 +2,13 @@ import pyiceberg.table from pyiceberg.catalog import Catalog from pyiceberg.expressions import AlwaysTrue -from pyiceberg.manifest import DataFile +from pyiceberg.io.pyarrow import ArrowScan from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC from pyiceberg.schema import Schema from pyiceberg.table import Table from typing import Optional, Iterator -from pyiceberg.io.pyarrow import ArrowScan - -from pyiceberg.typedef import Record +import core from core.models import ArrowTableTupleProvider, Tuple @@ -78,15 +76,30 @@ def load_table_metadata( return None -def read_data_file_as_iterator(planfile: pyiceberg.table.FileScanTask, iceberg_table: pyiceberg.table.Table) -> Iterator[Record]: +def read_data_file_as_arrow_table(planfile: pyiceberg.table.FileScanTask, iceberg_table: pyiceberg.table.Table) -> \ + pa.Table: """Reads a data file and returns an iterator over its records.""" arrow_table: pa.Table = ArrowScan( iceberg_table.metadata, iceberg_table.io, iceberg_table.schema(), AlwaysTrue(), True ).to_table([planfile]) + return arrow_table + + +def amber_tuples_to_arrow_table(iceberg_schema: Schema, tuple_list: Iterator[Tuple]) -> pa.Table: + return pa.Table.from_pydict( + { + name: [t[name] for t in tuple_list] + for name in iceberg_schema.names + }, + schema=iceberg_schema, + ) + + +def arrow_table_to_amber_tuples(iceberg_schema: Schema, arrow_table: pa.Table) -> Iterator[Tuple]: tuple_provider = ArrowTableTupleProvider(arrow_table) tuples = [ Tuple({name: field_accessor for name in arrow_table.column_names}) for field_accessor in tuple_provider ] - for tuples in tuples: - yield tuples + for t in tuples: + yield Tuple(t, schema=core.models.Schema(iceberg_schema)) diff --git a/core/amber/src/main/python/core/storage/test_iceberg_document.py b/core/amber/src/main/python/core/storage/test_iceberg_document.py index 5d75eca9388..0015ac68225 100644 --- a/core/amber/src/main/python/core/storage/test_iceberg_document.py +++ b/core/amber/src/main/python/core/storage/test_iceberg_document.py @@ -8,6 +8,7 @@ from core.models import Schema, Tuple from core.storage.iceberg_document import IcebergDocument +from core.storage.iceberg_utils import amber_tuples_to_arrow_table, arrow_table_to_amber_tuples # Function to generate random binary data @@ -36,7 +37,13 @@ def iceberg_document(self, amber_schema): operator_uuid = str(uuid.uuid4()).replace("-", "") uri = f"wid_0_eid_0_opid_test_table_{operator_uuid}_pid_0_E_result" arrow_schema = amber_schema.as_arrow_schema() - return IcebergDocument("operator-result", uri, arrow_schema) + return IcebergDocument[Tuple]( + "operator-result", + uri, + arrow_schema, + amber_tuples_to_arrow_table, + arrow_table_to_amber_tuples + ) @pytest.fixture def sample_items(self, amber_schema) -> [Tuple]: From 1f0e7dcf18a8b6460342416c63fe10ea885993eb Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Mon, 20 Jan 2025 13:34:52 -0800 Subject: [PATCH 03/29] Refactoring. --- core/amber/requirements.txt | 4 +++- .../python/core/storage/iceberg_document.py | 14 ++++++-------- .../python/core/storage/iceberg_table_writer.py | 4 ++-- .../main/python/core/storage/iceberg_utils.py | 17 +++++++++-------- 4 files changed, 20 insertions(+), 19 deletions(-) diff --git a/core/amber/requirements.txt b/core/amber/requirements.txt index f515135d3d2..f438e37dc4d 100644 --- a/core/amber/requirements.txt +++ b/core/amber/requirements.txt @@ -29,4 +29,6 @@ transformers==4.44.2 tzlocal==2.1 pyiceberg==0.8.1 grpclib==0.4.7 -readerwriterlock==1.0.9 \ No newline at end of file +readerwriterlock==1.0.9 +tenacity==9.0.0 +packaging==24.2 \ No newline at end of file diff --git a/core/amber/src/main/python/core/storage/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg_document.py index 84e4b962f7b..a881ca831c2 100644 --- a/core/amber/src/main/python/core/storage/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg_document.py @@ -1,20 +1,18 @@ from threading import RLock -from typing import Iterator, Optional, Callable, Iterable, List -from typing import TypeVar from urllib.parse import ParseResult, urlparse +import pyarrow as pa from pyiceberg.catalog import Catalog from pyiceberg.schema import Schema from pyiceberg.table import Table, FileScanTask -import pyarrow as pa from readerwriterlock import rwlock +from typing import Iterator, Optional, Callable, Iterable +from typing import TypeVar -import core.models -from core.models import Tuple from core.storage.iceberg_catalog_instance import IcebergCatalogInstance +from core.storage.iceberg_table_writer import IcebergTableWriter from core.storage.iceberg_utils import create_table, load_table_metadata, read_data_file_as_arrow_table from core.storage.model.virtual_document import VirtualDocument -from core.storage.iceberg_table_writer import IcebergTableWriter # Define a type variable T = TypeVar('T') @@ -38,8 +36,8 @@ def __init__( table_namespace: str, table_name: str, table_schema: Schema, - serde: Callable[[Schema, Iterator[T]], pa.Table], - deserde: Callable[[Schema, pa.Table], Iterator[T]], + serde: Callable[[Schema, Iterable[T]], pa.Table], + deserde: Callable[[Schema, pa.Table], Iterable[T]], catalog: Optional[Catalog] = None ): self.table_namespace = table_namespace diff --git a/core/amber/src/main/python/core/storage/iceberg_table_writer.py b/core/amber/src/main/python/core/storage/iceberg_table_writer.py index a661820e8f9..ee04b0c1b51 100644 --- a/core/amber/src/main/python/core/storage/iceberg_table_writer.py +++ b/core/amber/src/main/python/core/storage/iceberg_table_writer.py @@ -3,7 +3,7 @@ from pyiceberg.schema import Schema from pyiceberg.table import Table from tenacity import retry, wait_exponential, stop_after_attempt -from typing import List, TypeVar, Callable, Iterator, Iterable +from typing import List, TypeVar, Callable, Iterable from core.storage.model.buffered_item_writer import BufferedItemWriter @@ -35,7 +35,7 @@ def __init__( table_namespace: str, table_name: str, table_schema: pa.Schema, - serde: Callable[[Schema, Iterator[T]], pa.Table], + serde: Callable[[Schema, Iterable[T]], pa.Table], buffer_size: int = 4096 # Default buffer size TODO: move to config ): self.writer_identifier = writer_identifier diff --git a/core/amber/src/main/python/core/storage/iceberg_utils.py b/core/amber/src/main/python/core/storage/iceberg_utils.py index 187bc93ad54..3120c1467eb 100644 --- a/core/amber/src/main/python/core/storage/iceberg_utils.py +++ b/core/amber/src/main/python/core/storage/iceberg_utils.py @@ -6,7 +6,7 @@ from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC from pyiceberg.schema import Schema from pyiceberg.table import Table -from typing import Optional, Iterator +from typing import Optional, Iterable import core from core.models import ArrowTableTupleProvider, Tuple @@ -85,7 +85,7 @@ def read_data_file_as_arrow_table(planfile: pyiceberg.table.FileScanTask, iceber return arrow_table -def amber_tuples_to_arrow_table(iceberg_schema: Schema, tuple_list: Iterator[Tuple]) -> pa.Table: +def amber_tuples_to_arrow_table(iceberg_schema: Schema, tuple_list: Iterable[Tuple]) -> pa.Table: return pa.Table.from_pydict( { name: [t[name] for t in tuple_list] @@ -95,11 +95,12 @@ def amber_tuples_to_arrow_table(iceberg_schema: Schema, tuple_list: Iterator[Tup ) -def arrow_table_to_amber_tuples(iceberg_schema: Schema, arrow_table: pa.Table) -> Iterator[Tuple]: +def arrow_table_to_amber_tuples(iceberg_schema: Schema, arrow_table: pa.Table) -> Iterable[Tuple]: tuple_provider = ArrowTableTupleProvider(arrow_table) - tuples = [ - Tuple({name: field_accessor for name in arrow_table.column_names}) + return ( + Tuple( + {name: field_accessor for name in arrow_table.column_names}, + schema=core.models.Schema(iceberg_schema) + ) for field_accessor in tuple_provider - ] - for t in tuples: - yield Tuple(t, schema=core.models.Schema(iceberg_schema)) + ) From 2bd1e95f56c6e28067c5d8444aa05ad2e27b0213 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Tue, 21 Jan 2025 16:46:39 -0800 Subject: [PATCH 04/29] Add VFS URI Factory; modify retry config. --- .../core/storage/iceberg_table_writer.py | 2 +- .../core/storage/test_iceberg_document.py | 12 ++- .../python/core/storage/vfs_uri_factory.py | 84 +++++++++++++++++++ 3 files changed, 95 insertions(+), 3 deletions(-) create mode 100644 core/amber/src/main/python/core/storage/vfs_uri_factory.py diff --git a/core/amber/src/main/python/core/storage/iceberg_table_writer.py b/core/amber/src/main/python/core/storage/iceberg_table_writer.py index ee04b0c1b51..d8539c4afaf 100644 --- a/core/amber/src/main/python/core/storage/iceberg_table_writer.py +++ b/core/amber/src/main/python/core/storage/iceberg_table_writer.py @@ -82,7 +82,7 @@ def append_to_table_with_retry(pa_df: pa.Table) -> None: """Appends a pyarrow dataframe to the table in the catalog using tenacity exponential backoff.""" @retry( - wait=wait_exponential(multiplier=1, min=4, max=32), + wait=wait_exponential(multiplier=1, min=1, max=8), stop=stop_after_attempt(10), reraise=True ) diff --git a/core/amber/src/main/python/core/storage/test_iceberg_document.py b/core/amber/src/main/python/core/storage/test_iceberg_document.py index 0015ac68225..7ac790591b6 100644 --- a/core/amber/src/main/python/core/storage/test_iceberg_document.py +++ b/core/amber/src/main/python/core/storage/test_iceberg_document.py @@ -9,6 +9,8 @@ from core.models import Schema, Tuple from core.storage.iceberg_document import IcebergDocument from core.storage.iceberg_utils import amber_tuples_to_arrow_table, arrow_table_to_amber_tuples +from core.storage.vfs_uri_factory import VFSURIFactory +from proto.edu.uci.ics.amber.core import WorkflowIdentity, ExecutionIdentity, OperatorIdentity, PortIdentity # Function to generate random binary data @@ -35,7 +37,13 @@ def amber_schema(self): @pytest.fixture def iceberg_document(self, amber_schema): operator_uuid = str(uuid.uuid4()).replace("-", "") - uri = f"wid_0_eid_0_opid_test_table_{operator_uuid}_pid_0_E_result" + # uri = f"wid_0_eid_0_opid_test_table_{operator_uuid}_pid_0_E_result" + uri = VFSURIFactory.create_result_uri( + WorkflowIdentity(id=0), + ExecutionIdentity(id=0), + OperatorIdentity(id=f"test_table_{operator_uuid}"), + PortIdentity(id=0) + ) arrow_schema = amber_schema.as_arrow_schema() return IcebergDocument[Tuple]( "operator-result", @@ -136,7 +144,7 @@ def test_handle_empty_read(self, iceberg_document): def test_concurrent_writes_followed_by_read(self, iceberg_document, sample_items): all_items = sample_items - num_writers = 5 + num_writers = 10 # Calculate the batch size and the remainder batch_size = len(all_items) // num_writers remainder = len(all_items) % num_writers diff --git a/core/amber/src/main/python/core/storage/vfs_uri_factory.py b/core/amber/src/main/python/core/storage/vfs_uri_factory.py new file mode 100644 index 00000000000..043ae56ffac --- /dev/null +++ b/core/amber/src/main/python/core/storage/vfs_uri_factory.py @@ -0,0 +1,84 @@ +from enum import Enum +from typing import Optional +from urllib.parse import urlparse + +from proto.edu.uci.ics.amber.core import WorkflowIdentity, ExecutionIdentity, OperatorIdentity, PortIdentity + + +class VFSResourceType(str, Enum): + RESULT = "result" + MATERIALIZED_RESULT = "materialized_result" + + +class VFSURIFactory: + VFS_FILE_URI_SCHEME = "vfs" + + @staticmethod + def decode_uri(uri: str) -> (WorkflowIdentity, ExecutionIdentity, Optional[OperatorIdentity], PortIdentity, VFSResourceType): + parsed_uri = urlparse(uri) + + if parsed_uri.scheme != VFSURIFactory.VFS_FILE_URI_SCHEME: + raise ValueError(f"Invalid URI scheme: {parsed_uri.scheme}") + + segments = parsed_uri.path.lstrip("/").split("/") + + def extract_value(key: str) -> str: + try: + index = segments.index(key) + return segments[index + 1] + except (ValueError, IndexError): + raise ValueError(f"Missing value for key: {key} in URI: {uri}") + + workflow_id = WorkflowIdentity(int(extract_value("wid"))) + execution_id = ExecutionIdentity(int(extract_value("eid"))) + operator_id = OperatorIdentity(extract_value("opid")) + + port_identity = None + if "pid" in segments: + try: + pid_index = segments.index("pid") + port_id_str, port_type = segments[pid_index + 1].split("_") + port_id = int(port_id_str) + if port_type != "I" and port_type != "E": + raise ValueError(f"Invalid port type: {port_type} in URI: {uri}") + is_internal = port_type == "I" + port_identity = PortIdentity(port_id, is_internal) + except (ValueError, IndexError): + raise ValueError(f"Invalid port information in URI: {uri}") + + resource_type_str = segments[-1].lower() + try: + resource_type = VFSResourceType(resource_type_str) + except ValueError: + raise ValueError(f"Unknown resource type: {resource_type_str}") + + return workflow_id, execution_id, operator_id, port_identity, resource_type + + @staticmethod + def create_result_uri(workflow_id, execution_id, operator_id, port_identity) -> str: + return VFSURIFactory.create_vfs_uri( + VFSResourceType.RESULT, workflow_id, execution_id, operator_id, port_identity + ) + + @staticmethod + def create_materialized_result_uri(workflow_id, execution_id, operator_id, port_identity) -> str: + return VFSURIFactory.create_vfs_uri( + VFSResourceType.MATERIALIZED_RESULT, workflow_id, execution_id, operator_id, port_identity + ) + + @staticmethod + def create_vfs_uri(resource_type, workflow_id, execution_id, operator_id, port_identity=None) -> str: + if resource_type in (VFSResourceType.RESULT, VFSResourceType.MATERIALIZED_RESULT) and port_identity is None: + raise ValueError( + "PortIdentity must be provided when resourceType is RESULT or MATERIALIZED_RESULT." + ) + + base_uri = ( + f"{VFSURIFactory.VFS_FILE_URI_SCHEME}:///wid/{workflow_id.id}/eid/{execution_id.id}/opid/{operator_id.id}" + ) + + if port_identity: + port_type = "I" if port_identity.internal else "E" + base_uri += f"/pid/{port_identity.id}_{port_type}" + + return f"{base_uri}/{resource_type.value}" From 230a658a57fe6859a2b6b25a3d3d6c88d09ce81c Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 22 Jan 2025 16:47:58 -0800 Subject: [PATCH 05/29] Add document factory. --- .../core/models/schema/attribute_type.py | 2 + .../python/core/storage/document_factory.py | 84 +++++++++++++++++++ .../main/python/core/storage/iceberg_utils.py | 14 +--- .../core/storage/test_iceberg_document.py | 15 +--- 4 files changed, 94 insertions(+), 21 deletions(-) create mode 100644 core/amber/src/main/python/core/storage/document_factory.py diff --git a/core/amber/src/main/python/core/models/schema/attribute_type.py b/core/amber/src/main/python/core/models/schema/attribute_type.py index 3ea433a7d15..d2aac8fb55c 100644 --- a/core/amber/src/main/python/core/models/schema/attribute_type.py +++ b/core/amber/src/main/python/core/models/schema/attribute_type.py @@ -49,9 +49,11 @@ class AttributeType(Enum): lib.Type_INT32: AttributeType.INT, lib.Type_INT64: AttributeType.LONG, lib.Type_STRING: AttributeType.STRING, + lib.Type_LARGE_STRING: AttributeType.STRING, lib.Type_DOUBLE: AttributeType.DOUBLE, lib.Type_BOOL: AttributeType.BOOL, lib.Type_BINARY: AttributeType.BINARY, + lib.Type_LARGE_BINARY: AttributeType.BINARY, lib.Type_TIMESTAMP: AttributeType.TIMESTAMP, } diff --git a/core/amber/src/main/python/core/storage/document_factory.py b/core/amber/src/main/python/core/storage/document_factory.py new file mode 100644 index 00000000000..d8cc8a45985 --- /dev/null +++ b/core/amber/src/main/python/core/storage/document_factory.py @@ -0,0 +1,84 @@ +from urllib.parse import urlparse + +from typing import Optional + +from core.models import Schema, Tuple +from core.storage.iceberg_catalog_instance import IcebergCatalogInstance +from core.storage.iceberg_document import IcebergDocument +from core.storage.iceberg_utils import create_table, amber_tuples_to_arrow_table, arrow_table_to_amber_tuples, \ + load_table_metadata +from core.storage.model.virtual_document import VirtualDocument +from core.storage.vfs_uri_factory import VFSURIFactory, VFSResourceType + + +class DocumentFactory: + + ICEBERG = "iceberg" + + @staticmethod + def sanitize_uri_path(uri): + return uri.path.lstrip("/").replace("/", "_") + + @staticmethod + def create_document(uri: str, schema: Schema) -> VirtualDocument: + parsed_uri = urlparse(uri) + if parsed_uri.scheme == VFSURIFactory.VFS_FILE_URI_SCHEME: + _, _, _, _, resource_type = VFSURIFactory.decode_uri(uri) + + if resource_type in {VFSResourceType.RESULT, VFSResourceType.MATERIALIZED_RESULT}: + storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) + + iceberg_schema = Schema.as_arrow_schema(schema) + + create_table( + IcebergCatalogInstance.get_instance(), + "operator-result", + storage_key, + iceberg_schema, + override_if_exists=True + ) + + return IcebergDocument[Tuple]( + "operator-result", + storage_key, + iceberg_schema, + amber_tuples_to_arrow_table, + arrow_table_to_amber_tuples + ) + else: + raise ValueError(f"Resource type {resource_type} is not supported") + else: + raise NotImplementedError(f"Unsupported URI scheme: {parsed_uri.scheme} for creating the document") + + @staticmethod + def open_document(uri: str) -> (VirtualDocument, Optional[Schema]): + parsed_uri = urlparse(uri) + if parsed_uri.scheme == "vfs": + _, _, _, _, resource_type = VFSURIFactory.decode_uri(uri) + + if resource_type in {VFSResourceType.RESULT, VFSResourceType.MATERIALIZED_RESULT}: + storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) + + table = load_table_metadata( + IcebergCatalogInstance.get_instance(), + "operator-result", + storage_key + ) + + if table is None: + raise ValueError("No storage is found for the given URI") + + amber_schema = Schema(table.schema().as_arrow()) + + document = IcebergDocument( + "operator-result", + storage_key, + table.schema(), + amber_tuples_to_arrow_table, + arrow_table_to_amber_tuples + ) + return document, amber_schema + else: + raise ValueError(f"Resource type {resource_type} is not supported") + else: + raise NotImplementedError(f"Unsupported URI scheme: {parsed_uri.scheme} for opening the document") diff --git a/core/amber/src/main/python/core/storage/iceberg_utils.py b/core/amber/src/main/python/core/storage/iceberg_utils.py index 3120c1467eb..154a59759a9 100644 --- a/core/amber/src/main/python/core/storage/iceberg_utils.py +++ b/core/amber/src/main/python/core/storage/iceberg_utils.py @@ -31,11 +31,6 @@ def create_table( :param override_if_exists: Whether to drop and recreate the table if it exists. :return: The created Iceberg table. """ - table_properties = { - "commit.num-retries": str(5), # Replace with your configuration values - "commit.max-retry-wait-ms": str(5000), # Example retry wait time - "commit.min-retry-wait-ms": str(100) # Example retry wait time - } identifier = f"{table_namespace}.{table_name}" @@ -47,8 +42,7 @@ def create_table( table = catalog.create_table( identifier=identifier, schema=table_schema, - partition_spec=UNPARTITIONED_PARTITION_SPEC, - properties=table_properties + partition_spec=UNPARTITIONED_PARTITION_SPEC ) return table @@ -89,9 +83,9 @@ def amber_tuples_to_arrow_table(iceberg_schema: Schema, tuple_list: Iterable[Tup return pa.Table.from_pydict( { name: [t[name] for t in tuple_list] - for name in iceberg_schema.names + for name in iceberg_schema.as_arrow().names }, - schema=iceberg_schema, + schema=iceberg_schema.as_arrow(), ) @@ -100,7 +94,7 @@ def arrow_table_to_amber_tuples(iceberg_schema: Schema, arrow_table: pa.Table) - return ( Tuple( {name: field_accessor for name in arrow_table.column_names}, - schema=core.models.Schema(iceberg_schema) + schema=core.models.Schema(iceberg_schema.as_arrow()) ) for field_accessor in tuple_provider ) diff --git a/core/amber/src/main/python/core/storage/test_iceberg_document.py b/core/amber/src/main/python/core/storage/test_iceberg_document.py index 7ac790591b6..956b5614238 100644 --- a/core/amber/src/main/python/core/storage/test_iceberg_document.py +++ b/core/amber/src/main/python/core/storage/test_iceberg_document.py @@ -7,8 +7,7 @@ import pytest from core.models import Schema, Tuple -from core.storage.iceberg_document import IcebergDocument -from core.storage.iceberg_utils import amber_tuples_to_arrow_table, arrow_table_to_amber_tuples +from core.storage.document_factory import DocumentFactory from core.storage.vfs_uri_factory import VFSURIFactory from proto.edu.uci.ics.amber.core import WorkflowIdentity, ExecutionIdentity, OperatorIdentity, PortIdentity @@ -37,21 +36,15 @@ def amber_schema(self): @pytest.fixture def iceberg_document(self, amber_schema): operator_uuid = str(uuid.uuid4()).replace("-", "") - # uri = f"wid_0_eid_0_opid_test_table_{operator_uuid}_pid_0_E_result" uri = VFSURIFactory.create_result_uri( WorkflowIdentity(id=0), ExecutionIdentity(id=0), OperatorIdentity(id=f"test_table_{operator_uuid}"), PortIdentity(id=0) ) - arrow_schema = amber_schema.as_arrow_schema() - return IcebergDocument[Tuple]( - "operator-result", - uri, - arrow_schema, - amber_tuples_to_arrow_table, - arrow_table_to_amber_tuples - ) + DocumentFactory.create_document(uri, amber_schema) + document, _ = DocumentFactory.open_document(uri) + return document @pytest.fixture def sample_items(self, amber_schema) -> [Tuple]: From b3f2533056f4b2ad9ba32416c59f248e12b86630 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 22 Jan 2025 18:44:06 -0800 Subject: [PATCH 06/29] Add rest catalog. --- .../core/storage/iceberg_catalog_instance.py | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py index 4e7ec3ebc04..259ec7e3795 100644 --- a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py @@ -1,8 +1,10 @@ +from pyiceberg.catalog.rest import RestCatalog from typing import Optional from pyiceberg.catalog import Catalog from pyiceberg.catalog.sql import SqlCatalog +# replace with actual path warehouse_path = "/Users/xzliu/IdeaProjects/texera/core/amber/user-resources/workflow-results" @@ -23,12 +25,19 @@ def get_instance(cls): :return: the Iceberg catalog instance. """ if cls._instance is None: - cls._instance = SqlCatalog( + # cls._instance = SqlCatalog( + # "texera_iceberg", + # **{ + # "uri": "postgresql+psycopg2://iceberg_py_test:test@localhost/test_iceberg_catalog_jan16", + # "warehouse": f"file://{warehouse_path}", + # "init_catalog_tables": "true" + # } + # ) + cls._instance = RestCatalog( "texera_iceberg", **{ - "uri": "postgresql+psycopg2://iceberg_py_test:test@localhost/test_iceberg_catalog_jan16", - "warehouse": f"file://{warehouse_path}", - "init_catalog_tables": "true" + "uri": "localhost:8181", + "warehouse": f"file://{warehouse_path}" } ) return cls._instance @@ -41,8 +50,3 @@ def replace_instance(cls, catalog: Catalog): :param catalog: the new Iceberg catalog instance to replace the current one. """ cls._instance = catalog - - -if __name__ == '__main__': - ins = IcebergCatalogInstance.get_instance() - print(ins) From b1c31afe18fd5c540546fcfaf433348abee22dff Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 22 Jan 2025 18:46:48 -0800 Subject: [PATCH 07/29] Add rest catalog. --- .../src/main/python/core/storage/iceberg_catalog_instance.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py index 259ec7e3795..b4b1015e750 100644 --- a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py @@ -36,7 +36,7 @@ def get_instance(cls): cls._instance = RestCatalog( "texera_iceberg", **{ - "uri": "localhost:8181", + "uri": "http://localhost:8181", "warehouse": f"file://{warehouse_path}" } ) From 07f3bb2d8c5f28ea8b772b1568d0c63713d08003 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Thu, 23 Jan 2025 00:02:36 -0800 Subject: [PATCH 08/29] Temporary fix. --- core/amber/requirements.txt | 2 +- .../main/python/core/storage/iceberg_document.py | 6 ++---- .../src/main/python/core/storage/iceberg_utils.py | 13 +++++++------ 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/core/amber/requirements.txt b/core/amber/requirements.txt index f438e37dc4d..511434e4933 100644 --- a/core/amber/requirements.txt +++ b/core/amber/requirements.txt @@ -27,7 +27,7 @@ cached_property==1.5.2 psutil==5.9.0 transformers==4.44.2 tzlocal==2.1 -pyiceberg==0.8.1 +pyiceberg==0.7.1 grpclib==0.4.7 readerwriterlock==1.0.9 tenacity==9.0.0 diff --git a/core/amber/src/main/python/core/storage/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg_document.py index a881ca831c2..51c539d1f5e 100644 --- a/core/amber/src/main/python/core/storage/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg_document.py @@ -50,12 +50,10 @@ def __init__( self.catalog = catalog or self._load_catalog() # Create or override the table during initialization - create_table( + load_table_metadata( self.catalog, self.table_namespace, - self.table_name, - self.table_schema, - override_if_exists=True + self.table_name ) def _load_catalog(self) -> Catalog: diff --git a/core/amber/src/main/python/core/storage/iceberg_utils.py b/core/amber/src/main/python/core/storage/iceberg_utils.py index 154a59759a9..6905732f10a 100644 --- a/core/amber/src/main/python/core/storage/iceberg_utils.py +++ b/core/amber/src/main/python/core/storage/iceberg_utils.py @@ -2,7 +2,7 @@ import pyiceberg.table from pyiceberg.catalog import Catalog from pyiceberg.expressions import AlwaysTrue -from pyiceberg.io.pyarrow import ArrowScan +from pyiceberg.io.pyarrow import project_table from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC from pyiceberg.schema import Schema from pyiceberg.table import Table @@ -34,11 +34,11 @@ def create_table( identifier = f"{table_namespace}.{table_name}" + # catalog.create_namespace_if_not_exists(table_namespace) + if catalog.table_exists(identifier) and override_if_exists: catalog.drop_table(identifier) - catalog.create_namespace_if_not_exists(table_namespace) - table = catalog.create_table( identifier=identifier, schema=table_schema, @@ -73,9 +73,10 @@ def load_table_metadata( def read_data_file_as_arrow_table(planfile: pyiceberg.table.FileScanTask, iceberg_table: pyiceberg.table.Table) -> \ pa.Table: """Reads a data file and returns an iterator over its records.""" - arrow_table: pa.Table = ArrowScan( - iceberg_table.metadata, iceberg_table.io, iceberg_table.schema(), AlwaysTrue(), True - ).to_table([planfile]) + # arrow_table: pa.Table = ArrowScan( + # iceberg_table.metadata, iceberg_table.io, iceberg_table.schema(), AlwaysTrue(), True + # ).to_table([planfile]) + arrow_table: pa.Table = project_table([planfile], iceberg_table.metadata, iceberg_table.io, AlwaysTrue(), iceberg_table.schema(), True) return arrow_table From 8bc6050958568c6b77902dd46a1d3d30cc8557f7 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Thu, 23 Jan 2025 23:03:24 -0800 Subject: [PATCH 09/29] fix port uri. --- core/amber/src/main/python/core/storage/vfs_uri_factory.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/amber/src/main/python/core/storage/vfs_uri_factory.py b/core/amber/src/main/python/core/storage/vfs_uri_factory.py index 043ae56ffac..f6b9fe2e40e 100644 --- a/core/amber/src/main/python/core/storage/vfs_uri_factory.py +++ b/core/amber/src/main/python/core/storage/vfs_uri_factory.py @@ -77,7 +77,7 @@ def create_vfs_uri(resource_type, workflow_id, execution_id, operator_id, port_i f"{VFSURIFactory.VFS_FILE_URI_SCHEME}:///wid/{workflow_id.id}/eid/{execution_id.id}/opid/{operator_id.id}" ) - if port_identity: + if port_identity is not None: port_type = "I" if port_identity.internal else "E" base_uri += f"/pid/{port_identity.id}_{port_type}" From 16ed509ff991375bf8dbbb3def9df09787986936 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Fri, 24 Jan 2025 17:33:53 -0800 Subject: [PATCH 10/29] fix read from java; use sql catalog. --- .../core/storage/iceberg_catalog_instance.py | 17 +++++++++-------- .../python/core/storage/iceberg_document.py | 11 ++++++----- 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py index b4b1015e750..43477ffcd69 100644 --- a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py @@ -25,7 +25,15 @@ def get_instance(cls): :return: the Iceberg catalog instance. """ if cls._instance is None: - # cls._instance = SqlCatalog( + cls._instance = SqlCatalog( + "texera_iceberg", + **{ + "uri": "postgresql+psycopg2://iceberg_py_test:test@localhost/test_iceberg_catalog_jan16", + "warehouse": f"file://{warehouse_path}", + "init_catalog_tables": "true" + } + ) + # cls._instance = RestCatalog( # "texera_iceberg", # **{ # "uri": "postgresql+psycopg2://iceberg_py_test:test@localhost/test_iceberg_catalog_jan16", @@ -33,13 +41,6 @@ def get_instance(cls): # "init_catalog_tables": "true" # } # ) - cls._instance = RestCatalog( - "texera_iceberg", - **{ - "uri": "http://localhost:8181", - "warehouse": f"file://{warehouse_path}" - } - ) return cls._instance @classmethod diff --git a/core/amber/src/main/python/core/storage/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg_document.py index 51c539d1f5e..a6d49e59af0 100644 --- a/core/amber/src/main/python/core/storage/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg_document.py @@ -1,4 +1,6 @@ from threading import RLock +from typing import Iterator, Optional, Callable, Iterable +from typing import TypeVar from urllib.parse import ParseResult, urlparse import pyarrow as pa @@ -6,12 +8,10 @@ from pyiceberg.schema import Schema from pyiceberg.table import Table, FileScanTask from readerwriterlock import rwlock -from typing import Iterator, Optional, Callable, Iterable -from typing import TypeVar from core.storage.iceberg_catalog_instance import IcebergCatalogInstance from core.storage.iceberg_table_writer import IcebergTableWriter -from core.storage.iceberg_utils import create_table, load_table_metadata, read_data_file_as_arrow_table +from core.storage.iceberg_utils import load_table_metadata, read_data_file_as_arrow_table from core.storage.model.virtual_document import VirtualDocument # Define a type variable @@ -157,14 +157,14 @@ def _seek_to_usable_file(self) -> Iterator[FileScanTask]: try: self.table.refresh() # Retrieve the entries from the table - entries = self.table.inspect.entries() + entries = self.table.inspect.metadata_log_entries() # Convert to a Pandas DataFrame for easy manipulation entries_df = entries.to_pandas() # Sort by file_sequence_number file_sequence_map = { - row["data_file"]["file_path"]: row["file_sequence_number"] + row["file"]: row["latest_sequence_number"] for _, row in entries_df.iterrows() } @@ -182,6 +182,7 @@ def _seek_to_usable_file(self) -> Iterator[FileScanTask]: continue yield task except Exception: + print("Could not read iceberg table:\n", Exception) return iter([]) else: return iter([]) From 0e77b85b8c7d649eb78efa47754f5da350ef2e62 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Mon, 27 Jan 2025 10:37:07 -0800 Subject: [PATCH 11/29] Fix file_sequence_number retrieval logic. --- .../python/core/storage/iceberg_document.py | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/core/amber/src/main/python/core/storage/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg_document.py index a6d49e59af0..349f7d39a6c 100644 --- a/core/amber/src/main/python/core/storage/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg_document.py @@ -156,23 +156,22 @@ def _seek_to_usable_file(self) -> Iterator[FileScanTask]: if self.table: try: self.table.refresh() - # Retrieve the entries from the table - entries = self.table.inspect.metadata_log_entries() - - # Convert to a Pandas DataFrame for easy manipulation - entries_df = entries.to_pandas() - - # Sort by file_sequence_number - file_sequence_map = { - row["file"]: row["latest_sequence_number"] - for _, row in entries_df.iterrows() - } + # self.table.inspect.entries() does not work with java files, need to implement the logic + # to find file_sequence_number for each data file ourselves + file_sequence_map = {} + current_snapshot = self.table.current_snapshot() + if current_snapshot is None: + return iter([]) + for manifest in current_snapshot.manifests(self.table.io): + for entry in manifest.fetch_manifest_entry(io=self.table.io): + file_sequence_map[entry.data_file.file_path] = entry.sequence_number # Retrieve and sort the file scan tasks by file sequence number file_scan_tasks = list(self.table.scan().plan_files()) + # Sort files by their sequence number. Files without a sequence number will be read last. sorted_file_scan_tasks = sorted( file_scan_tasks, - key=lambda task: file_sequence_map.get(task.file.file_path, float('inf')) # Use float('inf') for missing files + key=lambda t: file_sequence_map.get(t.file.file_path, float('inf')) ) # Skip records in files before the `from_index` for task in sorted_file_scan_tasks: @@ -182,8 +181,8 @@ def _seek_to_usable_file(self) -> Iterator[FileScanTask]: continue yield task except Exception: - print("Could not read iceberg table:\n", Exception) - return iter([]) + print("Could not read iceberg table:\n") + raise Exception else: return iter([]) From edbb20e491e77b165fccf42b7760e56ae55c3da2 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Tue, 28 Jan 2025 10:46:17 -0800 Subject: [PATCH 12/29] Improve retry. --- .../src/main/python/core/storage/iceberg_table_writer.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/amber/src/main/python/core/storage/iceberg_table_writer.py b/core/amber/src/main/python/core/storage/iceberg_table_writer.py index d8539c4afaf..89a6d424702 100644 --- a/core/amber/src/main/python/core/storage/iceberg_table_writer.py +++ b/core/amber/src/main/python/core/storage/iceberg_table_writer.py @@ -2,7 +2,7 @@ from pyiceberg.catalog import Catalog from pyiceberg.schema import Schema from pyiceberg.table import Table -from tenacity import retry, wait_exponential, stop_after_attempt +from tenacity import retry, stop_after_attempt, wait_random from typing import List, TypeVar, Callable, Iterable from core.storage.model.buffered_item_writer import BufferedItemWriter @@ -82,12 +82,11 @@ def append_to_table_with_retry(pa_df: pa.Table) -> None: """Appends a pyarrow dataframe to the table in the catalog using tenacity exponential backoff.""" @retry( - wait=wait_exponential(multiplier=1, min=1, max=8), + wait=wait_random(0.001, 0.1), stop=stop_after_attempt(10), reraise=True ) def append_with_retry(): - # table = catalog.load_table(table_name) # <---- If a process appends between this line ... self.table.refresh() self.table.append(pa_df) # <----- and this line, then Tenacity will retry. self.filename_idx += 1 From 00500ecd23b93889c283327989de151117ecb951 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Tue, 28 Jan 2025 22:51:35 -0800 Subject: [PATCH 13/29] env setup. --- .github/workflows/github-action-build.yml | 2 ++ core/amber/requirements.txt | 6 ++---- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/github-action-build.yml b/.github/workflows/github-action-build.yml index 4c07bd728ca..57832a00177 100644 --- a/.github/workflows/github-action-build.yml +++ b/.github/workflows/github-action-build.yml @@ -111,6 +111,8 @@ jobs: if [ -f core/amber/requirements.txt ]; then pip install -r core/amber/requirements.txt; fi if [ -f core/amber/r-requirements.txt ]; then pip install -r core/amber/r-requirements.txt; fi if [ -f core/amber/operator-requirements.txt ]; then pip install -r core/amber/operator-requirements.txt; fi + - name: Install PostgreSQL + run: sudo apt-get update && sudo apt-get install -y postgresql - name: Lint with flake8 and black run: | cd core/amber/src/main/python && flake8 && black . --check diff --git a/core/amber/requirements.txt b/core/amber/requirements.txt index 511434e4933..5f54a186b46 100644 --- a/core/amber/requirements.txt +++ b/core/amber/requirements.txt @@ -27,8 +27,6 @@ cached_property==1.5.2 psutil==5.9.0 transformers==4.44.2 tzlocal==2.1 -pyiceberg==0.7.1 -grpclib==0.4.7 +pyiceberg==0.8.1 readerwriterlock==1.0.9 -tenacity==9.0.0 -packaging==24.2 \ No newline at end of file +tenacity==8.5.0 \ No newline at end of file From 41759f77c32f369e8ecf492a23040b565d545510 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Tue, 28 Jan 2025 23:34:36 -0800 Subject: [PATCH 14/29] env setup. --- .github/workflows/github-action-build.yml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/.github/workflows/github-action-build.yml b/.github/workflows/github-action-build.yml index 57832a00177..4edb2d627a0 100644 --- a/.github/workflows/github-action-build.yml +++ b/.github/workflows/github-action-build.yml @@ -113,6 +113,13 @@ jobs: if [ -f core/amber/operator-requirements.txt ]; then pip install -r core/amber/operator-requirements.txt; fi - name: Install PostgreSQL run: sudo apt-get update && sudo apt-get install -y postgresql + - name: Start PostgreSQL Service + run: sudo systemctl start postgresql + - name: Create Database and User + run: | + sudo -u postgres psql -c "CREATE DATABASE testdb;" + sudo -u postgres psql -c "CREATE USER testuser WITH ENCRYPTED PASSWORD 'testpassword';" + sudo -u postgres psql -c "GRANT ALL PRIVILEGES ON DATABASE testdb TO testuser;" - name: Lint with flake8 and black run: | cd core/amber/src/main/python && flake8 && black . --check From d1db92a12cf461757be25e99114fc17a8983d6c6 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 00:03:25 -0800 Subject: [PATCH 15/29] fix lint and fmt; fix path. --- core/amber/src/main/python/.flake8 | 2 +- .../python/core/storage/document_factory.py | 34 +++++-- .../core/storage/iceberg_catalog_instance.py | 20 ++-- .../python/core/storage/iceberg_document.py | 85 +++++++++++------ .../core/storage/iceberg_table_writer.py | 39 ++++---- .../main/python/core/storage/iceberg_utils.py | 46 ++++++---- .../storage/model/buffered_item_writer.py | 14 ++- .../model/readonly_virtual_document.py | 11 ++- .../core/storage/model/virtual_document.py | 8 +- .../core/storage/test_iceberg_document.py | 91 ++++++++++++------- .../python/core/storage/vfs_uri_factory.py | 49 ++++++++-- 11 files changed, 259 insertions(+), 140 deletions(-) diff --git a/core/amber/src/main/python/.flake8 b/core/amber/src/main/python/.flake8 index 071776ddc00..90b57b7591b 100644 --- a/core/amber/src/main/python/.flake8 +++ b/core/amber/src/main/python/.flake8 @@ -5,5 +5,5 @@ extend-ignore = E203, exclude = proto, -max-complexity = 10 +max-complexity = 11 max-line-length = 88 diff --git a/core/amber/src/main/python/core/storage/document_factory.py b/core/amber/src/main/python/core/storage/document_factory.py index d8cc8a45985..f84f9eea2e8 100644 --- a/core/amber/src/main/python/core/storage/document_factory.py +++ b/core/amber/src/main/python/core/storage/document_factory.py @@ -5,8 +5,12 @@ from core.models import Schema, Tuple from core.storage.iceberg_catalog_instance import IcebergCatalogInstance from core.storage.iceberg_document import IcebergDocument -from core.storage.iceberg_utils import create_table, amber_tuples_to_arrow_table, arrow_table_to_amber_tuples, \ - load_table_metadata +from core.storage.iceberg_utils import ( + create_table, + amber_tuples_to_arrow_table, + arrow_table_to_amber_tuples, + load_table_metadata, +) from core.storage.model.virtual_document import VirtualDocument from core.storage.vfs_uri_factory import VFSURIFactory, VFSResourceType @@ -25,7 +29,10 @@ def create_document(uri: str, schema: Schema) -> VirtualDocument: if parsed_uri.scheme == VFSURIFactory.VFS_FILE_URI_SCHEME: _, _, _, _, resource_type = VFSURIFactory.decode_uri(uri) - if resource_type in {VFSResourceType.RESULT, VFSResourceType.MATERIALIZED_RESULT}: + if resource_type in { + VFSResourceType.RESULT, + VFSResourceType.MATERIALIZED_RESULT, + }: storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) iceberg_schema = Schema.as_arrow_schema(schema) @@ -35,7 +42,7 @@ def create_document(uri: str, schema: Schema) -> VirtualDocument: "operator-result", storage_key, iceberg_schema, - override_if_exists=True + override_if_exists=True, ) return IcebergDocument[Tuple]( @@ -43,12 +50,14 @@ def create_document(uri: str, schema: Schema) -> VirtualDocument: storage_key, iceberg_schema, amber_tuples_to_arrow_table, - arrow_table_to_amber_tuples + arrow_table_to_amber_tuples, ) else: raise ValueError(f"Resource type {resource_type} is not supported") else: - raise NotImplementedError(f"Unsupported URI scheme: {parsed_uri.scheme} for creating the document") + raise NotImplementedError( + f"Unsupported URI scheme: {parsed_uri.scheme} for creating the document" + ) @staticmethod def open_document(uri: str) -> (VirtualDocument, Optional[Schema]): @@ -56,13 +65,16 @@ def open_document(uri: str) -> (VirtualDocument, Optional[Schema]): if parsed_uri.scheme == "vfs": _, _, _, _, resource_type = VFSURIFactory.decode_uri(uri) - if resource_type in {VFSResourceType.RESULT, VFSResourceType.MATERIALIZED_RESULT}: + if resource_type in { + VFSResourceType.RESULT, + VFSResourceType.MATERIALIZED_RESULT, + }: storage_key = DocumentFactory.sanitize_uri_path(parsed_uri) table = load_table_metadata( IcebergCatalogInstance.get_instance(), "operator-result", - storage_key + storage_key, ) if table is None: @@ -75,10 +87,12 @@ def open_document(uri: str) -> (VirtualDocument, Optional[Schema]): storage_key, table.schema(), amber_tuples_to_arrow_table, - arrow_table_to_amber_tuples + arrow_table_to_amber_tuples, ) return document, amber_schema else: raise ValueError(f"Resource type {resource_type} is not supported") else: - raise NotImplementedError(f"Unsupported URI scheme: {parsed_uri.scheme} for opening the document") + raise NotImplementedError( + f"Unsupported URI scheme: {parsed_uri.scheme} for opening the document" + ) diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py index 43477ffcd69..540a3fbdfeb 100644 --- a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py @@ -1,11 +1,9 @@ -from pyiceberg.catalog.rest import RestCatalog -from typing import Optional - from pyiceberg.catalog import Catalog from pyiceberg.catalog.sql import SqlCatalog +from typing import Optional # replace with actual path -warehouse_path = "/Users/xzliu/IdeaProjects/texera/core/amber/user-resources/workflow-results" +warehouse_path = "../../../../../core/amber/user-resources/workflow-results" class IcebergCatalogInstance: @@ -15,28 +13,32 @@ class IcebergCatalogInstance: - Lazily initializes the catalog on first access. - Supports replacing the catalog instance for testing or reconfiguration. """ + _instance: Optional[Catalog] = None @classmethod def get_instance(cls): """ Retrieves the singleton Iceberg catalog instance. - - If the catalog is not initialized, it is lazily created using the configured properties. + - If the catalog is not initialized, it is lazily created using the configured + properties. :return: the Iceberg catalog instance. """ if cls._instance is None: cls._instance = SqlCatalog( "texera_iceberg", **{ - "uri": "postgresql+psycopg2://iceberg_py_test:test@localhost/test_iceberg_catalog_jan16", + "uri": "postgresql+psycopg2://testdb:testuser@localhost/" + "testpassword", "warehouse": f"file://{warehouse_path}", - "init_catalog_tables": "true" - } + "init_catalog_tables": "true", + }, ) # cls._instance = RestCatalog( # "texera_iceberg", # **{ - # "uri": "postgresql+psycopg2://iceberg_py_test:test@localhost/test_iceberg_catalog_jan16", + # "uri": "postgresql+psycopg2://iceberg_py_test:test@localhost/test_ + # iceberg_catalog_jan16", # "warehouse": f"file://{warehouse_path}", # "init_catalog_tables": "true" # } diff --git a/core/amber/src/main/python/core/storage/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg_document.py index 349f7d39a6c..5d81cf0be39 100644 --- a/core/amber/src/main/python/core/storage/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg_document.py @@ -11,17 +11,21 @@ from core.storage.iceberg_catalog_instance import IcebergCatalogInstance from core.storage.iceberg_table_writer import IcebergTableWriter -from core.storage.iceberg_utils import load_table_metadata, read_data_file_as_arrow_table +from core.storage.iceberg_utils import ( + load_table_metadata, + read_data_file_as_arrow_table, +) from core.storage.model.virtual_document import VirtualDocument # Define a type variable -T = TypeVar('T') +T = TypeVar("T") class IcebergDocument(VirtualDocument[T]): """ IcebergDocument is used to read and write a set of T as an Iceberg table. - It provides iterator-based read methods and supports multiple writers to write to the same table. + It provides iterator-based read methods and supports multiple writers to write to + the same table. - On construction, the table will be created if it does not exist. - If the table exists, it will be overridden. @@ -32,13 +36,13 @@ class IcebergDocument(VirtualDocument[T]): """ def __init__( - self, - table_namespace: str, - table_name: str, - table_schema: Schema, - serde: Callable[[Schema, Iterable[T]], pa.Table], - deserde: Callable[[Schema, pa.Table], Iterable[T]], - catalog: Optional[Catalog] = None + self, + table_namespace: str, + table_name: str, + table_schema: Schema, + serde: Callable[[Schema, Iterable[T]], pa.Table], + deserde: Callable[[Schema, pa.Table], Iterable[T]], + catalog: Optional[Catalog] = None, ): self.table_namespace = table_namespace self.table_name = table_name @@ -50,11 +54,7 @@ def __init__( self.catalog = catalog or self._load_catalog() # Create or override the table during initialization - load_table_metadata( - self.catalog, - self.table_namespace, - self.table_name - ) + load_table_metadata(self.catalog, self.table_namespace, self.table_name) def _load_catalog(self) -> Catalog: """Load the Iceberg catalog.""" @@ -65,7 +65,9 @@ def get_uri(self) -> ParseResult: """Returns the URI of the table location.""" table = load_table_metadata(self.catalog, self.table_namespace, self.table_name) if not table: - raise Exception(f"table {self.table_namespace}.{self.table_name} doesn't exist.") + raise Exception( + f"table {self.table_namespace}.{self.table_name} doesn't exist." + ) return urlparse(table.location()) def clear(self): @@ -102,11 +104,11 @@ def writer(self, writer_identifier: str): table_namespace=self.table_namespace, table_name=self.table_name, table_schema=self.table_schema, - serde=self.serde + serde=self.serde, ) def _get_using_file_sequence_order( - self, from_index: int, until: Optional[int] + self, from_index: int, until: Optional[int] ) -> Iterator[T]: """Utility to get records within a specified range.""" with self.lock.gen_rlock(): @@ -117,13 +119,22 @@ def _get_using_file_sequence_order( self.table_namespace, self.table_name, self.table_schema, - self.deserde + self.deserde, ) return iterator class IcebergIterator(Iterator[T]): - def __init__(self, from_index, until, catalog, table_namespace, table_name, table_schema, deserde): + def __init__( + self, + from_index, + until, + catalog, + table_namespace, + table_name, + table_schema, + deserde, + ): self.from_index = from_index self.until = until self.catalog = catalog @@ -134,7 +145,9 @@ def __init__(self, from_index, until, catalog, table_namespace, table_name, tabl self.lock = RLock() self.num_of_skipped_records = 0 self.num_of_returned_records = 0 - self.total_records_to_return = self.until - self.from_index if until else float("inf") + self.total_records_to_return = ( + self.until - self.from_index if until else float("inf") + ) self.current_record_iterator = iter([]) self.table = self._load_table_metadata() self.usable_file_iterator = self._seek_to_usable_file() @@ -156,7 +169,8 @@ def _seek_to_usable_file(self) -> Iterator[FileScanTask]: if self.table: try: self.table.refresh() - # self.table.inspect.entries() does not work with java files, need to implement the logic + # self.table.inspect.entries() does not work with java files, need + # to implement the logic # to find file_sequence_number for each data file ourselves file_sequence_map = {} current_snapshot = self.table.current_snapshot() @@ -164,19 +178,27 @@ def _seek_to_usable_file(self) -> Iterator[FileScanTask]: return iter([]) for manifest in current_snapshot.manifests(self.table.io): for entry in manifest.fetch_manifest_entry(io=self.table.io): - file_sequence_map[entry.data_file.file_path] = entry.sequence_number + file_sequence_map[entry.data_file.file_path] = ( + entry.sequence_number + ) # Retrieve and sort the file scan tasks by file sequence number file_scan_tasks = list(self.table.scan().plan_files()) - # Sort files by their sequence number. Files without a sequence number will be read last. + # Sort files by their sequence number. Files without a sequence + # number will be read last. sorted_file_scan_tasks = sorted( file_scan_tasks, - key=lambda t: file_sequence_map.get(t.file.file_path, float('inf')) + key=lambda t: file_sequence_map.get( + t.file.file_path, float("inf") + ), ) # Skip records in files before the `from_index` for task in sorted_file_scan_tasks: record_count = task.file.record_count - if self.num_of_skipped_records + record_count <= self.from_index: + if ( + self.num_of_skipped_records + record_count + <= self.from_index + ): self.num_of_skipped_records += record_count continue yield task @@ -203,13 +225,16 @@ def __next__(self) -> T: try: next_file = next(self.usable_file_iterator) arrow_table = read_data_file_as_arrow_table(next_file, self.table) - self.current_record_iterator = self.deserde(self.table_schema, arrow_table) + self.current_record_iterator = self.deserde( + self.table_schema, arrow_table + ) # Skip records within the file if necessary - records_to_skip_in_file = self.from_index - self.num_of_skipped_records + records_to_skip_in_file = ( + self.from_index - self.num_of_skipped_records + ) if records_to_skip_in_file > 0: self.current_record_iterator = self._skip_records( - self.current_record_iterator, - records_to_skip_in_file + self.current_record_iterator, records_to_skip_in_file ) self.num_of_skipped_records += records_to_skip_in_file except StopIteration: diff --git a/core/amber/src/main/python/core/storage/iceberg_table_writer.py b/core/amber/src/main/python/core/storage/iceberg_table_writer.py index 89a6d424702..a9da56be11e 100644 --- a/core/amber/src/main/python/core/storage/iceberg_table_writer.py +++ b/core/amber/src/main/python/core/storage/iceberg_table_writer.py @@ -8,7 +8,7 @@ from core.storage.model.buffered_item_writer import BufferedItemWriter # Define a type variable for the data type T -T = TypeVar('T') +T = TypeVar("T") class IcebergTableWriter(BufferedItemWriter[T]): @@ -16,9 +16,11 @@ class IcebergTableWriter(BufferedItemWriter[T]): IcebergTableWriter writes data to the given Iceberg table in an append-only way. - Each time the buffer is flushed, a new data file is created with a unique name. - The `writer_identifier` is used to prefix the created files. - - Iceberg data files are immutable once created. So each flush will create a distinct file. + - Iceberg data files are immutable once created. So each flush will create a + distinct file. - **Thread Safety**: This writer is NOT thread-safe, so only one thread should call this writer. + **Thread Safety**: This writer is NOT thread-safe, so only one thread should call + this writer. :param writer_identifier: A unique identifier used to prefix the created files. :param catalog: The Iceberg catalog to manage table metadata. @@ -29,14 +31,14 @@ class IcebergTableWriter(BufferedItemWriter[T]): """ def __init__( - self, - writer_identifier: str, - catalog: Catalog, - table_namespace: str, - table_name: str, - table_schema: pa.Schema, - serde: Callable[[Schema, Iterable[T]], pa.Table], - buffer_size: int = 4096 # Default buffer size TODO: move to config + self, + writer_identifier: str, + catalog: Catalog, + table_namespace: str, + table_name: str, + table_schema: pa.Schema, + serde: Callable[[Schema, Iterable[T]], pa.Table], + buffer_size: int = 4096, # Default buffer size TODO: move to config ): self.writer_identifier = writer_identifier self.catalog = catalog @@ -52,7 +54,9 @@ def __init__( self.record_id = 0 # Load the Iceberg table - self.table: Table = self.catalog.load_table(f"{self.table_namespace}.{self.table_name}") + self.table: Table = self.catalog.load_table( + f"{self.table_namespace}.{self.table_name}" + ) @property def buffer_size(self) -> int: @@ -79,16 +83,17 @@ def flush_buffer(self) -> None: df = self.serde(self.table_schema, self.buffer) def append_to_table_with_retry(pa_df: pa.Table) -> None: - """Appends a pyarrow dataframe to the table in the catalog using tenacity exponential backoff.""" + """Appends a pyarrow dataframe to the table in the catalog using tenacity + exponential backoff.""" @retry( - wait=wait_random(0.001, 0.1), - stop=stop_after_attempt(10), - reraise=True + wait=wait_random(0.001, 0.1), stop=stop_after_attempt(10), reraise=True ) def append_with_retry(): self.table.refresh() - self.table.append(pa_df) # <----- and this line, then Tenacity will retry. + self.table.append( + pa_df + ) # <----- and this line, then Tenacity will retry. self.filename_idx += 1 append_with_retry() diff --git a/core/amber/src/main/python/core/storage/iceberg_utils.py b/core/amber/src/main/python/core/storage/iceberg_utils.py index 6905732f10a..4a6593187c5 100644 --- a/core/amber/src/main/python/core/storage/iceberg_utils.py +++ b/core/amber/src/main/python/core/storage/iceberg_utils.py @@ -13,15 +13,16 @@ def create_table( - catalog: Catalog, - table_namespace: str, - table_name: str, - table_schema: Schema, - override_if_exists: bool = False + catalog: Catalog, + table_namespace: str, + table_name: str, + table_schema: Schema, + override_if_exists: bool = False, ) -> Table: """ Creates a new Iceberg table with the specified schema and properties. - - Drops the existing table if `override_if_exists` is true and the table already exists. + - Drops the existing table if `override_if_exists` is true and the table already + exists. - Creates an unpartitioned table with custom commit retry properties. :param catalog: The Iceberg catalog to manage the table. @@ -42,16 +43,14 @@ def create_table( table = catalog.create_table( identifier=identifier, schema=table_schema, - partition_spec=UNPARTITIONED_PARTITION_SPEC + partition_spec=UNPARTITIONED_PARTITION_SPEC, ) return table def load_table_metadata( - catalog: Catalog, - table_namespace: str, - table_name: str + catalog: Catalog, table_namespace: str, table_name: str ) -> Optional[Table]: """ Loads metadata for an existing Iceberg table. @@ -70,17 +69,28 @@ def load_table_metadata( return None -def read_data_file_as_arrow_table(planfile: pyiceberg.table.FileScanTask, iceberg_table: pyiceberg.table.Table) -> \ - pa.Table: +def read_data_file_as_arrow_table( + planfile: pyiceberg.table.FileScanTask, iceberg_table: pyiceberg.table.Table +) -> pa.Table: """Reads a data file and returns an iterator over its records.""" # arrow_table: pa.Table = ArrowScan( - # iceberg_table.metadata, iceberg_table.io, iceberg_table.schema(), AlwaysTrue(), True + # iceberg_table.metadata, iceberg_table.io, iceberg_table.schema(), + # AlwaysTrue(), True # ).to_table([planfile]) - arrow_table: pa.Table = project_table([planfile], iceberg_table.metadata, iceberg_table.io, AlwaysTrue(), iceberg_table.schema(), True) + arrow_table: pa.Table = project_table( + [planfile], + iceberg_table.metadata, + iceberg_table.io, + AlwaysTrue(), + iceberg_table.schema(), + True, + ) return arrow_table -def amber_tuples_to_arrow_table(iceberg_schema: Schema, tuple_list: Iterable[Tuple]) -> pa.Table: +def amber_tuples_to_arrow_table( + iceberg_schema: Schema, tuple_list: Iterable[Tuple] +) -> pa.Table: return pa.Table.from_pydict( { name: [t[name] for t in tuple_list] @@ -90,12 +100,14 @@ def amber_tuples_to_arrow_table(iceberg_schema: Schema, tuple_list: Iterable[Tup ) -def arrow_table_to_amber_tuples(iceberg_schema: Schema, arrow_table: pa.Table) -> Iterable[Tuple]: +def arrow_table_to_amber_tuples( + iceberg_schema: Schema, arrow_table: pa.Table +) -> Iterable[Tuple]: tuple_provider = ArrowTableTupleProvider(arrow_table) return ( Tuple( {name: field_accessor for name in arrow_table.column_names}, - schema=core.models.Schema(iceberg_schema.as_arrow()) + schema=core.models.Schema(iceberg_schema.as_arrow()), ) for field_accessor in tuple_provider ) diff --git a/core/amber/src/main/python/core/storage/model/buffered_item_writer.py b/core/amber/src/main/python/core/storage/model/buffered_item_writer.py index 17f0b54a3ba..e7063ee58f8 100644 --- a/core/amber/src/main/python/core/storage/model/buffered_item_writer.py +++ b/core/amber/src/main/python/core/storage/model/buffered_item_writer.py @@ -2,13 +2,15 @@ from typing import Generic, TypeVar # Define a type variable -T = TypeVar('T') +T = TypeVar("T") class BufferedItemWriter(ABC, Generic[T]): """ - BufferedItemWriter provides an interface for writing items to a buffer and performing I/O operations. - The items are buffered before being written to the underlying storage to optimize performance. + BufferedItemWriter provides an interface for writing items to a buffer and + performing I/O operations. + The items are buffered before being written to the underlying storage to + optimize performance. :param T: the type of data items to be written. """ @@ -41,7 +43,8 @@ def close(self) -> None: @abstractmethod def put_one(self, item: T) -> None: """ - Put one item into the buffer. If the buffer is full, it should be flushed to the underlying storage. + Put one item into the buffer. If the buffer is full, it should be flushed to + the underlying storage. :param item: the data item to be written. """ pass @@ -49,7 +52,8 @@ def put_one(self, item: T) -> None: @abstractmethod def remove_one(self, item: T) -> None: """ - Remove one item from the buffer. If the item is not found in the buffer, an appropriate action should be taken, + Remove one item from the buffer. If the item is not found in the buffer, an + appropriate action should be taken, such as throwing an exception or ignoring the request. :param item: the data item to be removed. """ diff --git a/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py b/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py index a2f4074cb6d..71d3633bc4f 100644 --- a/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py +++ b/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py @@ -5,13 +5,15 @@ import os # For File equivalent # Define a type variable -T = TypeVar('T') +T = TypeVar("T") class ReadonlyVirtualDocument(ABC, Generic[T]): """ - ReadonlyVirtualDocument provides an abstraction for read operations over a single resource. - This class can be implemented by resources that only need to support read-related functionality. + ReadonlyVirtualDocument provides an abstraction for read operations over a single + esource. + This class can be implemented by resources that only need to support read-related + functionality. :param T: the type of data that can be accessed via an index. """ @@ -44,7 +46,8 @@ def get(self) -> Iterator[T]: @abstractmethod def get_range(self, from_index: int, until: int) -> Iterator[T]: """ - Get an iterator of a sequence starting from index `from_index`, until index `until`. + Get an iterator of a sequence starting from index `from_index`, until index + `until`. :param from_index: the starting index (inclusive) :param until: the ending index (exclusive) :return: an iterator that returns data items of type T diff --git a/core/amber/src/main/python/core/storage/model/virtual_document.py b/core/amber/src/main/python/core/storage/model/virtual_document.py index 4918466f8fe..ae10bddae8b 100644 --- a/core/amber/src/main/python/core/storage/model/virtual_document.py +++ b/core/amber/src/main/python/core/storage/model/virtual_document.py @@ -9,13 +9,15 @@ from typing import TypeVar, Iterator # Define a type variable -T = TypeVar('T') +T = TypeVar("T") class VirtualDocument(ReadonlyVirtualDocument[T], ABC): """ - VirtualDocument provides the abstraction of performing read/write/copy/delete operations over a single resource. - Note that all methods have a default implementation. This is because one document implementation may not be able + VirtualDocument provides the abstraction of performing read/write/copy/delete + operations over a single resource. + Note that all methods have a default implementation. This is because one document + implementation may not be able to reasonably support all methods. :param T: the type of data that can use index to read and write. diff --git a/core/amber/src/main/python/core/storage/test_iceberg_document.py b/core/amber/src/main/python/core/storage/test_iceberg_document.py index 956b5614238..2238efcc309 100644 --- a/core/amber/src/main/python/core/storage/test_iceberg_document.py +++ b/core/amber/src/main/python/core/storage/test_iceberg_document.py @@ -9,7 +9,12 @@ from core.models import Schema, Tuple from core.storage.document_factory import DocumentFactory from core.storage.vfs_uri_factory import VFSURIFactory -from proto.edu.uci.ics.amber.core import WorkflowIdentity, ExecutionIdentity, OperatorIdentity, PortIdentity +from proto.edu.uci.ics.amber.core import ( + WorkflowIdentity, + ExecutionIdentity, + OperatorIdentity, + PortIdentity, +) # Function to generate random binary data @@ -40,7 +45,7 @@ def iceberg_document(self, amber_schema): WorkflowIdentity(id=0), ExecutionIdentity(id=0), OperatorIdentity(id=f"test_table_{operator_uuid}"), - PortIdentity(id=0) + PortIdentity(id=0), ) DocumentFactory.create_document(uri, amber_schema) document, _ = DocumentFactory.open_document(uri) @@ -59,7 +64,7 @@ def sample_items(self, amber_schema) -> [Tuple]: "col-timestamp": datetime.datetime.now(), "col-binary": b"hello", }, - schema=amber_schema + schema=amber_schema, ), Tuple( { @@ -71,7 +76,7 @@ def sample_items(self, amber_schema) -> [Tuple]: "col-timestamp": datetime.datetime.fromtimestamp(100000000), "col-binary": bytearray([255, 0, 0, 64]), }, - schema=amber_schema + schema=amber_schema, ), Tuple( { @@ -79,11 +84,11 @@ def sample_items(self, amber_schema) -> [Tuple]: "col-int": 2147483647, "col-bool": True, "col-long": 9223372036854775807, - "col-double": 1.7976931348623157E308, + "col-double": 1.7976931348623157e308, "col-timestamp": datetime.datetime.fromtimestamp(1234567890), "col-binary": bytearray([1, 2, 3, 4, 5]), }, - schema=amber_schema + schema=amber_schema, ), ] @@ -96,8 +101,13 @@ def sample_items(self, amber_schema) -> [Tuple]: "col-bool": None if i % 6 == 0 else i % 2 == 0, "col-long": None if i % 4 == 0 else i * 1000000, "col-double": None if i % 3 == 0 else i * 0.12345, - "col-timestamp": None if i % 8 == 0 else datetime.datetime.fromtimestamp( - datetime.datetime.now().timestamp() + i), + "col-timestamp": ( + None + if i % 8 == 0 + else datetime.datetime.fromtimestamp( + datetime.datetime.now().timestamp() + i + ) + ), "col-binary": None if i % 9 == 0 else generate_random_binary(10), }, schema=amber_schema, @@ -107,18 +117,14 @@ def sample_items(self, amber_schema) -> [Tuple]: return base_tuples + additional_tuples - def test_basic_read_and_write( - self, - iceberg_document, - sample_items - ): + def test_basic_read_and_write(self, iceberg_document, sample_items): writer = iceberg_document.writer(str(uuid.uuid4())) writer.open() for item in sample_items: writer.put_one(item) writer.close() retrieved_items = list(iceberg_document.get()) - assert (sample_items == retrieved_items) + assert sample_items == retrieved_items def test_clear_document(self, iceberg_document, sample_items): writer = iceberg_document.writer(str(uuid.uuid4())) @@ -126,14 +132,14 @@ def test_clear_document(self, iceberg_document, sample_items): for item in sample_items: writer.put_one(item) writer.close() - assert (len(list(iceberg_document.get())) > 0) + assert len(list(iceberg_document.get())) > 0 iceberg_document.clear() - assert (len(list(iceberg_document.get())) == 0) + assert len(list(iceberg_document.get())) == 0 def test_handle_empty_read(self, iceberg_document): retrieved_items = list(iceberg_document.get()) - assert (retrieved_items == []) + assert retrieved_items == [] def test_concurrent_writes_followed_by_read(self, iceberg_document, sample_items): all_items = sample_items @@ -143,12 +149,19 @@ def test_concurrent_writes_followed_by_read(self, iceberg_document, sample_items remainder = len(all_items) % num_writers # Create writer's batches item_batches = [ - all_items[i * batch_size + min(i, remainder): i * batch_size + min(i, remainder) + batch_size + ( - 1 if i < remainder else 0)] + all_items[ + i * batch_size + + min(i, remainder) : i * batch_size + + min(i, remainder) + + batch_size + + (1 if i < remainder else 0) + ] 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): @@ -166,7 +179,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): writer = iceberg_document.writer(str(uuid.uuid4())) @@ -177,20 +192,24 @@ def test_read_using_range(self, iceberg_document, sample_items): # Read all items using ranges batch_size = 1500 # Generate ranges - ranges = [range(i, min(i + batch_size, len(sample_items))) for i in range(0, len(sample_items), batch_size)] + ranges = [ + range(i, min(i + batch_size, len(sample_items))) + for i in range(0, len(sample_items), batch_size) + ] # Retrieve items using ranges retrieved_items = [ - item - for r in ranges - for item in iceberg_document.get_range(r.start, r.stop) + 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): writer = iceberg_document.writer(str(uuid.uuid4())) @@ -207,17 +226,18 @@ def test_get_after(self, iceberg_document, sample_items): expected_items = [] retrieved_items = list(iceberg_document.get_after(offset)) - assert (retrieved_items == expected_items, ( + assert retrieved_items == expected_items, ( f"get_after({offset}) did not return the expected items. " f"Expected: {expected_items}, Got: {retrieved_items}" - )) + ) # Test get_after for an offset beyond the range invalid_offset = len(sample_items) retrieved_items = list(iceberg_document.get_after(invalid_offset)) - assert (not retrieved_items, ( - f"get_after({invalid_offset}) should return an empty list, but got: {retrieved_items}" - )) + assert not retrieved_items, ( + f"get_after({invalid_offset}) should return " + f"an empty list, but got: {retrieved_items}" + ) def test_get_counts(self, iceberg_document, sample_items): writer = iceberg_document.writer(str(uuid.uuid4())) @@ -226,5 +246,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 with " - "allItems") + assert iceberg_document.get_count() == len( + sample_items + ), "get_count should return the same number with allItems" diff --git a/core/amber/src/main/python/core/storage/vfs_uri_factory.py b/core/amber/src/main/python/core/storage/vfs_uri_factory.py index f6b9fe2e40e..dfaa064f78c 100644 --- a/core/amber/src/main/python/core/storage/vfs_uri_factory.py +++ b/core/amber/src/main/python/core/storage/vfs_uri_factory.py @@ -2,7 +2,12 @@ from typing import Optional from urllib.parse import urlparse -from proto.edu.uci.ics.amber.core import WorkflowIdentity, ExecutionIdentity, OperatorIdentity, PortIdentity +from proto.edu.uci.ics.amber.core import ( + WorkflowIdentity, + ExecutionIdentity, + OperatorIdentity, + PortIdentity, +) class VFSResourceType(str, Enum): @@ -14,7 +19,15 @@ class VFSURIFactory: VFS_FILE_URI_SCHEME = "vfs" @staticmethod - def decode_uri(uri: str) -> (WorkflowIdentity, ExecutionIdentity, Optional[OperatorIdentity], PortIdentity, VFSResourceType): + def decode_uri( + uri: str, + ) -> ( + WorkflowIdentity, + ExecutionIdentity, + Optional[OperatorIdentity], + PortIdentity, + VFSResourceType, + ): parsed_uri = urlparse(uri) if parsed_uri.scheme != VFSURIFactory.VFS_FILE_URI_SCHEME: @@ -57,24 +70,42 @@ def extract_value(key: str) -> str: @staticmethod def create_result_uri(workflow_id, execution_id, operator_id, port_identity) -> str: return VFSURIFactory.create_vfs_uri( - VFSResourceType.RESULT, workflow_id, execution_id, operator_id, port_identity + VFSResourceType.RESULT, + workflow_id, + execution_id, + operator_id, + port_identity, ) @staticmethod - def create_materialized_result_uri(workflow_id, execution_id, operator_id, port_identity) -> str: + def create_materialized_result_uri( + workflow_id, execution_id, operator_id, port_identity + ) -> str: return VFSURIFactory.create_vfs_uri( - VFSResourceType.MATERIALIZED_RESULT, workflow_id, execution_id, operator_id, port_identity + VFSResourceType.MATERIALIZED_RESULT, + workflow_id, + execution_id, + operator_id, + port_identity, ) @staticmethod - def create_vfs_uri(resource_type, workflow_id, execution_id, operator_id, port_identity=None) -> str: - if resource_type in (VFSResourceType.RESULT, VFSResourceType.MATERIALIZED_RESULT) and port_identity is None: + def create_vfs_uri( + resource_type, workflow_id, execution_id, operator_id, port_identity=None + ) -> str: + if ( + resource_type + in (VFSResourceType.RESULT, VFSResourceType.MATERIALIZED_RESULT) + and port_identity is None + ): raise ValueError( - "PortIdentity must be provided when resourceType is RESULT or MATERIALIZED_RESULT." + "PortIdentity must be provided when resourceType is RESULT or " + "MATERIALIZED_RESULT." ) base_uri = ( - f"{VFSURIFactory.VFS_FILE_URI_SCHEME}:///wid/{workflow_id.id}/eid/{execution_id.id}/opid/{operator_id.id}" + f"{VFSURIFactory.VFS_FILE_URI_SCHEME}:///wid/{workflow_id.id}" + f"/eid/{execution_id.id}/opid/{operator_id.id}" ) if port_identity is not None: From d3312eb640bf2f78f1ccb58ed9bfa7e15308f820 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 00:09:29 -0800 Subject: [PATCH 16/29] fix requirements.txt --- core/amber/requirements.txt | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/amber/requirements.txt b/core/amber/requirements.txt index 5f54a186b46..bc79a49365c 100644 --- a/core/amber/requirements.txt +++ b/core/amber/requirements.txt @@ -29,4 +29,6 @@ transformers==4.44.2 tzlocal==2.1 pyiceberg==0.8.1 readerwriterlock==1.0.9 -tenacity==8.5.0 \ No newline at end of file +tenacity==8.5.0 +SQLAlchemy==2.0.37 +psycopg2==2.9.10 \ No newline at end of file From 42622e2c1c52d29c7efca4bcb7b73e99dd6219b6 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 00:16:52 -0800 Subject: [PATCH 17/29] fix postgres config for test. --- .../python/core/storage/iceberg_catalog_instance.py | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py index 540a3fbdfeb..255ccc7b05b 100644 --- a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py @@ -28,21 +28,12 @@ def get_instance(cls): cls._instance = SqlCatalog( "texera_iceberg", **{ - "uri": "postgresql+psycopg2://testdb:testuser@localhost/" - "testpassword", + "uri": "postgresql+psycopg2://testuser:testpassword@localhost/" + "testdb", "warehouse": f"file://{warehouse_path}", "init_catalog_tables": "true", }, ) - # cls._instance = RestCatalog( - # "texera_iceberg", - # **{ - # "uri": "postgresql+psycopg2://iceberg_py_test:test@localhost/test_ - # iceberg_catalog_jan16", - # "warehouse": f"file://{warehouse_path}", - # "init_catalog_tables": "true" - # } - # ) return cls._instance @classmethod From 433a545c2a332e2f4e1d44a8d0a5eb6e5fefe236 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 00:23:38 -0800 Subject: [PATCH 18/29] fix postgres config for test. --- .github/workflows/github-action-build.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/github-action-build.yml b/.github/workflows/github-action-build.yml index 4edb2d627a0..b462a5dcf53 100644 --- a/.github/workflows/github-action-build.yml +++ b/.github/workflows/github-action-build.yml @@ -120,6 +120,8 @@ jobs: sudo -u postgres psql -c "CREATE DATABASE testdb;" sudo -u postgres psql -c "CREATE USER testuser WITH ENCRYPTED PASSWORD 'testpassword';" sudo -u postgres psql -c "GRANT ALL PRIVILEGES ON DATABASE testdb TO testuser;" + sudo -u postgres psql -c "ALTER DATABASE testdb OWNER TO testuser;" + sudo -u postgres psql -d testdb -c "GRANT ALL ON SCHEMA public TO testuser;" - name: Lint with flake8 and black run: | cd core/amber/src/main/python && flake8 && black . --check From 45f92d30676497437b309ddc3b78c22b6f4beae7 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 00:29:15 -0800 Subject: [PATCH 19/29] create namespace if not exists. --- core/amber/src/main/python/core/storage/iceberg_utils.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/amber/src/main/python/core/storage/iceberg_utils.py b/core/amber/src/main/python/core/storage/iceberg_utils.py index 4a6593187c5..150159b7d2a 100644 --- a/core/amber/src/main/python/core/storage/iceberg_utils.py +++ b/core/amber/src/main/python/core/storage/iceberg_utils.py @@ -35,7 +35,7 @@ def create_table( identifier = f"{table_namespace}.{table_name}" - # catalog.create_namespace_if_not_exists(table_namespace) + catalog.create_namespace_if_not_exists(table_namespace) if catalog.table_exists(identifier) and override_if_exists: catalog.drop_table(identifier) From 27a515e6229db60298d326008d9c2babe20e5358 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 00:40:38 -0800 Subject: [PATCH 20/29] fix test. --- core/amber/src/main/python/core/models/schema/test_schema.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/amber/src/main/python/core/models/schema/test_schema.py b/core/amber/src/main/python/core/models/schema/test_schema.py index 1b0798192af..ca9f14ae9a3 100644 --- a/core/amber/src/main/python/core/models/schema/test_schema.py +++ b/core/amber/src/main/python/core/models/schema/test_schema.py @@ -27,7 +27,7 @@ def arrow_schema(self): pa.field("field-3", pa.int64()), pa.field("field-4", pa.float64()), pa.field("field-5", pa.bool_()), - pa.field("field-6", pa.timestamp("ms", tz="UTC")), + pa.field("field-6", pa.timestamp("us")), pa.field("field-7", pa.binary()), ] ) From d7a70a2595812fa2f07bd7716ffc2bf215a1fc85 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 11:37:16 -0800 Subject: [PATCH 21/29] add psql script. --- .github/workflows/github-action-build.yml | 6 +----- .../core/storage/iceberg_catalog_instance.py | 5 ++--- core/scripts/sql/iceberg_postgres_catalog.sql | 16 ++++++++++++++++ 3 files changed, 19 insertions(+), 8 deletions(-) create mode 100755 core/scripts/sql/iceberg_postgres_catalog.sql diff --git a/.github/workflows/github-action-build.yml b/.github/workflows/github-action-build.yml index b462a5dcf53..3bdd0a40295 100644 --- a/.github/workflows/github-action-build.yml +++ b/.github/workflows/github-action-build.yml @@ -117,11 +117,7 @@ jobs: run: sudo systemctl start postgresql - name: Create Database and User run: | - sudo -u postgres psql -c "CREATE DATABASE testdb;" - sudo -u postgres psql -c "CREATE USER testuser WITH ENCRYPTED PASSWORD 'testpassword';" - sudo -u postgres psql -c "GRANT ALL PRIVILEGES ON DATABASE testdb TO testuser;" - sudo -u postgres psql -c "ALTER DATABASE testdb OWNER TO testuser;" - sudo -u postgres psql -d testdb -c "GRANT ALL ON SCHEMA public TO testuser;" + cd core/scripts/sql && sudo -u postgres psql -f iceberg_postgres_catalog.sql - name: Lint with flake8 and black run: | cd core/amber/src/main/python && flake8 && black . --check diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py index 255ccc7b05b..cb6bcb7ce08 100644 --- a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py @@ -2,7 +2,6 @@ from pyiceberg.catalog.sql import SqlCatalog from typing import Optional -# replace with actual path warehouse_path = "../../../../../core/amber/user-resources/workflow-results" @@ -28,8 +27,8 @@ def get_instance(cls): cls._instance = SqlCatalog( "texera_iceberg", **{ - "uri": "postgresql+psycopg2://testuser:testpassword@localhost/" - "testdb", + "uri": "postgresql+psycopg2://texera_iceberg_admin:password@localhost/" + "texera_iceberg_catalog", "warehouse": f"file://{warehouse_path}", "init_catalog_tables": "true", }, diff --git a/core/scripts/sql/iceberg_postgres_catalog.sql b/core/scripts/sql/iceberg_postgres_catalog.sql new file mode 100755 index 00000000000..374da5b0f82 --- /dev/null +++ b/core/scripts/sql/iceberg_postgres_catalog.sql @@ -0,0 +1,16 @@ +-- Important: replace "texera_iceberg_admin" and "password" with your own username +-- and password before executing the script. +\set db_user texera_iceberg_admin +\set db_password '\'password\'' + +CREATE DATABASE texera_iceberg_catalog; + +CREATE USER :db_user WITH ENCRYPTED PASSWORD :db_password; + +GRANT ALL PRIVILEGES ON DATABASE texera_iceberg_catalog TO :db_user; + +ALTER DATABASE texera_iceberg_catalog OWNER TO :db_user; + +\c texera_iceberg_catalog; + +GRANT ALL ON SCHEMA public TO :db_user; From 71644761f583461e79ab550f504c4e3d15e2c6bc Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 11:42:43 -0800 Subject: [PATCH 22/29] fix fmt. --- .../src/main/python/core/storage/iceberg_catalog_instance.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py index cb6bcb7ce08..84890a603b9 100644 --- a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py @@ -27,8 +27,8 @@ def get_instance(cls): cls._instance = SqlCatalog( "texera_iceberg", **{ - "uri": "postgresql+psycopg2://texera_iceberg_admin:password@localhost/" - "texera_iceberg_catalog", + "uri": "postgresql+psycopg2://texera_iceberg_admin:" + "password@localhost/texera_iceberg_catalog", "warehouse": f"file://{warehouse_path}", "init_catalog_tables": "true", }, From 4240577964a8789747641e5ac88f7917ed8cab8d Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 11:56:34 -0800 Subject: [PATCH 23/29] fix fmt. --- .../src/main/python/core/storage/iceberg_catalog_instance.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py index 84890a603b9..2977affb854 100644 --- a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py @@ -28,7 +28,7 @@ def get_instance(cls): "texera_iceberg", **{ "uri": "postgresql+psycopg2://texera_iceberg_admin:" - "password@localhost/texera_iceberg_catalog", + "password@localhost/texera_iceberg_catalog", "warehouse": f"file://{warehouse_path}", "init_catalog_tables": "true", }, From 307726374f9e30bff1934dc92e1d7bc3ebe5c5bf Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 13:57:11 -0800 Subject: [PATCH 24/29] add storage config. --- .../python/core/storage/document_factory.py | 9 +++--- .../core/storage/iceberg_catalog_instance.py | 15 ++++----- .../python/core/storage/iceberg_document.py | 4 +-- .../core/storage/iceberg_table_writer.py | 4 +-- .../main/python/core/storage/iceberg_utils.py | 14 ++++++++ .../python/core/storage/storage_config.py | 32 +++++++++++++++++++ .../core/storage/test_iceberg_document.py | 9 ++++++ 7 files changed, 69 insertions(+), 18 deletions(-) create mode 100644 core/amber/src/main/python/core/storage/storage_config.py diff --git a/core/amber/src/main/python/core/storage/document_factory.py b/core/amber/src/main/python/core/storage/document_factory.py index f84f9eea2e8..a73a4c0973c 100644 --- a/core/amber/src/main/python/core/storage/document_factory.py +++ b/core/amber/src/main/python/core/storage/document_factory.py @@ -12,6 +12,7 @@ load_table_metadata, ) from core.storage.model.virtual_document import VirtualDocument +from core.storage.storage_config import StorageConfig from core.storage.vfs_uri_factory import VFSURIFactory, VFSResourceType @@ -39,14 +40,14 @@ def create_document(uri: str, schema: Schema) -> VirtualDocument: create_table( IcebergCatalogInstance.get_instance(), - "operator-result", + StorageConfig.ICEBERG_TABLE_NAMESPACE, storage_key, iceberg_schema, override_if_exists=True, ) return IcebergDocument[Tuple]( - "operator-result", + StorageConfig.ICEBERG_TABLE_NAMESPACE, storage_key, iceberg_schema, amber_tuples_to_arrow_table, @@ -73,7 +74,7 @@ def open_document(uri: str) -> (VirtualDocument, Optional[Schema]): table = load_table_metadata( IcebergCatalogInstance.get_instance(), - "operator-result", + StorageConfig.ICEBERG_TABLE_NAMESPACE, storage_key, ) @@ -83,7 +84,7 @@ def open_document(uri: str) -> (VirtualDocument, Optional[Schema]): amber_schema = Schema(table.schema().as_arrow()) document = IcebergDocument( - "operator-result", + StorageConfig.ICEBERG_TABLE_NAMESPACE, storage_key, table.schema(), amber_tuples_to_arrow_table, diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py index 2977affb854..31e6b3e3625 100644 --- a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py @@ -1,8 +1,8 @@ from pyiceberg.catalog import Catalog -from pyiceberg.catalog.sql import SqlCatalog from typing import Optional -warehouse_path = "../../../../../core/amber/user-resources/workflow-results" +from core.storage.iceberg_utils import create_postgres_catalog +from core.storage.storage_config import StorageConfig class IcebergCatalogInstance: @@ -24,14 +24,11 @@ def get_instance(cls): :return: the Iceberg catalog instance. """ if cls._instance is None: - cls._instance = SqlCatalog( + cls._instance = create_postgres_catalog( "texera_iceberg", - **{ - "uri": "postgresql+psycopg2://texera_iceberg_admin:" - "password@localhost/texera_iceberg_catalog", - "warehouse": f"file://{warehouse_path}", - "init_catalog_tables": "true", - }, + StorageConfig.ICEBERG_FILE_STORAGE_DIRECTORY_PATH, + StorageConfig.ICEBERG_POSTGRES_CATALOG_USERNAME, + StorageConfig.ICEBERG_POSTGRES_CATALOG_PASSWORD, ) return cls._instance diff --git a/core/amber/src/main/python/core/storage/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg_document.py index 5d81cf0be39..2aec06c99be 100644 --- a/core/amber/src/main/python/core/storage/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg_document.py @@ -42,7 +42,6 @@ def __init__( table_schema: Schema, serde: Callable[[Schema, Iterable[T]], pa.Table], deserde: Callable[[Schema, pa.Table], Iterable[T]], - catalog: Optional[Catalog] = None, ): self.table_namespace = table_namespace self.table_name = table_name @@ -51,14 +50,13 @@ def __init__( self.deserde = deserde self.lock = rwlock.RWLockFair() - self.catalog = catalog or self._load_catalog() + self.catalog = self._load_catalog() # Create or override the table during initialization load_table_metadata(self.catalog, self.table_namespace, self.table_name) def _load_catalog(self) -> Catalog: """Load the Iceberg catalog.""" - # Implement catalog loading logic here, e.g., load from configuration return IcebergCatalogInstance.get_instance() def get_uri(self) -> ParseResult: diff --git a/core/amber/src/main/python/core/storage/iceberg_table_writer.py b/core/amber/src/main/python/core/storage/iceberg_table_writer.py index a9da56be11e..3f63bf139ba 100644 --- a/core/amber/src/main/python/core/storage/iceberg_table_writer.py +++ b/core/amber/src/main/python/core/storage/iceberg_table_writer.py @@ -6,6 +6,7 @@ from typing import List, TypeVar, Callable, Iterable from core.storage.model.buffered_item_writer import BufferedItemWriter +from core.storage.storage_config import StorageConfig # Define a type variable for the data type T T = TypeVar("T") @@ -38,7 +39,6 @@ def __init__( table_name: str, table_schema: pa.Schema, serde: Callable[[Schema, Iterable[T]], pa.Table], - buffer_size: int = 4096, # Default buffer size TODO: move to config ): self.writer_identifier = writer_identifier self.catalog = catalog @@ -46,7 +46,7 @@ def __init__( self.table_name = table_name self.table_schema = table_schema self.serde = serde - self.buffer_size = buffer_size + self.buffer_size = StorageConfig.ICEBERG_TABLE_COMMIT_BATCH_SIZE # Internal state self.buffer: List[T] = [] diff --git a/core/amber/src/main/python/core/storage/iceberg_utils.py b/core/amber/src/main/python/core/storage/iceberg_utils.py index 150159b7d2a..d3452ffc8e0 100644 --- a/core/amber/src/main/python/core/storage/iceberg_utils.py +++ b/core/amber/src/main/python/core/storage/iceberg_utils.py @@ -1,6 +1,7 @@ import pyarrow as pa import pyiceberg.table from pyiceberg.catalog import Catalog +from pyiceberg.catalog.sql import SqlCatalog from pyiceberg.expressions import AlwaysTrue from pyiceberg.io.pyarrow import project_table from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC @@ -12,6 +13,19 @@ from core.models import ArrowTableTupleProvider, Tuple +def create_postgres_catalog( + catalog_name: str, warehouse_path: str, username: str, password: str +) -> SqlCatalog: + return SqlCatalog( + catalog_name, + **{ + "uri": f"postgresql+psycopg2://{username}:" + f"{password}@localhost/texera_iceberg_catalog", + "warehouse": f"file://{warehouse_path}", + }, + ) + + def create_table( catalog: Catalog, table_namespace: str, diff --git a/core/amber/src/main/python/core/storage/storage_config.py b/core/amber/src/main/python/core/storage/storage_config.py new file mode 100644 index 00000000000..df9e5b56ea2 --- /dev/null +++ b/core/amber/src/main/python/core/storage/storage_config.py @@ -0,0 +1,32 @@ +class StorageConfig: + _initialized = False + + ICEBERG_POSTGRES_CATALOG_USERNAME = None + ICEBERG_POSTGRES_CATALOG_PASSWORD = None + ICEBERG_TABLE_NAMESPACE = None + ICEBERG_FILE_STORAGE_DIRECTORY_PATH = None + ICEBERG_TABLE_COMMIT_BATCH_SIZE = None + + @classmethod + def initialize( + cls, + postgres_username, + postgres_password, + table_namespace, + directory_path, + commit_batch_size, + ): + if cls._initialized: + raise RuntimeError( + "Storage config has already been initialized" "and cannot be modified." + ) + + cls.ICEBERG_POSTGRES_CATALOG_USERNAME = postgres_username + cls.ICEBERG_POSTGRES_CATALOG_PASSWORD = postgres_password + cls.ICEBERG_TABLE_NAMESPACE = table_namespace + cls.ICEBERG_FILE_STORAGE_DIRECTORY_PATH = directory_path + cls.ICEBERG_TABLE_COMMIT_BATCH_SIZE = commit_batch_size + cls._initialized = True + + def __new__(cls, *args, **kwargs): + raise TypeError(f"{cls.__name__} is a static class and cannot be instantiated.") diff --git a/core/amber/src/main/python/core/storage/test_iceberg_document.py b/core/amber/src/main/python/core/storage/test_iceberg_document.py index 2238efcc309..e752413391f 100644 --- a/core/amber/src/main/python/core/storage/test_iceberg_document.py +++ b/core/amber/src/main/python/core/storage/test_iceberg_document.py @@ -8,6 +8,7 @@ from core.models import Schema, Tuple from core.storage.document_factory import DocumentFactory +from core.storage.storage_config import StorageConfig from core.storage.vfs_uri_factory import VFSURIFactory from proto.edu.uci.ics.amber.core import ( WorkflowIdentity, @@ -16,6 +17,14 @@ PortIdentity, ) +StorageConfig.initialize( + postgres_username="texera_iceberg_admin", + postgres_password="password", + table_namespace="operator-port-result", + directory_path="../../../../../core/amber/user-resources/workflow-results", + commit_batch_size=4096, +) + # Function to generate random binary data def generate_random_binary(size): From 37175204fdc98f556b6bdc668df7378cf2eaf60b Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 14:22:26 -0800 Subject: [PATCH 25/29] refactoring. --- core/amber/src/main/python/core/storage/document_factory.py | 6 +++--- core/amber/src/main/python/core/storage/iceberg/__init__.py | 0 .../core/storage/{ => iceberg}/iceberg_catalog_instance.py | 2 +- .../python/core/storage/{ => iceberg}/iceberg_document.py | 6 +++--- .../core/storage/{ => iceberg}/iceberg_table_writer.py | 0 .../main/python/core/storage/{ => iceberg}/iceberg_utils.py | 0 .../core/storage/{ => iceberg}/test_iceberg_document.py | 2 +- 7 files changed, 8 insertions(+), 8 deletions(-) create mode 100644 core/amber/src/main/python/core/storage/iceberg/__init__.py rename core/amber/src/main/python/core/storage/{ => iceberg}/iceberg_catalog_instance.py (95%) rename core/amber/src/main/python/core/storage/{ => iceberg}/iceberg_document.py (97%) rename core/amber/src/main/python/core/storage/{ => iceberg}/iceberg_table_writer.py (100%) rename core/amber/src/main/python/core/storage/{ => iceberg}/iceberg_utils.py (100%) rename core/amber/src/main/python/core/storage/{ => iceberg}/test_iceberg_document.py (99%) diff --git a/core/amber/src/main/python/core/storage/document_factory.py b/core/amber/src/main/python/core/storage/document_factory.py index a73a4c0973c..d1819829516 100644 --- a/core/amber/src/main/python/core/storage/document_factory.py +++ b/core/amber/src/main/python/core/storage/document_factory.py @@ -3,9 +3,9 @@ from typing import Optional from core.models import Schema, Tuple -from core.storage.iceberg_catalog_instance import IcebergCatalogInstance -from core.storage.iceberg_document import IcebergDocument -from core.storage.iceberg_utils import ( +from core.storage.iceberg.iceberg_catalog_instance import IcebergCatalogInstance +from core.storage.iceberg.iceberg_document import IcebergDocument +from core.storage.iceberg.iceberg_utils import ( create_table, amber_tuples_to_arrow_table, arrow_table_to_amber_tuples, diff --git a/core/amber/src/main/python/core/storage/iceberg/__init__.py b/core/amber/src/main/python/core/storage/iceberg/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_catalog_instance.py similarity index 95% rename from core/amber/src/main/python/core/storage/iceberg_catalog_instance.py rename to core/amber/src/main/python/core/storage/iceberg/iceberg_catalog_instance.py index 31e6b3e3625..a70ad11ace5 100644 --- a/core/amber/src/main/python/core/storage/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg/iceberg_catalog_instance.py @@ -1,7 +1,7 @@ from pyiceberg.catalog import Catalog from typing import Optional -from core.storage.iceberg_utils import create_postgres_catalog +from core.storage.iceberg.iceberg_utils import create_postgres_catalog from core.storage.storage_config import StorageConfig diff --git a/core/amber/src/main/python/core/storage/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py similarity index 97% rename from core/amber/src/main/python/core/storage/iceberg_document.py rename to core/amber/src/main/python/core/storage/iceberg/iceberg_document.py index 2aec06c99be..06a6862b9df 100644 --- a/core/amber/src/main/python/core/storage/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py @@ -9,9 +9,9 @@ from pyiceberg.table import Table, FileScanTask from readerwriterlock import rwlock -from core.storage.iceberg_catalog_instance import IcebergCatalogInstance -from core.storage.iceberg_table_writer import IcebergTableWriter -from core.storage.iceberg_utils import ( +from core.storage.iceberg.iceberg_catalog_instance import IcebergCatalogInstance +from core.storage.iceberg.iceberg_table_writer import IcebergTableWriter +from core.storage.iceberg.iceberg_utils import ( load_table_metadata, read_data_file_as_arrow_table, ) diff --git a/core/amber/src/main/python/core/storage/iceberg_table_writer.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_table_writer.py similarity index 100% rename from core/amber/src/main/python/core/storage/iceberg_table_writer.py rename to core/amber/src/main/python/core/storage/iceberg/iceberg_table_writer.py diff --git a/core/amber/src/main/python/core/storage/iceberg_utils.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_utils.py similarity index 100% rename from core/amber/src/main/python/core/storage/iceberg_utils.py rename to core/amber/src/main/python/core/storage/iceberg/iceberg_utils.py diff --git a/core/amber/src/main/python/core/storage/test_iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py similarity index 99% rename from core/amber/src/main/python/core/storage/test_iceberg_document.py rename to core/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py index e752413391f..acaccb3f316 100644 --- a/core/amber/src/main/python/core/storage/test_iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py @@ -21,7 +21,7 @@ postgres_username="texera_iceberg_admin", postgres_password="password", table_namespace="operator-port-result", - directory_path="../../../../../core/amber/user-resources/workflow-results", + directory_path="../../../../../../core/amber/user-resources/workflow-results", commit_batch_size=4096, ) From 2231b9e9728305d6f8909a893f71be22f86eab26 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 14:40:09 -0800 Subject: [PATCH 26/29] refactoring. --- core/amber/src/main/python/.flake8 | 2 +- .../core/storage/iceberg/iceberg_document.py | 33 ++++++++++--------- .../python/core/storage/vfs_uri_factory.py | 29 +++++++++------- 3 files changed, 35 insertions(+), 29 deletions(-) diff --git a/core/amber/src/main/python/.flake8 b/core/amber/src/main/python/.flake8 index 90b57b7591b..071776ddc00 100644 --- a/core/amber/src/main/python/.flake8 +++ b/core/amber/src/main/python/.flake8 @@ -5,5 +5,5 @@ extend-ignore = E203, exclude = proto, -max-complexity = 11 +max-complexity = 10 max-line-length = 88 diff --git a/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py index 06a6862b9df..f340e77e625 100644 --- a/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py @@ -170,25 +170,11 @@ def _seek_to_usable_file(self) -> Iterator[FileScanTask]: # self.table.inspect.entries() does not work with java files, need # to implement the logic # to find file_sequence_number for each data file ourselves - file_sequence_map = {} current_snapshot = self.table.current_snapshot() if current_snapshot is None: return iter([]) - for manifest in current_snapshot.manifests(self.table.io): - for entry in manifest.fetch_manifest_entry(io=self.table.io): - file_sequence_map[entry.data_file.file_path] = ( - entry.sequence_number - ) - - # Retrieve and sort the file scan tasks by file sequence number - file_scan_tasks = list(self.table.scan().plan_files()) - # Sort files by their sequence number. Files without a sequence - # number will be read last. - sorted_file_scan_tasks = sorted( - file_scan_tasks, - key=lambda t: file_sequence_map.get( - t.file.file_path, float("inf") - ), + sorted_file_scan_tasks = self._extract_sorted_file_scan_tasks( + current_snapshot ) # Skip records in files before the `from_index` for task in sorted_file_scan_tasks: @@ -206,6 +192,21 @@ def _seek_to_usable_file(self) -> Iterator[FileScanTask]: else: return iter([]) + def _extract_sorted_file_scan_tasks(self, current_snapshot): + file_sequence_map = {} + for manifest in current_snapshot.manifests(self.table.io): + for entry in manifest.fetch_manifest_entry(io=self.table.io): + file_sequence_map[entry.data_file.file_path] = entry.sequence_number + # Retrieve and sort the file scan tasks by file sequence number + file_scan_tasks = list(self.table.scan().plan_files()) + # Sort files by their sequence number. Files without a sequence + # number will be read last. + sorted_file_scan_tasks = sorted( + file_scan_tasks, + key=lambda t: file_sequence_map.get(t.file.file_path, float("inf")), + ) + return sorted_file_scan_tasks + def __iter__(self) -> Iterator[T]: return self diff --git a/core/amber/src/main/python/core/storage/vfs_uri_factory.py b/core/amber/src/main/python/core/storage/vfs_uri_factory.py index dfaa064f78c..4bff06f78df 100644 --- a/core/amber/src/main/python/core/storage/vfs_uri_factory.py +++ b/core/amber/src/main/python/core/storage/vfs_uri_factory.py @@ -24,8 +24,8 @@ def decode_uri( ) -> ( WorkflowIdentity, ExecutionIdentity, - Optional[OperatorIdentity], - PortIdentity, + OperatorIdentity, + Optional[PortIdentity], VFSResourceType, ): parsed_uri = urlparse(uri) @@ -46,7 +46,18 @@ def extract_value(key: str) -> str: execution_id = ExecutionIdentity(int(extract_value("eid"))) operator_id = OperatorIdentity(extract_value("opid")) - port_identity = None + port_identity = VFSURIFactory._extract_optional_port_identity(segments, uri) + + resource_type_str = segments[-1].lower() + try: + resource_type = VFSResourceType(resource_type_str) + except ValueError: + raise ValueError(f"Unknown resource type: {resource_type_str}") + + return workflow_id, execution_id, operator_id, port_identity, resource_type + + @staticmethod + def _extract_optional_port_identity(segments, uri): if "pid" in segments: try: pid_index = segments.index("pid") @@ -55,17 +66,11 @@ def extract_value(key: str) -> str: if port_type != "I" and port_type != "E": raise ValueError(f"Invalid port type: {port_type} in URI: {uri}") is_internal = port_type == "I" - port_identity = PortIdentity(port_id, is_internal) + return PortIdentity(port_id, is_internal) except (ValueError, IndexError): raise ValueError(f"Invalid port information in URI: {uri}") - - resource_type_str = segments[-1].lower() - try: - resource_type = VFSResourceType(resource_type_str) - except ValueError: - raise ValueError(f"Unknown resource type: {resource_type_str}") - - return workflow_id, execution_id, operator_id, port_identity, resource_type + else: + return None @staticmethod def create_result_uri(workflow_id, execution_id, operator_id, port_identity) -> str: From f795ce3e8f058710424a35879d61362909034bc0 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 15:00:23 -0800 Subject: [PATCH 27/29] use wait random exponential for concurrent writers. --- .../python/core/storage/iceberg/iceberg_table_writer.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/amber/src/main/python/core/storage/iceberg/iceberg_table_writer.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_table_writer.py index 3f63bf139ba..cd464a447d1 100644 --- a/core/amber/src/main/python/core/storage/iceberg/iceberg_table_writer.py +++ b/core/amber/src/main/python/core/storage/iceberg/iceberg_table_writer.py @@ -2,7 +2,7 @@ from pyiceberg.catalog import Catalog from pyiceberg.schema import Schema from pyiceberg.table import Table -from tenacity import retry, stop_after_attempt, wait_random +from tenacity import retry, stop_after_attempt, wait_random_exponential from typing import List, TypeVar, Callable, Iterable from core.storage.model.buffered_item_writer import BufferedItemWriter @@ -28,7 +28,6 @@ class IcebergTableWriter(BufferedItemWriter[T]): :param table_namespace: The namespace of the Iceberg table. :param table_name: The name of the Iceberg table. :param table_schema: The schema of the Iceberg table. - :param buffer_size: The maximum size of the buffer before flushing. """ def __init__( @@ -84,10 +83,12 @@ def flush_buffer(self) -> None: def append_to_table_with_retry(pa_df: pa.Table) -> None: """Appends a pyarrow dataframe to the table in the catalog using tenacity - exponential backoff.""" + random exponential backoff.""" @retry( - wait=wait_random(0.001, 0.1), stop=stop_after_attempt(10), reraise=True + wait=wait_random_exponential(0.001, 10), + stop=stop_after_attempt(10), + reraise=True, ) def append_with_retry(): self.table.refresh() From d768921861ddff183f6dc2e0fed78bb541b14d91 Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Wed, 29 Jan 2025 21:52:25 -0800 Subject: [PATCH 28/29] refactorings and documentation. --- .../python/core/storage/document_factory.py | 4 + .../iceberg/iceberg_catalog_instance.py | 1 + .../core/storage/iceberg/iceberg_document.py | 76 +++++++++++-------- .../storage/iceberg/iceberg_table_writer.py | 28 ++++--- .../core/storage/iceberg/iceberg_utils.py | 34 ++++++--- .../storage/iceberg/test_iceberg_document.py | 41 ++++++++-- .../storage/model/buffered_item_writer.py | 2 - .../model/readonly_virtual_document.py | 31 ++------ .../core/storage/model/virtual_document.py | 48 ++++-------- .../python/core/storage/storage_config.py | 6 ++ .../python/core/storage/vfs_uri_factory.py | 12 ++- 11 files changed, 156 insertions(+), 127 deletions(-) diff --git a/core/amber/src/main/python/core/storage/document_factory.py b/core/amber/src/main/python/core/storage/document_factory.py index d1819829516..ed522df02b5 100644 --- a/core/amber/src/main/python/core/storage/document_factory.py +++ b/core/amber/src/main/python/core/storage/document_factory.py @@ -17,6 +17,10 @@ class DocumentFactory: + """ + Factory class to create and open documents. + Currently only iceberg documents are supported. + """ ICEBERG = "iceberg" diff --git a/core/amber/src/main/python/core/storage/iceberg/iceberg_catalog_instance.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_catalog_instance.py index a70ad11ace5..a00358a71ea 100644 --- a/core/amber/src/main/python/core/storage/iceberg/iceberg_catalog_instance.py +++ b/core/amber/src/main/python/core/storage/iceberg/iceberg_catalog_instance.py @@ -8,6 +8,7 @@ class IcebergCatalogInstance: """ IcebergCatalogInstance is a singleton that manages the Iceberg catalog instance. + Currently only postgres SQL catalog is supported. - Provides a single shared catalog for all Iceberg table-related operations. - Lazily initializes the catalog on first access. - Supports replacing the catalog instance for testing or reconfiguration. diff --git a/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py index f340e77e625..2a77307e2bb 100644 --- a/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py @@ -33,6 +33,9 @@ class IcebergDocument(VirtualDocument[T]): :param table_namespace: Namespace of the table. :param table_name: Name of the table. :param table_schema: Schema of the table. + :param serde: A function to convert a T iterable into a pyarrow Table. Note the + conversion is not based on a single T item (unlike Texera's Java IcebergDocument.) + :param deserde: A function to convert a pyarrow Table back into a T iterable. """ def __init__( @@ -50,14 +53,7 @@ def __init__( self.deserde = deserde self.lock = rwlock.RWLockFair() - self.catalog = self._load_catalog() - - # Create or override the table during initialization - load_table_metadata(self.catalog, self.table_namespace, self.table_name) - - def _load_catalog(self) -> Catalog: - """Load the Iceberg catalog.""" - return IcebergCatalogInstance.get_instance() + self.catalog = IcebergCatalogInstance.get_instance() def get_uri(self) -> ParseResult: """Returns the URI of the table location.""" @@ -79,9 +75,9 @@ def get(self) -> Iterator[T]: """Get an iterator for reading all records from the table.""" return self._get_using_file_sequence_order(0, None) - def get_range(self, from_index: int, until: int) -> Iterator[T]: + def get_range(self, from_index: int, until_index: int) -> Iterator[T]: """Get records within a specified range [from, until).""" - return self._get_using_file_sequence_order(from_index, until) + return self._get_using_file_sequence_order(from_index, until_index) def get_after(self, offset: int) -> Iterator[T]: """Get records starting after a specified offset.""" @@ -95,8 +91,13 @@ def get_count(self) -> int: return sum(f.file.record_count for f in table.scan().plan_files()) def writer(self, writer_identifier: str): - """Creates a BufferedItemWriter for writing data to the table.""" - return IcebergTableWriter( + """ + Creates a BufferedItemWriter for writing data to the table. + :param writer_identifier: The writer's ID. It should be unique within the same + table, as each writer will use it as the prefix of the files they append + :return: An IcebergTableWriter + """ + return IcebergTableWriter[T]( writer_identifier=writer_identifier, catalog=self.catalog, table_namespace=self.table_namespace, @@ -106,52 +107,61 @@ def writer(self, writer_identifier: str): ) def _get_using_file_sequence_order( - self, from_index: int, until: Optional[int] + self, from_index: int, until_index: Optional[int] ) -> Iterator[T]: """Utility to get records within a specified range.""" with self.lock.gen_rlock(): - iterator = IcebergIterator( + return IcebergIterator[T]( from_index, - until, + until_index, self.catalog, self.table_namespace, self.table_name, self.table_schema, self.deserde, ) - return iterator class IcebergIterator(Iterator[T]): + """ + A custom iterator class to read items from an iceberg table based on an index range. + """ + def __init__( self, - from_index, - until, - catalog, - table_namespace, - table_name, - table_schema, - deserde, + from_index: int, + until_index: int, + catalog: Catalog, + table_namespace: str, + table_name: str, + table_schema: Schema, + deserde: Callable[[Schema, pa.Table], Iterable[T]], ): self.from_index = from_index - self.until = until + self.until_index = until_index self.catalog = catalog self.table_namespace = table_namespace self.table_name = table_name self.table_schema = table_schema self.deserde = deserde self.lock = RLock() + # Counter for how many records have been skipped self.num_of_skipped_records = 0 + # Counter for how many records have been returned self.num_of_returned_records = 0 + # Total number of records to return, used for termination condition self.total_records_to_return = ( - self.until - self.from_index if until else float("inf") + self.until_index - self.from_index if until_index else float("inf") ) - self.current_record_iterator = iter([]) + # Load the table instance, initially the table instance may not exist self.table = self._load_table_metadata() + # Iterator for usable file scan tasks self.usable_file_iterator = self._seek_to_usable_file() + # Current record iterator for the active file + self.current_record_iterator = iter([]) def _load_table_metadata(self) -> Optional[Table]: - """Load table metadata.""" + """Util function to load the table's metadata.""" return load_table_metadata(self.catalog, self.table_namespace, self.table_name) def _seek_to_usable_file(self) -> Iterator[FileScanTask]: @@ -160,16 +170,14 @@ def _seek_to_usable_file(self) -> Iterator[FileScanTask]: if self.num_of_skipped_records > self.from_index: raise RuntimeError("seek operation should not be called") - # Refresh table snapshots + # Load the table for the first time if not self.table: self.table = self._load_table_metadata() + # If the table still does not exist after loading, end iterator. if self.table: try: self.table.refresh() - # self.table.inspect.entries() does not work with java files, need - # to implement the logic - # to find file_sequence_number for each data file ourselves current_snapshot = self.table.current_snapshot() if current_snapshot is None: return iter([]) @@ -193,6 +201,12 @@ def _seek_to_usable_file(self) -> Iterator[FileScanTask]: return iter([]) def _extract_sorted_file_scan_tasks(self, current_snapshot): + """ + As self.table.inspect.entries() does not work with java files, this method + implements the logic to find file_sequence_number for each data file ourselves + :param current_snapshot: The current snapshot of the table. + :return: The file scan tasks of the file sorted by file_sequence_number + """ file_sequence_map = {} for manifest in current_snapshot.manifests(self.table.io): for entry in manifest.fetch_manifest_entry(io=self.table.io): diff --git a/core/amber/src/main/python/core/storage/iceberg/iceberg_table_writer.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_table_writer.py index cd464a447d1..8f547a6af63 100644 --- a/core/amber/src/main/python/core/storage/iceberg/iceberg_table_writer.py +++ b/core/amber/src/main/python/core/storage/iceberg/iceberg_table_writer.py @@ -15,8 +15,7 @@ class IcebergTableWriter(BufferedItemWriter[T]): """ IcebergTableWriter writes data to the given Iceberg table in an append-only way. - - Each time the buffer is flushed, a new data file is created with a unique name. - - The `writer_identifier` is used to prefix the created files. + - Each time the buffer is flushed, a new data file is created using pyarrow - Iceberg data files are immutable once created. So each flush will create a distinct file. @@ -49,8 +48,6 @@ def __init__( # Internal state self.buffer: List[T] = [] - self.filename_idx = 0 - self.record_id = 0 # Load the Iceberg table self.table: Table = self.catalog.load_table( @@ -69,22 +66,26 @@ def put_one(self, item: T) -> None: """Add a single item to the buffer.""" self.buffer.append(item) if len(self.buffer) >= self.buffer_size: - self.flush_buffer() + self._flush_buffer() def remove_one(self, item: T) -> None: """Remove a single item from the buffer.""" self.buffer.remove(item) - def flush_buffer(self) -> None: - """Flush the current buffer to a new Iceberg data file.""" + def _flush_buffer(self) -> None: + """ + Flush the current buffer to a new Iceberg data file. The buffer is first + converted to a pyarrow table, and then appended to the iceberg table as a + parquet file. Note in the case of concurrent writers, as iceberg uses + optimistic concurrency control, we use a random exponential backoff mechanism + when commit failure happens because currently pyiceberg does not natively + support retry. + """ if not self.buffer: return df = self.serde(self.table_schema, self.buffer) def append_to_table_with_retry(pa_df: pa.Table) -> None: - """Appends a pyarrow dataframe to the table in the catalog using tenacity - random exponential backoff.""" - @retry( wait=wait_random_exponential(0.001, 10), stop=stop_after_attempt(10), @@ -92,10 +93,7 @@ def append_to_table_with_retry(pa_df: pa.Table) -> None: ) def append_with_retry(): self.table.refresh() - self.table.append( - pa_df - ) # <----- and this line, then Tenacity will retry. - self.filename_idx += 1 + self.table.append(pa_df) append_with_retry() @@ -105,7 +103,7 @@ def append_with_retry(): def close(self) -> None: """Close the writer, ensuring any remaining buffered items are flushed.""" if self.buffer: - self.flush_buffer() + self._flush_buffer() @buffer_size.setter def buffer_size(self, value): diff --git a/core/amber/src/main/python/core/storage/iceberg/iceberg_utils.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_utils.py index d3452ffc8e0..54db90f856b 100644 --- a/core/amber/src/main/python/core/storage/iceberg/iceberg_utils.py +++ b/core/amber/src/main/python/core/storage/iceberg/iceberg_utils.py @@ -3,7 +3,7 @@ from pyiceberg.catalog import Catalog from pyiceberg.catalog.sql import SqlCatalog from pyiceberg.expressions import AlwaysTrue -from pyiceberg.io.pyarrow import project_table +from pyiceberg.io.pyarrow import ArrowScan from pyiceberg.partitioning import UNPARTITIONED_PARTITION_SPEC from pyiceberg.schema import Schema from pyiceberg.table import Table @@ -16,6 +16,17 @@ def create_postgres_catalog( catalog_name: str, warehouse_path: str, username: str, password: str ) -> SqlCatalog: + """ + Creates a Postgres SQL catalog instance by connecting to the database named + "texera_iceberg_catalog". + - The only requirement of the database is that it already exists. Once pyiceberg + can connect to the database, it will handle the initializations. + :param catalog_name: the name of the catalog. + :param warehouse_path: the root path for the warehouse where the tables are stored. + :param username: the username of the postgres database. + :param password: the password of the postgres database. + :return: a SQLCatalog instance. + """ return SqlCatalog( catalog_name, **{ @@ -76,7 +87,7 @@ def load_table_metadata( :param table_name: The name of the table. :return: The table if found, or None if not found. """ - identifier = f"{table_namespace}.{table_name}" # Construct table identifier + identifier = f"{table_namespace}.{table_name}" try: return catalog.load_table(identifier) except Exception: @@ -86,25 +97,23 @@ def load_table_metadata( def read_data_file_as_arrow_table( planfile: pyiceberg.table.FileScanTask, iceberg_table: pyiceberg.table.Table ) -> pa.Table: - """Reads a data file and returns an iterator over its records.""" - # arrow_table: pa.Table = ArrowScan( - # iceberg_table.metadata, iceberg_table.io, iceberg_table.schema(), - # AlwaysTrue(), True - # ).to_table([planfile]) - arrow_table: pa.Table = project_table( - [planfile], + """Reads a data file as a pyarrow table and returns an iterator over its records.""" + arrow_table: pa.Table = ArrowScan( iceberg_table.metadata, iceberg_table.io, - AlwaysTrue(), iceberg_table.schema(), + AlwaysTrue(), True, - ) + ).to_table([planfile]) return arrow_table def amber_tuples_to_arrow_table( iceberg_schema: Schema, tuple_list: Iterable[Tuple] ) -> pa.Table: + """ + Converts a list of amber tuples to a pyarrow table for serialization. + """ return pa.Table.from_pydict( { name: [t[name] for t in tuple_list] @@ -117,6 +126,9 @@ def amber_tuples_to_arrow_table( 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. + """ tuple_provider = ArrowTableTupleProvider(arrow_table) return ( Tuple( diff --git a/core/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py index acaccb3f316..ef41c820615 100644 --- a/core/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg/test_iceberg_document.py @@ -17,6 +17,7 @@ PortIdentity, ) +# Hardcoded storage config only for test purposes. StorageConfig.initialize( postgres_username="texera_iceberg_admin", postgres_password="password", @@ -26,15 +27,11 @@ ) -# Function to generate random binary data -def generate_random_binary(size): - return bytearray(random.getrandbits(8) for _ in range(size)) - - class TestIcebergDocument: @pytest.fixture def amber_schema(self): + """Sample Amber schema""" return Schema( raw_schema={ "col-string": "STRING", @@ -49,6 +46,10 @@ def amber_schema(self): @pytest.fixture def iceberg_document(self, amber_schema): + """ + Creates an iceberg document of operator port results using the sample schema + with a random operator id + """ operator_uuid = str(uuid.uuid4()).replace("-", "") uri = VFSURIFactory.create_result_uri( WorkflowIdentity(id=0), @@ -62,6 +63,9 @@ def iceberg_document(self, amber_schema): @pytest.fixture def sample_items(self, amber_schema) -> [Tuple]: + """ + Generates a list of sample tuples + """ base_tuples = [ Tuple( { @@ -101,6 +105,10 @@ def sample_items(self, amber_schema) -> [Tuple]: ), ] + # Function to generate random binary data + def generate_random_binary(size): + return bytearray(random.getrandbits(8) for _ in range(size)) + # Generate additional tuples additional_tuples = [ Tuple( @@ -127,6 +135,9 @@ def sample_items(self, amber_schema) -> [Tuple]: return base_tuples + additional_tuples def test_basic_read_and_write(self, iceberg_document, sample_items): + """ + Create an iceberg document, write sample items, and read it back. + """ writer = iceberg_document.writer(str(uuid.uuid4())) writer.open() for item in sample_items: @@ -136,6 +147,9 @@ def test_basic_read_and_write(self, iceberg_document, sample_items): assert sample_items == retrieved_items def test_clear_document(self, iceberg_document, sample_items): + """ + Create an iceberg document, write sample items, and clear the document. + """ writer = iceberg_document.writer(str(uuid.uuid4())) writer.open() for item in sample_items: @@ -147,10 +161,16 @@ def test_clear_document(self, iceberg_document, sample_items): assert len(list(iceberg_document.get())) == 0 def test_handle_empty_read(self, iceberg_document): + """ + The iceberg document should handle empty reads gracefully + """ retrieved_items = list(iceberg_document.get()) assert retrieved_items == [] def test_concurrent_writes_followed_by_read(self, iceberg_document, sample_items): + """ + Tests multiple concurrent writers writing to the same iceberg document + """ all_items = sample_items num_writers = 10 # Calculate the batch size and the remainder @@ -193,6 +213,9 @@ def write_batch(batch): ), "All items should be read correctly after concurrent writes." def test_read_using_range(self, iceberg_document, sample_items): + """ + The iceberg document should read all items using rages correctly. + """ writer = iceberg_document.writer(str(uuid.uuid4())) writer.open() for item in sample_items: @@ -221,6 +244,9 @@ def test_read_using_range(self, iceberg_document, sample_items): ), "All items should be retrieved correctly using ranges." def test_get_after(self, iceberg_document, sample_items): + """ + The iceberg document should retrieve items correctly using get_after + """ writer = iceberg_document.writer(str(uuid.uuid4())) writer.open() for item in sample_items: @@ -249,6 +275,9 @@ def test_get_after(self, iceberg_document, sample_items): ) def test_get_counts(self, iceberg_document, sample_items): + """ + The iceberg document should correctly return the count of items. + """ writer = iceberg_document.writer(str(uuid.uuid4())) writer.open() for item in sample_items: @@ -257,4 +286,4 @@ def test_get_counts(self, iceberg_document, sample_items): assert iceberg_document.get_count() == len( sample_items - ), "get_count should return the same number with allItems" + ), "get_count should return the same number as the length of sample_items" diff --git a/core/amber/src/main/python/core/storage/model/buffered_item_writer.py b/core/amber/src/main/python/core/storage/model/buffered_item_writer.py index e7063ee58f8..7f00eb52d3a 100644 --- a/core/amber/src/main/python/core/storage/model/buffered_item_writer.py +++ b/core/amber/src/main/python/core/storage/model/buffered_item_writer.py @@ -11,8 +11,6 @@ class BufferedItemWriter(ABC, Generic[T]): performing I/O operations. The items are buffered before being written to the underlying storage to optimize performance. - - :param T: the type of data items to be written. """ @property diff --git a/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py b/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py index 71d3633bc4f..dcaca3b941b 100644 --- a/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py +++ b/core/amber/src/main/python/core/storage/model/readonly_virtual_document.py @@ -1,8 +1,7 @@ from abc import ABC, abstractmethod +from urllib.parse import ParseResult + from typing import Generic, TypeVar, Iterator -from urllib.parse import ParseResult # Python's URI equivalent -import io # For InputStream equivalent -import os # For File equivalent # Define a type variable T = TypeVar("T") @@ -11,18 +10,16 @@ class ReadonlyVirtualDocument(ABC, Generic[T]): """ ReadonlyVirtualDocument provides an abstraction for read operations over a single - esource. + resource. This class can be implemented by resources that only need to support read-related functionality. - - :param T: the type of data that can be accessed via an index. """ @abstractmethod def get_uri(self) -> ParseResult: """ Get the URI of the corresponding document. - :return: the URI of the document + :return: the URI of the document as a ParseResult object """ pass @@ -44,12 +41,12 @@ def get(self) -> Iterator[T]: pass @abstractmethod - def get_range(self, from_index: int, until: int) -> Iterator[T]: + def get_range(self, from_index: int, until_index: int) -> Iterator[T]: """ Get an iterator of a sequence starting from index `from_index`, until index `until`. :param from_index: the starting index (inclusive) - :param until: the ending index (exclusive) + :param until_index: the ending index (exclusive) :return: an iterator that returns data items of type T """ pass @@ -70,19 +67,3 @@ def get_count(self) -> int: :return: the count of items """ pass - - @abstractmethod - def as_input_stream(self) -> io.IOBase: - """ - Read the document as an input stream. - :return: the input stream - """ - pass - - @abstractmethod - def as_file(self) -> os.PathLike: - """ - Read or materialize the document as a file. - :return: the file object - """ - pass diff --git a/core/amber/src/main/python/core/storage/model/virtual_document.py b/core/amber/src/main/python/core/storage/model/virtual_document.py index ae10bddae8b..10b953eda97 100644 --- a/core/amber/src/main/python/core/storage/model/virtual_document.py +++ b/core/amber/src/main/python/core/storage/model/virtual_document.py @@ -1,13 +1,12 @@ -import io -import os from abc import ABC, abstractmethod from urllib.parse import ParseResult -from core.storage.model.buffered_item_writer import BufferedItemWriter -from core.storage.model.readonly_virtual_document import ReadonlyVirtualDocument from overrides import overrides from typing import TypeVar, Iterator +from core.storage.model.buffered_item_writer import BufferedItemWriter +from core.storage.model.readonly_virtual_document import ReadonlyVirtualDocument + # Define a type variable T = TypeVar("T") @@ -17,19 +16,12 @@ class VirtualDocument(ReadonlyVirtualDocument[T], ABC): VirtualDocument provides the abstraction of performing read/write/copy/delete operations over a single resource. Note that all methods have a default implementation. This is because one document - implementation may not be able - to reasonably support all methods. - - :param T: the type of data that can use index to read and write. + implementation may not be able to reasonably support all methods. """ - @abstractmethod + @overrides def get_uri(self) -> ParseResult: - """ - Get the URI of the corresponding document. - :return: the URI of the document - """ - pass + raise NotImplementedError("get_uri method is not implemented") @overrides def get_item(self, i: int) -> T: @@ -40,7 +32,7 @@ def get(self) -> Iterator[T]: raise NotImplementedError("get method is not implemented") @overrides - def get_range(self, from_index: int, until: int) -> Iterator[T]: + def get_range(self, from_index: int, until_index: int) -> Iterator[T]: raise NotImplementedError("get_range method is not implemented") @overrides @@ -51,27 +43,15 @@ def get_after(self, offset: int) -> Iterator[T]: def get_count(self) -> int: raise NotImplementedError("get_count method is not implemented") - def set_item(self, i: int, item: T) -> None: - raise NotImplementedError("set_item method is not implemented") - - def writer(self, writer_identifier: str) -> "BufferedItemWriter[T]": + def writer(self, writer_identifier: str) -> BufferedItemWriter[T]: + """ + return a writer that buffers the items and performs the flush operation at + close time + :param writer_identifier: the id of the writer + :return: a buffered item writer + """ raise NotImplementedError("writer method is not implemented") - def append(self, item: T) -> None: - raise NotImplementedError("append method is not implemented") - - def append_items(self, items: Iterator[T]) -> None: - raise NotImplementedError("append_items method is not implemented") - - def append_stream(self, input_stream: io.IOBase) -> None: - raise NotImplementedError("append_stream method is not implemented") - - def as_input_stream(self) -> io.IOBase: - raise NotImplementedError("as_input_stream method is not implemented") - - def as_file(self) -> os.PathLike: - raise NotImplementedError("as_file method is not implemented") - @abstractmethod def clear(self) -> None: """ diff --git a/core/amber/src/main/python/core/storage/storage_config.py b/core/amber/src/main/python/core/storage/storage_config.py index df9e5b56ea2..19300feaf51 100644 --- a/core/amber/src/main/python/core/storage/storage_config.py +++ b/core/amber/src/main/python/core/storage/storage_config.py @@ -1,4 +1,10 @@ class StorageConfig: + """ + A static class to keep the storage-related configs. + This class should be initialized with the configs passed from Java side and + is used by all storage-related classes. + """ + _initialized = False ICEBERG_POSTGRES_CATALOG_USERNAME = None diff --git a/core/amber/src/main/python/core/storage/vfs_uri_factory.py b/core/amber/src/main/python/core/storage/vfs_uri_factory.py index 4bff06f78df..59c95ad3d11 100644 --- a/core/amber/src/main/python/core/storage/vfs_uri_factory.py +++ b/core/amber/src/main/python/core/storage/vfs_uri_factory.py @@ -28,6 +28,9 @@ def decode_uri( Optional[PortIdentity], VFSResourceType, ): + """ + Parses a VFS URI and extracts its components. + """ parsed_uri = urlparse(uri) if parsed_uri.scheme != VFSURIFactory.VFS_FILE_URI_SCHEME: @@ -74,7 +77,8 @@ def _extract_optional_port_identity(segments, uri): @staticmethod def create_result_uri(workflow_id, execution_id, operator_id, port_identity) -> str: - return VFSURIFactory.create_vfs_uri( + """Creates a URI pointing to a result storage.""" + return VFSURIFactory._create_vfs_uri( VFSResourceType.RESULT, workflow_id, execution_id, @@ -86,7 +90,8 @@ def create_result_uri(workflow_id, execution_id, operator_id, port_identity) -> def create_materialized_result_uri( workflow_id, execution_id, operator_id, port_identity ) -> str: - return VFSURIFactory.create_vfs_uri( + """Creates a URI pointing to a materialized storage.""" + return VFSURIFactory._create_vfs_uri( VFSResourceType.MATERIALIZED_RESULT, workflow_id, execution_id, @@ -95,9 +100,10 @@ def create_materialized_result_uri( ) @staticmethod - def create_vfs_uri( + def _create_vfs_uri( resource_type, workflow_id, execution_id, operator_id, port_identity=None ) -> str: + """Internal helper to create URI pointing to a VFS resource.""" if ( resource_type in (VFSResourceType.RESULT, VFSResourceType.MATERIALIZED_RESULT) From c3b2fa3d806e6a3d40902a5b69eabab5f4b6bb2d Mon Sep 17 00:00:00 2001 From: Xiao-zhen-Liu Date: Thu, 30 Jan 2025 10:26:18 -0800 Subject: [PATCH 29/29] minor update. --- .../src/main/python/core/storage/iceberg/iceberg_document.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py b/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py index 2a77307e2bb..9d15336dff5 100644 --- a/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py +++ b/core/amber/src/main/python/core/storage/iceberg/iceberg_document.py @@ -1,3 +1,4 @@ +from itertools import islice from threading import RLock from typing import Iterator, Optional, Callable, Iterable from typing import TypeVar @@ -256,4 +257,4 @@ def __next__(self) -> T: @staticmethod def _skip_records(iterator, count): - return iter(list(iterator)[count:]) + return islice(iterator, count, None)