diff --git a/dev/dev-requirements.txt b/dev/dev-requirements.txt index 328c7e6..0094525 100755 --- a/dev/dev-requirements.txt +++ b/dev/dev-requirements.txt @@ -21,6 +21,7 @@ setuptools>=18.0 wheel py4j==0.10.9.7 pyarrow>=5.0.0 +polars>=1.31.0 fastavro>=1.9.0 zstandard>=0.23.0 pandas>=1.3.0 diff --git a/pypaimon/__init__.py b/pypaimon/__init__.py index c41cdf4..e154fad 100644 --- a/pypaimon/__init__.py +++ b/pypaimon/__init__.py @@ -15,37 +15,3 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################# - -from .api import Schema -from .py4j import Catalog -from .py4j import CommitMessage -from .py4j import Predicate -from .py4j import PredicateBuilder -from .py4j import ReadBuilder -from .py4j import RowType -from .py4j import Split -from .py4j import Table -from .py4j import BatchTableCommit -from .py4j import TableRead -from .py4j import TableScan -from .py4j import Plan -from .py4j import BatchTableWrite -from .py4j import BatchWriteBuilder - -__all__ = [ - 'Schema', - 'Catalog', - 'CommitMessage', - 'Predicate', - 'PredicateBuilder', - 'ReadBuilder', - 'RowType', - 'Split', - 'Table', - 'BatchTableCommit', - 'TableRead', - 'TableScan', - 'Plan', - 'BatchTableWrite', - 'BatchWriteBuilder' -] diff --git a/pypaimon/api/__init__.py b/pypaimon/api/__init__.py index 44717bf..4b57391 100644 --- a/pypaimon/api/__init__.py +++ b/pypaimon/api/__init__.py @@ -25,7 +25,9 @@ from .table_commit import BatchTableCommit from .table_write import BatchTableWrite from .write_builder import BatchWriteBuilder -from .table import Table, Schema +from .schema import Schema +from .table import Table +from .database import Database from .catalog import Catalog __all__ = [ @@ -40,6 +42,7 @@ 'BatchWriteBuilder', 'Table', 'Schema', + 'Database', 'Catalog', 'Predicate', 'PredicateBuilder' diff --git a/pypaimon/api/catalog.py b/pypaimon/api/catalog.py index 3132159..6e77a31 100644 --- a/pypaimon/api/catalog.py +++ b/pypaimon/api/catalog.py @@ -18,7 +18,7 @@ from abc import ABC, abstractmethod from typing import Optional -from pypaimon.api import Table, Schema +from pypaimon.api import Table, Schema, Database class Catalog(ABC): @@ -27,10 +27,9 @@ class Catalog(ABC): metadata such as database/table from a paimon catalog. """ - @staticmethod @abstractmethod - def create(catalog_options: dict) -> 'Catalog': - """Create catalog from configuration.""" + def get_database(self, name: str) -> 'Database': + """Get paimon database identified by the given name.""" @abstractmethod def get_table(self, identifier: str) -> Table: diff --git a/pypaimon/api/catalog_factory.py b/pypaimon/api/catalog_factory.py new file mode 100644 index 0000000..c497e22 --- /dev/null +++ b/pypaimon/api/catalog_factory.py @@ -0,0 +1,18 @@ +from pypaimon.api.catalog import Catalog + + +class CatalogFactory: + + @staticmethod + def create(catalog_options: dict) -> Catalog: + from pypaimon.pynative.catalog.catalog_option import CatalogOptions + from pypaimon.pynative.catalog.abstract_catalog import AbstractCatalog + from pypaimon.pynative.catalog.filesystem_catalog import FileSystemCatalog # noqa: F401 + from pypaimon.pynative.catalog.hive_catalog import HiveCatalog # noqa: F401 + + identifier = catalog_options.get(CatalogOptions.METASTORE, "filesystem") + subclasses = AbstractCatalog.__subclasses__() + for subclass in subclasses: + if subclass.identifier() == identifier: + return subclass(catalog_options) + raise ValueError(f"Unknown catalog identifier: {identifier}") diff --git a/pypaimon/api/database.py b/pypaimon/api/database.py new file mode 100644 index 0000000..db89430 --- /dev/null +++ b/pypaimon/api/database.py @@ -0,0 +1,28 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################# + +from typing import Optional + + +class Database: + """Structure of a Database.""" + + def __init__(self, name: str, properties: dict, comment: Optional[str] = None): + self.name = name + self.properties = properties + self.comment = comment diff --git a/pypaimon/api/schema.py b/pypaimon/api/schema.py new file mode 100644 index 0000000..e01b85b --- /dev/null +++ b/pypaimon/api/schema.py @@ -0,0 +1,37 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################# + +import pyarrow as pa + +from typing import Optional, List + + +class Schema: + """Schema of a table.""" + + def __init__(self, + pa_schema: pa.Schema, + partition_keys: Optional[List[str]] = None, + primary_keys: Optional[List[str]] = None, + options: Optional[dict] = None, + comment: Optional[str] = None): + self.pa_schema = pa_schema + self.partition_keys = partition_keys + self.primary_keys = primary_keys + self.options = options + self.comment = comment diff --git a/pypaimon/api/table.py b/pypaimon/api/table.py index 7eef7b4..310c7db 100644 --- a/pypaimon/api/table.py +++ b/pypaimon/api/table.py @@ -16,11 +16,8 @@ # limitations under the License. ################################################################################# -import pyarrow as pa - from abc import ABC, abstractmethod from pypaimon.api import ReadBuilder, BatchWriteBuilder -from typing import Optional, List class Table(ABC): @@ -33,19 +30,3 @@ def new_read_builder(self) -> ReadBuilder: @abstractmethod def new_batch_write_builder(self) -> BatchWriteBuilder: """Returns a builder for building batch table write and table commit.""" - - -class Schema: - """Schema of a table.""" - - def __init__(self, - pa_schema: pa.Schema, - partition_keys: Optional[List[str]] = None, - primary_keys: Optional[List[str]] = None, - options: Optional[dict] = None, - comment: Optional[str] = None): - self.pa_schema = pa_schema - self.partition_keys = partition_keys - self.primary_keys = primary_keys - self.options = options - self.comment = comment diff --git a/pypaimon/api/table_read.py b/pypaimon/api/table_read.py index 60b31e7..ca8847f 100644 --- a/pypaimon/api/table_read.py +++ b/pypaimon/api/table_read.py @@ -21,7 +21,7 @@ from abc import ABC, abstractmethod from pypaimon.api import Split -from typing import List, Optional, TYPE_CHECKING +from typing import List, Optional, TYPE_CHECKING, Iterator if TYPE_CHECKING: import ray @@ -31,6 +31,10 @@ class TableRead(ABC): """To read data from data splits.""" + @abstractmethod + def to_iterator(self, splits: List[Split]) -> Iterator[tuple]: + """Read data from splits and converted to pyarrow.Table format.""" + @abstractmethod def to_arrow(self, splits: List[Split]) -> pa.Table: """Read data from splits and converted to pyarrow.Table format.""" diff --git a/pypaimon/api/table_write.py b/pypaimon/api/table_write.py index 8839620..fcb4ff9 100644 --- a/pypaimon/api/table_write.py +++ b/pypaimon/api/table_write.py @@ -28,11 +28,11 @@ class BatchTableWrite(ABC): """A table write for batch processing. Recommended for one-time committing.""" @abstractmethod - def write_arrow(self, table: pa.Table): + def write_arrow(self, table: pa.Table, row_kind: List[int] = None): """ Write an arrow table to the writer.""" @abstractmethod - def write_arrow_batch(self, record_batch: pa.RecordBatch): + def write_arrow_batch(self, record_batch: pa.RecordBatch, row_kind: List[int] = None): """ Write an arrow record batch to the writer.""" @abstractmethod diff --git a/pypaimon/py4j/__init__.py b/pypaimon/py4j/__init__.py index 9152d3a..35d479d 100644 --- a/pypaimon/py4j/__init__.py +++ b/pypaimon/py4j/__init__.py @@ -20,7 +20,7 @@ from .java_implementation import \ (Catalog, Table, ReadBuilder, TableScan, Plan, RowType, Split, TableRead, BatchWriteBuilder, BatchTableWrite, CommitMessage, - BatchTableCommit, Predicate, PredicateBuilder) + BatchTableCommit, PredicateBuilder) __all__ = [ 'constants', @@ -36,6 +36,5 @@ 'BatchTableWrite', 'CommitMessage', 'BatchTableCommit', - 'Predicate', 'PredicateBuilder' ] diff --git a/pypaimon/py4j/java_gateway.py b/pypaimon/py4j/java_gateway.py index 41d7a0d..372b767 100644 --- a/pypaimon/py4j/java_gateway.py +++ b/pypaimon/py4j/java_gateway.py @@ -37,7 +37,6 @@ def get_gateway(): # type: () -> JavaGateway global _gateway - global _lock with _lock: if _gateway is None: # Set the level to WARN to mute the noisy INFO level logs diff --git a/pypaimon/py4j/java_implementation.py b/pypaimon/py4j/java_implementation.py index 34bd1c8..467e38d 100644 --- a/pypaimon/py4j/java_implementation.py +++ b/pypaimon/py4j/java_implementation.py @@ -15,7 +15,6 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import os # pypaimon.api implementation based on Java code & py4j lib @@ -28,14 +27,9 @@ from pypaimon.api import \ (catalog, table, read_builder, table_scan, split, row_type, table_read, write_builder, table_write, commit_message, - table_commit, Schema, predicate) + table_commit, Schema, predicate, Database, Table, Catalog) from typing import List, Iterator, Optional, Any, TYPE_CHECKING -from pypaimon.pynative.common.exception import PyNativeNotImplementedError -from pypaimon.pynative.common.predicate import PyNativePredicate -from pypaimon.pynative.common.row.internal_row import InternalRow -from pypaimon.pynative.util.reader_converter import ReaderConverter - if TYPE_CHECKING: import ray from duckdb.duckdb import DuckDBPyConnection @@ -48,13 +42,16 @@ def __init__(self, j_catalog, catalog_options: dict): self._catalog_options = catalog_options @staticmethod - def create(catalog_options: dict) -> 'Catalog': + def create(catalog_options: dict) -> Catalog: j_catalog_context = java_utils.to_j_catalog_context(catalog_options) gateway = get_gateway() j_catalog = gateway.jvm.CatalogFactory.createCatalog(j_catalog_context) return Catalog(j_catalog, catalog_options) - def get_table(self, identifier: str) -> 'Table': + def get_database(self, name: str) -> Database: + raise ValueError("No support method") + + def get_table(self, identifier: str) -> Table: j_identifier = java_utils.to_j_identifier(identifier) j_table = self._j_catalog.getTable(j_identifier) return Table(j_table, self._catalog_options) @@ -78,16 +75,7 @@ def __init__(self, j_table, catalog_options: dict): def new_read_builder(self) -> 'ReadBuilder': j_read_builder = get_gateway().jvm.InvocationUtil.getReadBuilder(self._j_table) - if self._j_table.primaryKeys().isEmpty(): - primary_keys = None - else: - primary_keys = [str(key) for key in self._j_table.primaryKeys()] - if self._j_table.partitionKeys().isEmpty(): - partition_keys = None - else: - partition_keys = [str(key) for key in self._j_table.partitionKeys()] - return ReadBuilder(j_read_builder, self._j_table.rowType(), self._catalog_options, - primary_keys, partition_keys) + return ReadBuilder(j_read_builder, self._j_table.rowType(), self._catalog_options) def new_batch_write_builder(self) -> 'BatchWriteBuilder': java_utils.check_batch_write(self._j_table) @@ -97,22 +85,16 @@ def new_batch_write_builder(self) -> 'BatchWriteBuilder': class ReadBuilder(read_builder.ReadBuilder): - def __init__(self, j_read_builder, j_row_type, catalog_options: dict, primary_keys: List[str], partition_keys: List[str]): + def __init__(self, j_read_builder, j_row_type, catalog_options: dict): self._j_read_builder = j_read_builder self._j_row_type = j_row_type self._catalog_options = catalog_options - self._primary_keys = primary_keys - self._partition_keys = partition_keys - self._predicate = None - self._projection = None def with_filter(self, predicate: 'Predicate'): - self._predicate = predicate self._j_read_builder.withFilter(predicate.to_j_predicate()) return self def with_projection(self, projection: List[str]) -> 'ReadBuilder': - self._projection = projection field_names = list(map(lambda field: field.name(), self._j_row_type.getFields())) int_projection = list(map(lambda p: field_names.index(p), projection)) gateway = get_gateway() @@ -132,8 +114,7 @@ def new_scan(self) -> 'TableScan': def new_read(self) -> 'TableRead': j_table_read = self._j_read_builder.newRead().executeFilter() - return TableRead(j_table_read, self._j_read_builder.readType(), self._catalog_options, - self._predicate, self._projection, self._primary_keys, self._partition_keys) + return TableRead(j_table_read, self._j_read_builder.readType(), self._catalog_options) def new_predicate_builder(self) -> 'PredicateBuilder': return PredicateBuilder(self._j_row_type) @@ -207,30 +188,17 @@ def file_paths(self) -> List[str]: class TableRead(table_read.TableRead): - def __init__(self, j_table_read, j_read_type, catalog_options, predicate, projection, - primary_keys: List[str], partition_keys: List[str]): - self._j_table_read = j_table_read - self._j_read_type = j_read_type - self._catalog_options = catalog_options - - self._predicate = predicate - self._projection = projection - self._primary_keys = primary_keys - self._partition_keys = partition_keys - + def __init__(self, j_table_read, j_read_type, catalog_options): self._arrow_schema = java_utils.to_arrow_schema(j_read_type) self._j_bytes_reader = get_gateway().jvm.InvocationUtil.createParallelBytesReader( j_table_read, j_read_type, TableRead._get_max_workers(catalog_options)) - def to_arrow(self, splits: List['Split']) -> pa.Table: - record_generator = self.to_record_generator(splits) + def to_iterator(self, splits: List[Split]) -> Iterator[tuple]: + raise ValueError("No support method") - # If necessary, set the env constants.IMPLEMENT_MODE to 'py4j' to forcibly use py4j reader - if os.environ.get(constants.IMPLEMENT_MODE, '') != 'py4j' and record_generator is not None: - return TableRead._iterator_to_pyarrow_table(record_generator, self._arrow_schema) - else: - record_batch_reader = self.to_arrow_batch_reader(splits) - return pa.Table.from_batches(record_batch_reader, schema=self._arrow_schema) + def to_arrow(self, splits): + record_batch_reader = self.to_arrow_batch_reader(splits) + return pa.Table.from_batches(record_batch_reader, schema=self._arrow_schema) def to_arrow_batch_reader(self, splits): j_splits = list(map(lambda s: s.to_j_split(), splits)) @@ -257,60 +225,6 @@ def to_ray(self, splits: List[Split]) -> "ray.data.dataset.Dataset": return ray.data.from_arrow(self.to_arrow(splits)) - def to_record_generator(self, splits: List['Split']) -> Optional[Iterator[Any]]: - """ - Returns a generator for iterating over records in the table. - If pynative reader is not available, returns None. - """ - try: - j_splits = list(s.to_j_split() for s in splits) - j_reader = get_gateway().jvm.InvocationUtil.createReader(self._j_table_read, j_splits) - converter = ReaderConverter(self._predicate, self._projection, self._primary_keys, self._partition_keys) - pynative_reader = converter.convert_java_reader(j_reader) - - def _record_generator(): - try: - batch = pynative_reader.read_batch() - while batch is not None: - record = batch.next() - while record is not None: - yield record - record = batch.next() - batch.release_batch() - batch = pynative_reader.read_batch() - finally: - pynative_reader.close() - - return _record_generator() - - except PyNativeNotImplementedError as e: - print(f"Generating pynative reader failed, will use py4j reader instead, " - f"error message: {str(e)}") - return None - - @staticmethod - def _iterator_to_pyarrow_table(record_generator, arrow_schema): - """ - Converts a record generator into a pyarrow Table using the provided Arrow schema. - """ - record_batches = [] - current_batch = [] - batch_size = 1024 # Can be adjusted according to needs for batch size - - for record in record_generator: - record_dict = {field: record.get_field(i) for i, field in enumerate(arrow_schema.names)} - current_batch.append(record_dict) - if len(current_batch) >= batch_size: - batch = pa.RecordBatch.from_pylist(current_batch, schema=arrow_schema) - record_batches.append(batch) - current_batch = [] - - if current_batch: - batch = pa.RecordBatch.from_pylist(current_batch, schema=arrow_schema) - record_batches.append(batch) - - return pa.Table.from_batches(record_batches, schema=arrow_schema) - @staticmethod def _get_max_workers(catalog_options): # default is sequential @@ -409,16 +323,12 @@ def close(self): class Predicate(predicate.Predicate): - def __init__(self, py_predicate: PyNativePredicate, j_predicate_bytes): - self.py_predicate = py_predicate + def __init__(self, j_predicate_bytes): self._j_predicate_bytes = j_predicate_bytes def to_j_predicate(self): return deserialize_java_object(self._j_predicate_bytes) - def test(self, record: InternalRow) -> bool: - return self.py_predicate.test(record) - class PredicateBuilder(predicate.PredicateBuilder): @@ -446,8 +356,7 @@ def _build(self, method: str, field: str, literals: Optional[List[Any]] = None): index, literals ) - return Predicate(PyNativePredicate(method, index, field, literals), - serialize_java_object(j_predicate)) + return Predicate(serialize_java_object(j_predicate)) def equal(self, field: str, literal: Any) -> Predicate: return self._build('equal', field, [literal]) @@ -493,13 +402,11 @@ def between(self, field: str, included_lower_bound: Any, included_upper_bound: A return self._build('between', field, [included_lower_bound, included_upper_bound]) def and_predicates(self, predicates: List[Predicate]) -> Predicate: - j_predicates = list(map(lambda p: p.to_j_predicate(), predicates)) - j_predicate = get_gateway().jvm.PredicationUtil.buildAnd(j_predicates) - return Predicate(PyNativePredicate('and', None, None, predicates), - serialize_java_object(j_predicate)) + predicates = list(map(lambda p: p.to_j_predicate(), predicates)) + j_predicate = get_gateway().jvm.PredicationUtil.buildAnd(predicates) + return Predicate(serialize_java_object(j_predicate)) def or_predicates(self, predicates: List[Predicate]) -> Predicate: - j_predicates = list(map(lambda p: p.to_j_predicate(), predicates)) - j_predicate = get_gateway().jvm.PredicationUtil.buildOr(j_predicates) - return Predicate(PyNativePredicate('or', None, None, predicates), - serialize_java_object(j_predicate)) + predicates = list(map(lambda p: p.to_j_predicate(), predicates)) + j_predicate = get_gateway().jvm.PredicationUtil.buildOr(predicates) + return Predicate(serialize_java_object(j_predicate)) diff --git a/pypaimon/py4j/tests/test_data_types.py b/pypaimon/py4j/tests/test_data_types.py index b0d0e41..44ee89b 100644 --- a/pypaimon/py4j/tests/test_data_types.py +++ b/pypaimon/py4j/tests/test_data_types.py @@ -20,7 +20,7 @@ import string import pyarrow as pa -from pypaimon import Schema +from pypaimon.api import Schema from pypaimon.py4j.tests import PypaimonTestBase from pypaimon.py4j.util import java_utils diff --git a/pypaimon/py4j/tests/test_object_metadata.py b/pypaimon/py4j/tests/test_object_metadata.py index e3591c9..b1aa5eb 100644 --- a/pypaimon/py4j/tests/test_object_metadata.py +++ b/pypaimon/py4j/tests/test_object_metadata.py @@ -19,7 +19,7 @@ import os import pyarrow as pa -from pypaimon import Schema +from pypaimon.api import Schema from pypaimon.py4j.tests import PypaimonTestBase diff --git a/pypaimon/py4j/tests/test_preicates.py b/pypaimon/py4j/tests/test_preicates.py index f538c93..13ad06f 100644 --- a/pypaimon/py4j/tests/test_preicates.py +++ b/pypaimon/py4j/tests/test_preicates.py @@ -20,7 +20,7 @@ import pandas as pd import pyarrow as pa -from pypaimon import Schema +from pypaimon.api import Schema from pypaimon.py4j.tests import PypaimonTestBase diff --git a/pypaimon/py4j/tests/test_write_and_read.py b/pypaimon/py4j/tests/test_write_and_read.py index 14e4138..4a6a56b 100644 --- a/pypaimon/py4j/tests/test_write_and_read.py +++ b/pypaimon/py4j/tests/test_write_and_read.py @@ -20,7 +20,7 @@ import pyarrow as pa from py4j.protocol import Py4JJavaError -from pypaimon import Schema +from pypaimon.api import Schema from pypaimon.py4j import Catalog from pypaimon.py4j.java_gateway import get_gateway from pypaimon.py4j.tests import PypaimonTestBase diff --git a/pypaimon/py4j/util/java_utils.py b/pypaimon/py4j/util/java_utils.py index 2a2aac9..0976b51 100644 --- a/pypaimon/py4j/util/java_utils.py +++ b/pypaimon/py4j/util/java_utils.py @@ -18,7 +18,7 @@ import pyarrow as pa -from pypaimon import Schema +from pypaimon.api import Schema from pypaimon.py4j.java_gateway import get_gateway diff --git a/pypaimon/pynative/common/row/__init__.py b/pypaimon/pynative/catalog/__init__.py similarity index 100% rename from pypaimon/pynative/common/row/__init__.py rename to pypaimon/pynative/catalog/__init__.py diff --git a/pypaimon/pynative/catalog/abstract_catalog.py b/pypaimon/pynative/catalog/abstract_catalog.py new file mode 100644 index 0000000..f19dfeb --- /dev/null +++ b/pypaimon/pynative/catalog/abstract_catalog.py @@ -0,0 +1,111 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################# + +from abc import abstractmethod +from pathlib import Path +from typing import Optional +from urllib.parse import urlparse + +from pypaimon.api import Schema, Table, Database +from pypaimon.api import Catalog +from pypaimon.pynative.common.exception import PyNativeNotImplementedError +from pypaimon.pynative.catalog.catalog_constant import CatalogConstants +from pypaimon.pynative.catalog.catalog_exception import DatabaseNotExistException, DatabaseAlreadyExistException, \ + TableAlreadyExistException, TableNotExistException +from pypaimon.pynative.catalog.catalog_option import CatalogOptions +from pypaimon.pynative.common.file_io import FileIO +from pypaimon.pynative.common.identifier import TableIdentifier +from pypaimon.pynative.common.core_option import CoreOptions +from pypaimon.pynative.table.file_store_table import FileStoreTable + + +class AbstractCatalog(Catalog): + def __init__(self, catalog_options: dict): + if CatalogOptions.WAREHOUSE not in catalog_options: + raise ValueError(f"Paimon '{CatalogOptions.WAREHOUSE}' path must be set") + self.warehouse = catalog_options.get(CatalogOptions.WAREHOUSE) + self.catalog_options = catalog_options + self.file_io = FileIO(self.warehouse, self.catalog_options) + + @staticmethod + @abstractmethod + def identifier() -> str: + """Catalog Identifier""" + + @abstractmethod + def create_database_impl(self, name: str, properties: Optional[dict] = None): + """Create DataBase Implementation""" + + @abstractmethod + def create_table_impl(self, table_identifier: TableIdentifier, schema: 'Schema'): + """Create Table Implementation""" + + @abstractmethod + def get_table_schema(self, table_identifier: TableIdentifier): + """Get Table Schema""" + + def create_database(self, name: str, ignore_if_exists: bool, properties: Optional[dict] = None): + try: + self.get_database(name) + if not ignore_if_exists: + raise DatabaseAlreadyExistException(name) + except DatabaseNotExistException: + self.create_database_impl(name, properties) + + def create_table(self, identifier: str, schema: 'Schema', ignore_if_exists: bool): + if schema.options and schema.options.get(CoreOptions.AUTO_CREATE): + raise ValueError(f"The value of {CoreOptions.AUTO_CREATE} property should be False.") + + table_identifier = TableIdentifier(identifier) + self.get_database(table_identifier.get_database_name()) + try: + self.get_table(identifier) + if not ignore_if_exists: + raise TableAlreadyExistException(identifier) + except TableNotExistException: + if schema.options and CoreOptions.TYPE in schema.options and schema.options.get( + CoreOptions.TYPE) != "table": + raise PyNativeNotImplementedError(f"Table Type {schema.options.get(CoreOptions.TYPE)}") + return self.create_table_impl(table_identifier, schema) + + def get_database(self, name: str) -> Database: + if self.file_io.exists(self.get_database_path(name)): + return Database(name, {}) + else: + raise DatabaseNotExistException(name) + + def get_table(self, identifier: str) -> Table: + table_identifier = TableIdentifier(identifier) + if CoreOptions.SCAN_FALLBACK_BRANCH in self.catalog_options: + raise PyNativeNotImplementedError(CoreOptions.SCAN_FALLBACK_BRANCH) + table_path = self.get_table_path(table_identifier) + table_schema = self.get_table_schema(table_identifier) + return FileStoreTable(self.file_io, table_identifier, table_path, table_schema) + + def get_database_path(self, name) -> Path: + return self._trim_schema(self.warehouse) / f"{name}{CatalogConstants.DB_SUFFIX}" + + def get_table_path(self, table_identifier: TableIdentifier) -> Path: + return self.get_database_path(table_identifier.get_database_name()) / table_identifier.get_table_name() + + @staticmethod + def _trim_schema(warehouse_url: str) -> Path: + parsed = urlparse(warehouse_url) + bucket = parsed.netloc + warehouse_dir = parsed.path.lstrip('/') + return Path(f"{bucket}/{warehouse_dir}" if warehouse_dir else bucket) diff --git a/pypaimon/pynative/reader/core/file_record_reader.py b/pypaimon/pynative/catalog/catalog_constant.py similarity index 66% rename from pypaimon/pynative/reader/core/file_record_reader.py rename to pypaimon/pynative/catalog/catalog_constant.py index 2d03cd1..32442a2 100644 --- a/pypaimon/pynative/reader/core/file_record_reader.py +++ b/pypaimon/pynative/catalog/catalog_constant.py @@ -16,22 +16,17 @@ # limitations under the License. ################################################################################ -from abc import abstractmethod -from typing import Optional, TypeVar +from enum import Enum -from pypaimon.pynative.reader.core.file_record_iterator import FileRecordIterator -from pypaimon.pynative.reader.core.record_reader import RecordReader -T = TypeVar('T') +class CatalogConstants(str, Enum): + def __str__(self): + return self.value -class FileRecordReader(RecordReader[T]): - """ - A RecordReader to support returning FileRecordIterator. - """ + COMMENT_PROP = "comment" + OWNER_PROP = "owner" - @abstractmethod - def read_batch(self) -> Optional[FileRecordIterator]: - """ - Reads one batch - """ + DEFAULT_DATABASE = "default" + DB_SUFFIX = ".db" + DB_LOCATION_PROP = "location" diff --git a/pypaimon/pynative/catalog/catalog_exception.py b/pypaimon/pynative/catalog/catalog_exception.py new file mode 100644 index 0000000..4216b14 --- /dev/null +++ b/pypaimon/pynative/catalog/catalog_exception.py @@ -0,0 +1,41 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +class ProcessSystemDatabaseException(ValueError): + def __init__(self): + super().__init__("Can't do operation on system database.") + + +class DatabaseNotExistException(ValueError): + def __init__(self, name): + super().__init__(f"Database {name} does not exist.") + + +class DatabaseAlreadyExistException(ValueError): + def __init__(self, name): + super().__init__(f"Database {name} already exists.") + + +class TableNotExistException(ValueError): + def __init__(self, name): + super().__init__(f"Table {name} does not exist.") + + +class TableAlreadyExistException(ValueError): + def __init__(self, name): + super().__init__(f"Table {name} already exists.") diff --git a/pypaimon/pynative/catalog/catalog_option.py b/pypaimon/pynative/catalog/catalog_option.py new file mode 100644 index 0000000..c44772f --- /dev/null +++ b/pypaimon/pynative/catalog/catalog_option.py @@ -0,0 +1,28 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from enum import Enum + + +class CatalogOptions(str, Enum): + + def __str__(self): + return self.value + + WAREHOUSE = "warehouse" + METASTORE = "metastore" diff --git a/pypaimon/pynative/catalog/filesystem_catalog.py b/pypaimon/pynative/catalog/filesystem_catalog.py new file mode 100644 index 0000000..1242f5b --- /dev/null +++ b/pypaimon/pynative/catalog/filesystem_catalog.py @@ -0,0 +1,63 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import Optional + +from pypaimon.api import Schema +from pypaimon.pynative.catalog.abstract_catalog import AbstractCatalog +from pypaimon.pynative.catalog.catalog_constant import CatalogConstants +from pypaimon.pynative.catalog.catalog_exception import TableNotExistException +from pypaimon.pynative.common.identifier import TableIdentifier +from pypaimon.pynative.table.schema_manager import SchemaManager + + +class FileSystemCatalog(AbstractCatalog): + + def __init__(self, catalog_options: dict): + super().__init__(catalog_options) + + @staticmethod + def identifier() -> str: + return "filesystem" + + def allow_custom_table_path(self) -> bool: + return False + + def create_database_impl(self, name: str, properties: Optional[dict] = None): + if properties and CatalogConstants.DB_LOCATION_PROP in properties: + raise ValueError("Cannot specify location for a database when using fileSystem catalog.") + path = self.get_database_path(name) + self.file_io.mkdirs(path) + + def create_table_impl(self, table_identifier: TableIdentifier, schema: Schema): + table_path = self.get_table_path(table_identifier) + schema_manager = SchemaManager(self.file_io, table_path) + schema_manager.create_table(schema) + + def get_table_schema(self, table_identifier: TableIdentifier): + table_path = self.get_table_path(table_identifier) + table_schema = SchemaManager(self.file_io, table_path).latest() + if table_schema is None: + raise TableNotExistException(table_identifier.get_full_name()) + return table_schema + + def lock_factory(self): + pass + + def metastore_client_factory(self): + return None diff --git a/pypaimon/pynative/reader/core/file_record_iterator.py b/pypaimon/pynative/catalog/hive_catalog.py similarity index 60% rename from pypaimon/pynative/reader/core/file_record_iterator.py rename to pypaimon/pynative/catalog/hive_catalog.py index 590a65e..da131c8 100644 --- a/pypaimon/pynative/reader/core/file_record_iterator.py +++ b/pypaimon/pynative/catalog/hive_catalog.py @@ -16,28 +16,25 @@ # limitations under the License. ################################################################################ -from abc import ABC, abstractmethod -from typing import TypeVar +from typing import Optional -from pypaimon.pynative.reader.core.record_iterator import RecordIterator +from pypaimon.api import Schema +from pypaimon.pynative.catalog.abstract_catalog import AbstractCatalog +from pypaimon.pynative.common.identifier import TableIdentifier -T = TypeVar('T') +class HiveCatalog(AbstractCatalog): + """Hive Catalog implementation for Paimon.""" -class FileRecordIterator(RecordIterator[T], ABC): - """ - A RecordIterator to support returning the record's row position and file Path. - """ + @staticmethod + def identifier() -> str: + return "hive" - @abstractmethod - def returned_position(self) -> int: - """ - Get the row position of the row returned by next(). - Returns: the row position from 0 to the number of rows in the file - """ + def create_database_impl(self, name: str, properties: Optional[dict] = None): + pass - @abstractmethod - def file_path(self) -> str: - """ - Returns: the file path - """ + def create_table_impl(self, table_identifier: TableIdentifier, schema: 'Schema'): + pass + + def get_table_schema(self, table_identifier: TableIdentifier): + pass diff --git a/pypaimon/pynative/common/core_option.py b/pypaimon/pynative/common/core_option.py new file mode 100644 index 0000000..bb47eff --- /dev/null +++ b/pypaimon/pynative/common/core_option.py @@ -0,0 +1,143 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from enum import Enum + + +class CoreOptions(str, Enum): + """Core options for paimon.""" + + def __str__(self): + return self.value + + # Basic options + AUTO_CREATE = "auto-create" + PATH = "path" + TYPE = "type" + BRANCH = "branch" + BUCKET = "bucket" + BUCKET_KEY = "bucket-key" + + # File format options + FILE_FORMAT = "file.format" + FILE_FORMAT_ORC = "orc" + FILE_FORMAT_AVRO = "avro" + FILE_FORMAT_PARQUET = "parquet" + FILE_COMPRESSION = "file.compression" + FILE_COMPRESSION_PER_LEVEL = "file.compression.per.level" + FILE_FORMAT_PER_LEVEL = "file.format.per.level" + FILE_BLOCK_SIZE = "file.block-size" + + # File index options + FILE_INDEX = "file-index" + FILE_INDEX_IN_MANIFEST_THRESHOLD = "file-index.in-manifest-threshold" + FILE_INDEX_READ_ENABLED = "file-index.read.enabled" + + # Manifest options + MANIFEST_FORMAT = "manifest.format" + MANIFEST_COMPRESSION = "manifest.compression" + MANIFEST_TARGET_FILE_SIZE = "manifest.target-file-size" + + # Sort options + SORT_SPILL_THRESHOLD = "sort-spill-threshold" + SORT_SPILL_BUFFER_SIZE = "sort-spill-buffer-size" + SPILL_COMPRESSION = "spill-compression" + SPILL_COMPRESSION_ZSTD_LEVEL = "spill-compression.zstd-level" + + # Write options + WRITE_ONLY = "write-only" + TARGET_FILE_SIZE = "target-file-size" + WRITE_BUFFER_SIZE = "write-buffer-size" + + # Level options + NUM_LEVELS = "num-levels" + + # Commit options + COMMIT_FORCE_COMPACT = "commit.force-compact" + COMMIT_TIMEOUT = "commit.timeout" + COMMIT_MAX_RETRIES = "commit.max-retries" + + # Compaction options + COMPACTION_MAX_SIZE_AMPLIFICATION_PERCENT = "compaction.max-size-amplification-percent" + + # Field options + DEFAULT_VALUE_SUFFIX = "default-value" + FIELDS_PREFIX = "fields" + FIELDS_SEPARATOR = "," + + # Aggregate options + AGG_FUNCTION = "aggregate-function" + DEFAULT_AGG_FUNCTION = "default-aggregate-function" + + # Other options + IGNORE_RETRACT = "ignore-retract" + NESTED_KEY = "nested-key" + DISTINCT = "distinct" + LIST_AGG_DELIMITER = "list-agg-delimiter" + COLUMNS = "columns" + + # Row kind options + ROWKIND_FIELD = "rowkind.field" + + # Scan options + SCAN_MODE = "scan.mode" + SCAN_TIMESTAMP = "scan.timestamp" + SCAN_TIMESTAMP_MILLIS = "scan.timestamp-millis" + SCAN_WATERMARK = "scan.watermark" + SCAN_FILE_CREATION_TIME_MILLIS = "scan.file-creation-time-millis" + SCAN_SNAPSHOT_ID = "scan.snapshot-id" + SCAN_TAG_NAME = "scan.tag-name" + SCAN_VERSION = "scan.version" + SCAN_BOUNDED_WATERMARK = "scan.bounded.watermark" + SCAN_MANIFEST_PARALLELISM = "scan.manifest.parallelism" + SCAN_FALLBACK_BRANCH = "scan.fallback-branch" + SCAN_MAX_SPLITS_PER_TASK = "scan.max-splits-per-task" + SCAN_PLAN_SORT_PARTITION = "scan.plan.sort-partition" + + # Startup mode options + INCREMENTAL_BETWEEN = "incremental-between" + INCREMENTAL_BETWEEN_TIMESTAMP = "incremental-between-timestamp" + + # Stream scan mode options + STREAM_SCAN_MODE = "stream-scan-mode" + + # Consumer options + CONSUMER_ID = "consumer-id" + CONSUMER_IGNORE_PROGRESS = "consumer-ignore-progress" + + # Changelog options + CHANGELOG_PRODUCER = "changelog-producer" + CHANGELOG_PRODUCER_ROW_DEDUPLICATE = "changelog-producer.row-deduplicate" + CHANGELOG_PRODUCER_ROW_DEDUPLICATE_IGNORE_FIELDS = "changelog-producer.row-deduplicate-ignore-fields" + CHANGELOG_LIFECYCLE_DECOUPLED = "changelog-lifecycle-decoupled" + + # Merge engine options + MERGE_ENGINE = "merge-engine" + IGNORE_DELETE = "ignore-delete" + PARTIAL_UPDATE_REMOVE_RECORD_ON_DELETE = "partial-update.remove-record-on-delete" + PARTIAL_UPDATE_REMOVE_RECORD_ON_SEQUENCE_GROUP = "partial-update.remove-record-on-sequence-group" + + # Lookup options + FORCE_LOOKUP = "force-lookup" + LOOKUP_WAIT = "lookup-wait" + + # Delete file options + DELETE_FILE_THREAD_NUM = "delete-file.thread-num" + + # Commit user options + COMMIT_USER_PREFIX = "commit.user-prefix" diff --git a/pypaimon/pynative/common/data_field.py b/pypaimon/pynative/common/data_field.py new file mode 100644 index 0000000..ab75586 --- /dev/null +++ b/pypaimon/pynative/common/data_field.py @@ -0,0 +1,71 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from dataclasses import dataclass +from typing import Optional + + +class DataType: + def __init__(self, type_name: str, nullable: bool = True): + self.type_name = type_name + self.nullable = nullable + + @classmethod + def from_string(cls, type_str: str) -> 'DataType': + parts = type_str.split() + type_name = parts[0].upper() + nullable = "NOT NULL" not in type_str.upper() + return cls(type_name, nullable) + + def __str__(self) -> str: + result = self.type_name + if not self.nullable: + result += " NOT NULL" + return result + + def __eq__(self, other): + if not isinstance(other, DataType): + return False + return self.type_name == other.type_name and self.nullable == other.nullable + + +@dataclass +class DataField: + id: int + name: str + type: DataType + description: Optional[str] = None + + @classmethod + def from_dict(cls, data: dict) -> 'DataField': + return cls( + id=data["id"], + name=data["name"], + type=DataType.from_string(data["type"]), + description=data.get("description") + ) + + def to_dict(self) -> dict: + result = { + "id": self.id, + "name": self.name, + "type": str(self.type) + } + if self.description is not None: + result["description"] = self.description + return result diff --git a/pypaimon/pynative/common/exception.py b/pypaimon/pynative/common/exception.py index 9f37729..d034fd1 100644 --- a/pypaimon/pynative/common/exception.py +++ b/pypaimon/pynative/common/exception.py @@ -16,6 +16,9 @@ # limitations under the License. ################################################################################ + class PyNativeNotImplementedError(NotImplementedError): - """ Method or function hasn't been implemented by py-native paimon yet. """ - pass + """ Method or property hasn't been implemented by py-native paimon yet. """ + + def __init__(self, name): + super().__init__(f"Feature '{name}' hasn't been implemented by PyNative Paimon.") diff --git a/pypaimon/pynative/common/file_io.py b/pypaimon/pynative/common/file_io.py new file mode 100644 index 0000000..6eb8c60 --- /dev/null +++ b/pypaimon/pynative/common/file_io.py @@ -0,0 +1,355 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import os +import logging +import subprocess +from pathlib import Path +from typing import Optional, List, Dict, Any +from urllib.parse import urlparse, splitport + +import pyarrow.fs +import pyarrow as pa +from pyarrow._fs import FileSystem + +from pypaimon.pynative.common.exception import PyNativeNotImplementedError +from pypaimon.pynative.common.core_option import CoreOptions + +S3_ENDPOINT = "s3.endpoint" +S3_ACCESS_KEY_ID = "s3.access-key" +S3_SECRET_ACCESS_KEY = "s3.secret-key" +S3_SESSION_TOKEN = "s3.session.token" +S3_REGION = "s3.region" +S3_PROXY_URI = "s3.proxy.uri" +S3_CONNECT_TIMEOUT = "s3.connect.timeout" +S3_REQUEST_TIMEOUT = "s3.request.timeout" +S3_ROLE_ARN = "s3.role.arn" +S3_ROLE_SESSION_NAME = "s3.role.session.name" +S3_FORCE_VIRTUAL_ADDRESSING = "s3.force.virtual.addressing" + +AWS_ROLE_ARN = "aws.role.arn" +AWS_ROLE_SESSION_NAME = "aws.role.session.name" + + +class FileIO: + def __init__(self, warehouse: str, catalog_options: dict): + self.properties = catalog_options + self.logger = logging.getLogger(__name__) + scheme, netloc, path = self.parse_location(warehouse) + if scheme in {"oss"}: + self.filesystem = self._initialize_oss_fs() + elif scheme in {"s3", "s3a", "s3n"}: + self.filesystem = self._initialize_s3_fs() + elif scheme in {"hdfs", "viewfs"}: + self.filesystem = self._initialize_hdfs_fs(scheme, netloc) + elif scheme in {"file"}: + self.filesystem = self._initialize_local_fs() + else: + raise ValueError(f"Unrecognized filesystem type in URI: {scheme}") + + @staticmethod + def parse_location(location: str): + uri = urlparse(location) + if not uri.scheme: + return "file", uri.netloc, os.path.abspath(location) + elif uri.scheme in ("hdfs", "viewfs"): + return uri.scheme, uri.netloc, uri.path + else: + return uri.scheme, uri.netloc, f"{uri.netloc}{uri.path}" + + def _initialize_oss_fs(self) -> FileSystem: + from pyarrow.fs import S3FileSystem + + client_kwargs = { + "endpoint_override": self.properties.get(S3_ENDPOINT), + "access_key": self.properties.get(S3_ACCESS_KEY_ID), + "secret_key": self.properties.get(S3_SECRET_ACCESS_KEY), + "session_token": self.properties.get(S3_SESSION_TOKEN), + "region": self.properties.get(S3_REGION), + "force_virtual_addressing": self.properties.get(S3_FORCE_VIRTUAL_ADDRESSING, True), + } + + if proxy_uri := self.properties.get(S3_PROXY_URI): + client_kwargs["proxy_options"] = proxy_uri + if connect_timeout := self.properties.get(S3_CONNECT_TIMEOUT): + client_kwargs["connect_timeout"] = float(connect_timeout) + if request_timeout := self.properties.get(S3_REQUEST_TIMEOUT): + client_kwargs["request_timeout"] = float(request_timeout) + if role_arn := self.properties.get(S3_ROLE_ARN): + client_kwargs["role_arn"] = role_arn + if session_name := self.properties.get(S3_ROLE_SESSION_NAME): + client_kwargs["session_name"] = session_name + + return S3FileSystem(**client_kwargs) + + def _initialize_s3_fs(self) -> FileSystem: + from pyarrow.fs import S3FileSystem + + client_kwargs = { + "endpoint_override": self.properties.get(S3_ENDPOINT), + "access_key": self.properties.get(S3_ACCESS_KEY_ID), + "secret_key": self.properties.get(S3_SECRET_ACCESS_KEY), + "session_token": self.properties.get(S3_SESSION_TOKEN), + "region": self.properties.get(S3_REGION), + } + + if proxy_uri := self.properties.get(S3_PROXY_URI): + client_kwargs["proxy_options"] = proxy_uri + if connect_timeout := self.properties.get(S3_CONNECT_TIMEOUT): + client_kwargs["connect_timeout"] = float(connect_timeout) + if request_timeout := self.properties.get(S3_REQUEST_TIMEOUT): + client_kwargs["request_timeout"] = float(request_timeout) + if role_arn := self.properties.get(S3_ROLE_ARN, AWS_ROLE_ARN): + client_kwargs["role_arn"] = role_arn + if session_name := self.properties.get(S3_ROLE_SESSION_NAME, AWS_ROLE_SESSION_NAME): + client_kwargs["session_name"] = session_name + + client_kwargs["force_virtual_addressing"] = self.properties.get(S3_FORCE_VIRTUAL_ADDRESSING, False) + return S3FileSystem(**client_kwargs) + + def _initialize_hdfs_fs(self, scheme: str, netloc: Optional[str]) -> FileSystem: + from pyarrow.fs import HadoopFileSystem + + if 'HADOOP_HOME' not in os.environ: + raise RuntimeError("HADOOP_HOME environment variable is not set.") + if 'HADOOP_CONF_DIR' not in os.environ: + raise RuntimeError("HADOOP_CONF_DIR environment variable is not set.") + + hadoop_home = os.environ.get("HADOOP_HOME") + native_lib_path = f"{hadoop_home}/lib/native" + os.environ['LD_LIBRARY_PATH'] = f"{native_lib_path}:{os.environ.get('LD_LIBRARY_PATH', '')}" + + class_paths = subprocess.run( + [f'{hadoop_home}/bin/hadoop', 'classpath', '--glob'], + capture_output=True, + text=True, + check=True + ) + os.environ['CLASSPATH'] = class_paths.stdout.strip() + + host, port_str = splitport(netloc) + return HadoopFileSystem( + host=host, + port=int(port_str), + user=os.environ.get('HADOOP_USER_NAME', 'hadoop') + ) + + def _initialize_local_fs(self) -> FileSystem: + from pyarrow.fs import LocalFileSystem + + return LocalFileSystem() + + def new_input_stream(self, path: Path): + return self.filesystem.open_input_file(str(path)) + + def new_output_stream(self, path: Path): + parent_dir = path.parent + if str(parent_dir) and not self.exists(parent_dir): + self.mkdirs(parent_dir) + + return self.filesystem.open_output_stream(str(path)) + + def get_file_status(self, path: Path): + file_infos = self.filesystem.get_file_info([str(path)]) + return file_infos[0] + + def list_status(self, path: Path): + selector = pyarrow.fs.FileSelector(str(path), recursive=False, allow_not_found=True) + return self.filesystem.get_file_info(selector) + + def list_directories(self, path: Path): + file_infos = self.list_status(path) + return [info for info in file_infos if info.type == pyarrow.fs.FileType.Directory] + + def exists(self, path: Path) -> bool: + try: + file_info = self.filesystem.get_file_info([str(path)])[0] + return file_info.type != pyarrow.fs.FileType.NotFound + except Exception: + return False + + def delete(self, path: Path, recursive: bool = False) -> bool: + try: + file_info = self.filesystem.get_file_info([str(path)])[0] + if file_info.type == pyarrow.fs.FileType.Directory: + if recursive: + self.filesystem.delete_dir_contents(str(path)) + else: + self.filesystem.delete_dir(str(path)) + else: + self.filesystem.delete_file(str(path)) + return True + except Exception as e: + self.logger.warning(f"Failed to delete {path}: {e}") + return False + + def mkdirs(self, path: Path) -> bool: + try: + self.filesystem.create_dir(str(path), recursive=True) + return True + except Exception as e: + self.logger.warning(f"Failed to create directory {path}: {e}") + return False + + def rename(self, src: Path, dst: Path) -> bool: + try: + dst_parent = dst.parent + if str(dst_parent) and not self.exists(dst_parent): + self.mkdirs(dst_parent) + + self.filesystem.move(str(src), str(dst)) + return True + except Exception as e: + self.logger.warning(f"Failed to rename {src} to {dst}: {e}") + return False + + def delete_quietly(self, path: Path): + if self.logger.isEnabledFor(logging.DEBUG): + self.logger.debug(f"Ready to delete {path}") + + try: + if not self.delete(path, False) and self.exists(path): + self.logger.warning(f"Failed to delete file {path}") + except Exception: + self.logger.warning(f"Exception occurs when deleting file {path}", exc_info=True) + + def delete_files_quietly(self, files: List[Path]): + for file_path in files: + self.delete_quietly(file_path) + + def delete_directory_quietly(self, directory: Path): + if self.logger.isEnabledFor(logging.DEBUG): + self.logger.debug(f"Ready to delete {directory}") + + try: + if not self.delete(directory, True) and self.exists(directory): + self.logger.warning(f"Failed to delete directory {directory}") + except Exception: + self.logger.warning(f"Exception occurs when deleting directory {directory}", exc_info=True) + + def get_file_size(self, path: Path) -> int: + file_info = self.get_file_status(path) + if file_info.size is None: + raise ValueError(f"File size not available for {path}") + return file_info.size + + def is_dir(self, path: Path) -> bool: + file_info = self.get_file_status(path) + return file_info.type == pyarrow.fs.FileType.Directory + + def check_or_mkdirs(self, path: Path): + if self.exists(path): + if not self.is_dir(path): + raise ValueError(f"The path '{path}' should be a directory.") + else: + self.mkdirs(path) + + def read_file_utf8(self, path: Path) -> str: + with self.new_input_stream(path) as input_stream: + return input_stream.read().decode('utf-8') + + def try_to_write_atomic(self, path: Path, content: str) -> bool: + temp_path = path.with_suffix(path.suffix + ".tmp") if path.suffix else Path(str(path) + ".tmp") + success = False + try: + self.write_file(temp_path, content, False) + success = self.rename(temp_path, path) + finally: + if not success: + self.delete_quietly(temp_path) + return success + + def write_file(self, path: Path, content: str, overwrite: bool = False): + with self.new_output_stream(path) as output_stream: + output_stream.write(content.encode('utf-8')) + + def overwrite_file_utf8(self, path: Path, content: str): + with self.new_output_stream(path) as output_stream: + output_stream.write(content.encode('utf-8')) + + def copy_file(self, source_path: Path, target_path: Path, overwrite: bool = False): + if not overwrite and self.exists(target_path): + raise FileExistsError(f"Target file {target_path} already exists and overwrite=False") + + self.filesystem.copy_file(str(source_path), str(target_path)) + + def copy_files(self, source_directory: Path, target_directory: Path, overwrite: bool = False): + file_infos = self.list_status(source_directory) + for file_info in file_infos: + if file_info.type == pyarrow.fs.FileType.File: + source_file = Path(file_info.path) + target_file = target_directory / source_file.name + self.copy_file(source_file, target_file, overwrite) + + def read_overwritten_file_utf8(self, path: Path) -> Optional[str]: + retry_number = 0 + exception = None + while retry_number < 5: + try: + return self.read_file_utf8(path) + except FileNotFoundError: + return None + except Exception as e: + if not self.exists(path): + return None + + if (str(type(e).__name__).endswith("RemoteFileChangedException") or + (str(e) and "Blocklist for" in str(e) and "has changed" in str(e))): + exception = e + retry_number += 1 + else: + raise e + + if exception: + if isinstance(exception, Exception): + raise exception + else: + raise RuntimeError(exception) + + return None + + def write_parquet(self, path: Path, data: pa.RecordBatch, compression: str = 'snappy', **kwargs): + try: + import pyarrow.parquet as pq + + with self.new_output_stream(path) as output_stream: + with pq.ParquetWriter(output_stream, data.schema, compression=compression, **kwargs) as pw: + pw.write_batch(data) + + except Exception as e: + self.delete_quietly(path) + raise RuntimeError(f"Failed to write Parquet file {path}: {e}") from e + + def write_orc(self, path: Path, data: pa.RecordBatch, compression: str = 'zstd', **kwargs): + try: + import pyarrow.orc as orc + + with self.new_output_stream(path) as output_stream: + orc.write_table( + data, + output_stream, + compression=compression, + **kwargs + ) + + except Exception as e: + self.delete_quietly(path) + raise RuntimeError(f"Failed to write ORC file {path}: {e}") from e + + def write_avro(self, path: Path, table: pa.RecordBatch, schema: Optional[Dict[str, Any]] = None, **kwargs): + raise PyNativeNotImplementedError(CoreOptions.FILE_FORMAT_AVRO) diff --git a/pypaimon/pynative/common/identifier.py b/pypaimon/pynative/common/identifier.py new file mode 100644 index 0000000..586493c --- /dev/null +++ b/pypaimon/pynative/common/identifier.py @@ -0,0 +1,71 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from pypaimon.pynative.common.exception import PyNativeNotImplementedError + +SYSTEM_TABLE_SPLITTER = '$' +SYSTEM_BRANCH_PREFIX = 'branch-' + + +class TableIdentifier: + + def __init__(self, full_name: str): + self._full_name = full_name + self._system_table = None + self._branch = None + + parts = full_name.split('.') + if len(parts) == 2: + self._database = parts[0] + self._object = parts[1] + else: + raise ValueError(f"Cannot get splits from '{full_name}' to get database and object") + + splits = self._object.split(SYSTEM_TABLE_SPLITTER) + if len(splits) == 1: + self._table = self._object + elif len(splits) == 2: + self._table = splits[0] + if splits[1].startswith(SYSTEM_BRANCH_PREFIX): + self._branch = splits[1][len(SYSTEM_BRANCH_PREFIX):] + else: + self._system_table = splits[1] + elif len(splits) == 3: + if not splits[1].startswith(SYSTEM_BRANCH_PREFIX): + raise ValueError(f"System table can only contain one '{SYSTEM_TABLE_SPLITTER}' separator, " + f"but this is: {self._object}") + self._table = splits[0] + self._branch = splits[1][len(SYSTEM_BRANCH_PREFIX):] + self._system_table = splits[2] + else: + raise ValueError(f"Invalid object name: {self._object}") + + if self._system_table is not None: + raise PyNativeNotImplementedError("SystemTable") + + elif self._branch is not None: + raise PyNativeNotImplementedError("BranchTable") + + def get_database_name(self): + return self._database + + def get_table_name(self): + return self._table + + def get_full_name(self): + return self._full_name diff --git a/pypaimon/pynative/common/predicate.py b/pypaimon/pynative/common/predicate.py index cadff46..3bb6a43 100644 --- a/pypaimon/pynative/common/predicate.py +++ b/pypaimon/pynative/common/predicate.py @@ -19,23 +19,18 @@ from dataclasses import dataclass from typing import Any, List, Optional -from pypaimon.pynative.common.row.internal_row import InternalRow +from pypaimon.api import Predicate +from pypaimon.pynative.row.internal_row import InternalRow @dataclass -class PyNativePredicate: +class PredicateImpl(Predicate): method: str - index: int - field: str + index: Optional[int] + field: str | None literals: Optional[List[Any]] = None def test(self, record: InternalRow) -> bool: - """ - # Test whether the record satisfies the predicate condition. - """ - if not hasattr(record, 'get_field'): - raise ValueError("Record must have get_field method") - if self.method == 'equal': return record.get_field(self.index) == self.literals[0] elif self.method == 'notEqual': diff --git a/pypaimon/pynative/common/predicate_builder.py b/pypaimon/pynative/common/predicate_builder.py new file mode 100644 index 0000000..e8a8669 --- /dev/null +++ b/pypaimon/pynative/common/predicate_builder.py @@ -0,0 +1,121 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import List, Any, Optional + +from pypaimon.api import PredicateBuilder +from pypaimon.pynative.common.data_field import DataField +from pypaimon.pynative.common.predicate import PredicateImpl + + +class PredicateBuilderImpl(PredicateBuilder): + """Implementation of PredicateBuilder using PredicateImpl.""" + + def __init__(self, row_field: List[DataField]): + self.field_names = [field.name for field in row_field] + + def _get_field_index(self, field: str) -> int: + """Get the index of a field in the schema.""" + try: + return self.field_names.index(field) + except ValueError: + raise ValueError(f'The field {field} is not in field list {self.field_names}.') + + def _build_predicate(self, method: str, field: str, literals: Optional[List[Any]] = None) -> PredicateImpl: + """Build a predicate with the given method, field, and literals.""" + index = self._get_field_index(field) + return PredicateImpl( + method=method, + index=index, + field=field, + literals=literals + ) + + def equal(self, field: str, literal: Any) -> PredicateImpl: + """Create an equality predicate.""" + return self._build_predicate('equal', field, [literal]) + + def not_equal(self, field: str, literal: Any) -> PredicateImpl: + """Create a not-equal predicate.""" + return self._build_predicate('notEqual', field, [literal]) + + def less_than(self, field: str, literal: Any) -> PredicateImpl: + """Create a less-than predicate.""" + return self._build_predicate('lessThan', field, [literal]) + + def less_or_equal(self, field: str, literal: Any) -> PredicateImpl: + """Create a less-or-equal predicate.""" + return self._build_predicate('lessOrEqual', field, [literal]) + + def greater_than(self, field: str, literal: Any) -> PredicateImpl: + """Create a greater-than predicate.""" + return self._build_predicate('greaterThan', field, [literal]) + + def greater_or_equal(self, field: str, literal: Any) -> PredicateImpl: + """Create a greater-or-equal predicate.""" + return self._build_predicate('greaterOrEqual', field, [literal]) + + def is_null(self, field: str) -> PredicateImpl: + """Create an is-null predicate.""" + return self._build_predicate('isNull', field) + + def is_not_null(self, field: str) -> PredicateImpl: + """Create an is-not-null predicate.""" + return self._build_predicate('isNotNull', field) + + def startswith(self, field: str, pattern_literal: Any) -> PredicateImpl: + """Create a starts-with predicate.""" + return self._build_predicate('startsWith', field, [pattern_literal]) + + def endswith(self, field: str, pattern_literal: Any) -> PredicateImpl: + """Create an ends-with predicate.""" + return self._build_predicate('endsWith', field, [pattern_literal]) + + def contains(self, field: str, pattern_literal: Any) -> PredicateImpl: + """Create a contains predicate.""" + return self._build_predicate('contains', field, [pattern_literal]) + + def is_in(self, field: str, literals: List[Any]) -> PredicateImpl: + """Create an in predicate.""" + return self._build_predicate('in', field, literals) + + def is_not_in(self, field: str, literals: List[Any]) -> PredicateImpl: + """Create a not-in predicate.""" + return self._build_predicate('notIn', field, literals) + + def between(self, field: str, included_lower_bound: Any, included_upper_bound: Any) -> PredicateImpl: + """Create a between predicate.""" + return self._build_predicate('between', field, [included_lower_bound, included_upper_bound]) + + def and_predicates(self, predicates: List[PredicateImpl]) -> PredicateImpl: + """Create an AND predicate from multiple predicates.""" + return PredicateImpl( + method='and', + index=None, + field=None, + literals=predicates + ) + + def or_predicates(self, predicates: List[PredicateImpl]) -> PredicateImpl: + """Create an OR predicate from multiple predicates.""" + return PredicateImpl( + method='or', + index=None, + field=None, + literals=predicates + ) diff --git a/pypaimon/pynative/common/row/columnar_row.py b/pypaimon/pynative/common/row/columnar_row.py deleted file mode 100644 index 244539d..0000000 --- a/pypaimon/pynative/common/row/columnar_row.py +++ /dev/null @@ -1,63 +0,0 @@ -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -from typing import Any - -import pyarrow as pa - -from pypaimon.pynative.common.row.internal_row import InternalRow -from pypaimon.pynative.common.row.key_value import RowKind - - -class ColumnarRow(InternalRow): - """ - Columnar row to support access to vector column data. It is a row based on PyArrow RecordBatch - """ - - def __init__(self, record_batch: pa.RecordBatch, row_id: int = 0, - row_kind: RowKind = RowKind.INSERT): - self._batch = record_batch - self._row_id = row_id - self._row_kind = row_kind - - def get_row_id(self) -> int: - return self._row_id - - def set_row_id(self, row_id: int) -> None: - self._row_id = row_id - - def batch(self) -> pa.RecordBatch: - return self._batch - - def get_field(self, pos: int) -> Any: - return self._batch.column(pos)[self._row_id].as_py() - - def is_null_at(self, pos: int) -> bool: - return self._batch.column(pos).is_null(self._row_id) - - def set_field(self, pos: int, value: Any) -> None: - raise NotImplementedError() - - def get_row_kind(self) -> RowKind: - return self._row_kind - - def set_row_kind(self, kind: RowKind) -> None: - self._row_kind = kind - - def __len__(self) -> int: - return self._batch.num_columns diff --git a/pypaimon/pynative/reader/__init__.py b/pypaimon/pynative/read/__init__.py similarity index 100% rename from pypaimon/pynative/reader/__init__.py rename to pypaimon/pynative/read/__init__.py diff --git a/pypaimon/pynative/read/interval_partition.py b/pypaimon/pynative/read/interval_partition.py new file mode 100644 index 0000000..0c38598 --- /dev/null +++ b/pypaimon/pynative/read/interval_partition.py @@ -0,0 +1,130 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import heapq +from dataclasses import dataclass +from functools import cmp_to_key +from typing import List, Callable + +from pypaimon.pynative.row.binary_row import BinaryRow +from pypaimon.pynative.table.data_file_meta import DataFileMeta + + +@dataclass +class SortedRun: + """ + A SortedRun is a list of files sorted by their keys. + The key intervals [minKey, maxKey] of these files do not overlap. + """ + files: List[DataFileMeta] + + +class IntervalPartition: + """ + Algorithm to partition several data files into the minimum number of SortedRuns. + """ + + def __init__(self, input_files: List[DataFileMeta]): + self.files = input_files.copy() + self.key_comparator = default_key_comparator + self.files.sort(key=cmp_to_key(self._compare_files)) + + def partition(self) -> List[List[SortedRun]]: + result = [] + section: List[DataFileMeta] = [] + bound = None + + for meta in self.files: + if section and self.key_comparator(meta.min_key, bound) > 0: + result.append(self._partition_section(section)) + section.clear() + bound = None + section.append(meta) + if bound is None or self.key_comparator(meta.max_key, bound) > 0: + bound = meta.max_key + + if section: + result.append(self._partition_section(section)) + return result + + def _partition_section(self, metas: List[DataFileMeta]) -> List[SortedRun]: + heap: List[HeapRun] = [] + first_run = [metas[0]] + heapq.heappush(heap, HeapRun(first_run, self.key_comparator)) + for i in range(1, len(metas)): + meta = metas[i] + + earliest_finishing_run = heap[0] + last_max_key = earliest_finishing_run.run[-1].max_key + if self.key_comparator(meta.min_key, last_max_key) > 0: + top = heapq.heappop(heap) + top.run.append(meta) + heapq.heappush(heap, top) + else: + new_run = [meta] + heapq.heappush(heap, HeapRun(new_run, self.key_comparator)) + + return [SortedRun(files=h.run) for h in heap] + + def _compare_files(self, f1: DataFileMeta, f2: DataFileMeta) -> int: + min_key_cmp = self.key_comparator(f1.min_key, f2.min_key) + if min_key_cmp != 0: + return min_key_cmp + return self.key_comparator(f1.max_key, f2.max_key) + + +@dataclass +class HeapRun: + run: List[DataFileMeta] + comparator: Callable[[BinaryRow, BinaryRow], int] + + def __lt__(self, other) -> bool: + my_last_max = self.run[-1].max_key + other_last_max = other.run[-1].max_key + return self.comparator(my_last_max, other_last_max) < 0 + + +def default_key_comparator(key1: BinaryRow, key2: BinaryRow) -> int: + if not key1 or not key1.values: + if not key2 or not key2.values: + return 0 + return -1 + if not key2 or not key2.values: + return 1 + + min_field_count = min(len(key1.values), len(key2.values)) + for i in range(min_field_count): + val1 = key1.values[i] + val2 = key2.values[i] + if val1 is None and val2 is None: + continue + if val1 is None: + return -1 + if val2 is None: + return 1 + if val1 < val2: + return -1 + elif val1 > val2: + return 1 + + if len(key1.values) < len(key2.values): + return -1 + elif len(key1.values) > len(key2.values): + return 1 + else: + return 0 diff --git a/pypaimon/pynative/read/partition_info.py b/pypaimon/pynative/read/partition_info.py new file mode 100644 index 0000000..40a4340 --- /dev/null +++ b/pypaimon/pynative/read/partition_info.py @@ -0,0 +1,46 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import Any, List + +from pypaimon.pynative.common.data_field import DataField +from pypaimon.pynative.row.binary_row import BinaryRow + + +class PartitionInfo: + """ + Partition information about how the row mapping of outer row. + """ + + def __init__(self, mapping: List[int], partition: BinaryRow): + self.mapping = mapping + self.partition_values = partition.values + self.partition_fields = partition.fields + + def size(self) -> int: + return len(self.mapping) - 1 + + def is_partition_row(self, pos: int) -> bool: + return self.mapping[pos] < 0 + + def get_real_index(self, pos: int) -> int: + return abs(self.mapping[pos]) - 1 + + def get_partition_value(self, pos: int) -> (Any, DataField): + real_index = self.get_real_index(pos) + return self.partition_values[real_index], self.partition_fields[real_index] diff --git a/pypaimon/pynative/read/read_builder_impl.py b/pypaimon/pynative/read/read_builder_impl.py new file mode 100644 index 0000000..b8d7c62 --- /dev/null +++ b/pypaimon/pynative/read/read_builder_impl.py @@ -0,0 +1,77 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import List, Optional + +from pypaimon.api import ReadBuilder, PredicateBuilder, TableRead, TableScan, Predicate +from pypaimon.pynative.common.data_field import DataField +from pypaimon.pynative.common.predicate import PredicateImpl +from pypaimon.pynative.common.predicate_builder import PredicateBuilderImpl +from pypaimon.pynative.read.table_scan_impl import TableScanImpl +from pypaimon.pynative.read.table_read_impl import TableReadImpl + + +class ReadBuilderImpl(ReadBuilder): + """Implementation of ReadBuilder for native Python reading.""" + + def __init__(self, table): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self._predicate: Optional[Predicate] = None + self._projection: Optional[List[str]] = None + self._limit: Optional[int] = None + + def with_filter(self, predicate: PredicateImpl) -> 'ReadBuilder': + self._predicate = predicate + return self + + def with_projection(self, projection: List[str]) -> 'ReadBuilder': + self._projection = projection + return self + + def with_limit(self, limit: int) -> 'ReadBuilder': + self._limit = limit + return self + + def new_scan(self) -> TableScan: + return TableScanImpl( + table=self.table, + predicate=self._predicate, + limit=self._limit, + read_type=self.read_type() + ) + + def new_read(self) -> TableRead: + return TableReadImpl( + table=self.table, + predicate=self._predicate, + read_type=self.read_type() + ) + + def new_predicate_builder(self) -> PredicateBuilder: + return PredicateBuilderImpl(self.read_type()) + + def read_type(self) -> List[DataField]: + table_fields = self.table.fields + + if not self._projection: + return table_fields + else: + field_map = {field.name: field for field in self.table.fields} + return [field_map[name] for name in self._projection if name in field_map] diff --git a/pypaimon/pynative/reader/core/__init__.py b/pypaimon/pynative/read/reader/__init__.py similarity index 100% rename from pypaimon/pynative/reader/core/__init__.py rename to pypaimon/pynative/read/reader/__init__.py diff --git a/pypaimon/pynative/read/reader/concat_batch_reader.py b/pypaimon/pynative/read/reader/concat_batch_reader.py new file mode 100644 index 0000000..5133fe8 --- /dev/null +++ b/pypaimon/pynative/read/reader/concat_batch_reader.py @@ -0,0 +1,51 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import collections +from typing import Optional, List, Callable + +from pyarrow import RecordBatch + +from pypaimon.pynative.read.reader.iface.record_batch_reader import RecordBatchReader + + +class ConcatBatchReader(RecordBatchReader): + + def __init__(self, reader_suppliers: List[Callable]): + self.queue = collections.deque(reader_suppliers) + self.current_reader: Optional[RecordBatchReader] = None + + def read_arrow_batch(self) -> Optional[RecordBatch]: + while True: + if self.current_reader is not None: + batch = self.current_reader.read_arrow_batch() + if batch is not None: + return batch + self.current_reader.close() + self.current_reader = None + elif self.queue: + supplier = self.queue.popleft() + self.current_reader = supplier() + else: + return None + + def close(self) -> None: + if self.current_reader: + self.current_reader.close() + self.current_reader = None + self.queue.clear() diff --git a/pypaimon/pynative/reader/concat_record_reader.py b/pypaimon/pynative/read/reader/concat_record_reader.py similarity index 50% rename from pypaimon/pynative/reader/concat_record_reader.py rename to pypaimon/pynative/read/reader/concat_record_reader.py index ccbffab..63becce 100644 --- a/pypaimon/pynative/reader/concat_record_reader.py +++ b/pypaimon/pynative/read/reader/concat_record_reader.py @@ -16,42 +16,35 @@ # limitations under the License. ################################################################################ -from typing import Optional +import collections +from typing import Optional, List, Callable -from py4j.java_gateway import JavaObject - -from pypaimon.pynative.reader.core.record_iterator import RecordIterator -from pypaimon.pynative.reader.core.record_reader import RecordReader +from pypaimon.pynative.read.reader.iface.record_iterator import RecordIterator +from pypaimon.pynative.read.reader.iface.record_reader import RecordReader class ConcatRecordReader(RecordReader): - """ - This reader is to concatenate a list of RecordReaders and read them sequentially. - The input list is already sorted by key and sequence number, and the key intervals do not - overlap each other. - """ - def __init__(self, converter, j_supplier_queue: JavaObject): - self.converter = converter - self.j_supplier_queue = j_supplier_queue - self.current: Optional[RecordReader] = None + def __init__(self, reader_suppliers: List[Callable]): + self.queue = collections.deque(reader_suppliers) + self.current_reader: Optional[RecordReader] = None def read_batch(self) -> Optional[RecordIterator]: while True: - if self.current is not None: - iterator = self.current.read_batch() + if self.current_reader is not None: + iterator = self.current_reader.read_batch() if iterator is not None: return iterator - self.current.close() - self.current = None - elif not self.j_supplier_queue.isEmpty(): - # If the Java supplier queue is not empty, initialize the reader by using py4j - j_supplier = self.j_supplier_queue.poll() - j_reader = j_supplier.get() - self.current = self.converter.convert_java_reader(j_reader) + self.current_reader.close() + self.current_reader = None + elif self.queue: + supplier = self.queue.popleft() + self.current_reader = supplier() else: return None def close(self) -> None: - if self.current is not None: - self.current.close() + if self.current_reader: + self.current_reader.close() + self.current_reader = None + self.queue.clear() diff --git a/pypaimon/pynative/read/reader/data_file_record_reader.py b/pypaimon/pynative/read/reader/data_file_record_reader.py new file mode 100644 index 0000000..0c536b7 --- /dev/null +++ b/pypaimon/pynative/read/reader/data_file_record_reader.py @@ -0,0 +1,90 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import Optional, List +import pyarrow as pa +from pyarrow import RecordBatch + +from pypaimon.pynative.read.partition_info import PartitionInfo +from pypaimon.pynative.read.reader.iface.record_batch_reader import RecordBatchReader + + +class DataFileBatchReader(RecordBatchReader): + """ + Reads record batch from data files. + """ + + def __init__(self, format_reader: RecordBatchReader, index_mapping: List[int], partition_info: PartitionInfo, + system_primary_key: Optional[List[str]]): + self.format_reader = format_reader + self.index_mapping = index_mapping + self.partition_info = partition_info + self.system_primary_key = system_primary_key + + def read_arrow_batch(self) -> Optional[RecordBatch]: + record_batch = self.format_reader.read_arrow_batch() + if record_batch is None: + return None + + if self.partition_info is None and self.index_mapping is None: + return record_batch + + inter_arrays = [] + inter_names = [] + num_rows = record_batch.num_rows + + if self.partition_info is not None: + for i in range(self.partition_info.size()): + if self.partition_info.is_partition_row(i): + partition_value, partition_field = self.partition_info.get_partition_value(i) + const_array = pa.repeat(partition_value, num_rows) + inter_arrays.append(const_array) + inter_names.append(partition_field.name) + else: + real_index = self.partition_info.get_real_index(i) + if real_index < record_batch.num_columns: + inter_arrays.append(record_batch.column(real_index)) + inter_names.append(record_batch.schema.field(real_index).name) + else: + inter_arrays = record_batch.columns + inter_names = record_batch.schema.names + + if self.index_mapping is not None: + mapped_arrays = [] + mapped_names = [] + for i, real_index in enumerate(self.index_mapping): + if 0 <= real_index < len(inter_arrays): + mapped_arrays.append(inter_arrays[real_index]) + mapped_names.append(inter_names[real_index]) + else: + null_array = pa.nulls(num_rows) + mapped_arrays.append(null_array) + mapped_names.append(f"null_col_{i}") + + if self.system_primary_key: + for i in range(len(self.system_primary_key)): + if not mapped_names[i].startswith("_KEY_"): + mapped_names[i] = f"_KEY_{mapped_names[i]}" + + inter_arrays = mapped_arrays + inter_names = mapped_names + + return pa.RecordBatch.from_arrays(inter_arrays, names=inter_names) + + def close(self) -> None: + self.format_reader.close() diff --git a/pypaimon/pynative/reader/drop_delete_reader.py b/pypaimon/pynative/read/reader/drop_delete_reader.py similarity index 77% rename from pypaimon/pynative/reader/drop_delete_reader.py rename to pypaimon/pynative/read/reader/drop_delete_reader.py index ccb70e0..9d203d4 100644 --- a/pypaimon/pynative/reader/drop_delete_reader.py +++ b/pypaimon/pynative/read/reader/drop_delete_reader.py @@ -18,28 +18,28 @@ from typing import Optional -from pypaimon.pynative.common.row.key_value import KeyValue -from pypaimon.pynative.reader.core.record_iterator import RecordIterator -from pypaimon.pynative.reader.core.record_reader import RecordReader +from pypaimon.pynative.row.key_value import KeyValue +from pypaimon.pynative.read.reader.iface.record_iterator import RecordIterator +from pypaimon.pynative.read.reader.iface.record_reader import RecordReader -class DropDeleteReader(RecordReader): +class DropDeleteRecordReader(RecordReader): """ A RecordReader which drops KeyValue that does not meet RowKind. isAdd from the wrapped reader. """ - def __init__(self, wrapped_reader: RecordReader[KeyValue]): - self.wrapped_reader = wrapped_reader + def __init__(self, kv_reader: RecordReader[KeyValue]): + self.kv_reader = kv_reader def read_batch(self) -> Optional[RecordIterator]: - batch = self.wrapped_reader.read_batch() + batch = self.kv_reader.read_batch() if batch is None: return None return DropDeleteIterator(batch) def close(self) -> None: - self.wrapped_reader.close() + self.kv_reader.close() class DropDeleteIterator(RecordIterator[KeyValue]): @@ -57,6 +57,3 @@ def next(self) -> Optional[KeyValue]: return None if kv.is_add(): return kv - - def release_batch(self) -> None: - self.batch.release_batch() diff --git a/pypaimon/pynative/reader/empty_record_reader.py b/pypaimon/pynative/read/reader/empty_record_reader.py similarity index 85% rename from pypaimon/pynative/reader/empty_record_reader.py rename to pypaimon/pynative/read/reader/empty_record_reader.py index 9883cb8..7874f95 100644 --- a/pypaimon/pynative/reader/empty_record_reader.py +++ b/pypaimon/pynative/read/reader/empty_record_reader.py @@ -18,11 +18,11 @@ from typing import Optional -from pypaimon.pynative.reader.core.file_record_reader import FileRecordReader -from pypaimon.pynative.reader.core.record_iterator import RecordIterator +from pypaimon.pynative.read.reader.iface.record_iterator import RecordIterator +from pypaimon.pynative.read.reader.iface.record_reader import RecordReader -class EmptyFileRecordReader(FileRecordReader): +class EmptyFileRecordReader(RecordReader): """ An empty FileRecordReader. """ diff --git a/pypaimon/pynative/reader/filter_record_reader.py b/pypaimon/pynative/read/reader/filter_record_reader.py similarity index 72% rename from pypaimon/pynative/reader/filter_record_reader.py rename to pypaimon/pynative/read/reader/filter_record_reader.py index ef57829..28a41dd 100644 --- a/pypaimon/pynative/reader/filter_record_reader.py +++ b/pypaimon/pynative/read/reader/filter_record_reader.py @@ -16,24 +16,23 @@ # limitations under the License. ################################################################################ -from typing import Optional, TypeVar +from typing import Optional -from pypaimon import Predicate -from pypaimon.pynative.reader.core.record_reader import RecordIterator, RecordReader +from pypaimon.pynative.common.predicate import PredicateImpl +from pypaimon.pynative.read.reader.iface.record_reader import RecordIterator, RecordReader +from pypaimon.pynative.row.internal_row import InternalRow -T = TypeVar('T') - -class FilterRecordReader(RecordReader[T]): +class FilterRecordReader(RecordReader[InternalRow]): """ A RecordReader that implements filtering functionality. """ - def __init__(self, reader: RecordReader[T], predicate: Predicate): + def __init__(self, reader: RecordReader[InternalRow], predicate: PredicateImpl): self.reader = reader self.predicate = predicate - def read_batch(self) -> Optional[RecordIterator[T]]: + def read_batch(self) -> Optional[RecordIterator[InternalRow]]: iterator = self.reader.read_batch() if iterator is None: return None @@ -43,22 +42,19 @@ def close(self) -> None: self.reader.close() -class FilterRecordIterator(RecordIterator[T]): +class FilterRecordIterator(RecordIterator[InternalRow]): """ A RecordIterator that implements filtering functionality. """ - def __init__(self, iterator: RecordIterator[T], predicate: Predicate): + def __init__(self, iterator: RecordIterator[InternalRow], predicate: PredicateImpl): self.iterator = iterator self.predicate = predicate - def next(self) -> Optional[T]: + def next(self) -> Optional[InternalRow]: while True: record = self.iterator.next() if record is None: return None if self.predicate.test(record): return record - - def release_batch(self) -> None: - self.iterator.release_batch() diff --git a/pypaimon/pynative/read/reader/format_avro_reader.py b/pypaimon/pynative/read/reader/format_avro_reader.py new file mode 100644 index 0000000..d145045 --- /dev/null +++ b/pypaimon/pynative/read/reader/format_avro_reader.py @@ -0,0 +1,71 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import List, Optional + +import fastavro +import pyarrow as pa +from pyarrow import RecordBatch + +from pypaimon.api import Predicate +from pypaimon.pynative.common.file_io import FileIO +from pypaimon.pynative.read.reader.iface.record_batch_reader import RecordBatchReader + + +class FormatAvroReader(RecordBatchReader): + """ + An ArrowBatchReader for reading Avro files using fastavro, filters records based on the + provided predicate and projection, and converts Avro records to RecordBatch format. + """ + + def __init__(self, file_io: FileIO, file_path: str, primary_keys: List[str], + fields: List[str], predicate: Predicate, batch_size: int = 4096): + self._file = file_io.filesystem.open_input_file(file_path) + self._reader = fastavro.reader(self._file) + + self._primary_keys = primary_keys + self._avro_schema = self._reader.writer_schema + self._predicate = predicate + self._batch_size = batch_size + self._fields = fields + + def read_arrow_batch(self) -> Optional[RecordBatch]: + num_columns = len(self._fields) + columns_data = [[] for _ in range(num_columns)] + records_in_batch = 0 + + for record in self._reader: + # TODO: converter from record to IternalRow, to call predicate.test + # if self._predicate and not self._predicate.test(record): + # continue + for i, col_name in enumerate(self._fields): + value = record.get(col_name) + columns_data[i].append(value) + + records_in_batch += 1 + if records_in_batch >= self._batch_size: + break + + if records_in_batch == 0: + return None + return pa.RecordBatch.from_arrays(columns_data, names=self._fields) + + def close(self): + if self._file: + self._file.close() + self._file = None diff --git a/pypaimon/pynative/reader/pyarrow_dataset_reader.py b/pypaimon/pynative/read/reader/format_pyarrow_reader.py similarity index 53% rename from pypaimon/pynative/reader/pyarrow_dataset_reader.py rename to pypaimon/pynative/read/reader/format_pyarrow_reader.py index 07ed9f7..c3bc8a1 100644 --- a/pypaimon/pynative/reader/pyarrow_dataset_reader.py +++ b/pypaimon/pynative/read/reader/format_pyarrow_reader.py @@ -19,53 +19,45 @@ from typing import Optional, List import pyarrow.dataset as ds +from pyarrow import RecordBatch + +from pypaimon.api import Predicate +from pypaimon.pynative.common.file_io import FileIO +from pypaimon.pynative.read.reader.iface.record_batch_reader import RecordBatchReader -from pypaimon import Predicate -from pypaimon.pynative.common.row.internal_row import InternalRow -from pypaimon.pynative.reader.core.columnar_row_iterator import ColumnarRowIterator -from pypaimon.pynative.reader.core.file_record_iterator import FileRecordIterator -from pypaimon.pynative.reader.core.file_record_reader import FileRecordReader from pypaimon.pynative.util.predicate_converter import convert_predicate -class PyArrowDatasetReader(FileRecordReader[InternalRow]): +class FormatPyArrowReader(RecordBatchReader): """ - A PyArrowDatasetReader that reads data from a dataset file using PyArrow, + A Format Reader that reads record batch from a Parquet or ORC file using PyArrow, and filters it based on the provided predicate and projection. """ - def __init__(self, format, file_path, batch_size, projection, - predicate: Predicate, primary_keys: List[str], fields: List[str]): + def __init__(self, file_io: FileIO, file_format: str, file_path: str, primary_keys: List[str], + fields: List[str], predicate: Predicate, batch_size: int = 4096): - if primary_keys is not None: + if primary_keys: # TODO: utilize predicate to improve performance predicate = None if predicate is not None: predicate = convert_predicate(predicate) - self._file_path = file_path - self.dataset = ds.dataset(file_path, format=format) - self.scanner = self.dataset.scanner( + self.dataset = ds.dataset(file_path, format=file_format, filesystem=file_io.filesystem) + self.reader = self.dataset.scanner( columns=fields, filter=predicate, batch_size=batch_size - ) - self.batch_iterator = self.scanner.to_batches() + ).to_reader() - def read_batch(self) -> Optional[FileRecordIterator[InternalRow]]: + def read_arrow_batch(self) -> Optional[RecordBatch]: try: - record_batch = next(self.batch_iterator, None) - if record_batch is None: - return None - - return ColumnarRowIterator( - self._file_path, - record_batch - ) - except Exception as e: - print(f"Error reading batch: {e}") - raise + return self.reader.read_next_batch() + except StopIteration: + return None def close(self): - pass + if self.reader is not None: + self.reader.close() + self.reader = None diff --git a/pypaimon/pynative/writer/__init__.py b/pypaimon/pynative/read/reader/iface/__init__.py similarity index 100% rename from pypaimon/pynative/writer/__init__.py rename to pypaimon/pynative/read/reader/iface/__init__.py diff --git a/pypaimon/pynative/read/reader/iface/record_batch_reader.py b/pypaimon/pynative/read/reader/iface/record_batch_reader.py new file mode 100644 index 0000000..5ec8229 --- /dev/null +++ b/pypaimon/pynative/read/reader/iface/record_batch_reader.py @@ -0,0 +1,74 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from abc import abstractmethod +from typing import Optional, TypeVar, Iterator + +import polars +from polars import DataFrame +from pyarrow import RecordBatch + +from pypaimon.pynative.read.reader.iface.record_iterator import RecordIterator +from pypaimon.pynative.read.reader.iface.record_reader import RecordReader +from pypaimon.pynative.row.internal_row import InternalRow +from pypaimon.pynative.row.offset_row import OffsetRow + +T = TypeVar('T') + + +class RecordBatchReader(RecordReader): + """ + The reader that reads the pyarrow batches of records. + """ + + @abstractmethod + def read_arrow_batch(self) -> Optional[RecordBatch]: + """ + Reads one batch. The method should return null when reaching the end of the input. + """ + + def _read_next_df(self) -> Optional[DataFrame]: + arrow_batch = self.read_arrow_batch() + if arrow_batch is None: + return None + return polars.from_arrow(arrow_batch) + + def tuple_iterator(self) -> Optional[Iterator[tuple]]: + df = self._read_next_df() + if df is None: + return None + return df.iter_rows() + + def read_batch(self) -> Optional[RecordIterator[InternalRow]]: + df = self._read_next_df() + if df is None: + return None + return InternalRowWrapperIterator(df.iter_rows(), df.width) + + +class InternalRowWrapperIterator(RecordIterator[InternalRow]): + def __init__(self, iterator: Iterator[tuple], width: int): + self._iterator = iterator + self._reused_row = OffsetRow(None, 0, width) + + def next(self) -> Optional[InternalRow]: + row_tuple = next(self._iterator, None) + if row_tuple is None: + return None + self._reused_row.replace(row_tuple) + return self._reused_row diff --git a/pypaimon/pynative/reader/core/record_iterator.py b/pypaimon/pynative/read/reader/iface/record_iterator.py similarity index 91% rename from pypaimon/pynative/reader/core/record_iterator.py rename to pypaimon/pynative/read/reader/iface/record_iterator.py index 4d3712c..6684d8f 100644 --- a/pypaimon/pynative/reader/core/record_iterator.py +++ b/pypaimon/pynative/read/reader/iface/record_iterator.py @@ -32,9 +32,3 @@ def next(self) -> Optional[T]: """ Gets the next record from the iterator. Returns null if this iterator has no more elements. """ - - @abstractmethod - def release_batch(self): - """ - Releases the batch that this iterator iterated over. - """ diff --git a/pypaimon/pynative/reader/core/record_reader.py b/pypaimon/pynative/read/reader/iface/record_reader.py similarity index 90% rename from pypaimon/pynative/reader/core/record_reader.py rename to pypaimon/pynative/read/reader/iface/record_reader.py index f7226fa..d93e480 100644 --- a/pypaimon/pynative/reader/core/record_reader.py +++ b/pypaimon/pynative/read/reader/iface/record_reader.py @@ -19,14 +19,14 @@ from abc import ABC, abstractmethod from typing import Generic, Optional, TypeVar -from pypaimon.pynative.reader.core.record_iterator import RecordIterator +from pypaimon.pynative.read.reader.iface.record_iterator import RecordIterator T = TypeVar('T') class RecordReader(Generic[T], ABC): """ - The reader that reads the batches of records. + The reader that reads the batches of records as RecordIterator. """ @abstractmethod diff --git a/pypaimon/pynative/reader/key_value_unwrap_reader.py b/pypaimon/pynative/read/reader/key_value_unwrap_reader.py similarity index 59% rename from pypaimon/pynative/reader/key_value_unwrap_reader.py rename to pypaimon/pynative/read/reader/key_value_unwrap_reader.py index 9add03e..f236d69 100644 --- a/pypaimon/pynative/reader/key_value_unwrap_reader.py +++ b/pypaimon/pynative/read/reader/key_value_unwrap_reader.py @@ -18,31 +18,31 @@ from typing import Any, Optional -from pypaimon.pynative.common.row.internal_row import InternalRow -from pypaimon.pynative.common.row.key_value import KeyValue -from pypaimon.pynative.common.row.row_kind import RowKind -from pypaimon.pynative.reader.core.record_iterator import RecordIterator -from pypaimon.pynative.reader.core.record_reader import RecordReader +from pypaimon.pynative.row.internal_row import InternalRow +from pypaimon.pynative.row.key_value import KeyValue +from pypaimon.pynative.read.reader.iface.record_iterator import RecordIterator +from pypaimon.pynative.read.reader.iface.record_reader import RecordReader +from pypaimon.pynative.row.offset_row import OffsetRow -class KeyValueUnwrapReader(RecordReader[InternalRow]): +class KeyValueUnwrapRecordReader(RecordReader[InternalRow]): """ A RecordReader that converts a KeyValue type record reader into an InternalRow type reader Corresponds to the KeyValueTableRead$1 in Java version. """ - def __init__(self, wrapped_reader: RecordReader[KeyValue]): - self.wrapped_reader = wrapped_reader + def __init__(self, kv_reader: RecordReader[KeyValue]): + self.kv_reader = kv_reader def read_batch(self) -> Optional[RecordIterator[InternalRow]]: - batch = self.wrapped_reader.read_batch() + batch = self.kv_reader.read_batch() if batch is None: return None return KeyValueUnwrapIterator(batch) def close(self) -> None: - self.wrapped_reader.close() + self.kv_reader.close() class KeyValueUnwrapIterator(RecordIterator[InternalRow]): @@ -52,23 +52,12 @@ class KeyValueUnwrapIterator(RecordIterator[InternalRow]): def __init__(self, batch: RecordIterator[KeyValue]): self.batch = batch - self.kv: KeyValue = None - self.pre_value_row_kind: RowKind = None def next(self) -> Optional[Any]: - # The row_data is reused in iterator, we should set back to real kind - if self.kv is not None: - self.kv.value.set_row_kind(self.pre_value_row_kind) - - self.kv = self.batch.next() - if self.kv is None: + kv = self.batch.next() + if kv is None: return None - row_data = self.kv.value - self.pre_value_row_kind = row_data.get_row_kind() - - row_data.set_row_kind(self.kv.value_kind) - return row_data - - def release_batch(self) -> None: - self.batch.release_batch() + row: OffsetRow = kv.value + row.set_row_kind_byte(kv.value_row_kind_byte) + return row diff --git a/pypaimon/pynative/read/reader/key_value_wrap_reader.py b/pypaimon/pynative/read/reader/key_value_wrap_reader.py new file mode 100644 index 0000000..a779d4d --- /dev/null +++ b/pypaimon/pynative/read/reader/key_value_wrap_reader.py @@ -0,0 +1,67 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import Optional, Iterator + +from pypaimon.pynative.read.reader.iface.record_batch_reader import RecordBatchReader +from pypaimon.pynative.read.reader.iface.record_iterator import RecordIterator +from pypaimon.pynative.read.reader.iface.record_reader import RecordReader +from pypaimon.pynative.row.key_value import KeyValue + + +class KeyValueWrapReader(RecordReader[KeyValue]): + """ + RecordReader for reading KeyValue data files. + Corresponds to the KeyValueDataFileRecordReader in Java version. + """ + + def __init__(self, data_reader: RecordBatchReader, key_arity, value_arity): + self.data_reader = data_reader + self.key_arity = key_arity + self.value_arity = value_arity + self.reused_kv = KeyValue(self.key_arity, self.value_arity) + + def read_batch(self) -> Optional[RecordIterator[KeyValue]]: + iterator = self.data_reader.tuple_iterator() + if iterator is None: + return None + return KeyValueWrapIterator(iterator, self.reused_kv) + + def close(self): + self.data_reader.close() + + +class KeyValueWrapIterator(RecordIterator[KeyValue]): + """ + An Iterator that converts an PrimaryKey InternalRow into a KeyValue + """ + + def __init__( + self, + iterator: Iterator, + reused_kv: KeyValue + ): + self.iterator = iterator + self.reused_kv = reused_kv + + def next(self) -> Optional[KeyValue]: + row_tuple = next(self.iterator, None) + if row_tuple is None: + return None + self.reused_kv.replace(row_tuple) + return self.reused_kv diff --git a/pypaimon/pynative/read/reader/sort_merge_reader.py b/pypaimon/pynative/read/reader/sort_merge_reader.py new file mode 100644 index 0000000..14688e1 --- /dev/null +++ b/pypaimon/pynative/read/reader/sort_merge_reader.py @@ -0,0 +1,225 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import heapq +from typing import Any, Callable, List, Optional + +import pyarrow as pa + +from pypaimon.api import Schema +from pypaimon.pynative.common.exception import PyNativeNotImplementedError +from pypaimon.pynative.read.reader.iface.record_iterator import RecordIterator +from pypaimon.pynative.read.reader.iface.record_reader import RecordReader +from pypaimon.pynative.row.internal_row import InternalRow +from pypaimon.pynative.row.key_value import KeyValue + + +class SortMergeReaderWithMinHeap(RecordReader): + """SortMergeReader implemented with min-heap.""" + + def __init__(self, readers: List[RecordReader[KeyValue]], schema: Schema): + self.next_batch_readers = list(readers) + self.merge_function = DeduplicateMergeFunction() + + if schema.partition_keys: + trimmed_primary_keys = [pk for pk in schema.primary_keys if pk not in schema.partition_keys] + if not trimmed_primary_keys: + raise ValueError(f"Primary key constraint {schema.primary_keys} same with partition fields") + else: + trimmed_primary_keys = schema.primary_keys + field_map = {field.name: field for field in schema.pa_schema} + key_schema = [field_map[name] for name in trimmed_primary_keys if name in field_map] + self.key_comparator = built_key_comparator(key_schema) + + self.min_heap = [] + self.polled = [] + + def read_batch(self) -> Optional[RecordIterator]: + for reader in self.next_batch_readers: + while True: + iterator = reader.read_batch() + if iterator is None: + reader.close() + break + + kv = iterator.next() + if kv is not None: + element = Element(kv, iterator, reader) + entry = HeapEntry(kv.key, element, self.key_comparator) + heapq.heappush(self.min_heap, entry) + break + + self.next_batch_readers.clear() + + if not self.min_heap: + return None + + return SortMergeIterator( + self, + self.polled, + self.min_heap, + self.merge_function, + self.key_comparator, + ) + + def close(self): + for reader in self.next_batch_readers: + reader.close() + + for entry in self.min_heap: + entry.element.reader.close() + + for element in self.polled: + element.reader.close() + + +class SortMergeIterator(RecordIterator): + def __init__(self, reader, polled: List['Element'], min_heap, merge_function, + key_comparator): + self.reader = reader + self.polled = polled + self.min_heap = min_heap + self.merge_function = merge_function + self.key_comparator = key_comparator + self.released = False + + def next(self): + while True: + if not self._next_impl(): + return None + result = self.merge_function.get_result() + if result is not None: + return result + + def _next_impl(self): + for element in self.polled: + if element.update(): + entry = HeapEntry(element.kv.key, element, self.key_comparator) + heapq.heappush(self.min_heap, entry) + self.polled.clear() + + if not self.min_heap: + return False + + self.merge_function.reset() + key = self.min_heap[0].key + while self.min_heap and self.key_comparator(key, self.min_heap[0].key) == 0: + entry = heapq.heappop(self.min_heap) + self.merge_function.add(entry.element.kv) + self.polled.append(entry.element) + + return True + + +class DeduplicateMergeFunction: + """A MergeFunction where key is primary key (unique) and value is the full record, only keep the latest one.""" + + def __init__(self): + self.latest_kv = None + + def reset(self) -> None: + self.latest_kv = None + + def add(self, kv: KeyValue): + self.latest_kv = kv + + def get_result(self) -> Optional[KeyValue]: + return self.latest_kv + + +class Element: + def __init__(self, kv: KeyValue, iterator: RecordIterator[KeyValue], reader: RecordReader[KeyValue]): + self.kv = kv + self.iterator = iterator + self.reader = reader + + def update(self) -> bool: + next_kv = self.iterator.next() + if next_kv is not None: + self.kv = next_kv + return True + + self.iterator = self.reader.read_batch() + if self.iterator is None: + self.reader.close() + return False + + next_kv_from_new_batch = self.iterator.next() + self.kv = next_kv_from_new_batch + return True + + +class HeapEntry: + def __init__(self, key: InternalRow, element: Element, key_comparator): + self.key = key + self.element = element + self.key_comparator = key_comparator + + def __lt__(self, other): + result = self.key_comparator(self.key, other.key) + if result < 0: + return True + elif result > 0: + return False + + return self.element.kv.sequence_number < other.element.kv.sequence_number + + +def built_key_comparator(key_schema: List) -> Callable[[Any, Any], int]: + def is_comparable_type(t): + return ( + pa.types.is_integer(t) or + pa.types.is_floating(t) or + pa.types.is_boolean(t) or + pa.types.is_string(t) or + pa.types.is_binary(t) or + pa.types.is_timestamp(t) or + pa.types.is_date(t) + ) + + # Precompute comparability flags to avoid repeated type checks + comparable_flags = [is_comparable_type(field.type) for field in key_schema] + + def comparator(key1: InternalRow, key2: InternalRow) -> int: + if key1 is None and key2 is None: + return 0 + if key1 is None: + return -1 + if key2 is None: + return 1 + for i, comparable in enumerate(comparable_flags): + val1 = key1.get_field(i) + val2 = key2.get_field(i) + + if val1 is None and val2 is None: + continue + if val1 is None: + return -1 + if val2 is None: + return 1 + + if not comparable: + raise PyNativeNotImplementedError(f"{key_schema[i].type} comparison") + + if val1 < val2: + return -1 + elif val1 > val2: + return 1 + return 0 + + return comparator diff --git a/pypaimon/pynative/read/split_impl.py b/pypaimon/pynative/read/split_impl.py new file mode 100644 index 0000000..87d1111 --- /dev/null +++ b/pypaimon/pynative/read/split_impl.py @@ -0,0 +1,57 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from dataclasses import dataclass +from typing import List + +from pypaimon.api import Split, Plan +from pypaimon.pynative.row.binary_row import BinaryRow +from pypaimon.pynative.table.data_file_meta import DataFileMeta + + +@dataclass +class SplitImpl(Split): + """Implementation of Split for native Python reading.""" + files: List[DataFileMeta] + partition: BinaryRow + bucket: int + _file_paths: List[str] + _row_count: int + _file_size: int + raw_convertible: bool = False + + @property + def row_count(self) -> int: + return self._row_count + + @property + def file_size(self) -> int: + return self._file_size + + @property + def file_paths(self) -> List[str]: + return self._file_paths + + +@dataclass +class PlanImpl(Plan): + """Implementation of Plan for native Python reading.""" + _splits: List[Split] + + def splits(self) -> List[Split]: + return self._splits diff --git a/pypaimon/pynative/read/split_read.py b/pypaimon/pynative/read/split_read.py new file mode 100644 index 0000000..3ddac29 --- /dev/null +++ b/pypaimon/pynative/read/split_read.py @@ -0,0 +1,283 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import os +from abc import ABC, abstractmethod +from functools import partial +from typing import Optional, List + +from pypaimon.pynative.common.predicate import PredicateImpl +from pypaimon.pynative.read.interval_partition import SortedRun, IntervalPartition +from pypaimon.pynative.read.partition_info import PartitionInfo +from pypaimon.pynative.read.reader.concat_batch_reader import ConcatBatchReader +from pypaimon.pynative.read.reader.filter_record_reader import FilterRecordReader +from pypaimon.pynative.read.reader.format_avro_reader import FormatAvroReader +from pypaimon.pynative.read.reader.iface.record_batch_reader import RecordBatchReader +from pypaimon.pynative.read.reader.concat_record_reader import ConcatRecordReader +from pypaimon.pynative.read.reader.data_file_record_reader import DataFileBatchReader +from pypaimon.pynative.read.reader.drop_delete_reader import DropDeleteRecordReader +from pypaimon.pynative.read.reader.empty_record_reader import EmptyFileRecordReader +from pypaimon.pynative.read.reader.iface.record_reader import RecordReader +from pypaimon.pynative.read.reader.key_value_unwrap_reader import KeyValueUnwrapRecordReader +from pypaimon.pynative.read.reader.key_value_wrap_reader import KeyValueWrapReader +from pypaimon.pynative.read.reader.format_pyarrow_reader import FormatPyArrowReader +from pypaimon.pynative.read.reader.sort_merge_reader import SortMergeReaderWithMinHeap +from pypaimon.pynative.read.split_impl import SplitImpl +from pypaimon.pynative.common.data_field import DataField, DataType + +KEY_PREFIX = "_KEY_" +KEY_FIELD_ID_START = 1000000 +NULL_FIELD_INDEX = -1 + + +class SplitRead(ABC): + """Abstract base class for split reading operations.""" + + def __init__(self, table, predicate: Optional[PredicateImpl], read_type: List[DataField], split: SplitImpl): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.predicate = predicate + self.split = split + self.value_arity = len(read_type) + + self.trimmed_primary_key = [field.name for field in self.table.table_schema.get_trimmed_primary_key_fields()] + self.read_fields = read_type + if isinstance(self, MergeFileSplitRead): + self.read_fields = self._create_key_value_fields(read_type) + + @abstractmethod + def create_reader(self) -> RecordReader: + """Create a record reader for the given split.""" + + def file_reader_supplier(self, file_path: str, for_merge_read: bool): + _, extension = os.path.splitext(file_path) + file_format = extension[1:] + + format_reader: RecordBatchReader + if file_format == "avro": + format_reader = FormatAvroReader(self.table.file_io, file_path, self.table.primary_keys, + self._get_final_read_data_fields(), self.predicate) + elif file_format == "parquet" or file_format == "orc": + format_reader = FormatPyArrowReader(self.table.file_io, file_format, file_path, self.table.primary_keys, + self._get_final_read_data_fields(), self.predicate) + else: + raise ValueError(f"Unexpected file format: {file_format}") + + index_mapping = self.create_index_mapping() + partition_info = self.create_partition_info() + if for_merge_read: + return DataFileBatchReader(format_reader, index_mapping, partition_info, self.trimmed_primary_key) + else: + return DataFileBatchReader(format_reader, index_mapping, partition_info, None) + + @abstractmethod + def _get_all_data_fields(self): + """Get all data fields""" + + def _get_read_data_fields(self): + read_data_fields = [] + read_field_ids = {field.id for field in self.read_fields} + for data_field in self._get_all_data_fields(): + if data_field.id in read_field_ids: + read_data_fields.append(data_field) + return read_data_fields + + def _create_key_value_fields(self, value_field: List[DataField]): + all_fields: List[DataField] = self.table.fields + all_data_fields = [] + + for field in all_fields: + if field.name in self.trimmed_primary_key: + key_field_name = f"{KEY_PREFIX}{field.name}" + key_field_id = field.id + KEY_FIELD_ID_START + key_field = DataField(key_field_id, key_field_name, field.type) + all_data_fields.append(key_field) + + sequence_field = DataField(2147483646, "_SEQUENCE_NUMBER", DataType("BIGINT", nullable=False)) + all_data_fields.append(sequence_field) + value_kind_field = DataField(2147483645, "_VALUE_KIND", DataType("TINYINT", nullable=False)) + all_data_fields.append(value_kind_field) + + for field in value_field: + all_data_fields.append(field) + + return all_data_fields + + def create_index_mapping(self): + base_index_mapping = self._create_base_index_mapping(self.read_fields, self._get_read_data_fields()) + trimmed_key_mapping, _ = self._get_trimmed_fields(self._get_read_data_fields(), self._get_all_data_fields()) + if base_index_mapping is None: + mapping = trimmed_key_mapping + elif trimmed_key_mapping is None: + mapping = base_index_mapping + else: + combined = [0] * len(base_index_mapping) + for i in range(len(base_index_mapping)): + if base_index_mapping[i] < 0: + combined[i] = base_index_mapping[i] + else: + combined[i] = trimmed_key_mapping[base_index_mapping[i]] + mapping = combined + + if mapping is not None: + for i in range(len(mapping)): + if mapping[i] != i: + return mapping + + return None + + def _create_base_index_mapping(self, table_fields: List[DataField], data_fields: List[DataField]): + index_mapping = [0] * len(table_fields) + field_id_to_index = {field.id: i for i, field in enumerate(data_fields)} + + for i, table_field in enumerate(table_fields): + field_id = table_field.id + data_field_index = field_id_to_index.get(field_id) + if data_field_index is not None: + index_mapping[i] = data_field_index + else: + index_mapping[i] = NULL_FIELD_INDEX + + for i in range(len(index_mapping)): + if index_mapping[i] != i: + return index_mapping + + return None + + def _get_final_read_data_fields(self) -> List[str]: + _, trimmed_fields = self._get_trimmed_fields( + self._get_read_data_fields(), self._get_all_data_fields() + ) + return self._remove_partition_fields(trimmed_fields) + + def _remove_partition_fields(self, fields: List[DataField]) -> List[str]: + partition_keys = self.table.partition_keys + if not partition_keys: + return [field.name for field in fields] + + fields_without_partition = [] + for field in fields: + if field.name not in partition_keys: + fields_without_partition.append(field) + + return [field.name for field in fields_without_partition] + + def _get_trimmed_fields(self, read_data_fields: List[DataField], + all_data_fields: List[DataField]) -> tuple[List[int], List[DataField]]: + trimmed_mapping = [0] * len(read_data_fields) + trimmed_fields = [] + + field_id_to_field = {field.id: field for field in all_data_fields} + position_map = {} + for i, field in enumerate(read_data_fields): + is_key_field = field.name.startswith(KEY_PREFIX) + if is_key_field: + original_id = field.id - KEY_FIELD_ID_START + else: + original_id = field.id + original_field = field_id_to_field.get(original_id) + + if original_id in position_map: + trimmed_mapping[i] = position_map[original_id] + else: + position = len(trimmed_fields) + position_map[original_id] = position + trimmed_mapping[i] = position + if is_key_field: + trimmed_fields.append(original_field) + else: + trimmed_fields.append(field) + + return trimmed_mapping, trimmed_fields + + def create_partition_info(self): + if not self.table.partition_keys: + return None + partition_mapping = self._construct_partition_mapping() + if not partition_mapping: + return None + return PartitionInfo(partition_mapping, self.split.partition) + + def _construct_partition_mapping(self) -> List[int]: + _, trimmed_fields = self._get_trimmed_fields( + self._get_read_data_fields(), self._get_all_data_fields() + ) + partition_names = self.table.partition_keys + + mapping = [0] * (len(trimmed_fields) + 1) + p_count = 0 + + for i, field in enumerate(trimmed_fields): + if field.name in partition_names: + partition_index = partition_names.index(field.name) + mapping[i] = -(partition_index + 1) + p_count += 1 + else: + mapping[i] = (i - p_count) + 1 + + return mapping + + +class RawFileSplitRead(SplitRead): + + def create_reader(self) -> RecordReader: + data_readers = [] + for file_path in self.split.file_paths: + supplier = partial(self.file_reader_supplier, file_path=file_path, for_merge_read=False) + data_readers.append(supplier) + + if not data_readers: + return EmptyFileRecordReader() + # no need for filter, all predicates has pushed down + return ConcatBatchReader(data_readers) + + def _get_all_data_fields(self): + return self.table.fields + + +class MergeFileSplitRead(SplitRead): + def kv_reader_supplier(self, file_path): + reader_supplier = partial(self.file_reader_supplier, file_path=file_path, for_merge_read=True) + return KeyValueWrapReader(reader_supplier(), len(self.trimmed_primary_key), self.value_arity) + + def section_reader_supplier(self, section: List[SortedRun]): + readers = [] + for sorter_run in section: + data_readers = [] + for file in sorter_run.files: + supplier = partial(self.kv_reader_supplier, file.file_path) + data_readers.append(supplier) + readers.append(ConcatRecordReader(data_readers)) + return SortMergeReaderWithMinHeap(readers, self.table.table_schema.to_schema()) + + def create_reader(self) -> RecordReader: + section_readers = [] + sections = IntervalPartition(self.split.files).partition() + for section in sections: + supplier = partial(self.section_reader_supplier, section) + section_readers.append(supplier) + concat_reader = ConcatRecordReader(section_readers) + kv_unwrap_reader = KeyValueUnwrapRecordReader(DropDeleteRecordReader(concat_reader)) + if self.predicate: + return FilterRecordReader(kv_unwrap_reader, self.predicate) + else: + return kv_unwrap_reader + + def _get_all_data_fields(self): + return self._create_key_value_fields(self.table.fields) diff --git a/pypaimon/pynative/read/table_read_impl.py b/pypaimon/pynative/read/table_read_impl.py new file mode 100644 index 0000000..6850e93 --- /dev/null +++ b/pypaimon/pynative/read/table_read_impl.py @@ -0,0 +1,135 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import List, Optional, TYPE_CHECKING, Iterator +import pandas as pd +import pyarrow as pa + +from pypaimon.api import TableRead, Split +from pypaimon.pynative.common.data_field import DataField +from pypaimon.pynative.common.exception import PyNativeNotImplementedError +from pypaimon.pynative.common.predicate import PredicateImpl +from pypaimon.pynative.read.reader.iface.record_batch_reader import RecordBatchReader +from pypaimon.pynative.row.offset_row import OffsetRow +from pypaimon.pynative.read.split_impl import SplitImpl +from pypaimon.pynative.read.split_read import SplitRead, RawFileSplitRead, MergeFileSplitRead +from pypaimon.pynative.table import schema_util + +if TYPE_CHECKING: + import ray + from duckdb.duckdb import DuckDBPyConnection + + +class TableReadImpl(TableRead): + """Implementation of TableRead for native Python reading.""" + + def __init__(self, table, predicate: Optional[PredicateImpl], read_type: List[DataField]): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.predicate = predicate + self.read_type = read_type + + def to_iterator(self, splits: List[Split]) -> Iterator: + def _record_generator(): + for split in splits: + if not isinstance(split, SplitImpl): + raise TypeError(f"Expected SplitImpl, but got {type(split).__name__}") + reader = self._create_split_read(split).create_reader() + try: + for batch in iter(reader.read_batch, None): + yield from iter(batch.next, None) + finally: + reader.close() + + return _record_generator() + + def to_arrow(self, splits: List[Split]) -> pa.Table: + chunk_size = 65536 + schema = schema_util.convert_data_fields_to_pa_schema(self.read_type) + arrow_batches = [] + + for split in splits: + if not isinstance(split, SplitImpl): + raise TypeError(f"Expected SplitImpl, but got {type(split).__name__}") + reader = self._create_split_read(split).create_reader() + try: + if isinstance(reader, RecordBatchReader): + for batch in iter(reader.read_arrow_batch, None): + arrow_batches.append(batch) + else: + row_tuple_chunk = [] + for iterator in iter(reader.read_batch, None): + for row in iter(iterator.next, None): + if not isinstance(row, OffsetRow): + raise TypeError(f"Expected OffsetRow, but got {type(row).__name__}") + row_tuple_chunk.append(row.row_tuple[row.offset: row.offset + row.arity]) + + if len(row_tuple_chunk) >= chunk_size: + batch = convert_rows_to_arrow_batch(row_tuple_chunk, schema) + arrow_batches.append(batch) + row_tuple_chunk = [] + + if row_tuple_chunk: + batch = convert_rows_to_arrow_batch(row_tuple_chunk, schema) + arrow_batches.append(batch) + finally: + reader.close() + + if not arrow_batches: + return pa.Table.from_arrays([], schema=schema) + + unified_schema = pa.unify_schemas([b.schema for b in arrow_batches]) + casted_batches = [b.cast(target_schema=unified_schema) for b in arrow_batches] + return pa.Table.from_batches(casted_batches) + + def to_arrow_batch_reader(self, splits: List[Split]) -> pa.RecordBatchReader: + raise PyNativeNotImplementedError("to_arrow_batch_reader") + + def to_pandas(self, splits: List[Split]) -> pd.DataFrame: + arrow_table = self.to_arrow(splits) + return arrow_table.to_pandas() + + def to_duckdb(self, splits: List[Split], table_name: str, + connection: Optional["DuckDBPyConnection"] = None) -> "DuckDBPyConnection": + raise PyNativeNotImplementedError("to_duckdb") + + def to_ray(self, splits: List[Split]) -> "ray.data.dataset.Dataset": + raise PyNativeNotImplementedError("to_ray") + + def _create_split_read(self, split: SplitImpl) -> SplitRead: + if self.table.is_primary_key_table and not split.raw_convertible: + return MergeFileSplitRead( + table=self.table, + predicate=self.predicate, + read_type=self.read_type, + split=split + ) + else: + return RawFileSplitRead( + table=self.table, + predicate=self.predicate, + read_type=self.read_type, + split=split + ) + + +def convert_rows_to_arrow_batch(row_tuples: List[tuple], schema: pa.Schema) -> pa.RecordBatch: + columns_data = zip(*row_tuples) + pydict = {name: list(column) for name, column in zip(schema.names, columns_data)} + return pa.RecordBatch.from_pydict(pydict, schema=schema) diff --git a/pypaimon/pynative/read/table_scan_impl.py b/pypaimon/pynative/read/table_scan_impl.py new file mode 100644 index 0000000..0427e19 --- /dev/null +++ b/pypaimon/pynative/read/table_scan_impl.py @@ -0,0 +1,290 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +from collections import defaultdict +from typing import List, Optional, Callable + +from pypaimon.api import TableScan, Plan +from pypaimon.pynative.common.data_field import DataField +from pypaimon.pynative.common.predicate import PredicateImpl +from pypaimon.pynative.read.interval_partition import IntervalPartition, SortedRun +from pypaimon.pynative.read.split_impl import SplitImpl, PlanImpl +from pypaimon.pynative.table.data_file_meta import DataFileMeta +from pypaimon.pynative.table.manifest_entry import ManifestEntry +from pypaimon.pynative.table.snapshot_manager import SnapshotManager +from pypaimon.pynative.table.manifest_list_manager import ManifestListManager +from pypaimon.pynative.table.manifest_file_manager import ManifestFileManager + + +class TableScanImpl(TableScan): + """Implementation of TableScan for native Python reading.""" + + def __init__(self, table, predicate: Optional[PredicateImpl], limit: Optional[int], read_type: List[DataField]): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.predicate = predicate + self.predicate = predicate + self.limit = limit + self.read_type = read_type + + self.snapshot_manager = SnapshotManager(table) + self.manifest_list_manager = ManifestListManager(table) + self.manifest_file_manager = ManifestFileManager(table) + + self.partition_conditions = self._extract_partition_conditions() + self.target_split_size = 128 * 1024 * 1024 + self.open_file_cost = 4 * 1024 * 1024 + + def plan(self) -> Plan: + latest_snapshot = self.snapshot_manager.get_latest_snapshot() + if not latest_snapshot: + return PlanImpl([]) + manifest_files = self.manifest_list_manager.read_all_manifest_files(latest_snapshot) + + file_entries = [] + for manifest_file_path in manifest_files: + manifest_entries = self.manifest_file_manager.read(manifest_file_path) + for entry in manifest_entries: + if entry.kind == 0: + file_entries.append(entry) + + if self.predicate: + file_entries = self._filter_by_predicate(file_entries) + + partitioned_split = defaultdict(list) + for entry in file_entries: + partitioned_split[(tuple(entry.partition.values), entry.bucket)].append(entry) + + splits = [] + for key, values in partitioned_split.items(): + if self.table.is_primary_key_table: + splits += self._create_primary_key_splits(values) + else: + splits += self._create_append_only_splits(values) + + splits = self._apply_push_down_limit(splits) + + return PlanImpl(splits) + + def _apply_push_down_limit(self, splits: List[SplitImpl]) -> List[SplitImpl]: + if self.limit is None: + return splits + scanned_row_count = 0 + limited_splits = [] + + for split in splits: + if split.raw_convertible: + limited_splits.append(split) + scanned_row_count += split.row_count + if scanned_row_count >= self.limit: + return limited_splits + + return limited_splits + + def _filter_by_predicate(self, file_entries: List[ManifestEntry]) -> List[ManifestEntry]: + if not self.predicate: + return file_entries + + filtered_files = [] + for file_entry in file_entries: + if self.partition_conditions and not self._filter_by_partition(file_entry): + continue + if not self._filter_by_stats(file_entry): + continue + filtered_files.append(file_entry) + + return filtered_files + + def _filter_by_partition(self, file_entry: ManifestEntry) -> bool: + # TODO: refactor with a better solution + partition_dict = file_entry.partition.to_dict() + for field_name, condition in self.partition_conditions.items(): + partition_value = partition_dict[field_name] + if condition['op'] == '=': + if str(partition_value) != str(condition['value']): + return False + elif condition['op'] == 'in': + if str(partition_value) not in [str(v) for v in condition['values']]: + return False + elif condition['op'] == 'notIn': + if str(partition_value) in [str(v) for v in condition['values']]: + return False + elif condition['op'] == '>': + if partition_value <= condition['values']: + return False + elif condition['op'] == '>=': + if partition_value < condition['values']: + return False + elif condition['op'] == '<': + if partition_value >= condition['values']: + return False + elif condition['op'] == '<=': + if partition_value > condition['values']: + return False + return True + + def _filter_by_stats(self, file_entry: ManifestEntry) -> bool: + # TODO: real support for filtering by stat + return True + + def _extract_partition_conditions(self) -> dict: + if not self.predicate or not self.table.partition_keys: + return {} + + conditions = {} + self._extract_conditions_from_predicate(self.predicate, conditions, self.table.partition_keys) + return conditions + + def _extract_conditions_from_predicate(self, predicate: 'PredicateImpl', conditions: dict, + partition_keys: List[str]): + if predicate.method == 'and': + for sub_predicate in predicate.literals: + self._extract_conditions_from_predicate(sub_predicate, conditions, partition_keys) + return + elif predicate.method == 'or': + all_partition_conditions = True + for sub_predicate in predicate.literals: + if sub_predicate.field not in partition_keys: + all_partition_conditions = False + break + if all_partition_conditions: + for sub_predicate in predicate.literals: + self._extract_conditions_from_predicate(sub_predicate, conditions, partition_keys) + return + + if predicate.field in partition_keys: + if predicate.method == 'equal': + conditions[predicate.field] = { + 'op': '=', + 'value': predicate.literals[0] if predicate.literals else None + } + elif predicate.method == 'in': + conditions[predicate.field] = { + 'op': 'in', + 'values': predicate.literals if predicate.literals else [] + } + elif predicate.method == 'notIn': + conditions[predicate.field] = { + 'op': 'notIn', + 'values': predicate.literals if predicate.literals else [] + } + elif predicate.method == 'greaterThan': + conditions[predicate.field] = { + 'op': '>', + 'value': predicate.literals[0] if predicate.literals else None + } + elif predicate.method == 'greaterOrEqual': + conditions[predicate.field] = { + 'op': '>=', + 'value': predicate.literals[0] if predicate.literals else None + } + elif predicate.method == 'lessThan': + conditions[predicate.field] = { + 'op': '<', + 'value': predicate.literals[0] if predicate.literals else None + } + elif predicate.method == 'lessOrEqual': + conditions[predicate.field] = { + 'op': '<=', + 'value': predicate.literals[0] if predicate.literals else None + } + + def _create_append_only_splits(self, file_entries: List[ManifestEntry]) -> List['SplitImpl']: + if not file_entries: + return [] + + data_files: List[DataFileMeta] = [e.file for e in file_entries] + + def weight_func(f: DataFileMeta) -> int: + return max(f.file_size, self.open_file_cost) + + packed_files: List[List[DataFileMeta]] = _pack_for_ordered(data_files, weight_func, self.target_split_size) + return self._build_split_from_pack(packed_files, file_entries, False) + + def _create_primary_key_splits(self, file_entries: List[ManifestEntry]) -> List['SplitImpl']: + if not file_entries: + return [] + + data_files: List[DataFileMeta] = [e.file for e in file_entries] + partition_sort_runs: List[List[SortedRun]] = IntervalPartition(data_files).partition() + sections: List[List[DataFileMeta]] = [ + [file for s in sl for file in s.files] + for sl in partition_sort_runs + ] + + def weight_func(fl: List[DataFileMeta]) -> int: + return max(sum(f.file_size for f in fl), self.open_file_cost) + + packed_files: List[List[List[DataFileMeta]]] = _pack_for_ordered(sections, weight_func, self.target_split_size) + flatten_packed_files: List[List[DataFileMeta]] = [ + [file for sub_pack in pack for file in sub_pack] + for pack in packed_files + ] + return self._build_split_from_pack(flatten_packed_files, file_entries, True) + + def _build_split_from_pack(self, packed_files, file_entries, for_primary_key_split: bool) -> List['SplitImpl']: + splits = [] + for file_group in packed_files: + raw_convertible = True + if for_primary_key_split: + raw_convertible = len(file_group) == 1 + + file_paths = [] + total_file_size = 0 + total_record_count = 0 + + for data_file in file_group: + data_file.set_file_path(self.table.table_path, file_entries[0].partition, + file_entries[0].bucket) + file_paths.append(data_file.file_path) + total_file_size += data_file.file_size + total_record_count += data_file.row_count + + if file_paths: + split = SplitImpl( + files=file_group, + partition=file_entries[0].partition, + bucket=file_entries[0].bucket, + _file_paths=file_paths, + _row_count=total_record_count, + _file_size=total_file_size, + raw_convertible=raw_convertible + ) + splits.append(split) + return splits + + +def _pack_for_ordered(items: List, weight_func: Callable, target_weight: int) -> List[List]: + packed = [] + bin_items = [] + bin_weight = 0 + + for item in items: + weight = weight_func(item) + if bin_weight + weight > target_weight and len(bin_items) > 0: + packed.append(bin_items) + bin_items.clear() + bin_weight = 0 + + bin_weight += weight + bin_items.append(item) + + if len(bin_items) > 0: + packed.append(bin_items) + + return packed diff --git a/pypaimon/pynative/reader/avro_format_reader.py b/pypaimon/pynative/reader/avro_format_reader.py deleted file mode 100644 index 6852516..0000000 --- a/pypaimon/pynative/reader/avro_format_reader.py +++ /dev/null @@ -1,83 +0,0 @@ -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -from typing import Any, Dict, List, Optional - -import fastavro -import pyarrow as pa - -from pypaimon.pynative.common.row.internal_row import InternalRow -from pypaimon.pynative.reader.core.columnar_row_iterator import ColumnarRowIterator -from pypaimon.pynative.reader.core.file_record_iterator import FileRecordIterator -from pypaimon.pynative.reader.core.file_record_reader import FileRecordReader - - -class AvroFormatReader(FileRecordReader[InternalRow]): - """ - A RecordReader implementation for reading Avro files using fastavro. - The reader converts Avro records to pyarrow.RecordBatch format, which is compatible with - the ColumnarRowIterator. - """ - - def __init__(self, file_path: str, batch_size: int, projected_type: Optional[List[str]] = None): - self._file_path = file_path - self._batch_size = batch_size - self._projected_type = projected_type - - self._reader = fastavro.reader(open(file_path, 'rb')) - self._schema = self._reader.schema - self._current_batch: List[Dict[str, Any]] = [] - - def read_batch(self) -> Optional[FileRecordIterator[InternalRow]]: - try: - self._current_batch = [] - for _ in range(self._batch_size): - try: - record = next(self._reader) - self._current_batch.append(record) - except StopIteration: - break - - if not self._current_batch: - return None - - # TODO: Temporarily converting results to pyarrow RecordBatch, reusing its logic. - # TODO: Custom adjustments will follow later. - record_batch = self._convert_to_record_batch(self._current_batch) - if record_batch is None: - return None - - return ColumnarRowIterator( - self._file_path, - record_batch - ) - except Exception as e: - print(f"Error reading Avro batch: {e}") - raise - - def _convert_to_record_batch(self, records: List[Dict[str, Any]]) -> pa.RecordBatch: - if not records: - return None - - if self._projected_type is not None: - records = [{k: r[k] for k in self._projected_type} for r in records] - - return pa.RecordBatch.from_pylist(records) - - def close(self): - pass diff --git a/pypaimon/pynative/reader/core/columnar_row_iterator.py b/pypaimon/pynative/reader/core/columnar_row_iterator.py deleted file mode 100644 index 124e4af..0000000 --- a/pypaimon/pynative/reader/core/columnar_row_iterator.py +++ /dev/null @@ -1,61 +0,0 @@ -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -from typing import Optional - -import pyarrow as pa - -from pypaimon.pynative.common.row.columnar_row import ColumnarRow -from pypaimon.pynative.common.row.key_value import InternalRow -from pypaimon.pynative.reader.core.file_record_iterator import FileRecordIterator - - -class ColumnarRowIterator(FileRecordIterator[InternalRow]): - """ - A RecordIterator that returns InternalRows. The next row is set by ColumnarRow.setRowId. - """ - - def __init__(self, file_path: str, record_batch: pa.RecordBatch): - self.file_path = file_path - self.record_batch = record_batch - self._row = ColumnarRow(record_batch) - - self.num_rows = record_batch.num_rows - self.next_pos = 0 - self.next_file_pos = 0 - - def next(self) -> Optional[InternalRow]: - if self.next_pos < self.num_rows: - self._row.set_row_id(self.next_pos) - self.next_pos += 1 - self.next_file_pos += 1 - return self._row - return None - - def returned_position(self) -> int: - return self.next_file_pos - 1 - - def file_path(self) -> str: - return self.file_path - - def reset(self, next_file_pos: int): - self.next_pos = 0 - self.next_file_pos = next_file_pos - - def release_batch(self): - del self.record_batch diff --git a/pypaimon/pynative/reader/data_file_record_reader.py b/pypaimon/pynative/reader/data_file_record_reader.py deleted file mode 100644 index a8b28eb..0000000 --- a/pypaimon/pynative/reader/data_file_record_reader.py +++ /dev/null @@ -1,135 +0,0 @@ -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -from typing import Optional, List, Any -import pyarrow as pa - -from pypaimon.pynative.common.exception import PyNativeNotImplementedError -from pypaimon.pynative.common.row.internal_row import InternalRow -from pypaimon.pynative.reader.core.file_record_iterator import FileRecordIterator -from pypaimon.pynative.reader.core.file_record_reader import FileRecordReader -from pypaimon.pynative.reader.core.record_reader import RecordReader -from pypaimon.pynative.reader.core.columnar_row_iterator import ColumnarRowIterator - - -class PartitionInfo: - """ - Partition information about how the row mapping of outer row. - """ - - def __init__(self, mapping: List[int], partition_values: List[Any]): - self.mapping = mapping # Mapping array similar to Java version - self.partition_values = partition_values # Partition values to be injected - - def size(self) -> int: - return len(self.mapping) - 1 - - def in_partition_row(self, pos: int) -> bool: - return self.mapping[pos] < 0 - - def get_real_index(self, pos: int) -> int: - return abs(self.mapping[pos]) - 1 - - def get_partition_value(self, pos: int) -> Any: - real_index = self.get_real_index(pos) - return self.partition_values[real_index] if real_index < len(self.partition_values) else None - - -class MappedColumnarRowIterator(ColumnarRowIterator): - """ - ColumnarRowIterator with mapping support for partition and index mapping. - """ - - def __init__(self, file_path: str, record_batch: pa.RecordBatch, - partition_info: Optional[PartitionInfo] = None, - index_mapping: Optional[List[int]] = None): - mapped_batch = self._apply_mappings(record_batch, partition_info, index_mapping) - super().__init__(file_path, mapped_batch) - - def _apply_mappings(self, record_batch: pa.RecordBatch, - partition_info: Optional[PartitionInfo], - index_mapping: Optional[List[int]]) -> pa.RecordBatch: - arrays = [] - names = [] - - if partition_info is not None: - for i in range(partition_info.size()): - if partition_info.in_partition_row(i): - partition_value = partition_info.get_partition_value(i) - const_array = pa.array([partition_value] * record_batch.num_rows) - arrays.append(const_array) - names.append(f"partition_field_{i}") - else: - real_index = partition_info.get_real_index(i) - if real_index < record_batch.num_columns: - arrays.append(record_batch.column(real_index)) - names.append(record_batch.column_names[real_index]) - else: - arrays = [record_batch.column(i) for i in range(record_batch.num_columns)] - names = record_batch.column_names[:] - - if index_mapping is not None: - mapped_arrays = [] - mapped_names = [] - for i, real_index in enumerate(index_mapping): - if real_index >= 0 and real_index < len(arrays): - mapped_arrays.append(arrays[real_index]) - mapped_names.append(names[real_index] if real_index < len(names) else f"field_{i}") - else: - null_array = pa.array([None] * record_batch.num_rows) - mapped_arrays.append(null_array) - mapped_names.append(f"null_field_{i}") - arrays = mapped_arrays - names = mapped_names - - final_batch = pa.RecordBatch.from_arrays(arrays, names=names) - return final_batch - - -class DataFileRecordReader(FileRecordReader[InternalRow]): - """ - Reads InternalRow from data files. - """ - - def __init__(self, wrapped_reader: RecordReader, - index_mapping: Optional[List[int]] = None, - partition_info: Optional[PartitionInfo] = None): - self.wrapped_reader = wrapped_reader - self.index_mapping = index_mapping - self.partition_info = partition_info - - def read_batch(self) -> Optional[FileRecordIterator['InternalRow']]: - iterator = self.wrapped_reader.read_batch() - if iterator is None: - return None - - if isinstance(iterator, ColumnarRowIterator): - if self.partition_info is not None or self.index_mapping is not None: - iterator = MappedColumnarRowIterator( - iterator.file_path, - iterator.record_batch, - self.partition_info, - self.index_mapping - ) - else: - raise PyNativeNotImplementedError("partition_info & index_mapping for non ColumnarRowIterator") - - return iterator - - def close(self) -> None: - self.wrapped_reader.close() diff --git a/pypaimon/pynative/reader/key_value_wrap_reader.py b/pypaimon/pynative/reader/key_value_wrap_reader.py deleted file mode 100644 index 980e7e5..0000000 --- a/pypaimon/pynative/reader/key_value_wrap_reader.py +++ /dev/null @@ -1,97 +0,0 @@ -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -from typing import Optional - -from pypaimon.pynative.common.row.internal_row import InternalRow -from pypaimon.pynative.common.row.key_value import KeyValue -from pypaimon.pynative.common.row.offset_row import OffsetRow -from pypaimon.pynative.common.row.row_kind import RowKind -from pypaimon.pynative.reader.core.file_record_iterator import FileRecordIterator -from pypaimon.pynative.reader.core.file_record_reader import FileRecordReader - - -class KeyValueWrapReader(FileRecordReader[KeyValue]): - """ - RecordReader for reading KeyValue data files. - Corresponds to the KeyValueDataFileRecordReader in Java version. - """ - - def __init__(self, wrapped_reader: FileRecordReader[InternalRow], - level, key_arity, value_arity): - self.wrapped_reader = wrapped_reader - self.level = level - self.key_arity = key_arity - self.value_arity = value_arity - - def read_batch(self) -> Optional[FileRecordIterator[KeyValue]]: - iterator = self.wrapped_reader.read_batch() - if iterator is None: - return None - return KeyValueWrapIterator(iterator, self.key_arity, self.value_arity, self.level) - - def close(self): - self.wrapped_reader.close() - - -class KeyValueWrapIterator(FileRecordIterator[KeyValue]): - """ - An Iterator that converts an PrimaryKey InternalRow into a KeyValue - """ - - def __init__( - self, - iterator: FileRecordIterator, - key_arity: int, - value_arity: int, - level: int - ): - self.iterator = iterator - self.key_arity = key_arity - self.value_arity = value_arity - self.level = level - - self.reused_key = OffsetRow(None, 0, key_arity) - self.reused_value = OffsetRow(None, key_arity + 2, value_arity) - - def next(self) -> Optional[KeyValue]: - row = self.iterator.next() - if row is None: - return None - - self.reused_key.replace(row) - self.reused_value.replace(row) - - sequence_number = row.get_field(self.key_arity) - value_kind = RowKind(row.get_field(self.key_arity + 1)) - - return KeyValue( - key=self.reused_key, - sequence_number=sequence_number, - value_kind=value_kind, - value=self.reused_value - ).set_level(self.level) - - def returned_position(self) -> int: - return self.iterator.returned_position() - - def file_path(self) -> str: - return self.iterator.file_path() - - def release_batch(self): - self.iterator.release_batch() diff --git a/pypaimon/pynative/reader/sort_merge_reader.py b/pypaimon/pynative/reader/sort_merge_reader.py deleted file mode 100644 index 30757b2..0000000 --- a/pypaimon/pynative/reader/sort_merge_reader.py +++ /dev/null @@ -1,271 +0,0 @@ -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -import heapq -from typing import Any, Callable, List, Optional - -import pyarrow as pa - -from pypaimon.pynative.common.row.key_value import KeyValue -from pypaimon.pynative.common.row.row_kind import RowKind -from pypaimon.pynative.reader.core.record_iterator import RecordIterator -from pypaimon.pynative.reader.core.record_reader import RecordReader - - -def built_comparator(key_schema: pa.Schema) -> Callable[[Any, Any], int]: - def comparator(key1, key2) -> int: - if key1 is None and key2 is None: - return 0 - if key1 is None: - return -1 - if key2 is None: - return 1 - - for i, field in enumerate(key_schema): - field_type = field.type - val1 = key1.get_field(i) - val2 = key2.get_field(i) - - if val1 is None and val2 is None: - continue - if val1 is None: - return -1 - if val2 is None: - return 1 - - if (pa.types.is_integer(field_type) or pa.types.is_floating(field_type) - or pa.types.is_boolean(field_type)): - if val1 < val2: - return -1 - elif val1 > val2: - return 1 - elif pa.types.is_string(field_type) or pa.types.is_binary(field_type): - if val1 < val2: - return -1 - elif val1 > val2: - return 1 - elif pa.types.is_timestamp(field_type) or pa.types.is_date(field_type): - if val1 < val2: - return -1 - elif val1 > val2: - return 1 - else: - str_val1 = str(val1) - str_val2 = str(val2) - if str_val1 < str_val2: - return -1 - elif str_val1 > str_val2: - return 1 - return 0 - return comparator - - -class DeduplicateMergeFunction: - def __init__(self, ignore_delete: bool = False): - self.ignore_delete = ignore_delete - self.latest_kv = None - self.is_initialized = False - self.initial_kv = None - - def reset(self) -> None: - self.latest_kv = None - self.is_initialized = False - self.initial_kv = None - - def add(self, kv: KeyValue) -> None: - if self.initial_kv is None: - self.initial_kv = kv - return - - if not self.is_initialized: - if not self.ignore_delete or not self.initial_kv.value_kind == RowKind.DELETE: - self.latest_kv = self.initial_kv - self.is_initialized = True - - if self.ignore_delete and kv.value_kind == RowKind.DELETE: - return - - self.latest_kv = kv - - def get_result(self) -> Optional[KeyValue]: - if not self.is_initialized: - return self.initial_kv - return self.latest_kv - - -class Element: - def __init__(self, kv, iterator: RecordIterator, reader: RecordReader): - self.kv = kv - self.iterator = iterator - self.reader = reader - - def update(self) -> bool: - next_kv = self.iterator.next() - if next_kv is None: - return False - self.kv = next_kv - return True - - -class HeapEntry: - def __init__(self, key, element: Element, key_comparator): - self.key = key - self.element = element - self.key_comparator = key_comparator - - def __lt__(self, other): - result = self.key_comparator(self.key, other.key) - if result < 0: - return True - elif result > 0: - return False - - return self.element.kv.sequence_number < other.element.kv.sequence_number - - -class SortMergeIterator(RecordIterator): - def __init__(self, reader, polled: List[Element], min_heap, merge_function, - user_key_comparator, next_batch_readers): - self.reader = reader - self.polled = polled - self.min_heap = min_heap - self.merge_function = merge_function - self.user_key_comparator = user_key_comparator - self.next_batch_readers = next_batch_readers - self.released = False - - def next(self): - while True: - has_more = self._next_impl() - if not has_more: - return None - result = self.merge_function.get_result() - if result is not None: - return result - - def _next_impl(self): - if self.released: - raise RuntimeError("SortMergeIterator.next called after release") - - if not self.next_batch_readers: - for element in self.polled: - if element.update(): - entry = HeapEntry(element.kv.key, element, self.user_key_comparator) - heapq.heappush(self.min_heap, entry) - else: - element.iterator.release_batch() - self.next_batch_readers.append(element.reader) - - self.polled.clear() - - if self.next_batch_readers: - return False - - if not self.min_heap: - return False - - self.merge_function.reset() - - first_entry = self.min_heap[0] - key = first_entry.key - - while self.min_heap and self.user_key_comparator(key, self.min_heap[0].key) == 0: - entry = heapq.heappop(self.min_heap) - self.merge_function.add(entry.element.kv) - self.polled.append(entry.element) - - return True - - def release_batch(self): - self.released = True - - -class SortMergeReader: - def __init__(self, readers, primary_keys, partition_keys): - self.next_batch_readers = list(readers) - self.merge_function = DeduplicateMergeFunction(False) - - if partition_keys: - trimmed_primary_keys = [pk for pk in primary_keys if pk not in partition_keys] - if not trimmed_primary_keys: - raise ValueError(f"Primary key constraint {primary_keys} same with partition fields") - else: - trimmed_primary_keys = primary_keys - - key_columns = [f"_KEY_{pk}" for pk in trimmed_primary_keys] - key_schema = pa.schema([pa.field(column, pa.string()) for column in key_columns]) - self.user_key_comparator = built_comparator(key_schema) - - def element_comparator(e1_tuple, e2_tuple): - key1, e1 = e1_tuple - key2, e2 = e2_tuple - - result = self.user_key_comparator(key1, key2) - if result != 0: - return result - - return e1.kv.sequence_number - e2.kv.sequence_number - - from functools import cmp_to_key - self.element_key = cmp_to_key(element_comparator) - - self.min_heap = [] - self.polled = [] - - def read_batch(self) -> Optional[RecordIterator]: - for reader in self.next_batch_readers: - while True: - iterator = reader.read_batch() - if iterator is None: - reader.close() - break - - kv = iterator.next() - if kv is None: - iterator.release_batch() - else: - element = Element(kv, iterator, reader) - entry = HeapEntry(kv.key, element, self.user_key_comparator) - heapq.heappush(self.min_heap, entry) - break - - self.next_batch_readers.clear() - - if not self.min_heap: - return None - - return SortMergeIterator( - self, - self.polled, - self.min_heap, - self.merge_function, - self.user_key_comparator, - self.next_batch_readers - ) - - def close(self): - for reader in self.next_batch_readers: - reader.close() - - for entry in self.min_heap: - entry.element.iterator.release_batch() - entry.element.reader.close() - - for element in self.polled: - element.iterator.release_batch() - element.reader.close() diff --git a/pypaimon/pynative/row/__init__.py b/pypaimon/pynative/row/__init__.py new file mode 100644 index 0000000..65b48d4 --- /dev/null +++ b/pypaimon/pynative/row/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/pypaimon/pynative/row/binary_row.py b/pypaimon/pynative/row/binary_row.py new file mode 100644 index 0000000..7800084 --- /dev/null +++ b/pypaimon/pynative/row/binary_row.py @@ -0,0 +1,465 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import struct +from dataclasses import dataclass +from datetime import datetime, timezone, timedelta +from decimal import Decimal +from typing import List, Any + +from pypaimon.pynative.common.data_field import DataField, DataType +from pypaimon.pynative.row.row_kind import RowKind + + +@dataclass +class BinaryRow: + values: List[Any] + fields: List[DataField] + row_kind: RowKind = RowKind.INSERT + + def to_dict(self): + return {self.fields[i].name: self.values[i] for i in range(len(self.fields))} + + +class BinaryRowDeserializer: + HEADER_SIZE_IN_BITS = 8 + MAX_FIX_PART_DATA_SIZE = 7 + HIGHEST_FIRST_BIT = 0x80 << 56 + HIGHEST_SECOND_TO_EIGHTH_BIT = 0x7F << 56 + + @classmethod + def from_bytes( + cls, + bytes_data: bytes, + data_fields: List[DataField] + ) -> BinaryRow: + if not bytes_data: + return BinaryRow([], data_fields) + + arity = len(data_fields) + actual_data = bytes_data + if len(bytes_data) >= 4: + arity_from_bytes = struct.unpack('>i', bytes_data[:4])[0] + if 0 < arity_from_bytes < 1000: + actual_data = bytes_data[4:] + + fields = [] + null_bits_size_in_bytes = cls._calculate_bit_set_width_in_bytes(arity) + for i, data_field in enumerate(data_fields): + value = None + if not cls._is_null_at(actual_data, 0, i): + value = cls._parse_field_value(actual_data, 0, null_bits_size_in_bytes, i, data_field.type) + fields.append(value) + + return BinaryRow(fields, data_fields, RowKind(actual_data[0])) + + @classmethod + def _calculate_bit_set_width_in_bytes(cls, arity: int) -> int: + return ((arity + 63 + cls.HEADER_SIZE_IN_BITS) // 64) * 8 + + @classmethod + def _is_null_at(cls, bytes_data: bytes, offset: int, pos: int) -> bool: + index = pos + cls.HEADER_SIZE_IN_BITS + byte_index = offset + (index // 8) + bit_index = index % 8 + return (bytes_data[byte_index] & (1 << bit_index)) != 0 + + @classmethod + def _parse_field_value( + cls, + bytes_data: bytes, + base_offset: int, + null_bits_size_in_bytes: int, + pos: int, + data_type: DataType + ) -> Any: + field_offset = base_offset + null_bits_size_in_bytes + pos * 8 + if field_offset >= len(bytes_data): + raise ValueError(f"Field offset {field_offset} exceeds data length {len(bytes_data)}") + type_name = data_type.type_name.upper() + + if type_name in ['BOOLEAN', 'BOOL']: + return cls._parse_boolean(bytes_data, field_offset) + elif type_name in ['TINYINT', 'BYTE']: + return cls._parse_byte(bytes_data, field_offset) + elif type_name in ['SMALLINT', 'SHORT']: + return cls._parse_short(bytes_data, field_offset) + elif type_name in ['INT', 'INTEGER']: + return cls._parse_int(bytes_data, field_offset) + elif type_name in ['BIGINT', 'LONG']: + return cls._parse_long(bytes_data, field_offset) + elif type_name in ['FLOAT', 'REAL']: + return cls._parse_float(bytes_data, field_offset) + elif type_name in ['DOUBLE']: + return cls._parse_double(bytes_data, field_offset) + elif type_name in ['VARCHAR', 'STRING', 'CHAR']: + return cls._parse_string(bytes_data, base_offset, field_offset) + elif type_name in ['BINARY', 'VARBINARY', 'BYTES']: + return cls._parse_binary(bytes_data, base_offset, field_offset) + elif type_name in ['DECIMAL', 'NUMERIC']: + return cls._parse_decimal(bytes_data, base_offset, field_offset, data_type) + elif type_name in ['TIMESTAMP', 'TIMESTAMP_WITHOUT_TIME_ZONE']: + return cls._parse_timestamp(bytes_data, base_offset, field_offset, data_type) + elif type_name in ['DATE']: + return cls._parse_date(bytes_data, field_offset) + elif type_name in ['TIME', 'TIME_WITHOUT_TIME_ZONE']: + return cls._parse_time(bytes_data, field_offset) + else: + return cls._parse_string(bytes_data, base_offset, field_offset) + + @classmethod + def _parse_boolean(cls, bytes_data: bytes, field_offset: int) -> bool: + return bytes_data[field_offset] != 0 + + @classmethod + def _parse_byte(cls, bytes_data: bytes, field_offset: int) -> int: + return struct.unpack(' int: + return struct.unpack(' int: + if field_offset + 4 > len(bytes_data): + raise ValueError(f"Not enough bytes for INT: need 4, have {len(bytes_data) - field_offset}") + return struct.unpack(' int: + if field_offset + 8 > len(bytes_data): + raise ValueError(f"Not enough bytes for LONG: need 8, have {len(bytes_data) - field_offset}") + return struct.unpack(' float: + return struct.unpack(' float: + if field_offset + 8 > len(bytes_data): + raise ValueError(f"Not enough bytes for DOUBLE: need 8, have {len(bytes_data) - field_offset}") + return struct.unpack(' str: + if field_offset + 8 > len(bytes_data): + raise ValueError(f"Not enough bytes for STRING offset: need 8, have {len(bytes_data) - field_offset}") + + offset_and_len = struct.unpack('> 32) & 0xFFFFFFFF + length = offset_and_len & 0xFFFFFFFF + actual_string_offset = base_offset + sub_offset + if actual_string_offset + length > len(bytes_data): + raise ValueError( + f"String data out of bounds: actual_offset={actual_string_offset}, length={length}, " + f"total_length={len(bytes_data)}") + string_data = bytes_data[actual_string_offset:actual_string_offset + length] + return string_data.decode('utf-8') + else: + length = (offset_and_len & cls.HIGHEST_SECOND_TO_EIGHTH_BIT) >> 56 + start_offset = field_offset + if start_offset + length > len(bytes_data): + raise ValueError(f"Compact string data out of bounds: length={length}") + string_data = bytes_data[start_offset:start_offset + length] + return string_data.decode('utf-8') + + @classmethod + def _parse_binary(cls, bytes_data: bytes, base_offset: int, field_offset: int) -> bytes: + offset_and_len = struct.unpack('> 32) & 0xFFFFFFFF + length = offset_and_len & 0xFFFFFFFF + return bytes_data[base_offset + sub_offset:base_offset + sub_offset + length] + else: + length = (offset_and_len & cls.HIGHEST_SECOND_TO_EIGHTH_BIT) >> 56 + return bytes_data[field_offset + 1:field_offset + 1 + length] + + @classmethod + def _parse_decimal(cls, bytes_data: bytes, base_offset: int, field_offset: int, data_type: DataType) -> Decimal: + unscaled_long = struct.unpack(' datetime: + millis = struct.unpack(' datetime: + days = struct.unpack(' datetime: + millis = struct.unpack(' bytes: + if not binary_row.values: + return b'' + + arity = len(binary_row.fields) + null_bits_size_in_bytes = cls._calculate_bit_set_width_in_bytes(arity) + fixed_part_size = null_bits_size_in_bytes + arity * 8 + fixed_part = bytearray(fixed_part_size) + fixed_part[0] = binary_row.row_kind.value + + for i, value in enumerate(binary_row.values): + if value is None: + cls._set_null_bit(fixed_part, 0, i) + + variable_data = [] + variable_offsets = [] + current_offset = 0 + + for i, (value, field) in enumerate(zip(binary_row.values, binary_row.fields)): + if value is None: + struct.pack_into(' 0x7FFFFFFFFFFFFFFF: + offset_and_len = offset_and_len - 0x10000000000000000 + struct.pack_into(' int: + return ((arity + 63 + cls.HEADER_SIZE_IN_BITS) // 64) * 8 + + @classmethod + def _set_null_bit(cls, bytes_data: bytearray, offset: int, pos: int) -> None: + index = pos + cls.HEADER_SIZE_IN_BITS + byte_index = offset + (index // 8) + bit_index = index % 8 + bytes_data[byte_index] |= (1 << bit_index) + + @classmethod + def _serialize_field_value(cls, value: Any, data_type: DataType) -> bytes: + type_name = data_type.type_name.upper() + + if type_name in ['BOOLEAN', 'BOOL']: + return cls._serialize_boolean(value) + elif type_name in ['TINYINT', 'BYTE']: + return cls._serialize_byte(value) + elif type_name in ['SMALLINT', 'SHORT']: + return cls._serialize_short(value) + elif type_name in ['INT', 'INTEGER']: + return cls._serialize_int(value) + elif type_name in ['BIGINT', 'LONG']: + return cls._serialize_long(value) + elif type_name in ['FLOAT', 'REAL']: + return cls._serialize_float(value) + elif type_name in ['DOUBLE']: + return cls._serialize_double(value) + elif type_name in ['VARCHAR', 'STRING', 'CHAR']: + return cls._serialize_string(value) + elif type_name in ['BINARY', 'VARBINARY', 'BYTES']: + return cls._serialize_binary(value) + elif type_name in ['DECIMAL', 'NUMERIC']: + return cls._serialize_decimal(value, data_type) + elif type_name in ['TIMESTAMP', 'TIMESTAMP_WITHOUT_TIME_ZONE']: + return cls._serialize_timestamp(value) + elif type_name in ['DATE']: + return cls._serialize_date(value) + elif type_name in ['TIME', 'TIME_WITHOUT_TIME_ZONE']: + return cls._serialize_time(value) + else: + return cls._serialize_string(str(value)) + + @classmethod + def _serialize_boolean(cls, value: bool) -> bytes: + return struct.pack(' bytes: + return struct.pack(' bytes: + return struct.pack(' bytes: + return struct.pack(' bytes: + return struct.pack(' bytes: + return struct.pack(' bytes: + return struct.pack(' bytes: + if isinstance(value, str): + value_bytes = value.encode('utf-8') + else: + value_bytes = bytes(value) + + length = len(value_bytes) + + offset_and_len = (0x80 << 56) | (length << 56) + if offset_and_len > 0x7FFFFFFFFFFFFFFF: + offset_and_len = offset_and_len - 0x10000000000000000 + return struct.pack(' bytes: + if isinstance(value, bytes): + data_bytes = value + else: + data_bytes = bytes(value) + length = len(data_bytes) + offset_and_len = (0x80 << 56) | (length << 56) + if offset_and_len > 0x7FFFFFFFFFFFFFFF: + offset_and_len = offset_and_len - 0x10000000000000000 + return struct.pack(' bytes: + type_str = str(data_type) + if '(' in type_str and ')' in type_str: + try: + precision_scale = type_str.split('(')[1].split(')')[0] + if ',' in precision_scale: + scale = int(precision_scale.split(',')[1]) + else: + scale = 0 + except: + scale = 0 + else: + scale = 0 + + unscaled_value = int(value * (10 ** scale)) + return struct.pack(' bytes: + if value.tzinfo is None: + value = value.replace(tzinfo=timezone.utc) + millis = int(value.timestamp() * 1000) + return struct.pack(' bytes: + if isinstance(value, datetime): + epoch = datetime(1970, 1, 1) + days = (value - epoch).days + else: + epoch = datetime(1970, 1, 1) + days = (value - epoch).days + return struct.pack(' bytes: + if isinstance(value, datetime): + midnight = value.replace(hour=0, minute=0, second=0, microsecond=0) + millis = int((value - midnight).total_seconds() * 1000) + else: + millis = value.hour * 3600000 + value.minute * 60000 + value.second * 1000 + value.microsecond // 1000 + return struct.pack(' bool: Returns true if the element is null at the given position. """ - @abstractmethod - def set_field(self, pos: int, value: Any) -> None: - """ - set element to a row at the given position. - """ - @abstractmethod def get_row_kind(self) -> RowKind: """ Returns the kind of change that this row describes in a changelog. """ - @abstractmethod - def set_row_kind(self, kind: RowKind) -> None: - """ - Sets the kind of change that this row describes in a changelog. - """ - @abstractmethod def __len__(self) -> int: """ diff --git a/pypaimon/pynative/row/key_value.py b/pypaimon/pynative/row/key_value.py new file mode 100644 index 0000000..b0ac342 --- /dev/null +++ b/pypaimon/pynative/row/key_value.py @@ -0,0 +1,57 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from pypaimon.pynative.row.row_kind import RowKind +from pypaimon.pynative.row.offset_row import OffsetRow + + +class KeyValue: + """A key value, including user key, sequence number, value kind and value.""" + + def __init__(self, key_arity: int, value_arity: int): + self.key_arity = key_arity + self.value_arity = value_arity + + self._row_tuple = None + self._reused_key = OffsetRow(None, 0, key_arity) + self._reused_value = OffsetRow(None, key_arity + 2, value_arity) + + def replace(self, row_tuple: tuple): + self._row_tuple = row_tuple + self._reused_key.replace(row_tuple) + self._reused_value.replace(row_tuple) + return self + + def is_add(self) -> bool: + return RowKind.is_add_byte(self.value_row_kind_byte) + + @property + def key(self) -> OffsetRow: + return self._reused_key + + @property + def value(self) -> OffsetRow: + return self._reused_value + + @property + def sequence_number(self) -> int: + return self._row_tuple[self.key_arity] + + @property + def value_row_kind_byte(self) -> int: + return self._row_tuple[self.key_arity + 1] diff --git a/pypaimon/pynative/common/row/offset_row.py b/pypaimon/pynative/row/offset_row.py similarity index 57% rename from pypaimon/pynative/common/row/offset_row.py rename to pypaimon/pynative/row/offset_row.py index 8ae21a2..e3975bd 100644 --- a/pypaimon/pynative/common/row/offset_row.py +++ b/pypaimon/pynative/row/offset_row.py @@ -16,44 +16,43 @@ # limitations under the License. ################################################################################ -from typing import Any +from typing import Optional -from pypaimon.pynative.common.row.internal_row import InternalRow -from pypaimon.pynative.common.row.row_kind import RowKind +from pypaimon.pynative.row.internal_row import InternalRow +from pypaimon.pynative.row.row_kind import RowKind class OffsetRow(InternalRow): - """ - A InternalRow to wrap row with offset. - """ + """A InternalRow to wrap row with offset.""" - def __init__(self, row: InternalRow, offset: int, arity: int): - self.row = row + def __init__(self, row_tuple: Optional[tuple], offset: int, arity: int): + self.row_tuple = row_tuple self.offset = offset self.arity = arity + self.row_kind_byte: int = 1 - def replace(self, row: InternalRow) -> 'OffsetRow': - self.row = row + def replace(self, row_tuple: tuple) -> 'OffsetRow': + self.row_tuple = row_tuple + if self.offset + self.arity > len(row_tuple): + raise ValueError(f"Offset {self.offset} plus arity {self.arity} is out of row length {len(row_tuple)}") return self def get_field(self, pos: int): if pos >= self.arity: - raise IndexError(f"Position {pos} is out of bounds for arity {self.arity}") - return self.row.get_field(pos + self.offset) + raise IndexError(f"Position {pos} is out of bounds for row arity {self.arity}") + return self.row_tuple[self.offset + pos] def is_null_at(self, pos: int) -> bool: - if pos >= self.arity: - raise IndexError(f"Position {pos} is out of bounds for arity {self.arity}") - return self.row.is_null_at(pos + self.offset) - - def set_field(self, pos: int, value: Any) -> None: - raise NotImplementedError() + return self.get_field(pos) is None def get_row_kind(self) -> RowKind: - return self.row.get_row_kind() + return RowKind(self.row_kind_byte) - def set_row_kind(self, kind: RowKind) -> None: - self.row.set_row_kind(kind) + def set_row_kind_byte(self, row_kind_byte: int) -> None: + """ + Store RowKind as a byte and instantiate it lazily to avoid performance overhead. + """ + self.row_kind_byte = row_kind_byte def __len__(self) -> int: return self.arity diff --git a/pypaimon/pynative/common/row/row_kind.py b/pypaimon/pynative/row/row_kind.py similarity index 90% rename from pypaimon/pynative/common/row/row_kind.py rename to pypaimon/pynative/row/row_kind.py index ff9b9b1..06a2904 100644 --- a/pypaimon/pynative/common/row/row_kind.py +++ b/pypaimon/pynative/row/row_kind.py @@ -20,9 +20,6 @@ class RowKind(Enum): - """ - Insertion operation. - """ INSERT = 0 # +I: Update operation with the previous content of the updated row. UPDATE_BEFORE = 1 # -U: Update operation with the previous content of the updated row UPDATE_AFTER = 2 # +U: Update operation with new content of the updated row @@ -55,3 +52,10 @@ def from_string(kind_str: str) -> 'RowKind': return RowKind.DELETE else: raise ValueError(f"Unknown row kind string: {kind_str}") + + @classmethod + def is_add_byte(cls, byte: int): + """ + Check RowKind type from byte, to avoid creation and destruction of RowKind objects, reducing GC pressure + """ + return byte == 0 or byte == 2 diff --git a/pypaimon/pynative/sample/__init__.py b/pypaimon/pynative/sample/__init__.py new file mode 100644 index 0000000..e69de29 diff --git a/pypaimon/pynative/sample/hdfs_read_and_write.py b/pypaimon/pynative/sample/hdfs_read_and_write.py new file mode 100644 index 0000000..e69de29 diff --git a/pypaimon/pynative/sample/oss_read_and_write.py b/pypaimon/pynative/sample/oss_read_and_write.py new file mode 100644 index 0000000..3be54bc --- /dev/null +++ b/pypaimon/pynative/sample/oss_read_and_write.py @@ -0,0 +1,70 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import pyarrow as pa + +from pypaimon.api import Schema +from pypaimon.api.catalog_factory import CatalogFactory + + +def oss_read_and_write(): + warehouse = 'oss:///' + catalog = CatalogFactory.create({ + 'warehouse': warehouse, + 's3.endpoint': 'oss-.aliyuncs.com', + 's3.access-key': '', + 's3.secret-key': '', + 's3.region': '' + }) + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()) + ]) + data = { + 'f0': [1, 2, 3], + 'f1': ['a', 'b', 'c'], + 'f2': ['X', 'Y', 'Z'] + } + catalog.create_database("test_db", True) + catalog.create_table("test_db.test_table", Schema(simple_pa_schema), True) + table = catalog.get_table("test_db.test_table") + + # write data + write_builder = table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + write_data = pa.Table.from_pydict(data, schema=simple_pa_schema) + table_write.write_arrow(write_data) + commit_messages = table_write.prepare_commit() + table_commit.commit(commit_messages) + table_write.close() + table_commit.close() + + # read data + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + splits = table_scan.plan().splits() + + result = table_read.to_arrow(splits) + print(result) + + +if __name__ == '__main__': + oss_read_and_write() diff --git a/pypaimon/pynative/table/__init__.py b/pypaimon/pynative/table/__init__.py new file mode 100644 index 0000000..65b48d4 --- /dev/null +++ b/pypaimon/pynative/table/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/pypaimon/pynative/table/bucket_mode.py b/pypaimon/pynative/table/bucket_mode.py new file mode 100644 index 0000000..df6c8e9 --- /dev/null +++ b/pypaimon/pynative/table/bucket_mode.py @@ -0,0 +1,29 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from enum import Enum, auto + + +class BucketMode(Enum): + def __str__(self): + return self.value + + HASH_FIXED = auto() + HASH_DYNAMIC = auto() + CROSS_PARTITION = auto() + BUCKET_UNAWARE = auto() diff --git a/pypaimon/pynative/table/data_file_meta.py b/pypaimon/pynative/table/data_file_meta.py new file mode 100644 index 0000000..86fab19 --- /dev/null +++ b/pypaimon/pynative/table/data_file_meta.py @@ -0,0 +1,60 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from dataclasses import dataclass +from pathlib import Path +from typing import List, Optional, Dict, Any +from datetime import datetime + +from pypaimon.pynative.row.binary_row import BinaryRow + + +@dataclass +class DataFileMeta: + file_name: str + file_size: int + row_count: int + min_key: Optional[BinaryRow] + max_key: Optional[BinaryRow] + key_stats: Optional[Dict[str, Any]] + value_stats: Optional[Dict[str, Any]] + min_sequence_number: int + max_sequence_number: int + schema_id: int + level: int + extra_files: Optional[List[str]] + + creation_time: Optional[datetime] = None + delete_row_count: Optional[int] = None + embedded_index: Optional[bytes] = None + file_source: Optional[str] = None + value_stats_cols: Optional[List[str]] = None + external_path: Optional[str] = None + + file_path: str = None + + def set_file_path(self, table_path: Path, partition: BinaryRow, bucket: int): + path_builder = table_path + + partition_dict = partition.to_dict() + for field_name, field_value in partition_dict.items(): + path_builder = path_builder / (field_name + "=" + str(field_value)) + + path_builder = path_builder / ("bucket-" + str(bucket)) / self.file_name + + self.file_path = str(path_builder) diff --git a/pypaimon/pynative/table/file_store_table.py b/pypaimon/pynative/table/file_store_table.py new file mode 100644 index 0000000..bc27a62 --- /dev/null +++ b/pypaimon/pynative/table/file_store_table.py @@ -0,0 +1,80 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from pathlib import Path + +from pypaimon.api import Table, BatchWriteBuilder, ReadBuilder +from pypaimon.pynative.common.core_option import CoreOptions +from pypaimon.pynative.common.exception import PyNativeNotImplementedError +from pypaimon.pynative.common.file_io import FileIO +from pypaimon.pynative.common.identifier import TableIdentifier +from pypaimon.pynative.read.read_builder_impl import ReadBuilderImpl +from pypaimon.pynative.table.bucket_mode import BucketMode +from pypaimon.pynative.write.row_key_extractor import RowKeyExtractor, FixedBucketRowKeyExtractor, \ + UnawareBucketRowKeyExtractor +from pypaimon.pynative.table.schema_manager import SchemaManager +from pypaimon.pynative.table.schema import TableSchema +from pypaimon.pynative.write.batch_write_builder import BatchWriteBuilderImpl + + +class FileStoreTable(Table): + def __init__(self, file_io: FileIO, table_identifier: TableIdentifier, table_path: Path, + table_schema: TableSchema): + self.file_io = file_io + self.table_identifier = table_identifier + self.table_path = table_path + + self.fields = table_schema.fields + self.primary_keys = table_schema.primary_keys + self.partition_keys = table_schema.partition_keys + + self.options = table_schema.options + self.table_schema = table_schema + self.schema_manager = SchemaManager(file_io, table_path) + self.is_primary_key_table = bool(self.primary_keys) + + def bucket_mode(self) -> BucketMode: + if self.is_primary_key_table: + if self.primary_keys == self.partition_keys: + return BucketMode.CROSS_PARTITION + if self.options.get(CoreOptions.BUCKET, -1) == -1: + return BucketMode.HASH_DYNAMIC + else: + return BucketMode.HASH_FIXED + else: + if self.options.get(CoreOptions.BUCKET, -1) == -1: + return BucketMode.BUCKET_UNAWARE + else: + return BucketMode.HASH_FIXED + + def new_read_builder(self) -> ReadBuilder: + return ReadBuilderImpl(self) + + def new_batch_write_builder(self) -> BatchWriteBuilder: + return BatchWriteBuilderImpl(self) + + def create_row_key_extractor(self) -> RowKeyExtractor: + bucket_mode = self.bucket_mode() + if bucket_mode == BucketMode.HASH_FIXED: + return FixedBucketRowKeyExtractor(self.table_schema) + elif bucket_mode == BucketMode.BUCKET_UNAWARE: + return UnawareBucketRowKeyExtractor(self.table_schema) + elif bucket_mode == BucketMode.HASH_DYNAMIC or bucket_mode == BucketMode.CROSS_PARTITION: + raise PyNativeNotImplementedError(bucket_mode) + else: + raise ValueError(f"Unsupported mode: {bucket_mode}") diff --git a/pypaimon/pynative/common/row/key_value.py b/pypaimon/pynative/table/manifest_entry.py similarity index 66% rename from pypaimon/pynative/common/row/key_value.py rename to pypaimon/pynative/table/manifest_entry.py index d8c9951..0edbad4 100644 --- a/pypaimon/pynative/common/row/key_value.py +++ b/pypaimon/pynative/table/manifest_entry.py @@ -18,25 +18,14 @@ from dataclasses import dataclass -from pypaimon.pynative.common.row.internal_row import InternalRow -from pypaimon.pynative.common.row.row_kind import RowKind - -""" -A key value, including user key, sequence number, value kind and value. -""" +from pypaimon.pynative.row.binary_row import BinaryRow +from pypaimon.pynative.table.data_file_meta import DataFileMeta @dataclass -class KeyValue: - key: InternalRow - sequence_number: int - value_kind: RowKind - value: InternalRow - level: int = -1 - - def set_level(self, level: int) -> 'KeyValue': - self.level = level - return self - - def is_add(self) -> bool: - return self.value_kind.is_add() +class ManifestEntry: + kind: int + partition: BinaryRow + bucket: int + total_buckets: int + file: DataFileMeta diff --git a/pypaimon/pynative/table/manifest_file_manager.py b/pypaimon/pynative/table/manifest_file_manager.py new file mode 100644 index 0000000..49ea44e --- /dev/null +++ b/pypaimon/pynative/table/manifest_file_manager.py @@ -0,0 +1,166 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import uuid +import fastavro +from typing import List +from io import BytesIO + +from pypaimon.pynative.row.binary_row import BinaryRowDeserializer, BinaryRowSerializer, BinaryRow +from pypaimon.pynative.table.data_file_meta import DataFileMeta +from pypaimon.pynative.table.manifest_entry import ManifestEntry +from pypaimon.pynative.write.commit_message_impl import CommitMessageImpl + + +class ManifestFileManager: + """Writer for manifest files in Avro format using unified FileIO.""" + + def __init__(self, table): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.manifest_path = table.table_path / "manifest" + self.file_io = table.file_io + self.partition_key_fields = self.table.table_schema.get_partition_key_fields() + self.primary_key_fields = self.table.table_schema.get_primary_key_fields() + self.trimmed_primary_key_fields = self.table.table_schema.get_trimmed_primary_key_fields() + + def read(self, manifest_file_name: str) -> List[ManifestEntry]: + manifest_file_path = self.manifest_path / manifest_file_name + + entries = [] + with self.file_io.new_input_stream(manifest_file_path) as input_stream: + avro_bytes = input_stream.read() + buffer = BytesIO(avro_bytes) + reader = fastavro.reader(buffer) + + for record in reader: + file_info = dict(record['_FILE']) + file_meta = DataFileMeta( + file_name=file_info['_FILE_NAME'], + file_size=file_info['_FILE_SIZE'], + row_count=file_info['_ROW_COUNT'], + min_key=BinaryRowDeserializer.from_bytes(file_info['_MIN_KEY'], self.trimmed_primary_key_fields), + max_key=BinaryRowDeserializer.from_bytes(file_info['_MAX_KEY'], self.trimmed_primary_key_fields), + key_stats=None, # TODO + value_stats=None, # TODO + min_sequence_number=file_info['_MIN_SEQUENCE_NUMBER'], + max_sequence_number=file_info['_MAX_SEQUENCE_NUMBER'], + schema_id=file_info['_SCHEMA_ID'], + level=file_info['_LEVEL'], + extra_files=None, # TODO + ) + entry = ManifestEntry( + kind=record['_KIND'], + partition=BinaryRowDeserializer.from_bytes(record['_PARTITION'], self.partition_key_fields), + bucket=record['_BUCKET'], + total_buckets=record['_TOTAL_BUCKETS'], + file=file_meta + ) + entries.append(entry) + return entries + + def write(self, commit_messages: List[CommitMessageImpl]) -> List[str]: + avro_records = [] + for message in commit_messages: + partition_bytes = BinaryRowSerializer.to_bytes( + BinaryRow(list(message.partition()), self.table.table_schema.get_partition_key_fields)) + for file in message.new_files(): + avro_record = { + "_KIND": 0, + "_PARTITION": partition_bytes, + "_BUCKET": message.bucket(), + "_TOTAL_BUCKETS": -1, # TODO + "_FILE": { + "_FILE_NAME": file.file_name, + "_FILE_SIZE": file.file_size, + "_ROW_COUNT": file.row_count, + "_MIN_KEY": BinaryRowSerializer.to_bytes(file.min_key), + "_MAX_KEY": BinaryRowSerializer.to_bytes(file.max_key), + "_KEY_STATS": 1, # TODO + "_VALUE_STATS": 1, + "_MIN_SEQUENCE_NUMBER": 0, + "_MAX_SEQUENCE_NUMBER": 0, + "_SCHEMA_ID": 0, + "_LEVEL": 0, + "_EXTRA_FILES": [], + } + } + avro_records.append(avro_record) + + manifest_filename = f"manifest-{str(uuid.uuid4())}.avro" + manifest_path = self.manifest_path / manifest_filename + try: + buffer = BytesIO() + fastavro.writer(buffer, MANIFEST_ENTRY_SCHEMA, avro_records) + avro_bytes = buffer.getvalue() + with self.file_io.new_output_stream(manifest_path) as output_stream: + output_stream.write(avro_bytes) + return [str(manifest_filename)] + except Exception as e: + self.file_io.delete_quietly(manifest_path) + raise RuntimeError(f"Failed to write manifest file: {e}") from e + + +SIMPLE_STATS_SCHEMA = { + "type": "record", + "name": "SimpleStats", + "namespace": "com.example.paimon", + "fields": [ + {"name": "null_count", "type": ["null", "long"], "default": None}, + {"name": "min_value", "type": ["null", "bytes"], "default": None}, + {"name": "max_value", "type": ["null", "bytes"], "default": None}, + ] +} + +DATA_FILE_META_SCHEMA = { + "type": "record", + "name": "DataFileMeta", + "fields": [ + {"name": "_FILE_NAME", "type": "string"}, + {"name": "_FILE_SIZE", "type": "long"}, + {"name": "_ROW_COUNT", "type": "long"}, + {"name": "_MIN_KEY", "type": "bytes"}, + {"name": "_MAX_KEY", "type": "bytes"}, + {"name": "_KEY_STATS", "type": "long"}, # TODO + {"name": "_VALUE_STATS", "type": "long"}, # TODO + {"name": "_MIN_SEQUENCE_NUMBER", "type": "long"}, + {"name": "_MAX_SEQUENCE_NUMBER", "type": "long"}, + {"name": "_SCHEMA_ID", "type": "long"}, + {"name": "_LEVEL", "type": "int"}, + {"name": "_EXTRA_FILES", "type": {"type": "array", "items": "string"}}, + {"name": "_CREATION_TIME", "type": ["null", "long"], "default": None}, + {"name": "_DELETE_ROW_COUNT", "type": ["null", "long"], "default": None}, + {"name": "_EMBEDDED_FILE_INDEX", "type": ["null", "bytes"], "default": None}, + {"name": "_FILE_SOURCE", "type": ["null", "int"], "default": None}, + {"name": "_VALUE_STATS_COLS", "type": ["null", {"type": "array", "items": "string"}], "default": None}, + {"name": "_EXTERNAL_PATH", "type": ["null", "string"], "default": None}, + ] +} + +MANIFEST_ENTRY_SCHEMA = { + "type": "record", + "name": "ManifestEntry", + "fields": [ + {"name": "_KIND", "type": "int"}, + {"name": "_PARTITION", "type": "bytes"}, + {"name": "_BUCKET", "type": "int"}, + {"name": "_TOTAL_BUCKETS", "type": "int"}, + {"name": "_FILE", "type": DATA_FILE_META_SCHEMA} + ] +} diff --git a/pypaimon/pynative/table/manifest_list_manager.py b/pypaimon/pynative/table/manifest_list_manager.py new file mode 100644 index 0000000..afd061c --- /dev/null +++ b/pypaimon/pynative/table/manifest_list_manager.py @@ -0,0 +1,100 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import uuid +import fastavro +from typing import List, Optional +from io import BytesIO + +from pypaimon.pynative.table.snapshot import Snapshot + + +class ManifestListManager: + """Manager for manifest list files in Avro format using unified FileIO.""" + + def __init__(self, table): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.manifest_path = self.table.table_path / "manifest" + self.file_io = self.table.file_io + + def read_all_manifest_files(self, snapshot: Snapshot) -> List[str]: + manifest_files = [] + base_manifests = self.read(snapshot.base_manifest_list) + manifest_files.extend(base_manifests) + delta_manifests = self.read(snapshot.delta_manifest_list) + manifest_files.extend(delta_manifests) + return list(set(manifest_files)) + + def read(self, manifest_list_name: str) -> List[str]: + manifest_list_path = self.manifest_path / manifest_list_name + manifest_paths = [] + + with self.file_io.new_input_stream(manifest_list_path) as input_stream: + avro_bytes = input_stream.read() + buffer = BytesIO(avro_bytes) + reader = fastavro.reader(buffer) + for record in reader: + file_name = record['_FILE_NAME'] + manifest_paths.append(file_name) + + return manifest_paths + + def write(self, manifest_file_names: List[str]) -> Optional[str]: + if not manifest_file_names: + return None + + avro_records = [] + for manifest_file_name in manifest_file_names: + avro_record = { + "_FILE_NAME": manifest_file_name, + "_FILE_SIZE": 0, # TODO + "_NUM_ADDED_FILES": 0, + "_NUM_DELETED_FILES": 0, + "_PARTITION_STATS": 0, + "_SCHEMA_ID": 0, + } + avro_records.append(avro_record) + + list_filename = f"manifest-list-{str(uuid.uuid4())}.avro" + list_path = self.manifest_path / list_filename + try: + buffer = BytesIO() + fastavro.writer(buffer, MANIFEST_FILE_META_SCHEMA, avro_records) + avro_bytes = buffer.getvalue() + with self.file_io.new_output_stream(list_path) as output_stream: + output_stream.write(avro_bytes) + return list_filename + except Exception as e: + self.file_io.delete_quietly(list_path) + raise RuntimeError(f"Failed to write manifest list file: {e}") from e + + +MANIFEST_FILE_META_SCHEMA = { + "type": "record", + "name": "ManifestFileMeta", + "fields": [ + {"name": "_FILE_NAME", "type": "string"}, + {"name": "_FILE_SIZE", "type": "long"}, + {"name": "_NUM_ADDED_FILES", "type": "long"}, + {"name": "_NUM_DELETED_FILES", "type": "long"}, + {"name": "_PARTITION_STATS", "type": "long"}, # TODO + {"name": "_SCHEMA_ID", "type": "long"}, + ] +} diff --git a/pypaimon/pynative/table/schema.py b/pypaimon/pynative/table/schema.py new file mode 100644 index 0000000..3b82582 --- /dev/null +++ b/pypaimon/pynative/table/schema.py @@ -0,0 +1,150 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import json +from pathlib import Path +from typing import List, Dict, Optional +import time + +from pypaimon.api import Schema +from pypaimon.pynative.common.file_io import FileIO +from pypaimon.pynative.table import schema_util +from pypaimon.pynative.common.core_option import CoreOptions +from pypaimon.pynative.common.data_field import DataField + + +class TableSchema: + PAIMON_07_VERSION = 1 + PAIMON_08_VERSION = 2 + CURRENT_VERSION = 3 + + def __init__(self, version: int, id: int, fields: List[DataField], highest_field_id: int, + partition_keys: List[str], primary_keys: List[str], options: Dict[str, str], + comment: Optional[str] = None, time_millis: Optional[int] = None): + self.version = version + self.id = id + self.fields = fields + self.highest_field_id = highest_field_id + self.partition_keys = partition_keys or [] + self.primary_keys = primary_keys or [] + self.options = options or {} + self.comment = comment + self.time_millis = time_millis if time_millis is not None else int(time.time() * 1000) + + @staticmethod + def from_path(file_io: FileIO, schema_path: Path): + try: + json_str = file_io.read_file_utf8(schema_path) + return TableSchema.from_json(json_str) + except FileNotFoundError as e: + raise RuntimeError(f"Schema file not found: {schema_path}") from e + except Exception as e: + raise RuntimeError(f"Failed to read schema from {schema_path}") from e + + @staticmethod + def from_json(json_str: str): + try: + data = json.loads(json_str) + + version = data.get("version", TableSchema.PAIMON_07_VERSION) + options = data["options"] + if version <= TableSchema.PAIMON_07_VERSION and CoreOptions.BUCKET not in options: + options[CoreOptions.BUCKET] = "1" + if version <= TableSchema.PAIMON_08_VERSION and CoreOptions.FILE_FORMAT not in options: + options[CoreOptions.FILE_FORMAT] = "orc" + fields = [DataField.from_dict(field) for field in data["fields"]] + + return TableSchema( + version=version, + id=data["id"], + fields=fields, + highest_field_id=data["highestFieldId"], + partition_keys=data["partitionKeys"], + primary_keys=data["primaryKeys"], + options=options, + comment=data.get("comment"), + time_millis=data.get("timeMillis") + ) + except json.JSONDecodeError as e: + raise RuntimeError(f"Invalid JSON format: {json_str}") from e + except KeyError as e: + raise RuntimeError(f"Missing required field in schema JSON: {e}") from e + except Exception as e: + raise RuntimeError(f"Failed to parse schema from JSON: {e}") from e + + @staticmethod + def from_schema(schema: Schema): + fields = schema_util.convert_pa_schema_to_data_fields(schema.pa_schema) + partition_keys = schema.partition_keys + primary_keys = schema.primary_keys + options = schema.options + highest_field_id = schema_util.get_highest_field_id(fields) + return TableSchema( + version=TableSchema.CURRENT_VERSION, + id=0, + fields=fields, + highest_field_id=highest_field_id, + partition_keys=partition_keys, + primary_keys=primary_keys, + options=options, + comment=schema.comment + ) + + def to_json(self) -> str: + data = { + "version": self.version, + "id": self.id, + "fields": [field.to_dict() for field in self.fields], + "highestFieldId": self.highest_field_id, + "partitionKeys": self.partition_keys, + "primaryKeys": self.primary_keys, + "options": self.options, + "timeMillis": self.time_millis + } + if self.comment is not None: + data["comment"] = self.comment + return json.dumps(data, indent=2, ensure_ascii=False) + + def to_schema(self) -> Schema: + pa_schema = schema_util.convert_data_fields_to_pa_schema(self.fields) + return Schema( + pa_schema=pa_schema, + partition_keys=self.partition_keys, + primary_keys=self.primary_keys, + options=self.options, + comment=self.comment + ) + + def get_primary_key_fields(self) -> List[DataField]: + if not self.primary_keys: + return [] + field_map = {field.name: field for field in self.fields} + return [field_map[name] for name in self.primary_keys if name in field_map] + + def get_partition_key_fields(self) -> List[DataField]: + if not self.partition_keys: + return [] + field_map = {field.name: field for field in self.fields} + return [field_map[name] for name in self.partition_keys if name in field_map] + + def get_trimmed_primary_key_fields(self) -> List[DataField]: + if not self.primary_keys or not self.partition_keys: + return self.get_primary_key_fields() + adjusted = [pk for pk in self.primary_keys if pk not in self.partition_keys] + field_map = {field.name: field for field in self.fields} + return [field_map[name] for name in adjusted if name in field_map] diff --git a/pypaimon/pynative/table/schema_manager.py b/pypaimon/pynative/table/schema_manager.py new file mode 100644 index 0000000..1c40511 --- /dev/null +++ b/pypaimon/pynative/table/schema_manager.py @@ -0,0 +1,96 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from pathlib import Path +from typing import Optional + +from pypaimon.api import Schema +from pypaimon.pynative.common.file_io import FileIO +from pypaimon.pynative.table import schema_util +from pypaimon.pynative.table.schema import TableSchema + + +class SchemaManager: + + def __init__(self, file_io: FileIO, table_path: Path): + self.schema_prefix = "schema-" + self.file_io = file_io + self.table_path = table_path + self.schema_path = table_path / "schema" + + def latest(self) -> Optional['TableSchema']: + try: + versions = self._list_versioned_files() + if not versions: + return None + + max_version = max(versions) + return self._read_schema(max_version) + except Exception as e: + raise RuntimeError(f"Failed to load schema from path: {self.schema_path}") from e + + def create_table(self, schema: Schema, external_table: bool = False) -> TableSchema: + while True: + latest = self.latest() + if latest is not None: + if external_table: + schema_util.check_schema_for_external_table(latest.to_schema(), schema) + return latest + else: + raise RuntimeError("Schema in filesystem exists, creation is not allowed.") + + table_schema = TableSchema.from_schema(schema) + success = self.commit(table_schema) + if success: + return table_schema + + def commit(self, new_schema: TableSchema) -> bool: + schema_path = self._to_schema_path(new_schema.id) + try: + return self.file_io.try_to_write_atomic(schema_path, new_schema.to_json()) + except Exception as e: + raise RuntimeError(f"Failed to commit schema: {e}") from e + + def _to_schema_path(self, schema_id: int) -> Path: + return self.schema_path / f"{self.schema_prefix}{schema_id}" + + def _read_schema(self, schema_id: int) -> Optional['TableSchema']: + schema_path = self._to_schema_path(schema_id) + if not self.file_io.exists(schema_path): + return None + + return TableSchema.from_path(self.file_io, schema_path) + + def _list_versioned_files(self) -> list[int]: + if not self.file_io.exists(self.schema_path): + return [] + + statuses = self.file_io.list_status(self.schema_path) + if statuses is None: + return [] + + versions = [] + for status in statuses: + name = Path(status.path).name + if name.startswith(self.schema_prefix): + try: + version = int(name[len(self.schema_prefix):]) + versions.append(version) + except ValueError: + continue + return versions diff --git a/pypaimon/pynative/table/schema_util.py b/pypaimon/pynative/table/schema_util.py new file mode 100644 index 0000000..7fb631e --- /dev/null +++ b/pypaimon/pynative/table/schema_util.py @@ -0,0 +1,140 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import pyarrow as pa +import re + +from pypaimon.api import Schema +from pypaimon.pynative.common.data_field import DataType, DataField + + +def convert_pa_schema_to_data_fields(pa_schema: pa.Schema) -> list[DataField]: + fields = [] + for i, field in enumerate(pa_schema): + field: pa.Field + type_name = str(field.type) + if type_name.startswith('int'): + type_name = 'INT' + elif type_name.startswith('float'): + type_name = 'FLOAT' + elif type_name.startswith('double'): + type_name = 'DOUBLE' + elif type_name.startswith('bool'): + type_name = 'BOOLEAN' + elif type_name.startswith('string'): + type_name = 'STRING' + elif type_name.startswith('binary'): + type_name = 'BINARY' + elif type_name.startswith('date'): + type_name = 'DATE' + elif type_name.startswith('timestamp'): + type_name = 'TIMESTAMP' + elif type_name.startswith('decimal'): + match = re.match(r'decimal\((\d+),\s*(\d+)\)', type_name) + if match: + precision, scale = map(int, match.groups()) + type_name = f'DECIMAL({precision},{scale})' + else: + type_name = 'DECIMAL(38,18)' + elif type_name.startswith('list'): + type_name = 'ARRAY' + elif type_name.startswith('struct'): + type_name = 'STRUCT' + elif type_name.startswith('map'): + type_name = 'MAP' + data_type = DataType(type_name, field.nullable) + + data_field = DataField( + id=i, + name=field.name, + type=data_type, + description=field.metadata.get(b'description', b'').decode + ('utf-8') if field.metadata and b'description' in field.metadata else None + ) + fields.append(data_field) + + return fields + + +def convert_data_fields_to_pa_schema(fields: list[DataField]) -> pa.Schema: + """Convert a list of DataField to PyArrow Schema.""" + pa_fields = [] + for field in fields: + type_name = field.type.type_name.upper() + if type_name == 'INT': + type_name = pa.int32() + elif type_name == 'BIGINT': + type_name = pa.int64() + elif type_name == 'FLOAT': + type_name = pa.float32() + elif type_name == 'DOUBLE': + type_name = pa.float64() + elif type_name == 'BOOLEAN': + type_name = pa.bool_() + elif type_name == 'STRING': + type_name = pa.string() + elif type_name == 'BINARY': + type_name = pa.binary() + elif type_name == 'DATE': + type_name = pa.date32() + elif type_name == 'TIMESTAMP': + type_name = pa.timestamp('ms') + elif type_name.startswith('DECIMAL'): + match = re.match(r'DECIMAL\((\d+),\s*(\d+)\)', type_name) + if match: + precision, scale = map(int, match.groups()) + type_name = pa.decimal128(precision, scale) + else: + type_name = pa.decimal128(38, 18) + elif type_name == 'ARRAY': + # TODO: support arra / struct / map element type + type_name = pa.list_(pa.string()) + elif type_name == 'STRUCT': + type_name = pa.struct([]) + elif type_name == 'MAP': + type_name = pa.map_(pa.string(), pa.string()) + else: + raise ValueError(f"Unsupported data type: {type_name}") + metadata = {} + if field.description: + metadata[b'description'] = field.description.encode('utf-8') + pa_fields.append(pa.field(field.name, type_name, nullable=field.type.nullable, metadata=metadata)) + return pa.schema(pa_fields) + + +def get_highest_field_id(fields: list) -> int: + return max(field.id for field in fields) + + +def check_schema_for_external_table(exists_schema: Schema, new_schema: Schema): + """Check if the new schema is compatible with the existing schema for external table.""" + if ((not new_schema.pa_schema or new_schema.pa_schema.equals(exists_schema.pa_schema)) + and (not new_schema.partition_keys or new_schema.partition_keys == exists_schema.partition_keys) + and (not new_schema.primary_keys or new_schema.primary_keys == exists_schema.primary_keys)): + exists_options = exists_schema.options + new_options = new_schema.options + for key, value in new_options.items(): + if (key != 'owner' and key != 'path' + and (key not in exists_options or exists_options[key] != value)): + raise ValueError( + f"New schema's options are not equal to the exists schema's, " + f"new schema: {new_options}, exists schema: {exists_options}") + else: + raise ValueError( + f"New schema is not equal to the exists schema, " + f"new schema: {new_schema}, exists schema: {exists_schema}") diff --git a/pypaimon/pynative/table/snapshot.py b/pypaimon/pynative/table/snapshot.py new file mode 100644 index 0000000..1bf571b --- /dev/null +++ b/pypaimon/pynative/table/snapshot.py @@ -0,0 +1,71 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import re +from dataclasses import dataclass, fields, asdict +from tracemalloc import Snapshot +from typing import Optional, Dict, Any + + +@dataclass +class Snapshot: + version: int + id: int + schema_id: int + base_manifest_list: str + delta_manifest_list: str + commit_user: str + commit_identifier: int + commit_kind: str + time_millis: int + log_offsets: Dict[int, int] + + changelog_manifest_list: Optional[str] = None + index_manifest: Optional[str] = None + total_record_count: Optional[int] = None + delta_record_count: Optional[int] = None + changelog_record_count: Optional[int] = None + watermark: Optional[int] = None + statistics: Optional[str] = None + + @staticmethod + def from_json(data: Dict[str, Any]) -> Snapshot: + known_fields = {field.name for field in fields(Snapshot)} + processed_data = { + camel_to_snake(key): value + for key, value in data.items() + if camel_to_snake(key) in known_fields + } + return Snapshot(**processed_data) + + def to_json(self) -> Dict[str, Any]: + snake_case_dict = asdict(self) + return { + snake_to_camel(key): value + for key, value in snake_case_dict.items() + } + + +def camel_to_snake(name: str) -> str: + s1 = re.sub('(.)([A-Z][a-z]+)', r'\1_\2', name) + return re.sub('([a-z0-9])([A-Z])', r'\1_\2', s1).lower() + + +def snake_to_camel(name: str) -> str: + parts = name.split('_') + return parts[0] + ''.join(word.capitalize() for word in parts[1:]) diff --git a/pypaimon/pynative/table/snapshot_manager.py b/pypaimon/pynative/table/snapshot_manager.py new file mode 100644 index 0000000..5d82108 --- /dev/null +++ b/pypaimon/pynative/table/snapshot_manager.py @@ -0,0 +1,68 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import json +from typing import Optional + +from pypaimon.pynative.common.file_io import FileIO +from pypaimon.pynative.table.snapshot import Snapshot + + +class SnapshotManager: + """Manager for snapshot files using unified FileIO.""" + + def __init__(self, table): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.file_io: FileIO = self.table.file_io + self.snapshot_dir = self.table.table_path / "snapshot" + + def get_latest_snapshot(self) -> Optional[Snapshot]: + latest_file = self.snapshot_dir / "LATEST" + if not self.file_io.exists(latest_file): + return None + + latest_content = self.file_io.read_file_utf8(latest_file) + latest_snapshot_id = int(latest_content.strip()) + + snapshot_file = self.snapshot_dir / f"snapshot-{latest_snapshot_id}" + if not self.file_io.exists(snapshot_file): + return None + + snapshot_content = self.file_io.read_file_utf8(snapshot_file) + snapshot_data = json.loads(snapshot_content) + return Snapshot.from_json(snapshot_data) + + def commit_snapshot(self, snapshot_id: int, snapshot_data: Snapshot): + snapshot_file = self.snapshot_dir / f"snapshot-{snapshot_id}" + latest_file = self.snapshot_dir / "LATEST" + + try: + snapshot_json = json.dumps(snapshot_data.to_json(), indent=2) + snapshot_success = self.file_io.try_to_write_atomic(snapshot_file, snapshot_json) + if not snapshot_success: + self.file_io.write_file(snapshot_file, snapshot_json, overwrite=True) + + latest_success = self.file_io.try_to_write_atomic(latest_file, str(snapshot_id)) + if not latest_success: + self.file_io.write_file(latest_file, str(snapshot_id), overwrite=True) + + except Exception as e: + self.file_io.delete_quietly(snapshot_file) + raise RuntimeError(f"Failed to commit snapshot {snapshot_id}: {e}") from e diff --git a/pypaimon/pynative/tests/__init__.py b/pypaimon/pynative/tests/__init__.py index e173487..92d694e 100644 --- a/pypaimon/pynative/tests/__init__.py +++ b/pypaimon/pynative/tests/__init__.py @@ -21,6 +21,7 @@ import tempfile import unittest +from pypaimon.api.catalog_factory import CatalogFactory from pypaimon.py4j import Catalog, constants @@ -42,6 +43,7 @@ def setUpClass(cls): cls.warehouse = os.path.join(cls.tempdir, 'warehouse') cls.catalog = Catalog.create({'warehouse': cls.warehouse}) cls.catalog.create_database('default', False) + cls.native_catalog = CatalogFactory.create({"warehouse": cls.warehouse}) @classmethod def tearDownClass(cls): diff --git a/pypaimon/pynative/tests/test_append_only_reader.py b/pypaimon/pynative/tests/test_append_only_reader.py new file mode 100644 index 0000000..2c94fbe --- /dev/null +++ b/pypaimon/pynative/tests/test_append_only_reader.py @@ -0,0 +1,204 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import pyarrow as pa + +from pypaimon.api import Schema +from pypaimon.pynative.tests import PypaimonTestBase + + +class NativeAppendOnlyReaderTest(PypaimonTestBase): + + @classmethod + def setUpClass(cls): + super().setUpClass() + cls.simple_pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('behavior', pa.string()), + ('dt', pa.string()) + ]) + cls.expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 3, 4, 5, 6, 7, 8], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008], + 'behavior': ['a', 'b', 'c', None, 'e', 'f', 'g', 'h'], + 'dt': ['p1', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p2'], + }, schema=cls.simple_pa_schema) + + def testParquetAppendOnlyReader(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_append_only_parquet', schema, False) + j_table = self.catalog.get_table('default.test_append_only_parquet') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_append_only_parquet") + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + self.assertEqual(actual, self.expected) + + def testOrcAppendOnlyReader(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt'], options={'file.format': 'orc'}) + self.catalog.create_table('default.test_append_only_orc', schema, False) + j_table = self.catalog.get_table('default.test_append_only_orc') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_append_only_orc") + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + self.assertEqual(actual, self.expected) + + def testAvroAppendOnlyReader(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt'], options={'file.format': 'avro'}) + self.catalog.create_table('default.test_append_only_avro', schema, False) + j_table = self.catalog.get_table('default.test_append_only_avro') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_append_only_avro") + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + self.assertEqual(actual, self.expected) + + def testAppendOnlyReaderWithFilter(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_append_only_filter', schema, False) + j_table = self.catalog.get_table('default.test_append_only_filter') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_append_only_filter") + predicate_builder = table.new_read_builder().new_predicate_builder() + p1 = predicate_builder.less_than('user_id', 7) + p2 = predicate_builder.greater_or_equal('user_id', 2) + p3 = predicate_builder.between('user_id', 0, 6) # [2/b, 3/c, 4/d, 5/e, 6/f] left + p4 = predicate_builder.is_not_in('behavior', ['b', 'e']) # [3/c, 4/d, 6/f] left + p5 = predicate_builder.is_in('dt', ['p1']) # exclude 3/c + p6 = predicate_builder.is_not_null('behavior') # exclude 4/d + g1 = predicate_builder.and_predicates([p1, p2, p3, p4, p5, p6]) + read_builder = table.new_read_builder().with_filter(g1) + actual = self._read_test_table(read_builder) + expected = pa.concat_tables([ + self.expected.slice(5, 1) # 6/f + ]) + self.assertEqual(actual.sort_by('user_id'), expected) + + p7 = predicate_builder.startswith('behavior', 'a') + p10 = predicate_builder.equal('item_id', 1002) + p11 = predicate_builder.is_null('behavior') + p9 = predicate_builder.contains('behavior', 'f') + p8 = predicate_builder.endswith('dt', 'p2') + g2 = predicate_builder.or_predicates([p7, p8, p9, p10, p11]) + read_builder = table.new_read_builder().with_filter(g2) + actual = self._read_test_table(read_builder) + self.assertEqual(actual.sort_by('user_id'), self.expected) + + g3 = predicate_builder.and_predicates([g1, g2]) + read_builder = table.new_read_builder().with_filter(g3) + actual = self._read_test_table(read_builder) + expected = pa.concat_tables([ + self.expected.slice(5, 1) # 6/f + ]) + self.assertEqual(actual.sort_by('user_id'), expected) + + # Same as java, 'not_equal' will also filter records of 'None' value + p12 = predicate_builder.not_equal('behavior', 'f') + read_builder = table.new_read_builder().with_filter(p12) + actual = self._read_test_table(read_builder) + expected = pa.concat_tables([ + # not only 6/f, but also 4/d will be filtered + self.expected.slice(0, 1), # 1/a + self.expected.slice(1, 1), # 2/b + self.expected.slice(2, 1), # 3/c + self.expected.slice(4, 1), # 5/e + self.expected.slice(6, 1), # 7/g + self.expected.slice(7, 1), # 8/h + ]) + self.assertEqual(actual.sort_by('user_id'), expected) + + def testAppendOnlyReaderWithProjection(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_append_only_projection', schema, False) + j_table = self.catalog.get_table('default.test_append_only_projection') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_append_only_projection") + read_builder = table.new_read_builder().with_projection(['dt', 'user_id']) + actual = self._read_test_table(read_builder).sort_by('user_id') + expected = self.expected.select(['dt', 'user_id']) + self.assertEqual(actual, expected) + + def testAvroAppendOnlyReaderWithProjection(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt'], options={'file.format': 'avro'}) + self.catalog.create_table('default.test_avro_append_only_projection', schema, False) + j_table = self.catalog.get_table('default.test_avro_append_only_projection') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_avro_append_only_projection") + read_builder = table.new_read_builder().with_projection(['dt', 'user_id']) + actual = self._read_test_table(read_builder).sort_by('user_id') + expected = self.expected.select(['dt', 'user_id']) + self.assertEqual(actual, expected) + + def testAppendOnlyReaderWithLimit(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_append_only_limit', schema, False) + j_table = self.catalog.get_table('default.test_append_only_limit') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_append_only_limit") + read_builder = table.new_read_builder().with_limit(1) + actual = self._read_test_table(read_builder) + # only records from 1st commit (1st split) will be read + # might be split of "dt=1" or split of "dt=2" + self.assertEqual(actual.num_rows, 4) + + def _write_test_table(self, table): + write_builder = table.new_batch_write_builder() + + # first write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4], + 'item_id': [1001, 1002, 1003, 1004], + 'behavior': ['a', 'b', 'c', None], + 'dt': ['p1', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=self.simple_pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # second write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data2 = { + 'user_id': [5, 6, 7, 8], + 'item_id': [1005, 1006, 1007, 1008], + 'behavior': ['e', 'f', 'g', 'h'], + 'dt': ['p2', 'p1', 'p2', 'p2'], + } + pa_table = pa.Table.from_pydict(data2, schema=self.simple_pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + def _read_test_table(self, read_builder): + table_read = read_builder.new_read() + splits = read_builder.new_scan().plan().splits() + return table_read.to_arrow(splits) diff --git a/pypaimon/pynative/tests/test_native_write_and_read.py b/pypaimon/pynative/tests/test_native_write_and_read.py new file mode 100644 index 0000000..a7fda71 --- /dev/null +++ b/pypaimon/pynative/tests/test_native_write_and_read.py @@ -0,0 +1,71 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import shutil + +import pyarrow as pa + +import os +import tempfile + +from pypaimon.api import Schema +from pypaimon.api.catalog_factory import CatalogFactory +from pypaimon.pynative.tests import PypaimonTestBase + + +class NativeFullTest(PypaimonTestBase): + + def testWriteAndRead(self): + tempdir = tempfile.mkdtemp() + warehouse = os.path.join(tempdir, 'warehouse') + catalog = CatalogFactory.create({ + "warehouse": warehouse + }) + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()) + ]) + data = { + 'f0': [1, 2, 3], + 'f1': ['a', 'b', 'c'], + 'f2': ['X', 'Y', 'Z'] + } + expect = pa.Table.from_pydict(data, schema=simple_pa_schema) + catalog.create_database("test_db", False) + catalog.create_table("test_db.native_full", Schema(simple_pa_schema, options={}), False) + table = catalog.get_table("test_db.native_full") + + # write + write_builder = table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + table_write.write_arrow(expect) + commit_messages = table_write.prepare_commit() + table_commit.commit(commit_messages) + table_write.close() + table_commit.close() + + # read + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + splits = table_scan.plan().splits() + actual = table_read.to_arrow(splits) + self.assertEqual(actual, expect) + + shutil.rmtree(tempdir, ignore_errors=True) diff --git a/pypaimon/pynative/tests/test_primary_key_reader.py b/pypaimon/pynative/tests/test_primary_key_reader.py new file mode 100644 index 0000000..02d32b1 --- /dev/null +++ b/pypaimon/pynative/tests/test_primary_key_reader.py @@ -0,0 +1,156 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import pyarrow as pa + +from pypaimon.api import Schema +from pypaimon.pynative.tests import PypaimonTestBase + + +class NativeReaderTest(PypaimonTestBase): + + @classmethod + def setUpClass(cls): + super().setUpClass() + cls.simple_pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('behavior', pa.string()), + ('dt', pa.string()) + ]) + cls.expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 3, 4, 5, 7, 8], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1007, 1008], + 'behavior': ['a', 'b-new', 'c', None, 'e', 'g', 'h'], + 'dt': ['p1', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2'], + }, schema=cls.simple_pa_schema) + + def testPkParquetReader(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt'], primary_keys=['user_id', 'dt'], + options={'bucket': '2'}) + self.catalog.create_table('default.test_pk_parquet', schema, False) + j_table = self.catalog.get_table('default.test_pk_parquet') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_pk_parquet") + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + self.assertEqual(actual, self.expected) + + def testPkOrcReader(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt'], primary_keys=['user_id', 'dt'], options={ + 'bucket': '1', + 'file.format': 'orc' + }) + self.catalog.create_table('default.test_pk_orc', schema, False) + j_table = self.catalog.get_table('default.test_pk_orc') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_pk_orc") + read_builder = table.new_read_builder() + actual: pa.Table = self._read_test_table(read_builder).sort_by('user_id') + + # when bucket=1, actual field name will contain 'not null', so skip comparing field name + for i in range(len(actual.columns)): + col_a = actual.column(i) + col_b = self.expected.column(i) + self.assertEqual(col_a, col_b) + + def testPkAvroReader(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt'], primary_keys=['user_id', 'dt'], options={ + 'bucket': '2', + 'file.format': 'avro' + }) + self.catalog.create_table('default.test_pk_avro', schema, False) + j_table = self.catalog.get_table('default.test_pk_avro') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_pk_avro") + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + self.assertEqual(actual, self.expected) + + def testPkReaderWithFilter(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt'], primary_keys=['user_id', 'dt'], + options={'bucket': '2'}) + self.catalog.create_table('default.test_pk_filter', schema, False) + j_table = self.catalog.get_table('default.test_pk_filter') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_pk_filter") + predicate_builder = table.new_read_builder().new_predicate_builder() + p1 = predicate_builder.is_in('dt', ['p1']) + p2 = predicate_builder.between('user_id', 2, 7) + p3 = predicate_builder.is_not_null('behavior') + g1 = predicate_builder.and_predicates([p1, p2, p3]) + read_builder = table.new_read_builder().with_filter(g1) + actual = self._read_test_table(read_builder).sort_by('user_id') + expected = pa.concat_tables([ + self.expected.slice(1, 1), # 2/b + self.expected.slice(5, 1) # 7/g + ]) + self.assertEqual(actual, expected) + + def testPkReaderWithProjection(self): + schema = Schema(self.simple_pa_schema, partition_keys=['dt'], primary_keys=['user_id', 'dt'], + options={'bucket': '2'}) + self.catalog.create_table('default.test_pk_projection', schema, False) + j_table = self.catalog.get_table('default.test_pk_projection') + self._write_test_table(j_table) + + table = self.native_catalog.get_table("default.test_pk_projection") + read_builder = table.new_read_builder().with_projection(['dt', 'user_id', 'behavior']) + actual = self._read_test_table(read_builder).sort_by('user_id') + expected = self.expected.select(['dt', 'user_id', 'behavior']) + self.assertEqual(actual, expected) + + def _write_test_table(self, table): + write_builder = table.new_batch_write_builder() + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4], + 'item_id': [1001, 1002, 1003, 1004], + 'behavior': ['a', 'b', 'c', None], + 'dt': ['p1', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=self.simple_pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [5, 2, 7, 8], + 'item_id': [1005, 1002, 1007, 1008], + 'behavior': ['e', 'b-new', 'g', 'h'], + 'dt': ['p2', 'p1', 'p1', 'p2'] + } + pa_table = pa.Table.from_pydict(data1, schema=self.simple_pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + def _read_test_table(self, read_builder): + table_read = read_builder.new_read() + splits = read_builder.new_scan().plan().splits() + return table_read.to_arrow(splits) diff --git a/pypaimon/pynative/tests/test_pynative_reader.py b/pypaimon/pynative/tests/test_pynative_reader.py deleted file mode 100644 index fe9efb3..0000000 --- a/pypaimon/pynative/tests/test_pynative_reader.py +++ /dev/null @@ -1,393 +0,0 @@ -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -import pandas as pd -import pyarrow as pa - -from pypaimon import Schema -from pypaimon.py4j.tests import PypaimonTestBase - - -class NativeReaderTest(PypaimonTestBase): - - @classmethod - def setUpClass(cls): - super().setUpClass() - cls.simple_pa_schema = pa.schema([ - ('f0', pa.int32()), - ('f1', pa.string()), - ('f2', pa.string()) - ]) - cls.pk_pa_schema = pa.schema([ - ('f0', pa.int32(), False), - ('f1', pa.string()), - ('f2', pa.string()) - ]) - cls.partition_pk_pa_schema = pa.schema([ - ('user_id', pa.int32(), False), - ('item_id', pa.int32()), - ('behavior', pa.string()), - ('dt', pa.string(), False) - ]) - cls._expected_full_data = pd.DataFrame({ - 'f0': [1, 2, 3, 4, 5, 6, 7, 8], - 'f1': ['a', 'b', 'c', None, 'e', 'f', 'g', 'h'], - 'f2': ['A', 'B', 'C', 'D', 'E', 'F', 'G', None], - }) - cls._expected_full_data['f0'] = cls._expected_full_data['f0'].astype('int32') - cls.expected_full = pa.Table.from_pandas(cls._expected_full_data, - schema=cls.simple_pa_schema) - cls._expected_full_data_pk = pd.DataFrame({ - 'f0': [1, 2, 3, 4, 6], - 'f1': ['a', 'x', 'y', None, 'z'], - 'f2': ['A', 'X', 'Y', 'D', 'Z'], - }) - cls._expected_full_data_pk['f0'] = cls._expected_full_data_pk['f0'].astype('int32') - cls.expected_full_pk = pa.Table.from_pandas(cls._expected_full_data_pk, - schema=cls.pk_pa_schema) - - def testParquetAppendOnlyReader(self): - schema = Schema(self.simple_pa_schema) - self.catalog.create_table('default.test_append_only_parquet', schema, False) - table = self.catalog.get_table('default.test_append_only_parquet') - self._write_test_table(table) - - read_builder = table.new_read_builder() - actual = self._read_test_table(read_builder) - self.assertEqual(actual, self.expected_full) - - def testOrcAppendOnlyReader(self): - schema = Schema(self.simple_pa_schema, options={'file.format': 'orc'}) - self.catalog.create_table('default.test_append_only_orc', schema, False) - table = self.catalog.get_table('default.test_append_only_orc') - self._write_test_table(table) - - read_builder = table.new_read_builder() - actual = self._read_test_table(read_builder) - self.assertEqual(actual, self.expected_full) - - def testAvroAppendOnlyReader(self): - schema = Schema(self.simple_pa_schema, options={'file.format': 'avro'}) - self.catalog.create_table('default.test_append_only_avro', schema, False) - table = self.catalog.get_table('default.test_append_only_avro') - self._write_test_table(table) - - read_builder = table.new_read_builder() - actual = self._read_test_table(read_builder) - self.assertEqual(actual, self.expected_full) - - def testAppendOnlyReaderWithFilter(self): - schema = Schema(self.simple_pa_schema) - self.catalog.create_table('default.test_append_only_filter', schema, False) - table = self.catalog.get_table('default.test_append_only_filter') - self._write_test_table(table) - predicate_builder = table.new_read_builder().new_predicate_builder() - - p1 = predicate_builder.less_than('f0', 7) - p2 = predicate_builder.greater_or_equal('f0', 2) - p3 = predicate_builder.between('f0', 0, 5) # from now, [2/b, 3/c, 4/d, 5/e] left - p4 = predicate_builder.is_not_in('f1', ['a', 'b']) # exclude 2/b - p5 = predicate_builder.is_in('f2', ['A', 'B', 'D', 'E', 'F', 'G']) # exclude 3/c - p6 = predicate_builder.is_not_null('f1') # exclude 4/d - g1 = predicate_builder.and_predicates([p1, p2, p3, p4, p5, p6]) - read_builder = table.new_read_builder().with_filter(g1) - actual = self._read_test_table(read_builder) - expected = pa.concat_tables([ - self.expected_full.slice(4, 1) # 5/e - ]) - self.assertEqual(actual, expected) - - p7 = predicate_builder.startswith('f1', 'a') - p8 = predicate_builder.endswith('f2', 'C') - p9 = predicate_builder.contains('f2', 'E') - p10 = predicate_builder.equal('f1', 'f') - p11 = predicate_builder.is_null('f2') - g2 = predicate_builder.or_predicates([p7, p8, p9, p10, p11]) - read_builder = table.new_read_builder().with_filter(g2) - actual = self._read_test_table(read_builder) - expected = pa.concat_tables([ - self.expected_full.slice(0, 1), # 1/a - self.expected_full.slice(2, 1), # 3/c - self.expected_full.slice(4, 1), # 5/e - self.expected_full.slice(5, 1), # 6/f - self.expected_full.slice(7, 1), # 8/h - ]) - self.assertEqual(actual, expected) - - g3 = predicate_builder.and_predicates([g1, g2]) - read_builder = table.new_read_builder().with_filter(g3) - actual = self._read_test_table(read_builder) - expected = pa.concat_tables([ - self.expected_full.slice(4, 1), # 5/e - ]) - self.assertEqual(actual, expected) - - # Same as java, 'not_equal' will also filter records of 'None' value - p12 = predicate_builder.not_equal('f1', 'f') - read_builder = table.new_read_builder().with_filter(p12) - actual = self._read_test_table(read_builder) - expected = pa.concat_tables([ - # not only 6/f, but also 4/d will be filtered - self.expected_full.slice(0, 1), # 1/a - self.expected_full.slice(1, 1), # 2/b - self.expected_full.slice(2, 1), # 3/c - self.expected_full.slice(4, 1), # 5/e - self.expected_full.slice(6, 1), # 7/g - self.expected_full.slice(7, 1), # 8/h - ]) - self.assertEqual(actual, expected) - - def testAppendOnlyReaderWithProjection(self): - schema = Schema(self.simple_pa_schema) - self.catalog.create_table('default.test_append_only_projection', schema, False) - table = self.catalog.get_table('default.test_append_only_projection') - self._write_test_table(table) - - read_builder = table.new_read_builder().with_projection(['f0', 'f2']) - actual = self._read_test_table(read_builder) - expected = self.expected_full.select(['f0', 'f2']) - self.assertEqual(actual, expected) - - def testAppendOnlyReaderWithLimit(self): - schema = Schema(self.simple_pa_schema, options={'source.split.target-size': '1mb'}) - self.catalog.create_table('default.test_append_only_limit', schema, False) - table = self.catalog.get_table('default.test_append_only_limit') - self._write_test_table(table) - - read_builder = table.new_read_builder().with_limit(1) - actual = self._read_test_table(read_builder) - # only records from 1st commit (1st split) will be read - expected = pa.concat_tables([ - self.expected_full.slice(0, 1), # 1/a - self.expected_full.slice(1, 1), # 2/b - self.expected_full.slice(2, 1), # 3/c - self.expected_full.slice(3, 1), # 4/d - ]) - self.assertEqual(actual, expected) - - # TODO: test cases for avro filter and projection - - def testPkParquetReader(self): - schema = Schema(self.pk_pa_schema, primary_keys=['f0'], options={ - 'bucket': '1' - }) - self.catalog.create_table('default.test_pk_parquet', schema, False) - table = self.catalog.get_table('default.test_pk_parquet') - self._write_test_table(table, for_pk=True) - - read_builder = table.new_read_builder() - actual = self._read_test_table(read_builder) - self.assertEqual(actual, self.expected_full_pk) - - def testPkParquetReaderWithMinHeap(self): - schema = Schema(self.pk_pa_schema, primary_keys=['f0'], options={ - 'bucket': '1', - 'sort-engine': 'min-heap' - }) - self.catalog.create_table('default.test_pk_parquet_loser_tree', schema, False) - table = self.catalog.get_table('default.test_pk_parquet_loser_tree') - self._write_test_table(table, for_pk=True) - - read_builder = table.new_read_builder() - actual = self._read_test_table(read_builder) - self.assertEqual(actual, self.expected_full_pk) - - def skip_testPkOrcReader(self): - schema = Schema(self.pk_pa_schema, primary_keys=['f0'], options={ - 'bucket': '1', - 'file.format': 'orc' - }) - self.catalog.create_table('default.test_pk_orc', schema, False) - table = self.catalog.get_table('default.test_pk_orc') - self._write_test_table(table, for_pk=True) - - read_builder = table.new_read_builder() - actual = self._read_test_table(read_builder) - self.assertEqual(actual, self.expected_full_pk) - - def skip_testPkAvroReader(self): - schema = Schema(self.pk_pa_schema, primary_keys=['f0'], options={ - 'bucket': '1', - 'file.format': 'avro' - }) - self.catalog.create_table('default.test_pk_avro', schema, False) - table = self.catalog.get_table('default.test_pk_avro') - self._write_test_table(table, for_pk=True) - - read_builder = table.new_read_builder() - actual = self._read_test_table(read_builder) - self.assertEqual(actual, self.expected_full_pk) - - def testPkReaderWithFilter(self): - schema = Schema(self.pk_pa_schema, primary_keys=['f0'], options={ - 'bucket': '1' - }) - self.catalog.create_table('default.test_pk_filter', schema, False) - table = self.catalog.get_table('default.test_pk_filter') - self._write_test_table(table, for_pk=True) - predicate_builder = table.new_read_builder().new_predicate_builder() - - p1 = predicate_builder.between('f0', 0, 5) - p2 = predicate_builder.is_not_in('f1', ['a', 'x']) - p3 = predicate_builder.is_not_null('f1') - g1 = predicate_builder.and_predicates([p1, p2, p3]) - p4 = predicate_builder.equal('f2', 'Z') - g2 = predicate_builder.or_predicates([g1, p4]) - read_builder = table.new_read_builder().with_filter(g2) - actual = self._read_test_table(read_builder) - expected = pa.concat_tables([ - self.expected_full_pk.slice(2, 1), # 3/y - self.expected_full_pk.slice(4, 1), # 6/z - ]) - self.assertEqual(actual, expected) - - def testPkReaderWithProjection(self): - schema = Schema(self.pk_pa_schema, primary_keys=['f0'], options={ - 'bucket': '1' - }) - self.catalog.create_table('default.test_pk_projection', schema, False) - table = self.catalog.get_table('default.test_pk_projection') - self._write_test_table(table, for_pk=True) - - read_builder = table.new_read_builder().with_projection(['f0', 'f2']) - actual = self._read_test_table(read_builder) - expected = self.expected_full_pk.select(['f0', 'f2']) - self.assertEqual(actual, expected) - - def testPartitionPkParquetReader(self): - schema = Schema(self.partition_pk_pa_schema, - partition_keys=['dt'], - primary_keys=['dt', 'user_id'], - options={ - 'bucket': '2' - }) - self.catalog.create_table('default.test_partition_pk_parquet', schema, False) - table = self.catalog.get_table('default.test_partition_pk_parquet') - self._write_partition_test_table(table) - - read_builder = table.new_read_builder() - actual = self._read_test_table(read_builder) - expected = pa.Table.from_pandas( - pd.DataFrame({ - 'user_id': [1, 2, 3, 4, 5, 7, 8], - 'item_id': [1, 2, 3, 4, 5, 7, 8], - 'behavior': ["b-1", "b-2-new", "b-3", None, "b-5", "b-7", None], - 'dt': ["p-1", "p-1", "p-1", "p-1", "p-2", "p-1", "p-2"] - }), - schema=self.partition_pk_pa_schema) - self.assertEqual(actual.sort_by('user_id'), expected) - - def testPartitionPkParquetReaderWriteOnce(self): - schema = Schema(self.partition_pk_pa_schema, - partition_keys=['dt'], - primary_keys=['dt', 'user_id'], - options={ - 'bucket': '1' - }) - self.catalog.create_table('default.test_partition_pk_parquet2', schema, False) - table = self.catalog.get_table('default.test_partition_pk_parquet2') - self._write_partition_test_table(table, write_once=True) - - read_builder = table.new_read_builder() - actual = self._read_test_table(read_builder) - expected = pa.Table.from_pandas( - pd.DataFrame({ - 'user_id': [1, 2, 3, 4], - 'item_id': [1, 2, 3, 4], - 'behavior': ['b-1', 'b-2', 'b-3', None], - 'dt': ['p-1', 'p-1', 'p-1', 'p-1'] - }), schema=self.partition_pk_pa_schema) - self.assertEqual(actual, expected) - - def _write_test_table(self, table, for_pk=False): - write_builder = table.new_batch_write_builder() - - # first write - table_write = write_builder.new_write() - table_commit = write_builder.new_commit() - data1 = { - 'f0': [1, 2, 3, 4], - 'f1': ['a', 'b', 'c', None], - 'f2': ['A', 'B', 'C', 'D'], - } - pa_table = pa.Table.from_pydict(data1, schema=self.simple_pa_schema) - table_write.write_arrow(pa_table) - table_commit.commit(table_write.prepare_commit()) - table_write.close() - table_commit.close() - - # second write - table_write = write_builder.new_write() - table_commit = write_builder.new_commit() - if for_pk: - data2 = { - 'f0': [2, 3, 6], - 'f1': ['x', 'y', 'z'], - 'f2': ['X', 'Y', 'Z'], - } - else: - data2 = { - 'f0': [5, 6, 7, 8], - 'f1': ['e', 'f', 'g', 'h'], - 'f2': ['E', 'F', 'G', None], - } - pa_table = pa.Table.from_pydict(data2, schema=self.simple_pa_schema) - table_write.write_arrow(pa_table) - table_commit.commit(table_write.prepare_commit()) - table_write.close() - table_commit.close() - - def _write_partition_test_table(self, table, write_once=False): - write_builder = table.new_batch_write_builder() - - table_write = write_builder.new_write() - table_commit = write_builder.new_commit() - data1 = { - 'user_id': [1, 2, 3, 4], - 'item_id': [1, 2, 3, 4], - 'behavior': ['b-1', 'b-2', 'b-3', None], - 'dt': ['p-1', 'p-1', 'p-1', 'p-1'] - } - pa_table = pa.Table.from_pydict(data1, schema=self.partition_pk_pa_schema) - table_write.write_arrow(pa_table) - table_commit.commit(table_write.prepare_commit()) - table_write.close() - table_commit.close() - - if write_once: - return - - table_write = write_builder.new_write() - table_commit = write_builder.new_commit() - data1 = { - 'user_id': [5, 2, 7, 8], - 'item_id': [5, 2, 7, 8], - 'behavior': ['b-5', 'b-2-new', 'b-7', None], - 'dt': ['p-2', 'p-1', 'p-1', 'p-2'] - } - pa_table = pa.Table.from_pydict(data1, schema=self.partition_pk_pa_schema) - table_write.write_arrow(pa_table) - table_commit.commit(table_write.prepare_commit()) - table_write.close() - table_commit.close() - - def _read_test_table(self, read_builder): - table_read = read_builder.new_read() - splits = read_builder.new_scan().plan().splits() - self.assertNotEqual(table_read.to_record_generator(splits), None) - return table_read.to_arrow(splits) diff --git a/pypaimon/pynative/util/global_timer.py b/pypaimon/pynative/util/global_timer.py new file mode 100644 index 0000000..e945ba7 --- /dev/null +++ b/pypaimon/pynative/util/global_timer.py @@ -0,0 +1,158 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import time +import functools +from threading import local +from contextlib import contextmanager + +_thread_local = local() + + +def _get_thread_storage(): + if not hasattr(_thread_local, 'storage'): + _thread_local.storage = { + 'is_enabled': True, + 'records': [], + 'scope_stack': [] + } + return _thread_local.storage + + +class GlobalTimer: + _instance = None + + def __new__(cls, *args, **kwargs): + if not cls._instance: + cls._instance = super().__new__(cls) + return cls._instance + + @property + def _storage(self): + return _get_thread_storage() + + def enable(self): + self._storage['is_enabled'] = True + + def disable(self): + self._storage['is_enabled'] = False + + def is_enabled(self): + return self._storage['is_enabled'] + + def reset(self): + self._storage['records'] = [] + self._storage['scope_stack'] = [] + self.mark("start") + + def __init__(self): + if not hasattr(self, '_initialized'): + self._initialized = True + self.reset() + + def mark(self, name: str): + if not self.is_enabled(): + return + + depth = len(self._storage['scope_stack']) + self._storage['records'].append({ + 'name': name, + 'type': 'mark', + 'time': time.perf_counter(), + 'depth': depth + }) + + @contextmanager + def scope(self, name: str): + if not self.is_enabled(): + yield + return + + depth = len(self._storage['scope_stack']) + start_time = time.perf_counter() + + self._storage['scope_stack'].append(name) + self._storage['records'].append({ + 'name': name, + 'type': 'scope_start', + 'time': start_time, + 'depth': depth + }) + + try: + yield + finally: + end_time = time.perf_counter() + self._storage['records'].append({ + 'name': name, + 'type': 'scope_end', + 'time': end_time, + 'duration': end_time - start_time, + 'depth': depth + }) + self._storage['scope_stack'].pop() + + def timed(self, name: str = None): + + def decorator(func): + @functools.wraps(func) + def wrapper(*args, **kwargs): + scope_name = name if name is not None else func.__name__ + with self.scope(scope_name): + return func(*args, **kwargs) + + return wrapper + + return decorator + + def report(self, sort_by='time'): + if not self.is_enabled(): + print("GlobalTimer is disabled.") + return + + records = self._storage['records'] + if not records: + print("No records to report.") + return + + print("\n--- Global Timer Report ---") + + print("\n[Timeline View]") + start_time = records[0]['time'] + for record in records: + indent = " " * record['depth'] + elapsed = record['time'] - start_time + if record['type'] == 'mark': + print(f"{indent}{record['name']:<20} | at {elapsed:8.4f}s") + elif record['type'] == 'scope_start': + print(f"{indent}Scope Start: {record['name']:<20} | at {elapsed:8.4f}s") + elif record['type'] == 'scope_end': + print( + f"{indent}Scope End: {record['name']:<20} | at {elapsed:8.4f}s (took {record['duration']:.4f}s)") + + print("\n[Top Scopes by Duration]") + scopes = [r for r in records if r['type'] == 'scope_end'] + sorted_scopes = sorted(scopes, key=lambda x: x['duration'], reverse=True) + + for scope in sorted_scopes[:10]: # 只显示耗时最长的前10个 + print(f"- {scope['name']:<25}: {scope['duration']:.4f}s") + + print("\n--- End of Report ---\n") + + +profiler = GlobalTimer() diff --git a/pypaimon/pynative/util/predicate_converter.py b/pypaimon/pynative/util/predicate_converter.py index e3c5499..d84c98f 100644 --- a/pypaimon/pynative/util/predicate_converter.py +++ b/pypaimon/pynative/util/predicate_converter.py @@ -23,55 +23,54 @@ import pyarrow.dataset as ds from pyarrow.dataset import Expression -from pypaimon import Predicate +from pypaimon.api import Predicate +from pypaimon.pynative.common.predicate import PredicateImpl def convert_predicate(predicate: Predicate) -> Expression | bool: """ # Convert Paimon's Predicate to PyArrow Dataset's filter """ - if not hasattr(predicate, 'py_predicate'): - raise ValueError("Predicate must have py_predicate attribute") + if not isinstance(predicate, PredicateImpl): + raise RuntimeError("Type of predicate should be PredicateImpl") - py_predicate = predicate.py_predicate - - if py_predicate.method == 'equal': - return ds.field(py_predicate.field) == py_predicate.literals[0] - elif py_predicate.method == 'notEqual': - return ds.field(py_predicate.field) != py_predicate.literals[0] - elif py_predicate.method == 'lessThan': - return ds.field(py_predicate.field) < py_predicate.literals[0] - elif py_predicate.method == 'lessOrEqual': - return ds.field(py_predicate.field) <= py_predicate.literals[0] - elif py_predicate.method == 'greaterThan': - return ds.field(py_predicate.field) > py_predicate.literals[0] - elif py_predicate.method == 'greaterOrEqual': - return ds.field(py_predicate.field) >= py_predicate.literals[0] - elif py_predicate.method == 'isNull': - return ds.field(py_predicate.field).is_null() - elif py_predicate.method == 'isNotNull': - return ds.field(py_predicate.field).is_valid() - elif py_predicate.method == 'in': - return ds.field(py_predicate.field).isin(py_predicate.literals) - elif py_predicate.method == 'notIn': - return ~ds.field(py_predicate.field).isin(py_predicate.literals) - elif py_predicate.method == 'startsWith': - pattern = py_predicate.literals[0] - return pc.starts_with(ds.field(py_predicate.field).cast(pa.string()), pattern) - elif py_predicate.method == 'endsWith': - pattern = py_predicate.literals[0] - return pc.ends_with(ds.field(py_predicate.field).cast(pa.string()), pattern) - elif py_predicate.method == 'contains': - pattern = py_predicate.literals[0] - return pc.match_substring(ds.field(py_predicate.field).cast(pa.string()), pattern) - elif py_predicate.method == 'between': - return (ds.field(py_predicate.field) >= py_predicate.literals[0]) & \ - (ds.field(py_predicate.field) <= py_predicate.literals[1]) - elif py_predicate.method == 'and': + if predicate.method == 'equal': + return ds.field(predicate.field) == predicate.literals[0] + elif predicate.method == 'notEqual': + return ds.field(predicate.field) != predicate.literals[0] + elif predicate.method == 'lessThan': + return ds.field(predicate.field) < predicate.literals[0] + elif predicate.method == 'lessOrEqual': + return ds.field(predicate.field) <= predicate.literals[0] + elif predicate.method == 'greaterThan': + return ds.field(predicate.field) > predicate.literals[0] + elif predicate.method == 'greaterOrEqual': + return ds.field(predicate.field) >= predicate.literals[0] + elif predicate.method == 'isNull': + return ds.field(predicate.field).is_null() + elif predicate.method == 'isNotNull': + return ds.field(predicate.field).is_valid() + elif predicate.method == 'in': + return ds.field(predicate.field).isin(predicate.literals) + elif predicate.method == 'notIn': + return ~ds.field(predicate.field).isin(predicate.literals) + elif predicate.method == 'startsWith': + pattern = predicate.literals[0] + return pc.starts_with(ds.field(predicate.field).cast(pa.string()), pattern) + elif predicate.method == 'endsWith': + pattern = predicate.literals[0] + return pc.ends_with(ds.field(predicate.field).cast(pa.string()), pattern) + elif predicate.method == 'contains': + pattern = predicate.literals[0] + return pc.match_substring(ds.field(predicate.field).cast(pa.string()), pattern) + elif predicate.method == 'between': + return (ds.field(predicate.field) >= predicate.literals[0]) & \ + (ds.field(predicate.field) <= predicate.literals[1]) + elif predicate.method == 'and': return reduce(lambda x, y: x & y, - [convert_predicate(p) for p in py_predicate.literals]) - elif py_predicate.method == 'or': + [convert_predicate(p) for p in predicate.literals]) + elif predicate.method == 'or': return reduce(lambda x, y: x | y, - [convert_predicate(p) for p in py_predicate.literals]) + [convert_predicate(p) for p in predicate.literals]) else: - raise ValueError(f"Unsupported predicate method: {py_predicate.method}") + raise ValueError(f"Unsupported predicate method: {predicate.method}") diff --git a/pypaimon/pynative/util/predicate_utils.py b/pypaimon/pynative/util/predicate_utils.py index 8178449..e312cf7 100644 --- a/pypaimon/pynative/util/predicate_utils.py +++ b/pypaimon/pynative/util/predicate_utils.py @@ -16,14 +16,12 @@ # limitations under the License. ################################################################################ -from pypaimon.pynative.common.predicate import PyNativePredicate - def filter_predicate_by_primary_keys(predicate, primary_keys): """ Filter out predicates that are not related to primary key fields. """ - from pypaimon import Predicate + from pypaimon.api import Predicate if predicate is None or primary_keys is None: return predicate @@ -43,7 +41,7 @@ def filter_predicate_by_primary_keys(predicate, primary_keys): if len(filtered_literals) == 1: return filtered_literals[0] - return Predicate(PyNativePredicate( + return Predicate(Predicate( method=py_predicate.method, index=py_predicate.index, field=py_predicate.field, diff --git a/pypaimon/pynative/util/reader_convert_func.py b/pypaimon/pynative/util/reader_convert_func.py deleted file mode 100644 index 0ccae0f..0000000 --- a/pypaimon/pynative/util/reader_convert_func.py +++ /dev/null @@ -1,273 +0,0 @@ -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - - -def create_concat_record_reader(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.concat_record_reader import ConcatRecordReader - reader_class = j_reader.getClass() - queue_field = reader_class.getDeclaredField("queue") - queue_field.setAccessible(True) - j_supplier_queue = queue_field.get(j_reader) - return ConcatRecordReader(converter, j_supplier_queue) - - -def create_data_file_record_reader(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.data_file_record_reader import DataFileRecordReader - reader_class = j_reader.getClass() - wrapped_reader_field = reader_class.getDeclaredField("reader") - wrapped_reader_field.setAccessible(True) - j_wrapped_reader = wrapped_reader_field.get(j_reader) - wrapped_reader = converter.convert_java_reader(j_wrapped_reader) - - index_mapping_field = reader_class.getDeclaredField("indexMapping") - index_mapping_field.setAccessible(True) - index_mapping = index_mapping_field.get(j_reader) - - partition_info_field = reader_class.getDeclaredField("partitionInfo") - partition_info_field.setAccessible(True) - j_partition_info = partition_info_field.get(j_reader) - partition_info = convert_partition_info(j_partition_info) - - return DataFileRecordReader(wrapped_reader, index_mapping, partition_info) - - -def create_filter_reader(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.filter_record_reader import FilterRecordReader - reader_class = j_reader.getClass() - wrapped_reader_field = reader_class.getDeclaredField("val$thisReader") - wrapped_reader_field.setAccessible(True) - j_wrapped_reader = wrapped_reader_field.get(j_reader) - wrapped_reader = converter.convert_java_reader(j_wrapped_reader) - if primary_keys is not None: - return FilterRecordReader(wrapped_reader, predicate) - else: - return wrapped_reader - - -def create_pyarrow_reader_for_parquet(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.pyarrow_dataset_reader import PyArrowDatasetReader - - reader_class = j_reader.getClass() - factory_field = reader_class.getDeclaredField("this$0") - factory_field.setAccessible(True) - j_factory = factory_field.get(j_reader) - factory_class = j_factory.getClass() - batch_size_field = factory_class.getDeclaredField("batchSize") - batch_size_field.setAccessible(True) - batch_size = batch_size_field.get(j_factory) - - file_reader_field = reader_class.getDeclaredField("reader") - file_reader_field.setAccessible(True) - j_file_reader = file_reader_field.get(j_reader) - file_reader_class = j_file_reader.getClass() - input_file_field = file_reader_class.getDeclaredField("file") - input_file_field.setAccessible(True) - j_input_file = input_file_field.get(j_file_reader) - file_path = j_input_file.getPath().toUri().toString() - - fields_field = reader_class.getDeclaredField("fields") - fields_field.setAccessible(True) - fields = fields_field.get(j_reader) - if fields is not None: - fields = [str(field.getDescriptor().getPrimitiveType().getName()) for field in fields] - - return PyArrowDatasetReader('parquet', file_path, batch_size, projection, - predicate, primary_keys, fields) - - -def create_pyarrow_reader_for_orc(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.pyarrow_dataset_reader import PyArrowDatasetReader - - reader_class = j_reader.getClass() - file_reader_field = reader_class.getDeclaredField("orcReader") - file_reader_field.setAccessible(True) - j_file_reader = file_reader_field.get(j_reader) - file_reader_class = j_file_reader.getClass() - path_field = file_reader_class.getDeclaredField("path") - path_field.setAccessible(True) - j_path = path_field.get(j_file_reader) - file_path = j_path.toUri().toString() - - # TODO: Temporarily hard-coded to 1024 as we cannot reflectively obtain this value yet - batch_size = 1024 - - return PyArrowDatasetReader('orc', file_path, batch_size, projection, predicate, primary_keys, None) - - -def create_avro_format_reader(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.avro_format_reader import AvroFormatReader - - reader_class = j_reader.getClass() - path_field = reader_class.getDeclaredField("filePath") - path_field.setAccessible(True) - j_path = path_field.get(j_reader) - file_path = j_path.toUri().toString() - - # TODO: Temporarily hard-coded to 1024 as we cannot reflectively obtain this value yet - batch_size = 1024 - - return AvroFormatReader(file_path, batch_size, None) - - -def create_key_value_unwrap_reader(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.key_value_unwrap_reader import KeyValueUnwrapReader - reader_class = j_reader.getClass() - wrapped_reader_field = reader_class.getDeclaredField("val$reader") - wrapped_reader_field.setAccessible(True) - j_wrapped_reader = wrapped_reader_field.get(j_reader) - wrapped_reader = converter.convert_java_reader(j_wrapped_reader) - return KeyValueUnwrapReader(wrapped_reader) - - -def create_transform_reader(j_reader, converter, predicate, projection, primary_keys, partition_keys): - reader_class = j_reader.getClass() - wrapped_reader_field = reader_class.getDeclaredField("val$thisReader") - wrapped_reader_field.setAccessible(True) - j_wrapped_reader = wrapped_reader_field.get(j_reader) - # TODO: implement projectKey and projectOuter - return converter.convert_java_reader(j_wrapped_reader) - - -def create_drop_delete_reader(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.drop_delete_reader import DropDeleteReader - reader_class = j_reader.getClass() - wrapped_reader_field = reader_class.getDeclaredField("reader") - wrapped_reader_field.setAccessible(True) - j_wrapped_reader = wrapped_reader_field.get(j_reader) - wrapped_reader = converter.convert_java_reader(j_wrapped_reader) - return DropDeleteReader(wrapped_reader) - - -def create_sort_merge_reader_minhep(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.sort_merge_reader import SortMergeReader - j_reader_class = j_reader.getClass() - batch_readers_field = j_reader_class.getDeclaredField("nextBatchReaders") - batch_readers_field.setAccessible(True) - j_batch_readers = batch_readers_field.get(j_reader) - readers = [] - for next_reader in j_batch_readers: - readers.append(converter.convert_java_reader(next_reader)) - return SortMergeReader(readers, primary_keys, partition_keys) - - -def create_sort_merge_reader_loser_tree(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.sort_merge_reader import SortMergeReader - j_reader_class = j_reader.getClass() - loser_tree_field = j_reader_class.getDeclaredField("loserTree") - loser_tree_field.setAccessible(True) - j_loser_tree = loser_tree_field.get(j_reader) - j_loser_tree_class = j_loser_tree.getClass() - leaves_field = j_loser_tree_class.getDeclaredField("leaves") - leaves_field.setAccessible(True) - j_leaves = leaves_field.get(j_loser_tree) - readers = [] - for j_leaf in j_leaves: - j_leaf_class = j_leaf.getClass() - j_leaf_reader_field = j_leaf_class.getDeclaredField("reader") - j_leaf_reader_field.setAccessible(True) - j_leaf_reader = j_leaf_reader_field.get(j_leaf) - readers.append(converter.convert_java_reader(j_leaf_reader)) - return SortMergeReader(readers, primary_keys, partition_keys) - - -def create_key_value_wrap_record_reader(j_reader, converter, predicate, projection, primary_keys, partition_keys): - from pypaimon.pynative.reader.key_value_wrap_reader import KeyValueWrapReader - reader_class = j_reader.getClass() - - wrapped_reader_field = reader_class.getDeclaredField("reader") - wrapped_reader_field.setAccessible(True) - j_wrapped_reader = wrapped_reader_field.get(j_reader) - wrapped_reader = converter.convert_java_reader(j_wrapped_reader) - - level_field = reader_class.getDeclaredField("level") - level_field.setAccessible(True) - level = level_field.get(j_reader) - - serializer_field = reader_class.getDeclaredField("serializer") - serializer_field.setAccessible(True) - j_serializer = serializer_field.get(j_reader) - serializer_class = j_serializer.getClass() - key_arity_field = serializer_class.getDeclaredField("keyArity") - key_arity_field.setAccessible(True) - key_arity = key_arity_field.get(j_serializer) - - reused_value_field = serializer_class.getDeclaredField("reusedValue") - reused_value_field.setAccessible(True) - j_reused_value = reused_value_field.get(j_serializer) - offset_row_class = j_reused_value.getClass() - arity_field = offset_row_class.getDeclaredField("arity") - arity_field.setAccessible(True) - value_arity = arity_field.get(j_reused_value) - return KeyValueWrapReader(wrapped_reader, level, key_arity, value_arity) - - -def convert_partition_info(j_partition_info): - if j_partition_info is None: - return None - - partition_info_class = j_partition_info.getClass() - - map_field = partition_info_class.getDeclaredField("map") - map_field.setAccessible(True) - j_mapping = map_field.get(j_partition_info) - mapping = list(j_mapping) if j_mapping is not None else [] - - partition_field = partition_info_class.getDeclaredField("partition") - partition_field.setAccessible(True) - j_binary_row = partition_field.get(j_partition_info) - - partition_type_field = partition_info_class.getDeclaredField("partitionType") - partition_type_field.setAccessible(True) - j_partition_type = partition_type_field.get(j_partition_info) - - partition_values = [] - if j_binary_row is not None and j_partition_type is not None: - field_count = j_binary_row.getFieldCount() - for i in range(field_count): - if j_binary_row.isNullAt(i): - partition_values.append(None) - else: - field_type = j_partition_type.getTypeAt(i) - type_info = field_type.getTypeRoot().toString() - - if "INTEGER" in type_info: - partition_values.append(j_binary_row.getInt(i)) - elif "BIGINT" in type_info: - partition_values.append(j_binary_row.getLong(i)) - elif "VARCHAR" in type_info or "CHAR" in type_info: - binary_string = j_binary_row.getString(i) - partition_values.append(str(binary_string) if binary_string is not None else None) - elif "BOOLEAN" in type_info: - partition_values.append(j_binary_row.getBoolean(i)) - elif "DOUBLE" in type_info: - partition_values.append(j_binary_row.getDouble(i)) - elif "FLOAT" in type_info: - partition_values.append(j_binary_row.getFloat(i)) - elif "DATE" in type_info: - partition_values.append(j_binary_row.getInt(i)) # Date stored as int - elif "TIMESTAMP" in type_info: - timestamp = j_binary_row.getTimestamp(i, 3) # precision=3 for millis - partition_values.append(timestamp.getMillisecond() if timestamp is not None else None) - else: - try: - partition_values.append(str(j_binary_row.getString(i) or "")) - except: - partition_values.append(None) - - from pypaimon.pynative.reader.data_file_record_reader import PartitionInfo - return PartitionInfo(mapping, partition_values) diff --git a/pypaimon/pynative/util/reader_converter.py b/pypaimon/pynative/util/reader_converter.py deleted file mode 100644 index 92c8ddf..0000000 --- a/pypaimon/pynative/util/reader_converter.py +++ /dev/null @@ -1,90 +0,0 @@ -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -import os -from typing import List - -from py4j.java_gateway import JavaObject - -from pypaimon.py4j.util import constants -from pypaimon.pynative.common.exception import PyNativeNotImplementedError -from pypaimon.pynative.reader.core.record_reader import RecordReader -from pypaimon.pynative.util.reader_convert_func import ( - create_avro_format_reader, - create_concat_record_reader, - create_data_file_record_reader, - create_drop_delete_reader, - create_filter_reader, - create_key_value_unwrap_reader, - create_key_value_wrap_record_reader, - create_pyarrow_reader_for_orc, - create_pyarrow_reader_for_parquet, - create_sort_merge_reader_minhep, - create_transform_reader, create_sort_merge_reader_loser_tree, -) - -reader_mapping = { - "org.apache.paimon.mergetree.compact.ConcatRecordReader": - create_concat_record_reader, - "org.apache.paimon.io.DataFileRecordReader": - create_data_file_record_reader, - "org.apache.paimon.reader.RecordReader$2": - create_filter_reader, - "org.apache.paimon.format.parquet.ParquetReaderFactory$ParquetReader": - create_pyarrow_reader_for_parquet, - "org.apache.paimon.format.orc.OrcReaderFactory$OrcVectorizedReader": - create_pyarrow_reader_for_orc, - "org.apache.paimon.format.avro.AvroBulkFormat$AvroReader": - create_avro_format_reader, - "org.apache.paimon.table.source.KeyValueTableRead$1": - create_key_value_unwrap_reader, - "org.apache.paimon.reader.RecordReader$1": - create_transform_reader, - "org.apache.paimon.mergetree.DropDeleteReader": - create_drop_delete_reader, - "org.apache.paimon.mergetree.compact.SortMergeReaderWithMinHeap": - create_sort_merge_reader_minhep, - "org.apache.paimon.mergetree.compact.SortMergeReaderWithLoserTree": - create_sort_merge_reader_loser_tree, - "org.apache.paimon.io.KeyValueDataFileRecordReader": - create_key_value_wrap_record_reader, - # Additional mappings can be added here -} - - -class ReaderConverter: - """ - # Convert Java RecordReader to Python RecordReader - """ - - def __init__(self, predicate, projection, primary_keys: List[str], partition_keys: List[str]): - self.reader_mapping = reader_mapping - self._predicate = predicate - self._projection = projection - self._primary_keys = primary_keys - self._partition_keys = partition_keys or [] - - def convert_java_reader(self, java_reader: JavaObject) -> RecordReader: - java_class_name = java_reader.getClass().getName() - if java_class_name in reader_mapping: - if os.environ.get(constants.PYPAIMON4J_TEST_MODE) == "true": - print("converting Java reader: " + str(java_class_name)) - return reader_mapping[java_class_name](java_reader, self, self._predicate, - self._projection, self._primary_keys, self._partition_keys) - else: - raise PyNativeNotImplementedError(f"Unsupported RecordReader type: {java_class_name}") diff --git a/pypaimon/pynative/write/__init__.py b/pypaimon/pynative/write/__init__.py new file mode 100644 index 0000000..65b48d4 --- /dev/null +++ b/pypaimon/pynative/write/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/pypaimon/pynative/write/batch_table_commit_impl.py b/pypaimon/pynative/write/batch_table_commit_impl.py new file mode 100644 index 0000000..9ee2e16 --- /dev/null +++ b/pypaimon/pynative/write/batch_table_commit_impl.py @@ -0,0 +1,73 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import time +from typing import List, Optional + +from pypaimon.api import BatchTableCommit +from pypaimon.pynative.write.commit_message_impl import CommitMessageImpl +from pypaimon.pynative.write.file_store_commit import FileStoreCommit + + +class BatchTableCommitImpl(BatchTableCommit): + """Python implementation of BatchTableCommit for batch writing scenarios.""" + + def __init__(self, table, commit_user: str, static_partition: Optional[dict]): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.commit_user = commit_user + self.overwrite_partition = static_partition + self.file_store_commit = FileStoreCommit(table, commit_user) + self.batch_committed = False + + def commit(self, commit_messages: List[CommitMessageImpl]): + self._check_committed() + + non_empty_messages = [msg for msg in commit_messages if not msg.is_empty()] + if not non_empty_messages: + return + + commit_identifier = int(time.time() * 1000) + + try: + if self.overwrite_partition is not None: + self.file_store_commit.overwrite( + partition=self.overwrite_partition, + commit_messages=non_empty_messages, + commit_identifier=commit_identifier + ) + else: + self.file_store_commit.commit( + commit_messages=non_empty_messages, + commit_identifier=commit_identifier + ) + except Exception as e: + self.file_store_commit.abort(commit_messages) + raise RuntimeError(f"Failed to commit: {str(e)}") from e + + def abort(self, commit_messages: List[CommitMessageImpl]): + self.file_store_commit.abort(commit_messages) + + def close(self): + self.file_store_commit.close() + + def _check_committed(self): + if self.batch_committed: + raise RuntimeError("BatchTableCommit only supports one-time committing.") + self.batch_committed = True diff --git a/pypaimon/pynative/write/batch_table_write_impl.py b/pypaimon/pynative/write/batch_table_write_impl.py new file mode 100644 index 0000000..0b05f3a --- /dev/null +++ b/pypaimon/pynative/write/batch_table_write_impl.py @@ -0,0 +1,64 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import pyarrow as pa +from collections import defaultdict + +from typing import List + +from pypaimon.api import BatchTableWrite, CommitMessage +from pypaimon.pynative.common.exception import PyNativeNotImplementedError +from pypaimon.pynative.write.file_store_write import FileStoreWrite + + +class BatchTableWriteImpl(BatchTableWrite): + def __init__(self, table): + self.file_store_write = FileStoreWrite(table) + self.row_key_extractor = table.create_row_key_extractor() + self.batch_committed = False + + def write_arrow(self, table: pa.Table, row_kind: List[int] = None): + # TODO: support row_kind + batches_iterator = table.to_batches() + for batch in batches_iterator: + self.write_arrow_batch(batch) + + def write_arrow_batch(self, data: pa.RecordBatch, row_kind: List[int] = None): + # TODO: support row_kind + partitions, buckets = self.row_key_extractor.extract_partition_bucket_batch(data) + + partition_bucket_groups = defaultdict(list) + for i in range(data.num_rows): + partition_bucket_groups[(tuple(partitions[i]), buckets[i])].append(i) + + for (partition, bucket), row_indices in partition_bucket_groups.items(): + indices_array = pa.array(row_indices, type=pa.int64()) + sub_table = pa.compute.take(data, indices_array) + self.file_store_write.write(partition, bucket, sub_table) + + def write_pandas(self, dataframe): + raise PyNativeNotImplementedError("write_pandas") + + def prepare_commit(self) -> List[CommitMessage]: + if self.batch_committed: + raise RuntimeError("BatchTableWrite only supports one-time committing.") + self.batch_committed = True + return self.file_store_write.prepare_commit() + + def close(self): + self.file_store_write.close() diff --git a/pypaimon/pynative/write/batch_write_builder.py b/pypaimon/pynative/write/batch_write_builder.py new file mode 100644 index 0000000..34d713a --- /dev/null +++ b/pypaimon/pynative/write/batch_write_builder.py @@ -0,0 +1,52 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import uuid + +from typing import Optional + +from pypaimon.api import BatchTableWrite, BatchWriteBuilder, BatchTableCommit +from pypaimon.pynative.common.core_option import CoreOptions +from pypaimon.pynative.write.batch_table_commit_impl import BatchTableCommitImpl +from pypaimon.pynative.write.batch_table_write_impl import BatchTableWriteImpl + + +class BatchWriteBuilderImpl(BatchWriteBuilder): + def __init__(self, table): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.commit_user = self._create_commit_user() + self.static_partition = None + + def overwrite(self, static_partition: Optional[dict] = None) -> BatchWriteBuilder: + self.static_partition = static_partition + return self + + def new_write(self) -> BatchTableWrite: + return BatchTableWriteImpl(self.table) + + def new_commit(self) -> BatchTableCommit: + commit = BatchTableCommitImpl(self.table, self.commit_user, self.static_partition) + return commit + + def _create_commit_user(self): + if CoreOptions.COMMIT_USER_PREFIX in self.table.options: + return f"{self.table.options.get(CoreOptions.COMMIT_USER_PREFIX)}_{uuid.uuid4()}" + else: + return str(uuid.uuid4()) diff --git a/pypaimon/pynative/write/commit_message_impl.py b/pypaimon/pynative/write/commit_message_impl.py new file mode 100644 index 0000000..a846edb --- /dev/null +++ b/pypaimon/pynative/write/commit_message_impl.py @@ -0,0 +1,46 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import Tuple, List + +from pypaimon.api import CommitMessage +from pypaimon.pynative.table.data_file_meta import DataFileMeta + + +class CommitMessageImpl(CommitMessage): + """Python implementation of CommitMessage""" + + def __init__(self, partition: Tuple, bucket: int, new_files: List[str]): + self._partition = partition + self._bucket = bucket + self._new_files = new_files or [] + + def partition(self) -> Tuple: + """Get the partition of this commit message.""" + return self._partition + + def bucket(self) -> int: + """Get the bucket of this commit message.""" + return self._bucket + + def new_files(self) -> List[DataFileMeta]: + """Get the list of new files.""" + return self._new_files + + def is_empty(self): + return not self._new_files diff --git a/pypaimon/pynative/write/file_store_commit.py b/pypaimon/pynative/write/file_store_commit.py new file mode 100644 index 0000000..cd57dc5 --- /dev/null +++ b/pypaimon/pynative/write/file_store_commit.py @@ -0,0 +1,120 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import time +from pathlib import Path +from typing import List + +from pypaimon.pynative.table.snapshot import Snapshot +from pypaimon.pynative.write.commit_message_impl import CommitMessageImpl +from pypaimon.pynative.table.manifest_file_manager import ManifestFileManager +from pypaimon.pynative.table.manifest_list_manager import ManifestListManager +from pypaimon.pynative.table.snapshot_manager import SnapshotManager + + +class FileStoreCommit: + """Core commit logic for file store operations.""" + + def __init__(self, table, commit_user: str): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.commit_user = commit_user + + self.snapshot_manager = SnapshotManager(table) + self.manifest_file_manager = ManifestFileManager(table) + self.manifest_list_manager = ManifestListManager(table) + + self.manifest_target_size = 8 * 1024 * 1024 + self.manifest_merge_min_count = 30 + + def commit(self, commit_messages: List[CommitMessageImpl], commit_identifier: int): + """Commit the given commit messages in normal append mode.""" + if not commit_messages: + return + + new_manifest_files = self.manifest_file_manager.write(commit_messages) + if not new_manifest_files: + return + latest_snapshot = self.snapshot_manager.get_latest_snapshot() + existing_manifest_files = [] + if latest_snapshot: + existing_manifest_files = self.manifest_list_manager.read_all_manifest_files(latest_snapshot) + new_manifest_files.extend(existing_manifest_files) + manifest_list = self.manifest_list_manager.write(new_manifest_files) + + new_snapshot_id = self._generate_snapshot_id() + snapshot_data = Snapshot( + version=3, + id=new_snapshot_id, + schema_id=0, + base_manifest_list=manifest_list, + delta_manifest_list=manifest_list, + commit_user=self.commit_user, + commit_identifier=commit_identifier, + commit_kind="APPEND", + time_millis=int(time.time() * 1000), + log_offsets={}, + ) + self.snapshot_manager.commit_snapshot(new_snapshot_id, snapshot_data) + + def overwrite(self, partition, commit_messages: List[CommitMessageImpl], commit_identifier: int): + if not commit_messages: + return + + new_manifest_files = self.manifest_file_manager.write(commit_messages) + if not new_manifest_files: + return + + # In overwrite mode, we don't merge with existing manifests + manifest_list = self.manifest_list_manager.write(new_manifest_files) + + new_snapshot_id = self._generate_snapshot_id() + snapshot_data = Snapshot( + version=3, + id=new_snapshot_id, + schema_id=0, + base_manifest_list=manifest_list, + delta_manifest_list=manifest_list, + commit_user=self.commit_user, + commit_identifier=commit_identifier, + commit_kind="OVERWRITE", + time_millis=int(time.time() * 1000), + log_offsets={}, + ) + self.snapshot_manager.commit_snapshot(new_snapshot_id, snapshot_data) + + def abort(self, commit_messages: List[CommitMessageImpl]): + for message in commit_messages: + for file in message.new_files(): + try: + file_path_obj = Path(file.file_path) + if file_path_obj.exists(): + file_path_obj.unlink() + except Exception as e: + print(f"Warning: Failed to clean up file {file.file_path}: {e}") + + def close(self): + pass + + def _generate_snapshot_id(self) -> int: + latest_snapshot = self.snapshot_manager.get_latest_snapshot() + if latest_snapshot: + return latest_snapshot.id + 1 + else: + return 1 diff --git a/pypaimon/pynative/write/file_store_write.py b/pypaimon/pynative/write/file_store_write.py new file mode 100644 index 0000000..b971f22 --- /dev/null +++ b/pypaimon/pynative/write/file_store_write.py @@ -0,0 +1,76 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import pyarrow as pa +from typing import Dict, Tuple, List + +from pypaimon.pynative.write.commit_message_impl import CommitMessageImpl +from pypaimon.pynative.write.writer.append_only_data_writer import AppendOnlyDataWriter +from pypaimon.pynative.write.writer.data_writer import DataWriter +from pypaimon.api import CommitMessage +from pypaimon.pynative.write.writer.key_value_data_writer import KeyValueDataWriter + + +class FileStoreWrite: + """Base class for file store write operations.""" + + def __init__(self, table): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.data_writers: Dict[Tuple, DataWriter] = {} + + def write(self, partition: Tuple, bucket: int, data: pa.RecordBatch): + key = (partition, bucket) + if key not in self.data_writers: + self.data_writers[key] = self._create_data_writer(partition, bucket) + writer = self.data_writers[key] + writer.write(data) + + def _create_data_writer(self, partition: Tuple, bucket: int) -> DataWriter: + if self.table.is_primary_key_table: + return KeyValueDataWriter( + table=self.table, + partition=partition, + bucket=bucket, + ) + else: + return AppendOnlyDataWriter( + table=self.table, + partition=partition, + bucket=bucket, + ) + + def prepare_commit(self) -> List[CommitMessage]: + commit_messages = [] + for (partition, bucket), writer in self.data_writers.items(): + committed_files = writer.prepare_commit() + if committed_files: + commit_message = CommitMessageImpl( + partition=partition, + bucket=bucket, + new_files=committed_files + ) + commit_messages.append(commit_message) + return commit_messages + + def close(self): + """Close all data writers and clean up resources.""" + for writer in self.data_writers.values(): + writer.close() + self.data_writers.clear() diff --git a/pypaimon/pynative/write/row_key_extractor.py b/pypaimon/pynative/write/row_key_extractor.py new file mode 100644 index 0000000..3cc9d5a --- /dev/null +++ b/pypaimon/pynative/write/row_key_extractor.py @@ -0,0 +1,102 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import pyarrow as pa +from typing import Tuple, List +from abc import ABC, abstractmethod + +from pypaimon.pynative.table.schema import TableSchema +from pypaimon.pynative.common.core_option import CoreOptions + + +class RowKeyExtractor(ABC): + """Base class for extracting partition and bucket information from PyArrow data.""" + + def __init__(self, table_schema: TableSchema): + self.table_schema = table_schema + self.partition_indices = self._get_field_indices(table_schema.partition_keys) + + def extract_partition_bucket_batch(self, data: pa.RecordBatch) -> Tuple[List[Tuple], List[int]]: + partitions = self._extract_partitions_batch(data) + buckets = self._extract_buckets_batch(data) + return partitions, buckets + + def _get_field_indices(self, field_names: List[str]) -> List[int]: + if not field_names: + return [] + field_map = {field.name: i for i, field in enumerate(self.table_schema.fields)} + return [field_map[name] for name in field_names if name in field_map] + + def _extract_partitions_batch(self, data: pa.RecordBatch) -> List[Tuple]: + if not self.partition_indices: + return [() for _ in range(data.num_rows)] + + partition_columns = [data.column(i) for i in self.partition_indices] + + partitions = [] + for row_idx in range(data.num_rows): + partition_values = tuple(col[row_idx].as_py() for col in partition_columns) + partitions.append(partition_values) + + return partitions + + @abstractmethod + def _extract_buckets_batch(self, table: pa.RecordBatch) -> List[int]: + """Extract bucket numbers for all rows. Must be implemented by subclasses.""" + pass + + +class FixedBucketRowKeyExtractor(RowKeyExtractor): + """Fixed bucket mode extractor with configurable number of buckets.""" + + def __init__(self, table_schema: TableSchema): + super().__init__(table_schema) + self.num_buckets = table_schema.options.get(CoreOptions.BUCKET, -1) + if self.num_buckets <= 0: + raise ValueError(f"Fixed bucket mode requires bucket > 0, got {self.num_buckets}") + + bucket_key_option = table_schema.options.get(CoreOptions.BUCKET_KEY, '') + if bucket_key_option.strip(): + self.bucket_keys = [k.strip() for k in bucket_key_option.split(',')] + else: + self.bucket_keys = [pk for pk in table_schema.primary_keys + if pk not in table_schema.partition_keys] + + self.bucket_key_indices = self._get_field_indices(self.bucket_keys) + + def _extract_buckets_batch(self, data: pa.RecordBatch) -> List[int]: + columns = [data.column(i) for i in self.bucket_key_indices] + hashes = [] + for row_idx in range(data.num_rows): + row_values = tuple(col[row_idx].as_py() for col in columns) + hashes.append(hash(row_values)) + return [abs(hash_val) % self.num_buckets for hash_val in hashes] + + +class UnawareBucketRowKeyExtractor(RowKeyExtractor): + """Extractor for unaware bucket mode (bucket = -1, no primary keys).""" + + def __init__(self, table_schema: TableSchema): + super().__init__(table_schema) + num_buckets = table_schema.options.get(CoreOptions.BUCKET, -1) + + if num_buckets != -1: + raise ValueError(f"Unaware bucket mode requires bucket = -1, got {num_buckets}") + + def _extract_buckets_batch(self, data: pa.RecordBatch) -> List[int]: + return [0] * data.num_rows diff --git a/pypaimon/pynative/write/writer/__init__.py b/pypaimon/pynative/write/writer/__init__.py new file mode 100644 index 0000000..65b48d4 --- /dev/null +++ b/pypaimon/pynative/write/writer/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/pypaimon/pynative/write/writer/append_only_data_writer.py b/pypaimon/pynative/write/writer/append_only_data_writer.py new file mode 100644 index 0000000..b5c9e47 --- /dev/null +++ b/pypaimon/pynative/write/writer/append_only_data_writer.py @@ -0,0 +1,31 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import pyarrow as pa + +from pypaimon.pynative.write.writer.data_writer import DataWriter + + +class AppendOnlyDataWriter(DataWriter): + """Data writer for append-only tables.""" + + def _process_data(self, data: pa.RecordBatch) -> pa.RecordBatch: + return data + + def _merge_data(self, existing_data: pa.RecordBatch, new_data: pa.RecordBatch) -> pa.RecordBatch: + return pa.concat_tables([existing_data, new_data]) diff --git a/pypaimon/pynative/write/writer/data_writer.py b/pypaimon/pynative/write/writer/data_writer.py new file mode 100644 index 0000000..bb0e1b9 --- /dev/null +++ b/pypaimon/pynative/write/writer/data_writer.py @@ -0,0 +1,162 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import uuid + +import pyarrow as pa +from typing import Tuple, Optional, List +from pathlib import Path +from abc import ABC, abstractmethod + +from pypaimon.pynative.common.core_option import CoreOptions +from pypaimon.pynative.row.binary_row import BinaryRow +from pypaimon.pynative.table.data_file_meta import DataFileMeta + + +class DataWriter(ABC): + """Base class for data writers that handle PyArrow tables directly.""" + + def __init__(self, table, partition: Tuple, bucket: int): + from pypaimon.pynative.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.partition = partition + self.bucket = bucket + + self.file_io = self.table.file_io + self.trimmed_primary_key_fields = self.table.table_schema.get_trimmed_primary_key_fields() + + options = self.table.options + self.target_file_size = 256 * 1024 * 1024 + self.file_format = options.get(CoreOptions.FILE_FORMAT, CoreOptions.FILE_FORMAT_PARQUET) + self.compression = options.get(CoreOptions.FILE_COMPRESSION, "zstd") + + self.pending_data: Optional[pa.RecordBatch] = None + self.committed_files: List[DataFileMeta] = [] + + def write(self, data: pa.RecordBatch): + processed_data = self._process_data(data) + + if self.pending_data is None: + self.pending_data = processed_data + else: + self.pending_data = self._merge_data(self.pending_data, processed_data) + + self._check_and_roll_if_needed() + + def prepare_commit(self) -> List[DataFileMeta]: + if self.pending_data is not None and self.pending_data.num_rows > 0: + self._write_data_to_file(self.pending_data) + self.pending_data = None + + return self.committed_files.copy() + + def close(self): + self.pending_data = None + self.committed_files.clear() + + @abstractmethod + def _process_data(self, data: pa.RecordBatch) -> pa.RecordBatch: + """Process incoming data (e.g., add system fields, sort). Must be implemented by subclasses.""" + + @abstractmethod + def _merge_data(self, existing_data: pa.RecordBatch, new_data: pa.RecordBatch) -> pa.RecordBatch: + """Merge existing data with new data. Must be implemented by subclasses.""" + + def _check_and_roll_if_needed(self): + if self.pending_data is None: + return + + current_size = self.pending_data.get_total_buffer_size() + if current_size > self.target_file_size: + split_row = _find_optimal_split_point(self.pending_data, self.target_file_size) + if split_row > 0: + data_to_write = self.pending_data.slice(0, split_row) + remaining_data = self.pending_data.slice(split_row) + + self._write_data_to_file(data_to_write) + self.pending_data = remaining_data + self._check_and_roll_if_needed() + + def _write_data_to_file(self, data: pa.RecordBatch): + if data.num_rows == 0: + return + file_name = f"data-{uuid.uuid4()}.{self.file_format}" + file_path = self._generate_file_path(file_name) + try: + if self.file_format == CoreOptions.FILE_FORMAT_PARQUET: + self.file_io.write_parquet(file_path, data, compression=self.compression) + elif self.file_format == CoreOptions.FILE_FORMAT_ORC: + self.file_io.write_orc(file_path, data, compression=self.compression) + elif self.file_format == CoreOptions.FILE_FORMAT_AVRO: + self.file_io.write_avro(file_path, data, compression=self.compression) + else: + raise ValueError(f"Unsupported file format: {self.file_format}") + + key_columns_batch = data.select(self.trimmed_primary_key_fields) + min_key_data = key_columns_batch.slice(0, 1).to_pylist()[0] + max_key_data = key_columns_batch.slice(key_columns_batch.num_rows - 1, 1).to_pylist()[0] + self.committed_files.append(DataFileMeta( + file_name=file_name, + file_size=self.file_io.get_file_size(file_path), + row_count=data.num_rows, + min_key=BinaryRow(min_key_data, self.trimmed_primary_key_fields), + max_key=BinaryRow(max_key_data, self.trimmed_primary_key_fields), + key_stats=None, # TODO + value_stats=None, + min_sequence_number=0, + max_sequence_number=0, + schema_id=0, + level=0, + extra_files=None, + file_path=str(file_path), + )) + + except Exception as e: + raise RuntimeError(f"Failed to write {self.file_format} file {file_path}: {e}") from e + + def _generate_file_path(self, file_name: str) -> Path: + path_builder = self.table.table_path + + for i, field_name in enumerate(self.table.partition_keys): + path_builder = path_builder / (field_name + "=" + self.partition[i]) + path_builder = path_builder / ("bucket-" + str(self.bucket)) / file_name + + return path_builder + + +def _find_optimal_split_point(data: pa.RecordBatch, target_size: int) -> int: + total_rows = data.num_rows + if total_rows <= 1: + return 0 + + left, right = 1, total_rows + best_split = 0 + + while left <= right: + mid = (left + right) // 2 + slice_data = data.slice(0, mid) + slice_size = slice_data.get_total_buffer_size() + + if slice_size <= target_size: + best_split = mid + left = mid + 1 + else: + right = mid - 1 + + return best_split diff --git a/pypaimon/pynative/write/writer/key_value_data_writer.py b/pypaimon/pynative/write/writer/key_value_data_writer.py new file mode 100644 index 0000000..f9dddab --- /dev/null +++ b/pypaimon/pynative/write/writer/key_value_data_writer.py @@ -0,0 +1,80 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import pyarrow as pa +import pyarrow.compute as pc +from typing import Tuple, Dict + +from pypaimon.pynative.write.writer.data_writer import DataWriter + + +class KeyValueDataWriter(DataWriter): + """Data writer for primary key tables with system fields and sorting.""" + + def __init__(self, partition: Tuple, bucket: int, file_io, table_schema, table_identifier, + target_file_size: int, options: Dict[str, str]): + super().__init__(partition, bucket, file_io, table_schema, table_identifier, + target_file_size, options) + self.sequence_generator = SequenceGenerator() + self.trimmed_primary_key = [field.name for field in self.table.table_schema.get_trimmed_primary_key_fields()] + + def _process_data(self, data: pa.RecordBatch) -> pa.RecordBatch: + enhanced_data = self._add_system_fields(data) + return self._sort_by_primary_key(enhanced_data) + + def _merge_data(self, existing_data: pa.RecordBatch, new_data: pa.RecordBatch) -> pa.RecordBatch: + combined = pa.concat_tables([existing_data, new_data]) + return self._sort_by_primary_key(combined) + + def _add_system_fields(self, data: pa.RecordBatch) -> pa.RecordBatch: + """Add system fields: _KEY_{pk_key}, _SEQUENCE_NUMBER, _VALUE_KIND.""" + num_rows = data.num_rows + enhanced_table = data + + for pk_key in reversed(self.trimmed_primary_key): + if pk_key in data.column_names: + key_column = data.column(pk_key) + enhanced_table = enhanced_table.add_column(0, f'_KEY_{pk_key}', key_column) + + sequence_column = pa.array([self.sequence_generator.next() for _ in range(num_rows)], type=pa.int64()) + enhanced_table = enhanced_table.add_column(len(self.trimmed_primary_key), '_SEQUENCE_NUMBER', sequence_column) + + # TODO: support real row kind here + value_kind_column = pa.repeat(0, num_rows) + enhanced_table = enhanced_table.add_column(len(self.trimmed_primary_key) + 1, '_VALUE_KIND', + value_kind_column) + + return enhanced_table + + def _sort_by_primary_key(self, data: pa.RecordBatch) -> pa.RecordBatch: + sort_keys = self.trimmed_primary_key + if '_SEQUENCE_NUMBER' in data.column_names: + sort_keys.append('_SEQUENCE_NUMBER') + + sorted_indices = pc.sort_indices(data, sort_keys=sort_keys) + sorted_batch = data.take(sorted_indices) + return sorted_batch + + +class SequenceGenerator: + def __init__(self, start: int = 0): + self.current = start + + def next(self) -> int: + self.current += 1 + return self.current diff --git a/setup.py b/setup.py index 0762a69..5795f53 100644 --- a/setup.py +++ b/setup.py @@ -45,7 +45,8 @@ install_requires = [ 'py4j==0.10.9.7', 'pandas>=1.3.0', - 'pyarrow>=5.0.0' + 'pyarrow>=5.0.0', + 'polars>=1.31.0' ] long_description = 'See Apache Paimon Python API \ diff --git a/tox.ini b/tox.ini index 8ce06f8..693c910 100644 --- a/tox.ini +++ b/tox.ini @@ -41,7 +41,7 @@ install_command = {toxinidir}/dev/install_command.sh {opts} {packages} # We follow PEP 8 (https://www.python.org/dev/peps/pep-0008/) with one exception: lines can be # up to 100 characters in length, not 79. ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 -max-line-length=100 +max-line-length=120 exclude=.tox/*,dev/*,build/*,dist/* [mypy]