#49 Implement Python native write with PyArrow (#51)
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 @@
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 @@
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/pynative/common/row/__init__.py b/pypaimon/api/database.py
similarity index 76%
copy from pypaimon/pynative/common/row/__init__.py
copy to pypaimon/api/database.py
index 65b48d4..db89430 100644
--- a/pypaimon/pynative/common/row/__init__.py
+++ b/pypaimon/api/database.py
@@ -14,4 +14,15 @@
# 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/pynative/reader/empty_record_reader.py b/pypaimon/api/schema.py
similarity index 61%
copy from pypaimon/pynative/reader/empty_record_reader.py
copy to pypaimon/api/schema.py
index 9883cb8..e01b85b 100644
--- a/pypaimon/pynative/reader/empty_record_reader.py
+++ b/pypaimon/api/schema.py
@@ -14,24 +14,24 @@
# 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.reader.core.file_record_reader import FileRecordReader
-from pypaimon.pynative.reader.core.record_iterator import RecordIterator
+from typing import Optional, List
-class EmptyFileRecordReader(FileRecordReader):
- """
- An empty FileRecordReader.
- """
+class Schema:
+ """Schema of a table."""
- def __init__(self):
- pass
-
- def read_batch(self) -> Optional[RecordIterator]:
- return None
-
- def close(self) -> None:
- pass
+ 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 @@
@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
@@ -32,6 +32,10 @@
"""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 @@
"""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 @@
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 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 @@
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_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 @@
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 @@
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 @@
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 @@
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 @@
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/reader/__init__.py b/pypaimon/pynative/catalog/__init__.py
similarity index 100%
copy from pypaimon/pynative/reader/__init__.py
copy 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/common/row/__init__.py b/pypaimon/pynative/catalog/catalog_constant.py
similarity index 78%
copy from pypaimon/pynative/common/row/__init__.py
copy to pypaimon/pynative/catalog/catalog_constant.py
index 65b48d4..32442a2 100644
--- a/pypaimon/pynative/common/row/__init__.py
+++ b/pypaimon/pynative/catalog/catalog_constant.py
@@ -15,3 +15,18 @@
# See the License for the specific language governing permissions and
# limitations under the License.
################################################################################
+
+from enum import Enum
+
+
+class CatalogConstants(str, Enum):
+
+ def __str__(self):
+ return self.value
+
+ COMMENT_PROP = "comment"
+ OWNER_PROP = "owner"
+
+ 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/common/row/__init__.py b/pypaimon/pynative/catalog/catalog_option.py
similarity index 84%
rename from pypaimon/pynative/common/row/__init__.py
rename to pypaimon/pynative/catalog/catalog_option.py
index 65b48d4..c44772f 100644
--- a/pypaimon/pynative/common/row/__init__.py
+++ b/pypaimon/pynative/catalog/catalog_option.py
@@ -15,3 +15,14 @@
# 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/empty_record_reader.py b/pypaimon/pynative/catalog/hive_catalog.py
similarity index 63%
copy from pypaimon/pynative/reader/empty_record_reader.py
copy to pypaimon/pynative/catalog/hive_catalog.py
index 9883cb8..da131c8 100644
--- a/pypaimon/pynative/reader/empty_record_reader.py
+++ b/pypaimon/pynative/catalog/hive_catalog.py
@@ -18,20 +18,23 @@
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.api import Schema
+from pypaimon.pynative.catalog.abstract_catalog import AbstractCatalog
+from pypaimon.pynative.common.identifier import TableIdentifier
-class EmptyFileRecordReader(FileRecordReader):
- """
- An empty FileRecordReader.
- """
+class HiveCatalog(AbstractCatalog):
+ """Hive Catalog implementation for Paimon."""
- def __init__(self):
+ @staticmethod
+ def identifier() -> str:
+ return "hive"
+
+ def create_database_impl(self, name: str, properties: Optional[dict] = None):
pass
- def read_batch(self) -> Optional[RecordIterator]:
- return None
+ def create_table_impl(self, table_identifier: TableIdentifier, schema: 'Schema'):
+ pass
- def close(self) -> None:
+ 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/common/row/key_value.py b/pypaimon/pynative/common/row/key_value.py
deleted file mode 100644
index d8c9951..0000000
--- a/pypaimon/pynative/common/row/key_value.py
+++ /dev/null
@@ -1,42 +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 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.
-"""
-
-
-@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()
diff --git a/pypaimon/pynative/common/row/offset_row.py b/pypaimon/pynative/common/row/offset_row.py
deleted file mode 100644
index 8ae21a2..0000000
--- a/pypaimon/pynative/common/row/offset_row.py
+++ /dev/null
@@ -1,59 +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
-
-from pypaimon.pynative.common.row.internal_row import InternalRow
-from pypaimon.pynative.common.row.row_kind import RowKind
-
-
-class OffsetRow(InternalRow):
- """
- A InternalRow to wrap row with offset.
- """
-
- def __init__(self, row: InternalRow, offset: int, arity: int):
- self.row = row
- self.offset = offset
- self.arity = arity
-
- def replace(self, row: InternalRow) -> 'OffsetRow':
- self.row = row
- 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)
-
- 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()
-
- def get_row_kind(self) -> RowKind:
- return self.row.get_row_kind()
-
- def set_row_kind(self, kind: RowKind) -> None:
- self.row.set_row_kind(kind)
-
- def __len__(self) -> int:
- return self.arity
diff --git a/pypaimon/pynative/reader/__init__.py b/pypaimon/pynative/read/__init__.py
similarity index 100%
copy from pypaimon/pynative/reader/__init__.py
copy 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/__init__.py b/pypaimon/pynative/read/reader/__init__.py
similarity index 100%
rename from pypaimon/pynative/reader/__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/read/reader/concat_record_reader.py b/pypaimon/pynative/read/reader/concat_record_reader.py
new file mode 100644
index 0000000..63becce
--- /dev/null
+++ b/pypaimon/pynative/read/reader/concat_record_reader.py
@@ -0,0 +1,50 @@
+################################################################################
+# 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 pypaimon.pynative.read.reader.iface.record_iterator import RecordIterator
+from pypaimon.pynative.read.reader.iface.record_reader import RecordReader
+
+
+class ConcatRecordReader(RecordReader):
+
+ 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_reader is not None:
+ iterator = self.current_reader.read_batch()
+ if iterator is not None:
+ return iterator
+ 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/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 @@
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
-
-T = TypeVar('T')
+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
-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 @@
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/read/reader/format_pyarrow_reader.py b/pypaimon/pynative/read/reader/format_pyarrow_reader.py
new file mode 100644
index 0000000..c3bc8a1
--- /dev/null
+++ b/pypaimon/pynative/read/reader/format_pyarrow_reader.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, 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.pynative.util.predicate_converter import convert_predicate
+
+
+class FormatPyArrowReader(RecordBatchReader):
+ """
+ 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, 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:
+ # TODO: utilize predicate to improve performance
+ predicate = None
+
+ if predicate is not None:
+ predicate = convert_predicate(predicate)
+
+ 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
+ ).to_reader()
+
+ def read_arrow_batch(self) -> Optional[RecordBatch]:
+ try:
+ return self.reader.read_next_batch()
+ except StopIteration:
+ return None
+
+ def close(self):
+ if self.reader is not None:
+ self.reader.close()
+ self.reader = None
diff --git a/pypaimon/pynative/reader/core/__init__.py b/pypaimon/pynative/read/reader/iface/__init__.py
similarity index 100%
rename from pypaimon/pynative/reader/core/__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 90%
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 @@
"""
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/read/reader/key_value_unwrap_reader.py b/pypaimon/pynative/read/reader/key_value_unwrap_reader.py
new file mode 100644
index 0000000..f236d69
--- /dev/null
+++ b/pypaimon/pynative/read/reader/key_value_unwrap_reader.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 Any, Optional
+
+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 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, kv_reader: RecordReader[KeyValue]):
+ self.kv_reader = kv_reader
+
+ def read_batch(self) -> Optional[RecordIterator[InternalRow]]:
+ batch = self.kv_reader.read_batch()
+ if batch is None:
+ return None
+
+ return KeyValueUnwrapIterator(batch)
+
+ def close(self) -> None:
+ self.kv_reader.close()
+
+
+class KeyValueUnwrapIterator(RecordIterator[InternalRow]):
+ """
+ An Iterator that converts a KeyValue into an InternalRow
+ """
+
+ def __init__(self, batch: RecordIterator[KeyValue]):
+ self.batch = batch
+
+ def next(self) -> Optional[Any]:
+ kv = self.batch.next()
+ if kv is None:
+ return None
+
+ 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/concat_record_reader.py b/pypaimon/pynative/reader/concat_record_reader.py
deleted file mode 100644
index ccbffab..0000000
--- a/pypaimon/pynative/reader/concat_record_reader.py
+++ /dev/null
@@ -1,57 +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 py4j.java_gateway import JavaObject
-
-from pypaimon.pynative.reader.core.record_iterator import RecordIterator
-from pypaimon.pynative.reader.core.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 read_batch(self) -> Optional[RecordIterator]:
- while True:
- if self.current is not None:
- iterator = self.current.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)
- else:
- return None
-
- def close(self) -> None:
- if self.current is not None:
- self.current.close()
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/core/file_record_iterator.py b/pypaimon/pynative/reader/core/file_record_iterator.py
deleted file mode 100644
index 590a65e..0000000
--- a/pypaimon/pynative/reader/core/file_record_iterator.py
+++ /dev/null
@@ -1,43 +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 abc import ABC, abstractmethod
-from typing import TypeVar
-
-from pypaimon.pynative.reader.core.record_iterator import RecordIterator
-
-T = TypeVar('T')
-
-
-class FileRecordIterator(RecordIterator[T], ABC):
- """
- A RecordIterator to support returning the record's row position and file Path.
- """
-
- @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
- """
-
- @abstractmethod
- def file_path(self) -> str:
- """
- Returns: the file path
- """
diff --git a/pypaimon/pynative/reader/core/file_record_reader.py b/pypaimon/pynative/reader/core/file_record_reader.py
deleted file mode 100644
index 2d03cd1..0000000
--- a/pypaimon/pynative/reader/core/file_record_reader.py
+++ /dev/null
@@ -1,37 +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 abc import abstractmethod
-from typing import Optional, TypeVar
-
-from pypaimon.pynative.reader.core.file_record_iterator import FileRecordIterator
-from pypaimon.pynative.reader.core.record_reader import RecordReader
-
-T = TypeVar('T')
-
-
-class FileRecordReader(RecordReader[T]):
- """
- A RecordReader to support returning FileRecordIterator.
- """
-
- @abstractmethod
- def read_batch(self) -> Optional[FileRecordIterator]:
- """
- Reads one 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_unwrap_reader.py b/pypaimon/pynative/reader/key_value_unwrap_reader.py
deleted file mode 100644
index 9add03e..0000000
--- a/pypaimon/pynative/reader/key_value_unwrap_reader.py
+++ /dev/null
@@ -1,74 +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, 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
-
-
-class KeyValueUnwrapReader(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 read_batch(self) -> Optional[RecordIterator[InternalRow]]:
- batch = self.wrapped_reader.read_batch()
- if batch is None:
- return None
-
- return KeyValueUnwrapIterator(batch)
-
- def close(self) -> None:
- self.wrapped_reader.close()
-
-
-class KeyValueUnwrapIterator(RecordIterator[InternalRow]):
- """
- An Iterator that converts a KeyValue into an 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:
- 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()
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/pyarrow_dataset_reader.py b/pypaimon/pynative/reader/pyarrow_dataset_reader.py
deleted file mode 100644
index 07ed9f7..0000000
--- a/pypaimon/pynative/reader/pyarrow_dataset_reader.py
+++ /dev/null
@@ -1,71 +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
-
-import pyarrow.dataset as ds
-
-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]):
- """
- A PyArrowDatasetReader that reads data from a dataset 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]):
-
- if primary_keys is not None:
- # 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(
- columns=fields,
- filter=predicate,
- batch_size=batch_size
- )
- self.batch_iterator = self.scanner.to_batches()
-
- def read_batch(self) -> Optional[FileRecordIterator[InternalRow]]:
- 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
-
- def close(self):
- pass
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/reader/__init__.py b/pypaimon/pynative/row/__init__.py
similarity index 100%
copy from pypaimon/pynative/reader/__init__.py
copy to pypaimon/pynative/row/__init__.py
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('<b', bytes_data[field_offset:field_offset + 1])[0]
+
+ @classmethod
+ def _parse_short(cls, bytes_data: bytes, field_offset: int) -> int:
+ return struct.unpack('<h', bytes_data[field_offset:field_offset + 2])[0]
+
+ @classmethod
+ def _parse_int(cls, bytes_data: bytes, field_offset: int) -> 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('<i', bytes_data[field_offset:field_offset + 4])[0]
+
+ @classmethod
+ def _parse_long(cls, bytes_data: bytes, field_offset: int) -> 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('<q', bytes_data[field_offset:field_offset + 8])[0]
+
+ @classmethod
+ def _parse_float(cls, bytes_data: bytes, field_offset: int) -> float:
+ return struct.unpack('<f', bytes_data[field_offset:field_offset + 4])[0]
+
+ @classmethod
+ def _parse_double(cls, bytes_data: bytes, field_offset: int) -> 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('<d', bytes_data[field_offset:field_offset + 8])[0]
+
+ @classmethod
+ def _parse_string(cls, bytes_data: bytes, base_offset: int, field_offset: int) -> 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('<q', bytes_data[field_offset:field_offset + 8])[0]
+ mark = offset_and_len & cls.HIGHEST_FIRST_BIT
+ if mark == 0:
+ sub_offset = (offset_and_len >> 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('<q', bytes_data[field_offset:field_offset + 8])[0]
+ mark = offset_and_len & cls.HIGHEST_FIRST_BIT
+ if mark == 0:
+ sub_offset = (offset_and_len >> 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('<q', bytes_data[field_offset:field_offset + 8])[0]
+ 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
+ return Decimal(unscaled_long) / (10 ** scale)
+
+ @classmethod
+ def _parse_timestamp(cls, bytes_data: bytes, base_offset: int, field_offset: int, data_type: DataType) -> datetime:
+ millis = struct.unpack('<q', bytes_data[field_offset:field_offset + 8])[0]
+ return datetime.fromtimestamp(millis / 1000.0, tz=timezone.utc)
+
+ @classmethod
+ def _parse_date(cls, bytes_data: bytes, field_offset: int) -> datetime:
+ days = struct.unpack('<i', bytes_data[field_offset:field_offset + 4])[0]
+ return datetime(1970, 1, 1) + timedelta(days=days)
+
+ @classmethod
+ def _parse_time(cls, bytes_data: bytes, field_offset: int) -> datetime:
+ millis = struct.unpack('<i', bytes_data[field_offset:field_offset + 4])[0]
+ seconds = millis // 1000
+ microseconds = (millis % 1000) * 1000
+ return datetime(1970, 1, 1).replace(
+ hour=seconds // 3600,
+ minute=(seconds % 3600) // 60,
+ second=seconds % 60,
+ microsecond=microseconds
+ )
+
+
+class BinaryRowSerializer:
+ 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 to_bytes(cls, binary_row: BinaryRow) -> 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('<q', fixed_part, null_bits_size_in_bytes + i * 8, 0)
+ variable_data.append(b'')
+ variable_offsets.append(0)
+ continue
+
+ field_offset = null_bits_size_in_bytes + i * 8
+ if field.type.type_name.upper() in ['VARCHAR', 'STRING', 'CHAR', 'BINARY', 'VARBINARY', 'BYTES']:
+ if field.type.type_name.upper() in ['VARCHAR', 'STRING', 'CHAR']:
+ if isinstance(value, str):
+ value_bytes = value.encode('utf-8')
+ else:
+ value_bytes = bytes(value)
+ else:
+ if isinstance(value, bytes):
+ value_bytes = value
+ else:
+ value_bytes = bytes(value)
+
+ length = len(value_bytes)
+ if length <= cls.MAX_FIX_PART_DATA_SIZE:
+ fixed_part[field_offset:field_offset + length] = value_bytes
+ for j in range(length, 8):
+ fixed_part[field_offset + j] = 0
+ packed_long = struct.unpack_from('<q', fixed_part, field_offset)[0]
+
+ offset_and_len = packed_long | (length << 56) | cls.HIGHEST_FIRST_BIT
+ if offset_and_len > 0x7FFFFFFFFFFFFFFF:
+ offset_and_len = offset_and_len - 0x10000000000000000
+ struct.pack_into('<q', fixed_part, field_offset, offset_and_len)
+ variable_data.append(b'')
+ variable_offsets.append(0)
+ else:
+ variable_data.append(value_bytes)
+ variable_offsets.append(current_offset)
+ current_offset += len(value_bytes)
+ offset_and_len = (variable_offsets[i] << 32) | len(variable_data[i])
+ struct.pack_into('<q', fixed_part, null_bits_size_in_bytes + i * 8, offset_and_len)
+ else:
+ if field.type.type_name.upper() in ['BOOLEAN', 'BOOL']:
+ struct.pack_into('<b', fixed_part, field_offset, 1 if value else 0)
+ elif field.type.type_name.upper() in ['TINYINT', 'BYTE']:
+ struct.pack_into('<b', fixed_part, field_offset, value)
+ elif field.type.type_name.upper() in ['SMALLINT', 'SHORT']:
+ struct.pack_into('<h', fixed_part, field_offset, value)
+ elif field.type.type_name.upper() in ['INT', 'INTEGER']:
+ struct.pack_into('<i', fixed_part, field_offset, value)
+ elif field.type.type_name.upper() in ['BIGINT', 'LONG']:
+ struct.pack_into('<q', fixed_part, field_offset, value)
+ elif field.type.type_name.upper() in ['FLOAT', 'REAL']:
+ struct.pack_into('<f', fixed_part, field_offset, value)
+ elif field.type.type_name.upper() in ['DOUBLE']:
+ struct.pack_into('<d', fixed_part, field_offset, value)
+ else:
+ field_bytes = cls._serialize_field_value(value, field.type)
+ fixed_part[field_offset:field_offset + len(field_bytes)] = field_bytes
+
+ variable_data.append(b'')
+ variable_offsets.append(0)
+
+ result = bytes(fixed_part) + b''.join(variable_data)
+ return result
+
+ @classmethod
+ def _calculate_bit_set_width_in_bytes(cls, arity: int) -> 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('<b', 1 if value else 0)
+
+ @classmethod
+ def _serialize_byte(cls, value: int) -> bytes:
+ return struct.pack('<b', value)
+
+ @classmethod
+ def _serialize_short(cls, value: int) -> bytes:
+ return struct.pack('<h', value)
+
+ @classmethod
+ def _serialize_int(cls, value: int) -> bytes:
+ return struct.pack('<i', value)
+
+ @classmethod
+ def _serialize_long(cls, value: int) -> bytes:
+ return struct.pack('<q', value)
+
+ @classmethod
+ def _serialize_float(cls, value: float) -> bytes:
+ return struct.pack('<f', value)
+
+ @classmethod
+ def _serialize_double(cls, value: float) -> bytes:
+ return struct.pack('<d', value)
+
+ @classmethod
+ def _serialize_string(cls, value: str) -> 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('<q', offset_and_len)
+
+ @classmethod
+ def _serialize_binary(cls, value: bytes) -> 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('<q', offset_and_len)
+
+ @classmethod
+ def _serialize_decimal(cls, value: Decimal, data_type: DataType) -> 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('<q', unscaled_value)
+
+ @classmethod
+ def _serialize_timestamp(cls, value: datetime) -> bytes:
+ if value.tzinfo is None:
+ value = value.replace(tzinfo=timezone.utc)
+ millis = int(value.timestamp() * 1000)
+ return struct.pack('<q', millis)
+
+ @classmethod
+ def _serialize_date(cls, value: datetime) -> 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('<i', days)
+
+ @classmethod
+ def _serialize_time(cls, value: datetime) -> 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('<i', millis)
diff --git a/pypaimon/pynative/common/row/internal_row.py b/pypaimon/pynative/row/internal_row.py
similarity index 83%
rename from pypaimon/pynative/common/row/internal_row.py
rename to pypaimon/pynative/row/internal_row.py
index 4c46ed9..7c7d0d2 100644
--- a/pypaimon/pynative/common/row/internal_row.py
+++ b/pypaimon/pynative/row/internal_row.py
@@ -19,7 +19,7 @@
from abc import ABC, abstractmethod
from typing import Any
-from pypaimon.pynative.common.row.row_kind import RowKind
+from pypaimon.pynative.row.row_kind import RowKind
class InternalRow(ABC):
@@ -40,24 +40,12 @@
"""
@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:
"""
Returns the number of fields in this row.
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/row/offset_row.py b/pypaimon/pynative/row/offset_row.py
new file mode 100644
index 0000000..e3975bd
--- /dev/null
+++ b/pypaimon/pynative/row/offset_row.py
@@ -0,0 +1,58 @@
+################################################################################
+# 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.row.internal_row import InternalRow
+from pypaimon.pynative.row.row_kind import RowKind
+
+
+class OffsetRow(InternalRow):
+ """A InternalRow to wrap row with offset."""
+
+ 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_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 row arity {self.arity}")
+ return self.row_tuple[self.offset + pos]
+
+ def is_null_at(self, pos: int) -> bool:
+ return self.get_field(pos) is None
+
+ def get_row_kind(self) -> RowKind:
+ return RowKind(self.row_kind_byte)
+
+ 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 @@
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
--- /dev/null
+++ b/pypaimon/pynative/sample/__init__.py
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
--- /dev/null
+++ b/pypaimon/pynative/sample/hdfs_read_and_write.py
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://<your-bucket>/<warehouse-path>'
+ catalog = CatalogFactory.create({
+ 'warehouse': warehouse,
+ 's3.endpoint': 'oss-<your-region>.aliyuncs.com',
+ 's3.access-key': '<your-ak>',
+ 's3.secret-key': '<your-sk>',
+ 's3.region': '<your-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/reader/__init__.py b/pypaimon/pynative/table/__init__.py
similarity index 100%
copy from pypaimon/pynative/reader/__init__.py
copy to pypaimon/pynative/table/__init__.py
diff --git a/pypaimon/pynative/common/row/__init__.py b/pypaimon/pynative/table/bucket_mode.py
similarity index 81%
copy from pypaimon/pynative/common/row/__init__.py
copy to pypaimon/pynative/table/bucket_mode.py
index 65b48d4..df6c8e9 100644
--- a/pypaimon/pynative/common/row/__init__.py
+++ b/pypaimon/pynative/table/bucket_mode.py
@@ -15,3 +15,15 @@
# 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/__init__.py b/pypaimon/pynative/table/manifest_entry.py
similarity index 76%
copy from pypaimon/pynative/common/row/__init__.py
copy to pypaimon/pynative/table/manifest_entry.py
index 65b48d4..0edbad4 100644
--- a/pypaimon/pynative/common/row/__init__.py
+++ b/pypaimon/pynative/table/manifest_entry.py
@@ -15,3 +15,17 @@
# See the License for the specific language governing permissions and
# limitations under the License.
################################################################################
+
+from dataclasses import dataclass
+
+from pypaimon.pynative.row.binary_row import BinaryRow
+from pypaimon.pynative.table.data_file_meta import DataFileMeta
+
+
+@dataclass
+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 @@
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 @@
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/writer/__init__.py b/pypaimon/pynative/write/__init__.py
similarity index 100%
copy from pypaimon/pynative/writer/__init__.py
copy to pypaimon/pynative/write/__init__.py
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/writer/__init__.py b/pypaimon/pynative/write/writer/__init__.py
similarity index 100%
rename from pypaimon/pynative/writer/__init__.py
rename to pypaimon/pynative/write/writer/__init__.py
diff --git a/pypaimon/pynative/reader/empty_record_reader.py b/pypaimon/pynative/write/writer/append_only_data_writer.py
similarity index 68%
copy from pypaimon/pynative/reader/empty_record_reader.py
copy to pypaimon/pynative/write/writer/append_only_data_writer.py
index 9883cb8..b5c9e47 100644
--- a/pypaimon/pynative/reader/empty_record_reader.py
+++ b/pypaimon/pynative/write/writer/append_only_data_writer.py
@@ -16,22 +16,16 @@
# limitations under the License.
################################################################################
-from typing import Optional
+import pyarrow as pa
-from pypaimon.pynative.reader.core.file_record_reader import FileRecordReader
-from pypaimon.pynative.reader.core.record_iterator import RecordIterator
+from pypaimon.pynative.write.writer.data_writer import DataWriter
-class EmptyFileRecordReader(FileRecordReader):
- """
- An empty FileRecordReader.
- """
+class AppendOnlyDataWriter(DataWriter):
+ """Data writer for append-only tables."""
- def __init__(self):
- pass
+ def _process_data(self, data: pa.RecordBatch) -> pa.RecordBatch:
+ return data
- def read_batch(self) -> Optional[RecordIterator]:
- return None
-
- def close(self) -> None:
- pass
+ 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 @@
# 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]