From d9e2dec55faf352c7812cd771824fcbb4be4f4a4 Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Sun, 3 May 2026 09:50:37 +0800 Subject: [PATCH 01/12] [python][compact] phase-1: extend commit protocol for compaction Lay the protocol-level groundwork for upcoming compaction work: - CommitMessage gains compact_before / compact_after fields so a single message can carry both deletion and addition of files in a compact result. - FileStoreCommit emits ADD entries for compact_after and DELETE entries for compact_before; commit() auto-selects COMPACT kind when no new_files are present, and a dedicated commit_compact() helper enforces COMPACT-only semantics with no row-id assignment. - DataFileMeta exposes to_dict / from_dict round-trip plus tagged-value encoding (bytes, decimal, datetime, date, time, Timestamp) so file metas can be shipped JSON-safely between processes. - New CommitMessageSerializer wraps the JSON form for use as a CompactTask payload (Phase 4 will consume it from the Ray executor). No write/read behavior changes for existing callers. --- .../manifest/schema/data_file_meta.py | 169 ++++++++++++++- .../tests/commit_message_serializer_test.py | 191 +++++++++++++++++ .../tests/file_store_commit_compact_test.py | 194 ++++++++++++++++++ .../pypaimon/write/commit_message.py | 8 +- .../write/commit_message_serializer.py | 78 +++++++ .../pypaimon/write/file_store_commit.py | 86 ++++++-- 6 files changed, 708 insertions(+), 18 deletions(-) create mode 100644 paimon-python/pypaimon/tests/commit_message_serializer_test.py create mode 100644 paimon-python/pypaimon/tests/file_store_commit_compact_test.py create mode 100644 paimon-python/pypaimon/write/commit_message_serializer.py diff --git a/paimon-python/pypaimon/manifest/schema/data_file_meta.py b/paimon-python/pypaimon/manifest/schema/data_file_meta.py index 870dde7aa7b8..63d3f10e5f64 100644 --- a/paimon-python/pypaimon/manifest/schema/data_file_meta.py +++ b/paimon-python/pypaimon/manifest/schema/data_file_meta.py @@ -17,15 +17,19 @@ ################################################################################ from dataclasses import dataclass -from datetime import datetime -from typing import List, Optional +from datetime import date, datetime, time as dt_time +from decimal import Decimal +from base64 import b64decode, b64encode +from typing import Any, Dict, List, Optional import time from pypaimon.utils.range import Range from pypaimon.data.timestamp import Timestamp from pypaimon.manifest.schema.simple_stats import (KEY_STATS_SCHEMA, VALUE_STATS_SCHEMA, SimpleStats) +from pypaimon.schema.data_types import DataField from pypaimon.table.row.generic_row import GenericRow +from pypaimon.table.row.internal_row import RowKind from pypaimon.utils.file_store_path_factory import _is_null_or_whitespace_only @@ -224,6 +228,167 @@ def assign_sequence_number(self, min_sequence_number: int, max_sequence_number: file_path=self.file_path ) + def to_dict(self) -> Dict[str, Any]: + """Serialize to a JSON-friendly dict for cross-process transport (e.g. Ray task payloads). + + Field types preserved via tagged objects (see _encode_value/_decode_value). + """ + return { + "file_name": self.file_name, + "file_size": self.file_size, + "row_count": self.row_count, + "min_key": _generic_row_to_dict(self.min_key), + "max_key": _generic_row_to_dict(self.max_key), + "key_stats": _simple_stats_to_dict(self.key_stats), + "value_stats": _simple_stats_to_dict(self.value_stats), + "min_sequence_number": self.min_sequence_number, + "max_sequence_number": self.max_sequence_number, + "schema_id": self.schema_id, + "level": self.level, + "extra_files": list(self.extra_files) if self.extra_files is not None else [], + "creation_time": _timestamp_to_dict(self.creation_time), + "delete_row_count": self.delete_row_count, + "embedded_index": _bytes_to_str(self.embedded_index), + "file_source": self.file_source, + "value_stats_cols": list(self.value_stats_cols) if self.value_stats_cols is not None else None, + "external_path": self.external_path, + "first_row_id": self.first_row_id, + "write_cols": list(self.write_cols) if self.write_cols is not None else None, + "file_path": self.file_path, + } + + @classmethod + def from_dict(cls, data: Dict[str, Any]) -> "DataFileMeta": + return cls( + file_name=data["file_name"], + file_size=data["file_size"], + row_count=data["row_count"], + min_key=_generic_row_from_dict(data.get("min_key")), + max_key=_generic_row_from_dict(data.get("max_key")), + key_stats=_simple_stats_from_dict(data.get("key_stats")), + value_stats=_simple_stats_from_dict(data.get("value_stats")), + min_sequence_number=data["min_sequence_number"], + max_sequence_number=data["max_sequence_number"], + schema_id=data["schema_id"], + level=data["level"], + extra_files=list(data.get("extra_files") or []), + creation_time=_timestamp_from_dict(data.get("creation_time")), + delete_row_count=data.get("delete_row_count"), + embedded_index=_bytes_from_str(data.get("embedded_index")), + file_source=data.get("file_source"), + value_stats_cols=list(data["value_stats_cols"]) if data.get("value_stats_cols") is not None else None, + external_path=data.get("external_path"), + first_row_id=data.get("first_row_id"), + write_cols=list(data["write_cols"]) if data.get("write_cols") is not None else None, + file_path=data.get("file_path"), + ) + + +def _bytes_to_str(value: Optional[bytes]) -> Optional[str]: + if value is None: + return None + return b64encode(value).decode("ascii") + + +def _bytes_from_str(value: Optional[str]) -> Optional[bytes]: + if value is None: + return None + return b64decode(value.encode("ascii")) + + +def _timestamp_to_dict(ts: Optional[Timestamp]) -> Optional[Dict[str, int]]: + if ts is None: + return None + return {"ms": ts.get_millisecond(), "ns": ts.get_nano_of_millisecond()} + + +def _timestamp_from_dict(data: Optional[Dict[str, int]]) -> Optional[Timestamp]: + if data is None: + return None + return Timestamp(data["ms"], data.get("ns", 0)) + + +def _encode_value(value: Any) -> Any: + """Encode a GenericRow / SimpleStats field value into a JSON-friendly form. + + Tagged dicts mark non-JSON-native types so _decode_value can round-trip them. + """ + if value is None or isinstance(value, (bool, int, float, str)): + return value + if isinstance(value, bytes): + return {"__t__": "bytes", "v": b64encode(value).decode("ascii")} + if isinstance(value, Decimal): + return {"__t__": "decimal", "v": str(value)} + if isinstance(value, Timestamp): + return {"__t__": "ts", "ms": value.get_millisecond(), "ns": value.get_nano_of_millisecond()} + if isinstance(value, datetime): + return {"__t__": "datetime", "v": value.isoformat()} + if isinstance(value, date): + return {"__t__": "date", "v": value.isoformat()} + if isinstance(value, dt_time): + return {"__t__": "time", "v": value.isoformat()} + raise TypeError( + f"Unsupported value type for DataFileMeta serialization: {type(value).__name__}" + ) + + +def _decode_value(value: Any) -> Any: + if not isinstance(value, dict) or "__t__" not in value: + return value + tag = value["__t__"] + if tag == "bytes": + return b64decode(value["v"].encode("ascii")) + if tag == "decimal": + return Decimal(value["v"]) + if tag == "ts": + return Timestamp(value["ms"], value.get("ns", 0)) + if tag == "datetime": + return datetime.fromisoformat(value["v"]) + if tag == "date": + return date.fromisoformat(value["v"]) + if tag == "time": + return dt_time.fromisoformat(value["v"]) + raise ValueError(f"Unknown tagged value type: {tag}") + + +def _generic_row_to_dict(row: Optional[GenericRow]) -> Optional[Dict[str, Any]]: + if row is None: + return None + return { + "values": [_encode_value(v) for v in row.values], + "fields": [f.to_dict() for f in row.fields] if row.fields else [], + "row_kind": row.row_kind.value, + } + + +def _generic_row_from_dict(data: Optional[Dict[str, Any]]) -> Optional[GenericRow]: + if data is None: + return None + fields = [DataField.from_dict(f) for f in data.get("fields", [])] + values = [_decode_value(v) for v in data.get("values", [])] + row_kind = RowKind(data.get("row_kind", RowKind.INSERT.value)) + return GenericRow(values, fields, row_kind) + + +def _simple_stats_to_dict(stats: Optional[SimpleStats]) -> Optional[Dict[str, Any]]: + if stats is None: + return None + return { + "min_values": _generic_row_to_dict(stats.min_values), + "max_values": _generic_row_to_dict(stats.max_values), + "null_counts": list(stats.null_counts) if stats.null_counts is not None else [], + } + + +def _simple_stats_from_dict(data: Optional[Dict[str, Any]]) -> Optional[SimpleStats]: + if data is None: + return None + return SimpleStats( + min_values=_generic_row_from_dict(data.get("min_values")), + max_values=_generic_row_from_dict(data.get("max_values")), + null_counts=list(data.get("null_counts") or []), + ) + DATA_FILE_META_SCHEMA = { "type": "record", diff --git a/paimon-python/pypaimon/tests/commit_message_serializer_test.py b/paimon-python/pypaimon/tests/commit_message_serializer_test.py new file mode 100644 index 000000000000..5cb8b73afdf2 --- /dev/null +++ b/paimon-python/pypaimon/tests/commit_message_serializer_test.py @@ -0,0 +1,191 @@ +################################################################################ +# 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 unittest +from datetime import date, datetime, time as dt_time +from decimal import Decimal + +from pypaimon.data.timestamp import Timestamp +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.manifest.schema.simple_stats import SimpleStats +from pypaimon.schema.data_types import AtomicType, DataField +from pypaimon.table.row.generic_row import GenericRow +from pypaimon.table.row.internal_row import RowKind +from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.commit_message_serializer import CommitMessageSerializer + + +def _key_field(idx: int, name: str, type_str: str) -> DataField: + return DataField(idx, name, AtomicType(type_str)) + + +def _build_data_file_meta(file_name: str = "data-1.parquet") -> DataFileMeta: + pk_fields = [_key_field(0, "id", "BIGINT"), _key_field(1, "name", "STRING")] + min_key = GenericRow([1, "alice"], pk_fields) + max_key = GenericRow([99, "zoe"], pk_fields) + key_stats = SimpleStats( + min_values=GenericRow([1, "alice"], pk_fields), + max_values=GenericRow([99, "zoe"], pk_fields), + null_counts=[0, 0], + ) + value_stats = SimpleStats( + min_values=GenericRow([], []), + max_values=GenericRow([], []), + null_counts=[], + ) + return DataFileMeta.create( + file_name=file_name, + file_size=4096, + row_count=99, + min_key=min_key, + max_key=max_key, + key_stats=key_stats, + value_stats=value_stats, + min_sequence_number=10, + max_sequence_number=200, + schema_id=0, + level=0, + extra_files=["index-1.idx"], + creation_time=Timestamp.from_epoch_millis(1_700_000_000_000, 123_456), + delete_row_count=2, + embedded_index=b"\x00\x01\x02\x03embedded", + file_source=1, + value_stats_cols=["c1"], + external_path="oss://bucket/path/to/file", + first_row_id=1000, + write_cols=["id", "name"], + file_path="/abs/path/data-1.parquet", + ) + + +class DataFileMetaSerdeTest(unittest.TestCase): + + def test_to_from_dict_roundtrip(self): + original = _build_data_file_meta() + rebuilt = DataFileMeta.from_dict(original.to_dict()) + + self.assertEqual(original, rebuilt) + # spot check of complex sub-fields that use tagged encoding + self.assertEqual(original.embedded_index, rebuilt.embedded_index) + self.assertEqual(original.creation_time, rebuilt.creation_time) + self.assertEqual(original.min_key.values, rebuilt.min_key.values) + self.assertEqual(original.min_key.row_kind, rebuilt.min_key.row_kind) + self.assertEqual( + [f.to_dict() for f in original.min_key.fields], + [f.to_dict() for f in rebuilt.min_key.fields], + ) + + def test_value_encoding_supports_decimal_and_temporal_types(self): + fields = [ + _key_field(0, "amount", "DECIMAL(10, 2)"), + _key_field(1, "ts", "TIMESTAMP(6)"), + _key_field(2, "d", "DATE"), + _key_field(3, "t", "TIME"), + _key_field(4, "blob", "BYTES"), + ] + row = GenericRow( + values=[ + Decimal("12.34"), + datetime(2024, 1, 2, 3, 4, 5, 678901), + date(2024, 1, 2), + dt_time(13, 45, 30, 250000), + b"binary-payload", + ], + fields=fields, + row_kind=RowKind.UPDATE_AFTER, + ) + # Reuse the GenericRow encode path through SimpleStats + stats = SimpleStats(min_values=row, max_values=row, null_counts=[0, 0, 0, 0, 0]) + meta = _build_data_file_meta() + meta.key_stats = stats + + rebuilt = DataFileMeta.from_dict(meta.to_dict()) + + self.assertEqual(rebuilt.key_stats.min_values.values[0], Decimal("12.34")) + self.assertEqual(rebuilt.key_stats.min_values.values[1], datetime(2024, 1, 2, 3, 4, 5, 678901)) + self.assertEqual(rebuilt.key_stats.min_values.values[2], date(2024, 1, 2)) + self.assertEqual(rebuilt.key_stats.min_values.values[3], dt_time(13, 45, 30, 250000)) + self.assertEqual(rebuilt.key_stats.min_values.values[4], b"binary-payload") + self.assertEqual(rebuilt.key_stats.min_values.row_kind, RowKind.UPDATE_AFTER) + + +class CommitMessageSerializerTest(unittest.TestCase): + + def test_serialize_deserialize_roundtrip_for_compact_message(self): + before_files = [_build_data_file_meta(f"old-{i}.parquet") for i in range(3)] + after_files = [_build_data_file_meta("new-merged.parquet")] + message = CommitMessage( + partition=("2024-01-01", "us"), + bucket=2, + new_files=[], + compact_before=before_files, + compact_after=after_files, + check_from_snapshot=42, + ) + + payload = CommitMessageSerializer.serialize(message) + rebuilt = CommitMessageSerializer.deserialize(payload) + + self.assertIsInstance(payload, bytes) + self.assertEqual(message.partition, rebuilt.partition) + self.assertEqual(message.bucket, rebuilt.bucket) + self.assertEqual(message.new_files, rebuilt.new_files) + self.assertEqual(message.compact_before, rebuilt.compact_before) + self.assertEqual(message.compact_after, rebuilt.compact_after) + self.assertEqual(message.check_from_snapshot, rebuilt.check_from_snapshot) + + def test_serialize_deserialize_roundtrip_for_append_message(self): + message = CommitMessage( + partition=(), + bucket=0, + new_files=[_build_data_file_meta("append-1.parquet")], + ) + + rebuilt = CommitMessageSerializer.deserialize(CommitMessageSerializer.serialize(message)) + + self.assertEqual(message.partition, rebuilt.partition) + self.assertEqual(message.bucket, rebuilt.bucket) + self.assertEqual(message.new_files, rebuilt.new_files) + self.assertEqual([], rebuilt.compact_before) + self.assertEqual([], rebuilt.compact_after) + + def test_unsupported_version_is_rejected(self): + message = CommitMessage(partition=(), bucket=0, new_files=[_build_data_file_meta()]) + payload_dict = CommitMessageSerializer.to_dict(message) + payload_dict["version"] = CommitMessageSerializer.VERSION + 1 + + with self.assertRaises(ValueError): + CommitMessageSerializer.from_dict(payload_dict) + + def test_serialize_list_round_trip(self): + messages = [ + CommitMessage(partition=(f"p{i}",), bucket=i, new_files=[_build_data_file_meta(f"f{i}.parquet")]) + for i in range(3) + ] + payloads = CommitMessageSerializer.serialize_list(messages) + rebuilt = CommitMessageSerializer.deserialize_list(payloads) + + self.assertEqual(len(messages), len(rebuilt)) + for original, copy in zip(messages, rebuilt): + self.assertEqual(original.partition, copy.partition) + self.assertEqual(original.bucket, copy.bucket) + self.assertEqual(original.new_files, copy.new_files) + + +if __name__ == "__main__": + unittest.main() diff --git a/paimon-python/pypaimon/tests/file_store_commit_compact_test.py b/paimon-python/pypaimon/tests/file_store_commit_compact_test.py new file mode 100644 index 000000000000..e3643ae4ff47 --- /dev/null +++ b/paimon-python/pypaimon/tests/file_store_commit_compact_test.py @@ -0,0 +1,194 @@ +################################################################################ +# 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 unittest +from datetime import datetime +from unittest.mock import Mock, patch + +from pypaimon.data.timestamp import Timestamp +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.manifest.schema.simple_stats import SimpleStats +from pypaimon.schema.data_types import AtomicType, DataField +from pypaimon.table.row.generic_row import GenericRow +from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.file_store_commit import FileStoreCommit + + +def _make_file(name: str, *, first_row_id=None) -> DataFileMeta: + return DataFileMeta.create( + file_name=name, + file_size=4096, + row_count=10, + min_key=GenericRow([], []), + max_key=GenericRow([], []), + key_stats=SimpleStats.empty_stats(), + value_stats=SimpleStats.empty_stats(), + min_sequence_number=1, + max_sequence_number=10, + schema_id=0, + level=0, + extra_files=[], + creation_time=Timestamp.from_local_date_time(datetime(2024, 1, 15, 10, 30, 0)), + first_row_id=first_row_id, + ) + + +@patch('pypaimon.write.file_store_commit.SnapshotManager') +@patch('pypaimon.write.file_store_commit.ManifestFileManager') +@patch('pypaimon.write.file_store_commit.ManifestListManager') +class TestFileStoreCommitCompact(unittest.TestCase): + """Phase 1 protocol-level tests: verify compact_before/after entries flow correctly through commit(). + + These tests stub _try_commit so we only verify the entry-construction and commit_kind selection. + Full e2e (with real manifest writes / scans) is covered in Phase 2 once the rewriter exists. + """ + + def setUp(self): + self.mock_table = Mock() + self.mock_table.partition_keys = ['dt'] + self.mock_table.partition_keys_fields = [DataField(0, 'dt', AtomicType('STRING'))] + self.mock_table.total_buckets = 4 + self.mock_table.current_branch.return_value = 'main' + self.mock_table.identifier = 'default.t' + self.mock_snapshot_commit = Mock() + + def _create_commit(self): + return FileStoreCommit( + snapshot_commit=self.mock_snapshot_commit, + table=self.mock_table, + commit_user='test_user', + ) + + def test_build_entries_emits_add_for_new_files(self, *_): + commit = self._create_commit() + msg = CommitMessage( + partition=('2024-01-15',), + bucket=2, + new_files=[_make_file('a.parquet')], + ) + + entries = commit._build_commit_entries([msg]) + + self.assertEqual(1, len(entries)) + self.assertEqual(0, entries[0].kind) + self.assertEqual(2, entries[0].bucket) + self.assertEqual('a.parquet', entries[0].file.file_name) + self.assertEqual(['2024-01-15'], list(entries[0].partition.values)) + + def test_build_entries_emits_delete_for_compact_before_and_add_for_compact_after(self, *_): + commit = self._create_commit() + msg = CommitMessage( + partition=('2024-01-15',), + bucket=1, + compact_before=[_make_file('old-1.parquet'), _make_file('old-2.parquet')], + compact_after=[_make_file('merged.parquet')], + ) + + entries = commit._build_commit_entries([msg]) + + kinds = [e.kind for e in entries] + names = [e.file.file_name for e in entries] + self.assertEqual([1, 1, 0], kinds) + self.assertEqual(['old-1.parquet', 'old-2.parquet', 'merged.parquet'], names) + self.assertTrue(all(e.bucket == 1 for e in entries)) + + def test_commit_with_only_compact_messages_uses_compact_kind(self, *_): + commit = self._create_commit() + commit._try_commit = Mock() + msg = CommitMessage( + partition=('p1',), + bucket=0, + compact_before=[_make_file('old.parquet')], + compact_after=[_make_file('new.parquet')], + ) + + commit.commit([msg], commit_identifier=100) + + commit._try_commit.assert_called_once() + call_kwargs = commit._try_commit.call_args.kwargs + self.assertEqual('COMPACT', call_kwargs['commit_kind']) + self.assertEqual(100, call_kwargs['commit_identifier']) + + def test_commit_with_new_files_keeps_append_kind_even_when_compact_fields_present(self, *_): + commit = self._create_commit() + commit._try_commit = Mock() + msg = CommitMessage( + partition=('p1',), + bucket=0, + new_files=[_make_file('new.parquet')], + compact_before=[_make_file('old.parquet')], + compact_after=[_make_file('merged.parquet')], + ) + + commit.commit([msg], commit_identifier=200) + + call_kwargs = commit._try_commit.call_args.kwargs + self.assertEqual('APPEND', call_kwargs['commit_kind']) + + def test_commit_compact_uses_compact_kind_and_no_conflict_detection(self, *_): + commit = self._create_commit() + commit._try_commit = Mock() + msg = CommitMessage( + partition=('p1',), + bucket=3, + compact_before=[_make_file('old.parquet')], + compact_after=[_make_file('new.parquet')], + ) + + commit.commit_compact([msg], commit_identifier=300) + + commit._try_commit.assert_called_once() + kwargs = commit._try_commit.call_args.kwargs + self.assertEqual('COMPACT', kwargs['commit_kind']) + self.assertEqual(300, kwargs['commit_identifier']) + self.assertFalse(kwargs['detect_conflicts']) + self.assertFalse(kwargs['allow_rollback']) + + def test_commit_compact_rejects_messages_with_new_files(self, *_): + commit = self._create_commit() + msg = CommitMessage( + partition=('p1',), + bucket=0, + new_files=[_make_file('append.parquet')], + compact_before=[_make_file('old.parquet')], + compact_after=[_make_file('new.parquet')], + ) + + with self.assertRaises(ValueError): + commit.commit_compact([msg], commit_identifier=400) + + def test_commit_compact_skips_when_no_messages(self, *_): + commit = self._create_commit() + commit._try_commit = Mock() + + commit.commit_compact([], commit_identifier=500) + + commit._try_commit.assert_not_called() + + def test_commit_compact_skips_when_messages_have_no_files(self, *_): + commit = self._create_commit() + commit._try_commit = Mock() + empty_msg = CommitMessage(partition=('p1',), bucket=0) + + commit.commit_compact([empty_msg], commit_identifier=600) + + commit._try_commit.assert_not_called() + + +if __name__ == '__main__': + unittest.main() diff --git a/paimon-python/pypaimon/write/commit_message.py b/paimon-python/pypaimon/write/commit_message.py index d560c5a2479f..77344058c3b4 100644 --- a/paimon-python/pypaimon/write/commit_message.py +++ b/paimon-python/pypaimon/write/commit_message.py @@ -16,7 +16,7 @@ # limitations under the License. ################################################################################ -from dataclasses import dataclass +from dataclasses import dataclass, field from typing import List, Tuple, Optional from pypaimon.manifest.schema.data_file_meta import DataFileMeta @@ -26,8 +26,10 @@ class CommitMessage: partition: Tuple bucket: int - new_files: List[DataFileMeta] + new_files: List[DataFileMeta] = field(default_factory=list) + compact_before: List[DataFileMeta] = field(default_factory=list) + compact_after: List[DataFileMeta] = field(default_factory=list) check_from_snapshot: Optional[int] = -1 def is_empty(self): - return not self.new_files + return not self.new_files and not self.compact_before and not self.compact_after diff --git a/paimon-python/pypaimon/write/commit_message_serializer.py b/paimon-python/pypaimon/write/commit_message_serializer.py new file mode 100644 index 000000000000..51613d22a2fe --- /dev/null +++ b/paimon-python/pypaimon/write/commit_message_serializer.py @@ -0,0 +1,78 @@ +################################################################################ +# 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 Any, Dict, List + +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.write.commit_message import CommitMessage + + +class CommitMessageSerializer: + """Cross-process serializer for CommitMessage payloads. + + JSON-based on purpose: human-debuggable, version-tolerant across worker + Python versions, and avoids the security/compat pitfalls of pickle when + shipping CompactTask outputs from Ray workers back to the driver. + """ + + VERSION = 1 + + @classmethod + def serialize(cls, message: CommitMessage) -> bytes: + return json.dumps(cls.to_dict(message), separators=(",", ":")).encode("utf-8") + + @classmethod + def deserialize(cls, payload: bytes) -> CommitMessage: + return cls.from_dict(json.loads(payload.decode("utf-8"))) + + @classmethod + def to_dict(cls, message: CommitMessage) -> Dict[str, Any]: + return { + "version": cls.VERSION, + "partition": list(message.partition) if message.partition is not None else [], + "bucket": message.bucket, + "new_files": [f.to_dict() for f in message.new_files], + "compact_before": [f.to_dict() for f in message.compact_before], + "compact_after": [f.to_dict() for f in message.compact_after], + "check_from_snapshot": message.check_from_snapshot, + } + + @classmethod + def from_dict(cls, data: Dict[str, Any]) -> CommitMessage: + version = data.get("version", cls.VERSION) + if version != cls.VERSION: + raise ValueError( + f"Unsupported CommitMessage payload version: {version} (expected {cls.VERSION})" + ) + return CommitMessage( + partition=tuple(data.get("partition") or ()), + bucket=data["bucket"], + new_files=[DataFileMeta.from_dict(f) for f in data.get("new_files", [])], + compact_before=[DataFileMeta.from_dict(f) for f in data.get("compact_before", [])], + compact_after=[DataFileMeta.from_dict(f) for f in data.get("compact_after", [])], + check_from_snapshot=data.get("check_from_snapshot", -1), + ) + + @classmethod + def serialize_list(cls, messages: List[CommitMessage]) -> List[bytes]: + return [cls.serialize(m) for m in messages] + + @classmethod + def deserialize_list(cls, payloads: List[bytes]) -> List[CommitMessage]: + return [cls.deserialize(p) for p in payloads] diff --git a/paimon-python/pypaimon/write/file_store_commit.py b/paimon-python/pypaimon/write/file_store_commit.py index 832a39ba6887..28f4535bcdac 100644 --- a/paimon-python/pypaimon/write/file_store_commit.py +++ b/paimon-python/pypaimon/write/file_store_commit.py @@ -111,7 +111,12 @@ def __init__(self, snapshot_commit: SnapshotCommit, table, commit_user: str): self.rollback = CommitRollback(table_rollback) if table_rollback is not None else None def commit(self, commit_messages: List[CommitMessage], commit_identifier: int): - """Commit the given commit messages in normal append mode.""" + """Commit the given commit messages in normal append mode. + + new_files in each message generate ADD entries; compact_before/compact_after + generate DELETE/ADD entries respectively. If only compact_* fields are present + across all messages (no new_files), commit_kind becomes COMPACT. + """ if not commit_messages: return @@ -126,21 +131,12 @@ def commit(self, commit_messages: List[CommitMessage], commit_identifier: int): self.table.identifier, len(commit_messages), ) - commit_entries = [] - for msg in commit_messages: - partition = GenericRow(list(msg.partition), self.table.partition_keys_fields) - for file in msg.new_files: - commit_entries.append(ManifestEntry( - kind=0, - partition=partition, - bucket=msg.bucket, - total_buckets=self.table.total_buckets, - file=file - )) + commit_entries = self._build_commit_entries(commit_messages) + has_new_files = any(msg.new_files for msg in commit_messages) logger.info("Finished collecting changes, including: %d entries", len(commit_entries)) - commit_kind = "APPEND" + commit_kind = "APPEND" if has_new_files else "COMPACT" detect_conflicts = False allow_rollback = False if self.conflict_detection.should_be_overwrite_commit(): @@ -157,6 +153,70 @@ def commit(self, commit_messages: List[CommitMessage], commit_identifier: int): detect_conflicts=detect_conflicts, allow_rollback=allow_rollback) + def commit_compact(self, commit_messages: List[CommitMessage], commit_identifier: int): + """Commit compaction results (compact_before/compact_after only). + + Each message must carry no new_files. compact_before generate DELETE entries, + compact_after generate ADD entries. Snapshot kind is COMPACT. + """ + if not commit_messages: + return + + for msg in commit_messages: + if msg.new_files: + raise ValueError( + "commit_compact rejects messages with new_files; use commit() instead." + ) + + logger.info( + "Ready to commit compact to table %s, number of commit messages: %d", + self.table.identifier, + len(commit_messages), + ) + commit_entries = self._build_commit_entries(commit_messages) + if not commit_entries: + return + + logger.info("Finished collecting compact changes: %d entries", len(commit_entries)) + + self._try_commit( + commit_kind="COMPACT", + commit_identifier=commit_identifier, + commit_entries_plan=lambda snapshot: commit_entries, + detect_conflicts=False, + allow_rollback=False, + ) + + def _build_commit_entries(self, commit_messages: List[CommitMessage]) -> List[ManifestEntry]: + entries: List[ManifestEntry] = [] + for msg in commit_messages: + partition = GenericRow(list(msg.partition), self.table.partition_keys_fields) + for file in msg.new_files: + entries.append(ManifestEntry( + kind=0, + partition=partition, + bucket=msg.bucket, + total_buckets=self.table.total_buckets, + file=file, + )) + for file in msg.compact_before: + entries.append(ManifestEntry( + kind=1, + partition=partition, + bucket=msg.bucket, + total_buckets=self.table.total_buckets, + file=file, + )) + for file in msg.compact_after: + entries.append(ManifestEntry( + kind=0, + partition=partition, + bucket=msg.bucket, + total_buckets=self.table.total_buckets, + file=file, + )) + return entries + def overwrite(self, overwrite_partition, commit_messages: List[CommitMessage], commit_identifier: int): """Commit the given commit messages in overwrite mode.""" if not commit_messages: From a28b497ae2483782a9aa5cb0dd273bd3a0bb1e67 Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Sun, 3 May 2026 09:57:21 +0800 Subject: [PATCH 02/12] [python][compact] phase-1 fixup: round-trip non-JSON-native partition values Promote encode_value/decode_value to the public DataFileMeta API and reuse them for CommitMessage.partition. Without this, partitions containing DATE/DECIMAL/bytes/Timestamp would crash json.dumps once Phase 4 ships CommitMessage payloads through Ray workers. Tests: round-trip date/Decimal/bytes and Timestamp partition tuples. --- .../manifest/schema/data_file_meta.py | 16 +++++++------ .../tests/commit_message_serializer_test.py | 24 +++++++++++++++++++ .../write/commit_message_serializer.py | 9 ++++--- 3 files changed, 39 insertions(+), 10 deletions(-) diff --git a/paimon-python/pypaimon/manifest/schema/data_file_meta.py b/paimon-python/pypaimon/manifest/schema/data_file_meta.py index 63d3f10e5f64..18a5df3cd1d9 100644 --- a/paimon-python/pypaimon/manifest/schema/data_file_meta.py +++ b/paimon-python/pypaimon/manifest/schema/data_file_meta.py @@ -231,7 +231,7 @@ def assign_sequence_number(self, min_sequence_number: int, max_sequence_number: def to_dict(self) -> Dict[str, Any]: """Serialize to a JSON-friendly dict for cross-process transport (e.g. Ray task payloads). - Field types preserved via tagged objects (see _encode_value/_decode_value). + Field types preserved via tagged objects (see encode_value/decode_value). """ return { "file_name": self.file_name, @@ -308,10 +308,12 @@ def _timestamp_from_dict(data: Optional[Dict[str, int]]) -> Optional[Timestamp]: return Timestamp(data["ms"], data.get("ns", 0)) -def _encode_value(value: Any) -> Any: - """Encode a GenericRow / SimpleStats field value into a JSON-friendly form. +def encode_value(value: Any) -> Any: + """Encode a GenericRow / SimpleStats / partition field value into a JSON-friendly form. - Tagged dicts mark non-JSON-native types so _decode_value can round-trip them. + Tagged dicts mark non-JSON-native types so decode_value can round-trip them. + Public so that callers serializing other field-bearing structures (e.g. partitions + in CommitMessage) can reuse the same tagged encoding. """ if value is None or isinstance(value, (bool, int, float, str)): return value @@ -332,7 +334,7 @@ def _encode_value(value: Any) -> Any: ) -def _decode_value(value: Any) -> Any: +def decode_value(value: Any) -> Any: if not isinstance(value, dict) or "__t__" not in value: return value tag = value["__t__"] @@ -355,7 +357,7 @@ def _generic_row_to_dict(row: Optional[GenericRow]) -> Optional[Dict[str, Any]]: if row is None: return None return { - "values": [_encode_value(v) for v in row.values], + "values": [encode_value(v) for v in row.values], "fields": [f.to_dict() for f in row.fields] if row.fields else [], "row_kind": row.row_kind.value, } @@ -365,7 +367,7 @@ def _generic_row_from_dict(data: Optional[Dict[str, Any]]) -> Optional[GenericRo if data is None: return None fields = [DataField.from_dict(f) for f in data.get("fields", [])] - values = [_decode_value(v) for v in data.get("values", [])] + values = [decode_value(v) for v in data.get("values", [])] row_kind = RowKind(data.get("row_kind", RowKind.INSERT.value)) return GenericRow(values, fields, row_kind) diff --git a/paimon-python/pypaimon/tests/commit_message_serializer_test.py b/paimon-python/pypaimon/tests/commit_message_serializer_test.py index 5cb8b73afdf2..25782b50b25a 100644 --- a/paimon-python/pypaimon/tests/commit_message_serializer_test.py +++ b/paimon-python/pypaimon/tests/commit_message_serializer_test.py @@ -172,6 +172,30 @@ def test_unsupported_version_is_rejected(self): with self.assertRaises(ValueError): CommitMessageSerializer.from_dict(payload_dict) + def test_serialize_supports_partition_with_non_json_native_types(self): + # Partitions can carry DATE/DECIMAL/bytes columns; serializer must round-trip them. + message = CommitMessage( + partition=(date(2024, 1, 2), Decimal("99.50"), b"raw"), + bucket=0, + compact_after=[_build_data_file_meta()], + ) + + rebuilt = CommitMessageSerializer.deserialize(CommitMessageSerializer.serialize(message)) + + self.assertEqual((date(2024, 1, 2), Decimal("99.50"), b"raw"), rebuilt.partition) + + def test_serialize_supports_timestamp_partition(self): + ts = Timestamp.from_epoch_millis(1_700_000_000_000, 500_000) + message = CommitMessage( + partition=(ts,), + bucket=0, + compact_after=[_build_data_file_meta()], + ) + + rebuilt = CommitMessageSerializer.deserialize(CommitMessageSerializer.serialize(message)) + + self.assertEqual((ts,), rebuilt.partition) + def test_serialize_list_round_trip(self): messages = [ CommitMessage(partition=(f"p{i}",), bucket=i, new_files=[_build_data_file_meta(f"f{i}.parquet")]) diff --git a/paimon-python/pypaimon/write/commit_message_serializer.py b/paimon-python/pypaimon/write/commit_message_serializer.py index 51613d22a2fe..13b60e201316 100644 --- a/paimon-python/pypaimon/write/commit_message_serializer.py +++ b/paimon-python/pypaimon/write/commit_message_serializer.py @@ -19,7 +19,8 @@ import json from typing import Any, Dict, List -from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.manifest.schema.data_file_meta import (DataFileMeta, decode_value, + encode_value) from pypaimon.write.commit_message import CommitMessage @@ -43,9 +44,10 @@ def deserialize(cls, payload: bytes) -> CommitMessage: @classmethod def to_dict(cls, message: CommitMessage) -> Dict[str, Any]: + partition = message.partition if message.partition is not None else () return { "version": cls.VERSION, - "partition": list(message.partition) if message.partition is not None else [], + "partition": [encode_value(v) for v in partition], "bucket": message.bucket, "new_files": [f.to_dict() for f in message.new_files], "compact_before": [f.to_dict() for f in message.compact_before], @@ -60,8 +62,9 @@ def from_dict(cls, data: Dict[str, Any]) -> CommitMessage: raise ValueError( f"Unsupported CommitMessage payload version: {version} (expected {cls.VERSION})" ) + partition_values = data.get("partition") or [] return CommitMessage( - partition=tuple(data.get("partition") or ()), + partition=tuple(decode_value(v) for v in partition_values), bucket=data["bucket"], new_files=[DataFileMeta.from_dict(f) for f in data.get("new_files", [])], compact_before=[DataFileMeta.from_dict(f) for f in data.get("compact_before", [])], From 21365a74a9c08ca51d7f5700780d57d4322384eb Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Sun, 3 May 2026 10:08:55 +0800 Subject: [PATCH 03/12] [python][compact] phase-2: append-only compaction with LocalExecutor MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit End-to-end Append-only compaction in-process, exposed as table.new_compact_job(...).execute(). The plumbing follows the same Coordinator → Executor → Driver-commit shape Spark uses, so plugging in a Ray backend in Phase 4 only swaps out the executor. New compact package layout (kept stable as Phase 3 will plug PK): pypaimon/compact/ options.py coordinator/{coordinator.py, append_compact_coordinator.py} task/{compact_task.py, append_compact_task.py} rewriter/{rewriter.py, append_compact_rewriter.py} executor/{executor.py, local_executor.py} job/compact_job.py Behavior: - Coordinator scans the latest snapshot via FileScanner.plan_files, groups by (partition, bucket), filters out already target_file_size+ files, and chunks each bucket at max_file_num. full_compaction=True rewrites every file regardless of size or count. - Rewriter feeds files batch-by-batch into AppendOnlyDataWriter so the writer's existing target_file_size rolling produces correctly sized output without a separate rolling layer. - AppendCompactTask captures the in-process FileStoreTable directly; to_dict/from_dict are stubbed and raise — Phase 4 will fill them in once Ray needs serialization. - CompactJob assembles CommitMessage(compact_before, compact_after) from each task and calls FileStoreCommit.commit_compact for a single atomic snapshot tagged COMPACT. Tests cover threshold behavior, full_compaction override, max_file_num chunking, PK rejection, partitioned/unpartitioned e2e (file count shrinks, data identity preserved, snapshot kind=COMPACT), and the no-op path. --- paimon-python/pypaimon/compact/__init__.py | 17 ++ .../pypaimon/compact/coordinator/__init__.py | 17 ++ .../coordinator/append_compact_coordinator.py | 136 +++++++++++++ .../compact/coordinator/coordinator.py | 40 ++++ .../pypaimon/compact/executor/__init__.py | 17 ++ .../pypaimon/compact/executor/executor.py | 40 ++++ .../compact/executor/local_executor.py | 34 ++++ .../pypaimon/compact/job/__init__.py | 17 ++ .../pypaimon/compact/job/compact_job.py | 109 +++++++++++ paimon-python/pypaimon/compact/options.py | 59 ++++++ .../pypaimon/compact/rewriter/__init__.py | 17 ++ .../rewriter/append_compact_rewriter.py | 102 ++++++++++ .../pypaimon/compact/rewriter/rewriter.py | 30 +++ .../pypaimon/compact/task/__init__.py | 17 ++ .../compact/task/append_compact_task.py | 89 +++++++++ .../pypaimon/compact/task/compact_task.py | 83 ++++++++ .../pypaimon/table/file_store_table.py | 21 ++ .../pypaimon/tests/compact/__init__.py | 17 ++ .../append_compact_coordinator_test.py | 161 +++++++++++++++ .../tests/compact/append_compact_e2e_test.py | 183 ++++++++++++++++++ 20 files changed, 1206 insertions(+) create mode 100644 paimon-python/pypaimon/compact/__init__.py create mode 100644 paimon-python/pypaimon/compact/coordinator/__init__.py create mode 100644 paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py create mode 100644 paimon-python/pypaimon/compact/coordinator/coordinator.py create mode 100644 paimon-python/pypaimon/compact/executor/__init__.py create mode 100644 paimon-python/pypaimon/compact/executor/executor.py create mode 100644 paimon-python/pypaimon/compact/executor/local_executor.py create mode 100644 paimon-python/pypaimon/compact/job/__init__.py create mode 100644 paimon-python/pypaimon/compact/job/compact_job.py create mode 100644 paimon-python/pypaimon/compact/options.py create mode 100644 paimon-python/pypaimon/compact/rewriter/__init__.py create mode 100644 paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py create mode 100644 paimon-python/pypaimon/compact/rewriter/rewriter.py create mode 100644 paimon-python/pypaimon/compact/task/__init__.py create mode 100644 paimon-python/pypaimon/compact/task/append_compact_task.py create mode 100644 paimon-python/pypaimon/compact/task/compact_task.py create mode 100644 paimon-python/pypaimon/tests/compact/__init__.py create mode 100644 paimon-python/pypaimon/tests/compact/append_compact_coordinator_test.py create mode 100644 paimon-python/pypaimon/tests/compact/append_compact_e2e_test.py diff --git a/paimon-python/pypaimon/compact/__init__.py b/paimon-python/pypaimon/compact/__init__.py new file mode 100644 index 000000000000..65b48d4d79b4 --- /dev/null +++ b/paimon-python/pypaimon/compact/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/paimon-python/pypaimon/compact/coordinator/__init__.py b/paimon-python/pypaimon/compact/coordinator/__init__.py new file mode 100644 index 000000000000..65b48d4d79b4 --- /dev/null +++ b/paimon-python/pypaimon/compact/coordinator/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py b/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py new file mode 100644 index 000000000000..630cb5ac83a5 --- /dev/null +++ b/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py @@ -0,0 +1,136 @@ +################################################################################ +# 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 Dict, List, Optional, Tuple + +from pypaimon.common.predicate import Predicate +from pypaimon.compact.coordinator.coordinator import CompactCoordinator +from pypaimon.compact.options import CompactOptions +from pypaimon.compact.task.append_compact_task import AppendCompactTask +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.read.scanner.file_scanner import FileScanner + + +class AppendCompactCoordinator(CompactCoordinator): + """Plan compaction tasks for append-only tables (HASH_FIXED or BUCKET_UNAWARE). + + For each (partition, bucket) we compact files smaller than the table's + target_file_size. A bucket is eligible only when it has at least + `min_file_num` such files (default 5) — matching the Java + AppendCompactCoordinator threshold and avoiding pointless rewrites of a + single small file. `full_compaction=True` overrides the threshold and + rewrites every file in every bucket regardless of size. + + The coordinator caps each task at `max_file_num` files; oversized buckets + produce multiple tasks so an executor can spread the work in parallel + instead of hot-spotting one worker on a huge bucket. + """ + + def __init__( + self, + table, + compact_options: Optional[CompactOptions] = None, + partition_predicate: Optional[Predicate] = None, + ): + if table.is_primary_key_table: + raise ValueError( + "AppendCompactCoordinator only handles append-only tables; " + "use the merge-tree coordinator for primary-key tables." + ) + self.table = table + self.options = compact_options or CompactOptions() + self.partition_predicate = partition_predicate + + def plan(self) -> List[AppendCompactTask]: + manifest_entries = self._scan_live_files() + if not manifest_entries: + return [] + + # Reduce the manifest entry stream to (partition, bucket) → live files. + # We trust the manifest scanner to have already merged ADD/DELETE + # entries; whatever survives here is currently in the snapshot. + bucket_files: Dict[Tuple[Tuple, int], List[DataFileMeta]] = defaultdict(list) + for entry in manifest_entries: + key = (tuple(entry.partition.values), entry.bucket) + bucket_files[key].append(entry.file) + + target_file_size = self.table.options.target_file_size(False) + + tasks: List[AppendCompactTask] = [] + for (partition, bucket), files in bucket_files.items(): + for chunk in self._pick_files_for_bucket(files, target_file_size): + tasks.append( + AppendCompactTask( + partition=partition, + bucket=bucket, + files=chunk, + table=self.table, + ) + ) + return tasks + + def _pick_files_for_bucket( + self, + files: List[DataFileMeta], + target_file_size: int, + ) -> List[List[DataFileMeta]]: + """Choose which files in a single bucket get compacted, batching if needed. + + Files >= target_file_size are skipped (they're already at output size and + rewriting them only spends IO). full_compaction overrides this skip. + """ + if self.options.full_compaction: + candidates = list(files) + if not candidates: + return [] + else: + candidates = [f for f in files if f.file_size < target_file_size] + if len(candidates) < self.options.min_file_num: + return [] + + # Stable order: oldest sequence first so rewrites preserve append order + # if the executor later relies on file ordering for something. + candidates.sort(key=lambda f: f.min_sequence_number) + + chunks: List[List[DataFileMeta]] = [] + max_per_task = max(self.options.max_file_num, self.options.min_file_num) + for start in range(0, len(candidates), max_per_task): + chunk = candidates[start:start + max_per_task] + if len(chunk) >= self.options.min_file_num or self.options.full_compaction: + chunks.append(chunk) + return chunks + + def _scan_live_files(self): + """Read manifest entries from the latest snapshot, applying partition filter.""" + snapshot = self.table.snapshot_manager().get_latest_snapshot() + if snapshot is None: + return [] + + from pypaimon.manifest.manifest_list_manager import ManifestListManager + manifest_list_manager = ManifestListManager(self.table) + + def manifest_scanner(): + return manifest_list_manager.read_all(snapshot), snapshot + + scanner = FileScanner( + self.table, + manifest_scanner, + partition_predicate=self.partition_predicate, + ) + return scanner.plan_files() diff --git a/paimon-python/pypaimon/compact/coordinator/coordinator.py b/paimon-python/pypaimon/compact/coordinator/coordinator.py new file mode 100644 index 000000000000..478ca1f8a470 --- /dev/null +++ b/paimon-python/pypaimon/compact/coordinator/coordinator.py @@ -0,0 +1,40 @@ +################################################################################ +# 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 List + +from pypaimon.compact.task.compact_task import CompactTask + + +class CompactCoordinator(ABC): + """Driver-side planner that turns a snapshot into a list of CompactTask. + + The coordinator runs once per CompactJob, in the driver, and **does not** + rewrite data itself. Splitting plan() from task.run() lets us pin the + snapshot scan to a single process (no concurrent manifest re-reads) while + still letting the executor distribute the resulting tasks however it likes. + """ + + @abstractmethod + def plan(self) -> List[CompactTask]: + """Return a possibly-empty list of compact tasks for the current snapshot. + + An empty list means there is nothing worth compacting at this moment; + the job should commit nothing rather than produce an empty snapshot. + """ diff --git a/paimon-python/pypaimon/compact/executor/__init__.py b/paimon-python/pypaimon/compact/executor/__init__.py new file mode 100644 index 000000000000..65b48d4d79b4 --- /dev/null +++ b/paimon-python/pypaimon/compact/executor/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/paimon-python/pypaimon/compact/executor/executor.py b/paimon-python/pypaimon/compact/executor/executor.py new file mode 100644 index 000000000000..4fc988ff01b9 --- /dev/null +++ b/paimon-python/pypaimon/compact/executor/executor.py @@ -0,0 +1,40 @@ +################################################################################ +# 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 List + +from pypaimon.compact.task.compact_task import CompactTask +from pypaimon.write.commit_message import CommitMessage + + +class CompactExecutor(ABC): + """Pluggable backend that runs a list of CompactTask and returns CommitMessages. + + Implementations decide where the work happens (current process, thread pool, + Ray cluster, ...). The contract is intentionally narrow so adding RayExecutor + in Phase 4 doesn't ripple through the coordinator/job layers. + """ + + @abstractmethod + def execute(self, tasks: List[CompactTask]) -> List[CommitMessage]: + """Run all tasks and gather one CommitMessage per task. + + Order of returned messages is not significant — the driver merges them + into a single atomic commit. + """ diff --git a/paimon-python/pypaimon/compact/executor/local_executor.py b/paimon-python/pypaimon/compact/executor/local_executor.py new file mode 100644 index 000000000000..acc01f353355 --- /dev/null +++ b/paimon-python/pypaimon/compact/executor/local_executor.py @@ -0,0 +1,34 @@ +################################################################################ +# 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 + +from pypaimon.compact.executor.executor import CompactExecutor +from pypaimon.compact.task.compact_task import CompactTask +from pypaimon.write.commit_message import CommitMessage + + +class LocalExecutor(CompactExecutor): + """Synchronous in-process executor — runs each task in series. + + Useful for tests, single-machine compactions, and as the default backend + so a CompactJob is functional out of the box without any cluster setup. + """ + + def execute(self, tasks: List[CompactTask]) -> List[CommitMessage]: + return [task.run() for task in tasks] diff --git a/paimon-python/pypaimon/compact/job/__init__.py b/paimon-python/pypaimon/compact/job/__init__.py new file mode 100644 index 000000000000..65b48d4d79b4 --- /dev/null +++ b/paimon-python/pypaimon/compact/job/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/paimon-python/pypaimon/compact/job/compact_job.py b/paimon-python/pypaimon/compact/job/compact_job.py new file mode 100644 index 000000000000..95083928bd44 --- /dev/null +++ b/paimon-python/pypaimon/compact/job/compact_job.py @@ -0,0 +1,109 @@ +################################################################################ +# 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 logging +import uuid +from typing import List, Optional + +from pypaimon.common.predicate import Predicate +from pypaimon.compact.coordinator.append_compact_coordinator import \ + AppendCompactCoordinator +from pypaimon.compact.coordinator.coordinator import CompactCoordinator +from pypaimon.compact.executor.executor import CompactExecutor +from pypaimon.compact.executor.local_executor import LocalExecutor +from pypaimon.compact.options import CompactOptions +from pypaimon.snapshot.snapshot import BATCH_COMMIT_IDENTIFIER +from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.file_store_commit import FileStoreCommit + +logger = logging.getLogger(__name__) + + +class CompactJob: + """Driver-side orchestrator: plan → distribute → commit, in three steps. + + The flow purposely mirrors what Spark's CompactProcedure does: + 1. A Coordinator runs once on the driver and produces independent tasks. + 2. An Executor (Local in Phase 2, Ray in Phase 4) runs the tasks and + returns CommitMessage(compact_before, compact_after) per task. + 3. The driver collects all messages and commits them atomically with + commit_kind="COMPACT". + """ + + def __init__( + self, + table, + compact_options: Optional[CompactOptions] = None, + executor: Optional[CompactExecutor] = None, + partition_predicate: Optional[Predicate] = None, + commit_user: Optional[str] = None, + ): + self.table = table + self.compact_options = compact_options or CompactOptions() + self.executor = executor or LocalExecutor() + self.partition_predicate = partition_predicate + self.commit_user = commit_user or str(uuid.uuid4()) + + def execute(self) -> List[CommitMessage]: + """Run the compaction job and return the messages that were committed. + + Returns an empty list when there is nothing to compact. + """ + coordinator = self._build_coordinator() + tasks = coordinator.plan() + if not tasks: + logger.info( + "No compaction work for table %s at the current snapshot", + self.table.identifier, + ) + return [] + + logger.info( + "Compacting table %s: %d task(s) via %s", + self.table.identifier, + len(tasks), + type(self.executor).__name__, + ) + messages = self.executor.execute(tasks) + self._commit(messages) + return messages + + def _build_coordinator(self) -> CompactCoordinator: + if self.table.is_primary_key_table: + # Phase 3 will plug in the merge-tree coordinator here. + raise NotImplementedError( + "Primary-key compaction is not implemented yet (Phase 3)." + ) + return AppendCompactCoordinator( + table=self.table, + compact_options=self.compact_options, + partition_predicate=self.partition_predicate, + ) + + def _commit(self, messages: List[CommitMessage]) -> None: + non_empty = [m for m in messages if not m.is_empty()] + if not non_empty: + return + snapshot_commit = self.table.new_snapshot_commit() + if snapshot_commit is None: + raise RuntimeError("Table does not provide a SnapshotCommit instance") + file_store_commit = FileStoreCommit(snapshot_commit, self.table, self.commit_user) + try: + file_store_commit.commit_compact(non_empty, BATCH_COMMIT_IDENTIFIER) + finally: + file_store_commit.close() diff --git a/paimon-python/pypaimon/compact/options.py b/paimon-python/pypaimon/compact/options.py new file mode 100644 index 000000000000..dd2a050b7bf0 --- /dev/null +++ b/paimon-python/pypaimon/compact/options.py @@ -0,0 +1,59 @@ +################################################################################ +# 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 Any, Dict, Optional + +# Defaults mirror Java's append-only compaction options where possible. Only the +# subset that drives append/PK planning is exposed here; per-table options +# (file format, compression, target_file_size) still come from CoreOptions on +# the table itself. +DEFAULT_MIN_FILE_NUM = 5 +DEFAULT_MAX_FILE_NUM = 50 +DEFAULT_FORCE_FULL = False + + +@dataclass +class CompactOptions: + """Knobs that drive compaction planning. + + target_file_size is intentionally absent — it is sourced from the table's + own CoreOptions (via DataWriter rolling) so a job inherits whatever the + writer would use, keeping output sizes consistent across write/compact. + """ + + min_file_num: int = DEFAULT_MIN_FILE_NUM + max_file_num: int = DEFAULT_MAX_FILE_NUM + full_compaction: bool = DEFAULT_FORCE_FULL + + def to_dict(self) -> Dict[str, Any]: + return { + "min_file_num": self.min_file_num, + "max_file_num": self.max_file_num, + "full_compaction": self.full_compaction, + } + + @classmethod + def from_dict(cls, data: Optional[Dict[str, Any]]) -> "CompactOptions": + if not data: + return cls() + return cls( + min_file_num=data.get("min_file_num", DEFAULT_MIN_FILE_NUM), + max_file_num=data.get("max_file_num", DEFAULT_MAX_FILE_NUM), + full_compaction=data.get("full_compaction", DEFAULT_FORCE_FULL), + ) diff --git a/paimon-python/pypaimon/compact/rewriter/__init__.py b/paimon-python/pypaimon/compact/rewriter/__init__.py new file mode 100644 index 000000000000..65b48d4d79b4 --- /dev/null +++ b/paimon-python/pypaimon/compact/rewriter/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py b/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py new file mode 100644 index 000000000000..f8c8b578e598 --- /dev/null +++ b/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.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. +################################################################################ + +from typing import Iterator, List, Tuple + +import pyarrow as pa +import pyarrow.dataset as ds + +from pypaimon.compact.rewriter.rewriter import CompactRewriter +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.read.split_read import format_identifier +from pypaimon.write.writer.append_only_data_writer import AppendOnlyDataWriter + + +class AppendCompactRewriter(CompactRewriter): + """Reads input append-only files and re-writes them via the table's normal + AppendOnlyDataWriter, leveraging its built-in target_file_size rolling so + output files are sized consistently with what the table would produce on a + fresh INSERT. + + Notes on sequence numbers: append-only writers do not advance the per-row + sequence generator (only PK writers do), so input/output min_seq == max_seq. + To keep the new files >= every consumed file, we seed the generator with + max(input.max_sequence_number). + """ + + def __init__(self, table): + self.table = table + + def rewrite( + self, + partition: Tuple, + bucket: int, + files: List[DataFileMeta], + ) -> List[DataFileMeta]: + if not files: + return [] + + # Materialize the file paths the writer's downstream readers expect. + # The coordinator hands us DataFileMeta straight off the manifest, so + # file_path is None until we resolve it against the table layout. + path_factory = self.table.path_factory() + bucket_path = path_factory.bucket_path(partition, bucket).rstrip("/") + for f in files: + if f.file_path is None: + f.file_path = f"{bucket_path}/{f.file_name}" + + max_seq = max(f.max_sequence_number for f in files) + writer = AppendOnlyDataWriter( + table=self.table, + partition=partition, + bucket=bucket, + max_seq_number=max_seq, + options=self.table.options, + write_cols=None, + ) + try: + for batch in self._read_input_batches(files): + if batch.num_rows > 0: + writer.write(batch) + new_files = writer.prepare_commit() + finally: + writer.close() + + for f in new_files: + # Mark provenance — Java's CompactWriter also tags compact output. + f.file_source = 1 # 1 = COMPACT in pypaimon DataFileMeta convention + + return new_files + + def _read_input_batches(self, files: List[DataFileMeta]) -> Iterator[pa.RecordBatch]: + """Stream record batches from each input file in order. + + We feed the writer batch-by-batch (rather than loading every file into + memory first) so the AppendOnlyDataWriter's target_file_size rolling + kicks in even for large inputs. + """ + for f in files: + file_format = format_identifier(f.file_name) + file_path_for_pyarrow = self.table.file_io.to_filesystem_path(f.file_path) + dataset = ds.dataset( + file_path_for_pyarrow, + format=file_format, + filesystem=self.table.file_io.filesystem, + ) + for batch in dataset.to_batches(): + yield batch diff --git a/paimon-python/pypaimon/compact/rewriter/rewriter.py b/paimon-python/pypaimon/compact/rewriter/rewriter.py new file mode 100644 index 000000000000..05eb22c6b39f --- /dev/null +++ b/paimon-python/pypaimon/compact/rewriter/rewriter.py @@ -0,0 +1,30 @@ +################################################################################ +# 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 + +# CompactRewriter is intentionally a marker base class without an abstract +# rewrite() signature: append-only and merge-tree rewriters take very different +# arguments (flat file list vs. pre-partitioned sections + dropDelete + output +# level) and forcing a unified signature here would obscure their semantics. +# Concrete rewriters are still expected to expose a rewrite(...) entry point so +# callers can grep for one consistent verb. + + +class CompactRewriter(ABC): + """Marker base for compact rewriters (append-only, merge-tree, ...).""" diff --git a/paimon-python/pypaimon/compact/task/__init__.py b/paimon-python/pypaimon/compact/task/__init__.py new file mode 100644 index 000000000000..65b48d4d79b4 --- /dev/null +++ b/paimon-python/pypaimon/compact/task/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/paimon-python/pypaimon/compact/task/append_compact_task.py b/paimon-python/pypaimon/compact/task/append_compact_task.py new file mode 100644 index 000000000000..42524b750b23 --- /dev/null +++ b/paimon-python/pypaimon/compact/task/append_compact_task.py @@ -0,0 +1,89 @@ +################################################################################ +# 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, Tuple + +from pypaimon.compact.rewriter.append_compact_rewriter import AppendCompactRewriter +from pypaimon.compact.task.compact_task import CompactTask, register_compact_task +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.write.commit_message import CommitMessage + + +@register_compact_task +class AppendCompactTask(CompactTask): + """Compact a single (partition, bucket) of an append-only table. + + The driver attaches the in-process FileStoreTable so LocalExecutor can run + without rebuilding catalog state. Distributed executors (RayExecutor, added + in Phase 4) must instead populate the loader fields so the worker can + rebuild the table — see to_dict()/from_dict() for the contract. + """ + + TYPE = "append-compact" + + def __init__( + self, + partition: Tuple, + bucket: int, + files: List[DataFileMeta], + table=None, + ): + self.partition = tuple(partition) + self.bucket = bucket + self.files = list(files) + self._table = table + + def with_table(self, table) -> "AppendCompactTask": + self._table = table + return self + + def run(self) -> CommitMessage: + table = self._resolve_table() + rewriter = AppendCompactRewriter(table) + after = rewriter.rewrite(self.partition, self.bucket, self.files) + return CommitMessage( + partition=self.partition, + bucket=self.bucket, + compact_before=list(self.files), + compact_after=list(after), + ) + + def to_dict(self) -> Dict[str, Any]: + # Distributed executors will be wired up in Phase 4 (Ray). At that + # point we'll replace this stub with catalog-options + identifier so + # workers can rebuild the table; until then the LocalExecutor path + # never serializes a task and this method should raise loudly. + raise NotImplementedError( + "AppendCompactTask.to_dict() is reserved for Phase 4 distributed " + "execution; LocalExecutor runs tasks in-process without serialization." + ) + + @classmethod + def from_dict(cls, data: Dict[str, Any]) -> "AppendCompactTask": + raise NotImplementedError( + "AppendCompactTask.from_dict() is reserved for Phase 4 distributed " + "execution; LocalExecutor runs tasks in-process without serialization." + ) + + def _resolve_table(self): + if self._table is None: + raise RuntimeError( + "AppendCompactTask has no table attached. The CompactJob/driver " + "must call with_table(table) before handing tasks to an executor." + ) + return self._table diff --git a/paimon-python/pypaimon/compact/task/compact_task.py b/paimon-python/pypaimon/compact/task/compact_task.py new file mode 100644 index 000000000000..63dd018afa0c --- /dev/null +++ b/paimon-python/pypaimon/compact/task/compact_task.py @@ -0,0 +1,83 @@ +################################################################################ +# 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 abc import ABC, abstractmethod +from typing import Any, Dict + +from pypaimon.write.commit_message import CommitMessage + + +class CompactTask(ABC): + """A self-contained compaction unit dispatched to a worker. + + Implementations must be JSON-serializable so the same payload can be + shipped to a Ray task in Phase 4 without touching the executor side. + The constructor argument list is the contract: anything captured here + is what the worker has to rebuild its execution context. + """ + + TYPE: str = "" + + @abstractmethod + def run(self) -> CommitMessage: + """Execute the compaction unit on the local process and return a CommitMessage. + + The CommitMessage carries compact_before / compact_after files for the + driver to assemble into a single atomic commit. + """ + + @abstractmethod + def to_dict(self) -> Dict[str, Any]: + """Return a JSON-friendly payload identifying everything the worker needs.""" + + def serialize(self) -> bytes: + return json.dumps(self.to_dict(), separators=(",", ":")).encode("utf-8") + + @classmethod + def deserialize(cls, payload: bytes) -> "CompactTask": + data = json.loads(payload.decode("utf-8")) + task_type = data.get("type") + impl = _TASK_REGISTRY.get(task_type) + if impl is None: + raise ValueError(f"Unknown CompactTask type: {task_type}") + return impl.from_dict(data) + + @classmethod + @abstractmethod + def from_dict(cls, data: Dict[str, Any]) -> "CompactTask": + """Rebuild a task from its to_dict() payload.""" + + +_TASK_REGISTRY: Dict[str, type] = {} + + +def register_compact_task(impl: type) -> type: + """Decorator to register a CompactTask subclass under its TYPE string. + + The registry powers CompactTask.deserialize() so the executor can route + payloads back to the correct subclass without a hard import. + """ + if not issubclass(impl, CompactTask): + raise TypeError(f"{impl} is not a CompactTask subclass") + if not impl.TYPE: + raise ValueError(f"{impl} must define a non-empty TYPE") + if impl.TYPE in _TASK_REGISTRY and _TASK_REGISTRY[impl.TYPE] is not impl: + raise ValueError(f"CompactTask TYPE {impl.TYPE!r} already registered") + _TASK_REGISTRY[impl.TYPE] = impl + return impl diff --git a/paimon-python/pypaimon/table/file_store_table.py b/paimon-python/pypaimon/table/file_store_table.py index 4dadb234db2a..da06d35ac507 100644 --- a/paimon-python/pypaimon/table/file_store_table.py +++ b/paimon-python/pypaimon/table/file_store_table.py @@ -366,6 +366,27 @@ def new_stream_read_builder(self) -> 'StreamReadBuilder': def new_batch_write_builder(self) -> BatchWriteBuilder: return BatchWriteBuilder(self) + def new_compact_job( + self, + compact_options=None, + executor=None, + partition_predicate=None, + commit_user: Optional[str] = None, + ): + """Create a CompactJob bound to this table. + + Args mirror CompactJob — passed through so callers can construct + coordinators/executors elsewhere when they need cross-table sharing. + """ + from pypaimon.compact.job.compact_job import CompactJob + return CompactJob( + table=self, + compact_options=compact_options, + executor=executor, + partition_predicate=partition_predicate, + commit_user=commit_user, + ) + def new_stream_write_builder(self) -> StreamWriteBuilder: return StreamWriteBuilder(self) diff --git a/paimon-python/pypaimon/tests/compact/__init__.py b/paimon-python/pypaimon/tests/compact/__init__.py new file mode 100644 index 000000000000..65b48d4d79b4 --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/paimon-python/pypaimon/tests/compact/append_compact_coordinator_test.py b/paimon-python/pypaimon/tests/compact/append_compact_coordinator_test.py new file mode 100644 index 000000000000..2c920008589c --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/append_compact_coordinator_test.py @@ -0,0 +1,161 @@ +################################################################################ +# 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 shutil +import tempfile +import unittest + +import pyarrow as pa + +from pypaimon import CatalogFactory, Schema +from pypaimon.compact.coordinator.append_compact_coordinator import \ + AppendCompactCoordinator +from pypaimon.compact.options import CompactOptions +from pypaimon.common.options.core_options import CoreOptions + + +class AppendCompactCoordinatorTest(unittest.TestCase): + + @classmethod + def setUpClass(cls): + cls.temp_dir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.temp_dir, "warehouse") + cls.catalog = CatalogFactory.create({"warehouse": cls.warehouse}) + cls.catalog.create_database("compact_db", False) + + @classmethod + def tearDownClass(cls): + shutil.rmtree(cls.temp_dir, ignore_errors=True) + + def _create_unaware_table(self, table_name: str, options=None) -> "FileStoreTable": # noqa: F821 + full_name = f"compact_db.{table_name}" + try: + self.catalog.file_io.delete(self.catalog.get_table_path( + self.catalog.identifier_from_string(full_name) if hasattr( + self.catalog, "identifier_from_string") else None), recursive=True) + except Exception: + pass + # Force a small target_file_size so a few rows are already "small enough" + # to be candidates without writing thousands of rows per test. + opts = { + CoreOptions.BUCKET.key(): "-1", + CoreOptions.TARGET_FILE_SIZE.key(): "1mb", + } + if options: + opts.update(options) + pa_schema = pa.schema([("id", pa.int32()), ("name", pa.string())]) + schema = Schema.from_pyarrow_schema(pa_schema, options=opts) + self.catalog.create_table(full_name, schema, True) + return self.catalog.get_table(full_name) + + def _write_n_files(self, table, n: int, rows_per_file: int = 5): + builder = table.new_batch_write_builder() + for i in range(n): + write = builder.new_write() + commit = builder.new_commit() + data = pa.Table.from_pydict( + { + "id": pa.array( + list(range(i * rows_per_file, (i + 1) * rows_per_file)), + type=pa.int32(), + ), + "name": [f"row-{j}" for j in range(rows_per_file)], + } + ) + write.write_arrow(data) + commit.commit(write.prepare_commit()) + write.close() + commit.close() + + def test_no_tasks_when_below_min_file_num(self): + table = self._create_unaware_table("below_min") + self._write_n_files(table, n=3) # default min_file_num=5 + # Re-fetch table so it sees the new snapshots. + table = self.catalog.get_table("compact_db.below_min") + + coordinator = AppendCompactCoordinator(table, CompactOptions(min_file_num=5)) + tasks = coordinator.plan() + + self.assertEqual(0, len(tasks), + "Coordinator should not plan when fewer than min_file_num small files exist") + + def test_one_task_when_threshold_met(self): + table = self._create_unaware_table("at_threshold") + self._write_n_files(table, n=6) + table = self.catalog.get_table("compact_db.at_threshold") + + coordinator = AppendCompactCoordinator(table, CompactOptions(min_file_num=5)) + tasks = coordinator.plan() + + self.assertEqual(1, len(tasks)) + self.assertEqual((), tasks[0].partition) + self.assertEqual(0, tasks[0].bucket) + self.assertGreaterEqual(len(tasks[0].files), 5) + + def test_full_compaction_overrides_threshold(self): + table = self._create_unaware_table("full_compact") + self._write_n_files(table, n=2) # well below min_file_num=5 + table = self.catalog.get_table("compact_db.full_compact") + + coordinator = AppendCompactCoordinator( + table, + CompactOptions(min_file_num=5, full_compaction=True), + ) + tasks = coordinator.plan() + + self.assertEqual(1, len(tasks), + "full_compaction should produce a task even below min_file_num") + self.assertEqual(2, len(tasks[0].files)) + + def test_chunks_when_exceeding_max_file_num(self): + table = self._create_unaware_table("chunked") + self._write_n_files(table, n=12) + table = self.catalog.get_table("compact_db.chunked") + + coordinator = AppendCompactCoordinator( + table, + CompactOptions(min_file_num=5, max_file_num=5), + ) + tasks = coordinator.plan() + + # 12 files, 5 per chunk → chunks of [5, 5, 2]; the last chunk (2) is + # below min_file_num so it should be dropped. + self.assertEqual(2, len(tasks)) + for t in tasks: + self.assertLessEqual(len(t.files), 5) + + def test_pk_table_rejected(self): + full_name = "compact_db.pk_rejected" + pa_schema = pa.schema([("id", pa.int32()), ("name", pa.string())]) + schema = Schema.from_pyarrow_schema( + pa_schema, + primary_keys=["id"], + options={CoreOptions.BUCKET.key(): "1"}, + ) + try: + self.catalog.create_table(full_name, schema, True) + except Exception: + pass + table = self.catalog.get_table(full_name) + with self.assertRaises(ValueError): + AppendCompactCoordinator(table) + + +if __name__ == "__main__": + unittest.main() diff --git a/paimon-python/pypaimon/tests/compact/append_compact_e2e_test.py b/paimon-python/pypaimon/tests/compact/append_compact_e2e_test.py new file mode 100644 index 000000000000..67043db5f465 --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/append_compact_e2e_test.py @@ -0,0 +1,183 @@ +################################################################################ +# 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 shutil +import tempfile +import unittest + +import pyarrow as pa + +from pypaimon import CatalogFactory, Schema +from pypaimon.common.options.core_options import CoreOptions +from pypaimon.compact.options import CompactOptions + + +class AppendCompactE2ETest(unittest.TestCase): + """End-to-end test: write many small files, run a CompactJob, verify + the table reads back identical data with fewer underlying files and a + new snapshot tagged commit_kind=COMPACT. + """ + + @classmethod + def setUpClass(cls): + cls.temp_dir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.temp_dir, "warehouse") + cls.catalog = CatalogFactory.create({"warehouse": cls.warehouse}) + cls.catalog.create_database("e2e_db", False) + + @classmethod + def tearDownClass(cls): + shutil.rmtree(cls.temp_dir, ignore_errors=True) + + def _make_table(self, name: str, partitioned: bool = False): + full = f"e2e_db.{name}" + opts = { + CoreOptions.BUCKET.key(): "-1", # unaware bucket + CoreOptions.TARGET_FILE_SIZE.key(): "10mb", # plenty of headroom for small writes + } + if partitioned: + pa_schema = pa.schema([ + ("id", pa.int32()), + ("name", pa.string()), + ("dt", pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + pa_schema, partition_keys=["dt"], options=opts, + ) + else: + pa_schema = pa.schema([ + ("id", pa.int32()), + ("name", pa.string()), + ]) + schema = Schema.from_pyarrow_schema(pa_schema, options=opts) + self.catalog.create_table(full, schema, True) + return self.catalog.get_table(full) + + def _write_one(self, table, batch: pa.Table): + builder = table.new_batch_write_builder() + write = builder.new_write() + commit = builder.new_commit() + write.write_arrow(batch) + commit.commit(write.prepare_commit()) + write.close() + commit.close() + + def _read_sorted(self, table, sort_col: str = "id") -> pa.Table: + rb = table.new_read_builder() + scan = rb.new_scan() + splits = scan.plan().splits() + return rb.new_read().to_arrow(splits).sort_by(sort_col) + + def _count_live_files(self, table) -> int: + from pypaimon.read.scanner.file_scanner import FileScanner + from pypaimon.manifest.manifest_list_manager import ManifestListManager + snapshot = table.snapshot_manager().get_latest_snapshot() + if snapshot is None: + return 0 + mlm = ManifestListManager(table) + + def manifest_scanner(): + return mlm.read_all(snapshot), snapshot + return len(FileScanner(table, manifest_scanner).plan_files()) + + def test_unpartitioned_compact_reduces_file_count_and_preserves_data(self): + table = self._make_table("flat") + + rows_per_write = 4 + n_writes = 6 + for i in range(n_writes): + self._write_one(table, pa.Table.from_pydict({ + "id": pa.array( + list(range(i * rows_per_write, (i + 1) * rows_per_write)), + type=pa.int32(), + ), + "name": [f"r-{j}" for j in range(rows_per_write)], + })) + + table = self.catalog.get_table("e2e_db.flat") + before_files = self._count_live_files(table) + self.assertGreaterEqual(before_files, n_writes, + "Each write should leave at least one file") + before_data = self._read_sorted(table) + + job = table.new_compact_job(compact_options=CompactOptions(min_file_num=5)) + messages = job.execute() + + self.assertEqual(1, len(messages), + "Single (partition, bucket) → single CommitMessage") + msg = messages[0] + self.assertEqual(n_writes, len(msg.compact_before), + "All n_writes small files should have been picked up") + self.assertGreaterEqual(len(msg.compact_after), 1) + + table = self.catalog.get_table("e2e_db.flat") + after_files = self._count_live_files(table) + self.assertLess(after_files, before_files, + f"Compact must reduce live file count ({before_files} → {after_files})") + + after_data = self._read_sorted(table) + self.assertEqual(before_data, after_data, + "Compact must preserve data identity") + + latest = table.snapshot_manager().get_latest_snapshot() + self.assertEqual("COMPACT", latest.commit_kind) + + def test_partitioned_compact_emits_per_partition_messages(self): + table = self._make_table("partitioned", partitioned=True) + for partition in ["p1", "p2"]: + for i in range(5): + self._write_one(table, pa.Table.from_pydict({ + "id": pa.array([i * 10 + k for k in range(3)], type=pa.int32()), + "name": [f"x-{k}" for k in range(3)], + "dt": [partition] * 3, + })) + + table = self.catalog.get_table("e2e_db.partitioned") + messages = table.new_compact_job( + compact_options=CompactOptions(min_file_num=5), + ).execute() + + partitions = sorted(m.partition for m in messages) + self.assertEqual([("p1",), ("p2",)], partitions) + for m in messages: + self.assertEqual(5, len(m.compact_before)) + self.assertGreaterEqual(len(m.compact_after), 1) + + def test_no_op_when_nothing_to_compact(self): + table = self._make_table("noop") + # Only 2 writes — below default min_file_num. + for i in range(2): + self._write_one(table, pa.Table.from_pydict({ + "id": pa.array([i], type=pa.int32()), + "name": [f"x-{i}"], + })) + table = self.catalog.get_table("e2e_db.noop") + snapshot_before = table.snapshot_manager().get_latest_snapshot().id + + messages = table.new_compact_job().execute() + + self.assertEqual([], messages) + table = self.catalog.get_table("e2e_db.noop") + self.assertEqual(snapshot_before, + table.snapshot_manager().get_latest_snapshot().id, + "No-op compact must not produce a new snapshot") + + +if __name__ == "__main__": + unittest.main() From 4427beae37d0689faf396ed053173d1dbede6db2 Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Sun, 3 May 2026 10:16:19 +0800 Subject: [PATCH 04/12] [python][compact] phase-2 fixup: address review findings MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Rewriter: stop mutating manifest-owned DataFileMeta. Resolve the read path locally each iteration, preferring external_path (matches SplitRead.file_reader_supplier) over file_path, and never write back. - Rewriter: seed sequence_generator per bucket_mode — 0 for BUCKET_UNAWARE and max(input.max_seq) for HASH_FIXED — matching FileStoreWrite._create_data_writer instead of always using max. - Rewriter: abort the AppendOnlyDataWriter on failure so partial output files don't leak when an executor raises mid-rewrite. - CompactOptions: validate min_file_num >= 1 and max_file_num >= min_file_num at construction so misconfiguration fails loudly instead of being silently rounded up. - AppendCompactCoordinator: drop the silent max(min, max) rescue and document that the trailing chunk below min_file_num is intentionally dropped (deferred to a future change). - CompactTask: align docstring with reality — JSON serialization is declared on the base class but concrete subclasses may defer it until distributed execution arrives in Phase 4. Tests: rewriter must not mutate input metadata; rewriter must abort output on failure; CompactOptions validation. All 15 compact tests plus 60 commit/manifest/scanner regression tests pass. --- .../coordinator/append_compact_coordinator.py | 7 +- paimon-python/pypaimon/compact/options.py | 10 ++ .../rewriter/append_compact_rewriter.py | 70 ++++++---- .../pypaimon/compact/task/compact_task.py | 10 +- .../compact/append_compact_rewriter_test.py | 120 ++++++++++++++++++ .../tests/compact/compact_options_test.py | 51 ++++++++ 6 files changed, 243 insertions(+), 25 deletions(-) create mode 100644 paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py create mode 100644 paimon-python/pypaimon/tests/compact/compact_options_test.py diff --git a/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py b/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py index 630cb5ac83a5..d3b709f492aa 100644 --- a/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py +++ b/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py @@ -108,8 +108,13 @@ def _pick_files_for_bucket( # if the executor later relies on file ordering for something. candidates.sort(key=lambda f: f.min_sequence_number) + # NOTE: under non-full compaction, the trailing chunk may have fewer + # than min_file_num files and is dropped here — those files stay live + # and will be picked up next time more small files accumulate. This is + # a deliberate trade-off (vs. Java, which always picks them up); a + # future change can revisit it. chunks: List[List[DataFileMeta]] = [] - max_per_task = max(self.options.max_file_num, self.options.min_file_num) + max_per_task = self.options.max_file_num for start in range(0, len(candidates), max_per_task): chunk = candidates[start:start + max_per_task] if len(chunk) >= self.options.min_file_num or self.options.full_compaction: diff --git a/paimon-python/pypaimon/compact/options.py b/paimon-python/pypaimon/compact/options.py index dd2a050b7bf0..5f5a8f2cbf6d 100644 --- a/paimon-python/pypaimon/compact/options.py +++ b/paimon-python/pypaimon/compact/options.py @@ -41,6 +41,16 @@ class CompactOptions: max_file_num: int = DEFAULT_MAX_FILE_NUM full_compaction: bool = DEFAULT_FORCE_FULL + def __post_init__(self): + if self.min_file_num < 1: + raise ValueError(f"min_file_num must be >= 1, got {self.min_file_num}") + if self.max_file_num < self.min_file_num: + raise ValueError( + f"max_file_num ({self.max_file_num}) must be >= " + f"min_file_num ({self.min_file_num}); silently raising it would " + f"hide the misconfiguration." + ) + def to_dict(self) -> Dict[str, Any]: return { "min_file_num": self.min_file_num, diff --git a/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py b/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py index f8c8b578e598..38642a536756 100644 --- a/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py +++ b/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py @@ -24,6 +24,7 @@ from pypaimon.compact.rewriter.rewriter import CompactRewriter from pypaimon.manifest.schema.data_file_meta import DataFileMeta from pypaimon.read.split_read import format_identifier +from pypaimon.table.bucket_mode import BucketMode from pypaimon.write.writer.append_only_data_writer import AppendOnlyDataWriter @@ -34,9 +35,15 @@ class AppendCompactRewriter(CompactRewriter): fresh INSERT. Notes on sequence numbers: append-only writers do not advance the per-row - sequence generator (only PK writers do), so input/output min_seq == max_seq. - To keep the new files >= every consumed file, we seed the generator with - max(input.max_sequence_number). + sequence generator (only PK writers do). To stay consistent with the + write path (FileStoreWrite._create_data_writer), we seed the generator + with 0 for BUCKET_UNAWARE tables and with max(input.max_sequence_number) + for HASH_FIXED tables. + + NOTE (deferred): schema evolution across input files is not handled — we + feed batches at their on-disk schema directly into the writer. Inputs + spanning a schema change will be addressed alongside the broader schema + evolution support in a later phase. """ def __init__(self, table): @@ -51,16 +58,7 @@ def rewrite( if not files: return [] - # Materialize the file paths the writer's downstream readers expect. - # The coordinator hands us DataFileMeta straight off the manifest, so - # file_path is None until we resolve it against the table layout. - path_factory = self.table.path_factory() - bucket_path = path_factory.bucket_path(partition, bucket).rstrip("/") - for f in files: - if f.file_path is None: - f.file_path = f"{bucket_path}/{f.file_name}" - - max_seq = max(f.max_sequence_number for f in files) + max_seq = self._initial_max_seq(files) writer = AppendOnlyDataWriter( table=self.table, partition=partition, @@ -70,10 +68,16 @@ def rewrite( write_cols=None, ) try: - for batch in self._read_input_batches(files): - if batch.num_rows > 0: - writer.write(batch) - new_files = writer.prepare_commit() + try: + for batch in self._read_input_batches(partition, bucket, files): + if batch.num_rows > 0: + writer.write(batch) + new_files = writer.prepare_commit() + except Exception: + # Roll back any rewriter output written so far so a failed task + # doesn't leave orphan files in the warehouse. + writer.abort() + raise finally: writer.close() @@ -83,16 +87,38 @@ def rewrite( return new_files - def _read_input_batches(self, files: List[DataFileMeta]) -> Iterator[pa.RecordBatch]: + def _initial_max_seq(self, files: List[DataFileMeta]) -> int: + """Pick the writer's seed sequence number consistent with the write path. + + Mirrors FileStoreWrite._create_data_writer: BUCKET_UNAWARE always seeds + with 0; bucketed append seeds with max(input.max_seq) so subsequent + writes / reads observe a monotonically advancing seq. + """ + if self.table.bucket_mode() == BucketMode.BUCKET_UNAWARE: + return 0 + return max(f.max_sequence_number for f in files) + + def _read_input_batches( + self, + partition: Tuple, + bucket: int, + files: List[DataFileMeta], + ) -> Iterator[pa.RecordBatch]: """Stream record batches from each input file in order. - We feed the writer batch-by-batch (rather than loading every file into - memory first) so the AppendOnlyDataWriter's target_file_size rolling - kicks in even for large inputs. + We resolve each file's read path locally (preferring external_path, + matching SplitRead.file_reader_supplier) instead of mutating the + DataFileMeta returned by the manifest — those objects may be cached or + shared with other readers. """ + path_factory = self.table.path_factory() + bucket_path = path_factory.bucket_path(partition, bucket).rstrip("/") for f in files: + read_path = f.external_path if f.external_path else ( + f.file_path if f.file_path else f"{bucket_path}/{f.file_name}" + ) file_format = format_identifier(f.file_name) - file_path_for_pyarrow = self.table.file_io.to_filesystem_path(f.file_path) + file_path_for_pyarrow = self.table.file_io.to_filesystem_path(read_path) dataset = ds.dataset( file_path_for_pyarrow, format=file_format, diff --git a/paimon-python/pypaimon/compact/task/compact_task.py b/paimon-python/pypaimon/compact/task/compact_task.py index 63dd018afa0c..69d155be4c2a 100644 --- a/paimon-python/pypaimon/compact/task/compact_task.py +++ b/paimon-python/pypaimon/compact/task/compact_task.py @@ -26,10 +26,16 @@ class CompactTask(ABC): """A self-contained compaction unit dispatched to a worker. - Implementations must be JSON-serializable so the same payload can be - shipped to a Ray task in Phase 4 without touching the executor side. The constructor argument list is the contract: anything captured here is what the worker has to rebuild its execution context. + + JSON serialization (to_dict / from_dict / serialize / deserialize) is + declared on the base class so distributed executors have a single hook + to call, but concrete subclasses are free to defer the implementation + until distributed execution actually arrives — Phase 4 will fill in the + AppendCompactTask serialization once Ray is the executor. Until then + those methods may raise NotImplementedError; LocalExecutor never + serializes a task and is unaffected. """ TYPE: str = "" diff --git a/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py b/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py new file mode 100644 index 000000000000..0e377f69c534 --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.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 os +import shutil +import tempfile +import unittest +from unittest.mock import patch + +import pyarrow as pa + +from pypaimon import CatalogFactory, Schema +from pypaimon.common.options.core_options import CoreOptions +from pypaimon.compact.coordinator.append_compact_coordinator import \ + AppendCompactCoordinator +from pypaimon.compact.options import CompactOptions +from pypaimon.compact.rewriter.append_compact_rewriter import \ + AppendCompactRewriter + + +class AppendCompactRewriterTest(unittest.TestCase): + + @classmethod + def setUpClass(cls): + cls.temp_dir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.temp_dir, "warehouse") + cls.catalog = CatalogFactory.create({"warehouse": cls.warehouse}) + cls.catalog.create_database("rw_db", False) + + @classmethod + def tearDownClass(cls): + shutil.rmtree(cls.temp_dir, ignore_errors=True) + + def _make_unaware_table(self, name: str): + full = f"rw_db.{name}" + opts = { + CoreOptions.BUCKET.key(): "-1", + CoreOptions.TARGET_FILE_SIZE.key(): "10mb", + } + pa_schema = pa.schema([("id", pa.int32()), ("name", pa.string())]) + schema = Schema.from_pyarrow_schema(pa_schema, options=opts) + self.catalog.create_table(full, schema, True) + return self.catalog.get_table(full) + + def _write_n(self, table, n: int): + builder = table.new_batch_write_builder() + for i in range(n): + w = builder.new_write() + c = builder.new_commit() + data = pa.Table.from_pydict({ + "id": pa.array([i], type=pa.int32()), + "name": [f"row-{i}"], + }) + w.write_arrow(data) + c.commit(w.prepare_commit()) + w.close() + c.close() + + def test_does_not_mutate_input_metadata(self): + table = self._make_unaware_table("no_mutate") + self._write_n(table, n=5) + table = self.catalog.get_table("rw_db.no_mutate") + coord = AppendCompactCoordinator(table, CompactOptions(min_file_num=5)) + tasks = coord.plan() + self.assertEqual(1, len(tasks)) + files = tasks[0].files + + original_paths = [f.file_path for f in files] + self.assertTrue(all(p is None for p in original_paths), + "Coordinator should hand off manifest entries with file_path=None") + + rewriter = AppendCompactRewriter(table) + rewriter.rewrite(tasks[0].partition, tasks[0].bucket, files) + + # Rewriter must not write file_path back onto manifest-owned objects. + self.assertEqual(original_paths, [f.file_path for f in files]) + + def test_aborts_partial_output_on_failure(self): + table = self._make_unaware_table("abort_on_failure") + self._write_n(table, n=5) + table = self.catalog.get_table("rw_db.abort_on_failure") + coord = AppendCompactCoordinator(table, CompactOptions(min_file_num=5)) + tasks = coord.plan() + self.assertEqual(1, len(tasks)) + + rewriter = AppendCompactRewriter(table) + # Force AppendOnlyDataWriter.prepare_commit to blow up after some + # batches have already been buffered/flushed; rewriter must abort + # those outputs rather than leave them on disk. + with patch( + "pypaimon.write.writer.append_only_data_writer.AppendOnlyDataWriter.prepare_commit", + side_effect=RuntimeError("boom"), + ): + with self.assertRaises(RuntimeError): + rewriter.rewrite(tasks[0].partition, tasks[0].bucket, list(tasks[0].files)) + + # Snapshot id should not have advanced (no successful commit happened), + # and no new compaction snapshot should exist. + latest = table.snapshot_manager().get_latest_snapshot() + self.assertNotEqual("COMPACT", latest.commit_kind, + "Failed compaction must not produce a COMPACT snapshot") + + +if __name__ == "__main__": + unittest.main() diff --git a/paimon-python/pypaimon/tests/compact/compact_options_test.py b/paimon-python/pypaimon/tests/compact/compact_options_test.py new file mode 100644 index 000000000000..6217c93871ea --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/compact_options_test.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 unittest + +from pypaimon.compact.options import CompactOptions + + +class CompactOptionsTest(unittest.TestCase): + + def test_defaults(self): + opts = CompactOptions() + self.assertEqual(5, opts.min_file_num) + self.assertEqual(50, opts.max_file_num) + self.assertFalse(opts.full_compaction) + + def test_max_below_min_rejected(self): + # Silently raising max to min would mask user misconfiguration. + with self.assertRaises(ValueError): + CompactOptions(min_file_num=5, max_file_num=3) + + def test_min_zero_rejected(self): + with self.assertRaises(ValueError): + CompactOptions(min_file_num=0) + + def test_to_from_dict_roundtrip(self): + opts = CompactOptions(min_file_num=2, max_file_num=10, full_compaction=True) + rebuilt = CompactOptions.from_dict(opts.to_dict()) + self.assertEqual(opts, rebuilt) + + def test_from_dict_none_returns_defaults(self): + self.assertEqual(CompactOptions(), CompactOptions.from_dict(None)) + + +if __name__ == "__main__": + unittest.main() From 2bb6c0ffe6a24d0ab9f6171963d8f315ba6322da Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Sun, 3 May 2026 10:30:48 +0800 Subject: [PATCH 05/12] [python][compact] phase-3: primary-key LSM compaction with universal strategy MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit End-to-end primary-key compaction in-process. table.new_compact_job(...) on a PK table now plans a MergeTreeCompactTask per (partition, bucket) that is eligible under UniversalCompaction's three-stage decision (size-amp / size-ratio / file-num), rewrites it via SortMergeReader + MergeFunction, and commits the result with snapshot kind=COMPACT. New modules: pypaimon/compact/levels.py Direct port of Java mergetree.Levels — L0 ordered by maxSeq DESC, L1..N hold one SortedRun each, update() routes per-level. pypaimon/compact/strategy/ compact_unit.py + strategy.py + universal_compaction.py — full Universal Compaction algorithm (size-amp, size-ratio, file-num, force-pick-L0). EarlyFullCompaction / OffPeak left for later. pypaimon/compact/rewriter/merge_tree_rolling_writer.py Subclass of DataWriter that consumes pre-merged KV batches; rewrites each appended file's metadata with the strategy's output_level, the actual min/max sequence numbers and retract count. pypaimon/compact/rewriter/merge_tree_compact_rewriter.py Drives IntervalPartition → per-section ConcatRecordReader → SortMergeReader (with the table's MergeFunction) → optional DropDeleteRecordReader → buffered RecordBatch → rolling writer. pypaimon/compact/coordinator/merge_tree_compact_coordinator.py Per-(partition, bucket) Levels build + strategy.pick + drop_delete rule (output_level >= non_empty_highest_level). pypaimon/compact/task/merge_tree_compact_task.py Carries one CompactUnit; assembles CommitMessage(compact_before, compact_after) for the driver to commit atomically. Read path: pypaimon/read/reader/merge_function.py Abstract MergeFunction + Factory; DeduplicateMergeFunction migrated from sort_merge_reader.py. PartialUpdate / Aggregate / FirstRow are stubbed so configured tables fail loudly with a Phase 6 message. SortMergeReaderWithMinHeap accepts an optional merge_function (default DeduplicateMergeFunction → existing read path unchanged). KeyValue.row_tuple exposes the underlying physical tuple so the rewriter can buffer KVs back into a RecordBatch. CompactJob now routes PK tables to MergeTreeCompactCoordinator. Tests: 19 unit (Levels semantics, UniversalCompaction trigger algorithm, MergeFunction registry + stubs) + 2 PK e2e (full-compaction dedup keeps latest values & promotes level; below-trigger no-op). 99-test combined regression on commit/manifest/scanner/reader paths. --- .../merge_tree_compact_coordinator.py | 153 ++++++++++++ .../pypaimon/compact/job/compact_job.py | 9 +- paimon-python/pypaimon/compact/levels.py | 216 +++++++++++++++++ .../rewriter/merge_tree_compact_rewriter.py | 228 ++++++++++++++++++ .../rewriter/merge_tree_rolling_writer.py | 129 ++++++++++ .../pypaimon/compact/strategy/__init__.py | 17 ++ .../pypaimon/compact/strategy/compact_unit.py | 55 +++++ .../pypaimon/compact/strategy/strategy.py | 62 +++++ .../compact/strategy/universal_compaction.py | 180 ++++++++++++++ .../compact/task/merge_tree_compact_task.py | 105 ++++++++ .../pypaimon/read/reader/merge_function.py | 155 ++++++++++++ .../pypaimon/read/reader/sort_merge_reader.py | 36 ++- paimon-python/pypaimon/table/row/key_value.py | 9 + .../pypaimon/tests/compact/levels_test.py | 127 ++++++++++ .../tests/compact/merge_function_test.py | 85 +++++++ .../tests/compact/pk_compact_e2e_test.py | 160 ++++++++++++ .../compact/universal_compaction_test.py | 124 ++++++++++ 17 files changed, 1827 insertions(+), 23 deletions(-) create mode 100644 paimon-python/pypaimon/compact/coordinator/merge_tree_compact_coordinator.py create mode 100644 paimon-python/pypaimon/compact/levels.py create mode 100644 paimon-python/pypaimon/compact/rewriter/merge_tree_compact_rewriter.py create mode 100644 paimon-python/pypaimon/compact/rewriter/merge_tree_rolling_writer.py create mode 100644 paimon-python/pypaimon/compact/strategy/__init__.py create mode 100644 paimon-python/pypaimon/compact/strategy/compact_unit.py create mode 100644 paimon-python/pypaimon/compact/strategy/strategy.py create mode 100644 paimon-python/pypaimon/compact/strategy/universal_compaction.py create mode 100644 paimon-python/pypaimon/compact/task/merge_tree_compact_task.py create mode 100644 paimon-python/pypaimon/read/reader/merge_function.py create mode 100644 paimon-python/pypaimon/tests/compact/levels_test.py create mode 100644 paimon-python/pypaimon/tests/compact/merge_function_test.py create mode 100644 paimon-python/pypaimon/tests/compact/pk_compact_e2e_test.py create mode 100644 paimon-python/pypaimon/tests/compact/universal_compaction_test.py diff --git a/paimon-python/pypaimon/compact/coordinator/merge_tree_compact_coordinator.py b/paimon-python/pypaimon/compact/coordinator/merge_tree_compact_coordinator.py new file mode 100644 index 000000000000..b1397f640a71 --- /dev/null +++ b/paimon-python/pypaimon/compact/coordinator/merge_tree_compact_coordinator.py @@ -0,0 +1,153 @@ +################################################################################ +# 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. +################################################################################ + +"""Driver-side planner for primary-key (merge-tree) compaction.""" + +from collections import defaultdict +from typing import Dict, List, Optional, Tuple + +from pypaimon.common.predicate import Predicate +from pypaimon.compact.coordinator.coordinator import CompactCoordinator +from pypaimon.compact.levels import Levels +from pypaimon.compact.options import CompactOptions +from pypaimon.compact.strategy.strategy import (CompactStrategy, + pick_full_compaction) +from pypaimon.compact.strategy.universal_compaction import UniversalCompaction +from pypaimon.compact.task.merge_tree_compact_task import MergeTreeCompactTask +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.read.reader.sort_merge_reader import builtin_key_comparator +from pypaimon.read.scanner.file_scanner import FileScanner + +DEFAULT_NUM_LEVELS = 5 + + +class MergeTreeCompactCoordinator(CompactCoordinator): + """Plan one MergeTreeCompactTask per (partition, bucket) that the strategy says to compact. + + The coordinator owns the Levels object for a bucket and asks the + strategy.pick(...) which runs to combine. full_compaction=True bypasses + the strategy entirely and picks every file in every bucket. + """ + + def __init__( + self, + table, + compact_options: Optional[CompactOptions] = None, + partition_predicate: Optional[Predicate] = None, + strategy: Optional[CompactStrategy] = None, + ): + if not table.is_primary_key_table: + raise ValueError( + "MergeTreeCompactCoordinator only handles primary-key tables; " + "use AppendCompactCoordinator for append-only tables." + ) + self.table = table + self.options = compact_options or CompactOptions() + self.partition_predicate = partition_predicate + self.num_levels = self._resolve_num_levels() + self.strategy = strategy or self._default_strategy() + self.key_comparator = builtin_key_comparator(self.table.trimmed_primary_keys_fields) + + def plan(self) -> List[MergeTreeCompactTask]: + manifest_entries = self._scan_live_files() + if not manifest_entries: + return [] + + bucket_files: Dict[Tuple[Tuple, int], List[DataFileMeta]] = defaultdict(list) + for entry in manifest_entries: + key = (tuple(entry.partition.values), entry.bucket) + bucket_files[key].append(entry.file) + + tasks: List[MergeTreeCompactTask] = [] + for (partition, bucket), files in bucket_files.items(): + levels = Levels(self.key_comparator, files, self.num_levels) + unit = self._pick_unit(levels) + if unit is None: + continue + drop_delete = self._should_drop_delete(unit, levels) + tasks.append( + MergeTreeCompactTask( + partition=partition, + bucket=bucket, + files=unit.files, + output_level=unit.output_level, + drop_delete=drop_delete, + table=self.table, + ) + ) + return tasks + + # ---- internals --------------------------------------------------------- + + def _pick_unit(self, levels: Levels): + runs = levels.level_sorted_runs() + if self.options.full_compaction: + return pick_full_compaction(levels.number_of_levels(), runs) + return self.strategy.pick(levels.number_of_levels(), runs) + + def _should_drop_delete(self, unit, levels: Levels) -> bool: + # Mirrors MergeTreeCompactManager.triggerCompaction's dropDelete rule: + # we may drop retract rows only when nothing older could need them, i.e. + # we are writing to a level >= the highest non-empty level (and never + # to L0, which by definition can have older data above and below). + if unit.output_level == 0: + return False + return unit.output_level >= levels.non_empty_highest_level() + + def _resolve_num_levels(self) -> int: + # Java reads num-levels off CoreOptions; pypaimon's CoreOptions doesn't + # surface it as a typed accessor yet, so we read the raw map and fall + # back to Java's CoreOptions.NUM_LEVELS default (5). Any input file + # already at a higher level wins during Levels construction anyway. + raw = self._raw_options_map().get("num-levels") + return int(raw) if raw is not None else DEFAULT_NUM_LEVELS + + def _default_strategy(self) -> CompactStrategy: + raw = self._raw_options_map() + max_size_amp = int(raw.get("compaction.max-size-amplification-percent") or 200) + size_ratio = int(raw.get("compaction.size-ratio") or 1) + trigger = int(raw.get("num-sorted-run.compaction-trigger") or 5) + return UniversalCompaction( + max_size_amp=max_size_amp, + size_ratio=size_ratio, + num_run_compaction_trigger=trigger, + ) + + def _raw_options_map(self) -> dict: + opts = self.table.options.options + if hasattr(opts, "to_map"): + return opts.to_map() + return dict(opts) if opts else {} + + def _scan_live_files(self): + snapshot = self.table.snapshot_manager().get_latest_snapshot() + if snapshot is None: + return [] + + from pypaimon.manifest.manifest_list_manager import ManifestListManager + manifest_list_manager = ManifestListManager(self.table) + + def manifest_scanner(): + return manifest_list_manager.read_all(snapshot), snapshot + + scanner = FileScanner( + self.table, + manifest_scanner, + partition_predicate=self.partition_predicate, + ) + return scanner.plan_files() diff --git a/paimon-python/pypaimon/compact/job/compact_job.py b/paimon-python/pypaimon/compact/job/compact_job.py index 95083928bd44..3a2eef44165e 100644 --- a/paimon-python/pypaimon/compact/job/compact_job.py +++ b/paimon-python/pypaimon/compact/job/compact_job.py @@ -24,6 +24,8 @@ from pypaimon.compact.coordinator.append_compact_coordinator import \ AppendCompactCoordinator from pypaimon.compact.coordinator.coordinator import CompactCoordinator +from pypaimon.compact.coordinator.merge_tree_compact_coordinator import \ + MergeTreeCompactCoordinator from pypaimon.compact.executor.executor import CompactExecutor from pypaimon.compact.executor.local_executor import LocalExecutor from pypaimon.compact.options import CompactOptions @@ -85,9 +87,10 @@ def execute(self) -> List[CommitMessage]: def _build_coordinator(self) -> CompactCoordinator: if self.table.is_primary_key_table: - # Phase 3 will plug in the merge-tree coordinator here. - raise NotImplementedError( - "Primary-key compaction is not implemented yet (Phase 3)." + return MergeTreeCompactCoordinator( + table=self.table, + compact_options=self.compact_options, + partition_predicate=self.partition_predicate, ) return AppendCompactCoordinator( table=self.table, diff --git a/paimon-python/pypaimon/compact/levels.py b/paimon-python/pypaimon/compact/levels.py new file mode 100644 index 000000000000..2ce21cb934d0 --- /dev/null +++ b/paimon-python/pypaimon/compact/levels.py @@ -0,0 +1,216 @@ +################################################################################ +# 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. +################################################################################ + +"""Multi-level file management for primary-key tables. + +Direct port of paimon-core/.../mergetree/Levels.java semantics: +- Level 0: every file is its own SortedRun, sorted by maxSequenceNumber DESC + (newest first) so the universal compaction strategy can read it in age + order. +- Levels 1..N: each level holds a single SortedRun whose files have + non-overlapping [min_key, max_key] intervals (compaction maintains this + invariant on output). +""" + +from collections import defaultdict +from dataclasses import dataclass +from functools import cmp_to_key +from typing import Callable, List + +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.read.interval_partition import SortedRun +from pypaimon.table.row.generic_row import GenericRow + + +# Re-export SortedRun under the compact namespace so callers can import a +# single, stable name. +__all__ = ["LevelSortedRun", "Levels", "SortedRun"] + + +@dataclass +class LevelSortedRun: + """Pairs a SortedRun with the level it came from.""" + + level: int + run: SortedRun + + def total_size(self) -> int: + return sum(f.file_size for f in self.run.files) + + +KeyComparator = Callable[[GenericRow, GenericRow], int] + + +class Levels: + """Maintains the L0 + per-level structure of a single (partition, bucket).""" + + def __init__( + self, + key_comparator: KeyComparator, + input_files: List[DataFileMeta], + num_levels: int, + ): + self.key_comparator = key_comparator + + max_seen = max((f.level for f in input_files), default=-1) + restored_num_levels = max(num_levels, max_seen + 1) + if restored_num_levels < 2: + raise ValueError( + f"Number of levels must be at least 2, got {restored_num_levels}" + ) + + # Level 0: list ordered by (max_seq DESC, min_seq ASC, creation_time ASC, + # file_name ASC). We use a sorted python list rather than SortedList + # because additions are rare and full re-sorting is cheap relative to + # data sizes here. + self._level0: List[DataFileMeta] = [] + # Levels 1..N: index 0 is L1, index 1 is L2, ... + self._levels: List[SortedRun] = [SortedRun(files=[]) for _ in range(restored_num_levels - 1)] + + grouped: dict = defaultdict(list) + for f in input_files: + grouped[f.level].append(f) + for level, files in grouped.items(): + self._update_level(level, [], files) + + # Sanity check parallels Java's same Preconditions.checkState. + stored = len(self._level0) + sum(len(r.files) for r in self._levels) + if stored != len(input_files): + raise RuntimeError( + f"Levels stored {stored} files but inputs had {len(input_files)} — " + f"this is a bug in level grouping." + ) + + @property + def level0(self) -> List[DataFileMeta]: + return list(self._level0) + + def run_of_level(self, level: int) -> SortedRun: + if level <= 0: + raise ValueError("Level0 does not have one single sorted run.") + return self._levels[level - 1] + + def number_of_levels(self) -> int: + return len(self._levels) + 1 + + def max_level(self) -> int: + return len(self._levels) + + def number_of_sorted_runs(self) -> int: + n = len(self._level0) + for r in self._levels: + if r.files: + n += 1 + return n + + def non_empty_highest_level(self) -> int: + """Highest level index with at least one file, or -1 if everything is empty.""" + for i in range(len(self._levels) - 1, -1, -1): + if self._levels[i].files: + return i + 1 + return 0 if self._level0 else -1 + + def total_file_size(self) -> int: + return sum(f.file_size for f in self._level0) + sum( + sum(f.file_size for f in r.files) for r in self._levels + ) + + def all_files(self) -> List[DataFileMeta]: + out: List[DataFileMeta] = [] + for run in self.level_sorted_runs(): + out.extend(run.run.files) + return out + + def level_sorted_runs(self) -> List[LevelSortedRun]: + """L0 contributes one LevelSortedRun per file; other levels contribute + their single non-empty SortedRun.""" + runs: List[LevelSortedRun] = [] + for f in self._level0: + runs.append(LevelSortedRun(0, SortedRun(files=[f]))) + for i, run in enumerate(self._levels): + if run.files: + runs.append(LevelSortedRun(i + 1, run)) + return runs + + def update(self, before: List[DataFileMeta], after: List[DataFileMeta]) -> None: + """Apply a CompactResult: remove `before` files and add `after` files, + preserving each file's level. Mirrors Java's Levels.update(). + """ + before_by_level: dict = defaultdict(list) + after_by_level: dict = defaultdict(list) + for f in before: + before_by_level[f.level].append(f) + for f in after: + after_by_level[f.level].append(f) + for level in range(self.number_of_levels()): + self._update_level( + level, + before_by_level.get(level, []), + after_by_level.get(level, []), + ) + + def _update_level( + self, + level: int, + before: List[DataFileMeta], + after: List[DataFileMeta], + ) -> None: + if not before and not after: + return + if level == 0: + before_names = {f.file_name for f in before} + self._level0 = [f for f in self._level0 if f.file_name not in before_names] + self._level0.extend(after) + self._level0.sort(key=cmp_to_key(_level0_compare)) + else: + current = list(self._levels[level - 1].files) + before_names = {f.file_name for f in before} + current = [f for f in current if f.file_name not in before_names] + current.extend(after) + current.sort(key=cmp_to_key(_min_key_compare(self.key_comparator))) + self._levels[level - 1] = SortedRun(files=current) + + +def _level0_compare(a: DataFileMeta, b: DataFileMeta) -> int: + """Order L0: file with the largest maxSequenceNumber comes first. + + Ties (concurrent writers) are broken by minSequenceNumber, then creation + time, then file name — same priority chain as Levels.java's TreeSet + comparator, so a Python recovery from a manifest with conflicting + timestamps lays out files identically to the Java side. + """ + if a.max_sequence_number != b.max_sequence_number: + return -1 if a.max_sequence_number > b.max_sequence_number else 1 + if a.min_sequence_number != b.min_sequence_number: + return -1 if a.min_sequence_number < b.min_sequence_number else 1 + if a.creation_time != b.creation_time: + # Treat None as smallest so it sorts first deterministically. + if a.creation_time is None: + return -1 + if b.creation_time is None: + return 1 + return -1 if a.creation_time < b.creation_time else 1 + if a.file_name == b.file_name: + return 0 + return -1 if a.file_name < b.file_name else 1 + + +def _min_key_compare(key_comparator: KeyComparator) -> Callable[[DataFileMeta, DataFileMeta], int]: + def cmp(a: DataFileMeta, b: DataFileMeta) -> int: + return key_comparator(a.min_key, b.min_key) + return cmp diff --git a/paimon-python/pypaimon/compact/rewriter/merge_tree_compact_rewriter.py b/paimon-python/pypaimon/compact/rewriter/merge_tree_compact_rewriter.py new file mode 100644 index 000000000000..950efca3c042 --- /dev/null +++ b/paimon-python/pypaimon/compact/rewriter/merge_tree_compact_rewriter.py @@ -0,0 +1,228 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +"""Primary-key (merge-tree) compaction rewriter. + +Reads each section of the input plan via SortMergeReader, applies the +table's MergeFunction (Deduplicate by default), optionally drops retract +rows, and writes the merged stream out via MergeTreeRollingWriter so the +target_file_size rolling stays consistent with the regular write path. + +Sections are produced by IntervalPartition before reaching us — that's the +existing utility used by MergeFileSplitRead, so we get identical "key +intervals don't overlap inside a sorted run" guarantees here. +""" + +from functools import partial +from typing import Callable, List, Optional + +import pyarrow as pa + +from pypaimon.compact.rewriter.merge_tree_rolling_writer import \ + MergeTreeRollingWriter +from pypaimon.compact.rewriter.rewriter import CompactRewriter +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.read.interval_partition import SortedRun +from pypaimon.read.reader.concat_record_reader import ConcatRecordReader +from pypaimon.read.reader.drop_delete_reader import DropDeleteRecordReader +from pypaimon.read.reader.format_avro_reader import FormatAvroReader +from pypaimon.read.reader.format_pyarrow_reader import FormatPyArrowReader +from pypaimon.read.reader.iface.record_reader import RecordReader +from pypaimon.read.reader.key_value_wrap_reader import KeyValueWrapReader +from pypaimon.read.reader.merge_function import MergeFunctionFactory +from pypaimon.read.reader.sort_merge_reader import SortMergeReaderWithMinHeap +from pypaimon.read.split_read import (KEY_FIELD_ID_START, KEY_PREFIX, + format_identifier) +from pypaimon.schema.data_types import DataField +from pypaimon.table.special_fields import SpecialFields + +# Buffer KVs from the merge stream this many at a time before handing the +# resulting RecordBatch to the writer. Sized to amortize per-row Python +# overhead without ballooning peak memory for wide PK rows. +DEFAULT_BUFFER_ROWS = 4096 + + +class MergeTreeCompactRewriter(CompactRewriter): + + def __init__( + self, + table, + mf_factory: MergeFunctionFactory, + buffer_rows: int = DEFAULT_BUFFER_ROWS, + ): + self.table = table + self.mf_factory = mf_factory + self.buffer_rows = buffer_rows + + # Schema of the on-disk KV file: [_KEY_pk, _SEQUENCE_NUMBER, + # _VALUE_KIND, value_cols...]. Computed once per rewriter to avoid + # repeated per-section work. + self._kv_fields: List[DataField] = self._build_kv_fields() + self._kv_field_names: List[str] = [f.name for f in self._kv_fields] + self._key_arity = sum( + 1 for f in self._kv_fields if f.name.startswith(KEY_PREFIX) + ) + self._value_arity = ( + len(self._kv_fields) - self._key_arity - 2 # minus seq + kind + ) + self._arrow_schema = self._build_arrow_schema() + + def rewrite( + self, + partition, + bucket: int, + output_level: int, + sections: List[List[SortedRun]], + drop_delete: bool, + ) -> List[DataFileMeta]: + if not sections: + return [] + + writer = MergeTreeRollingWriter( + table=self.table, + partition=partition, + bucket=bucket, + output_level=output_level, + options=self.table.options, + ) + + try: + try: + for section in sections: + self._consume_section(section, drop_delete, writer) + files = writer.prepare_commit() + except Exception: + writer.abort() + raise + finally: + writer.close() + + return files + + # ---- internals --------------------------------------------------------- + + def _consume_section( + self, + section: List[SortedRun], + drop_delete: bool, + writer: MergeTreeRollingWriter, + ) -> None: + # Each rewrite() call already knows its (partition, bucket); compute + # the bucket directory once here so each file's read_path is a cheap + # string concat instead of repeating path-factory work per file. + partition = writer.partition + bucket_path = self.table.path_factory().bucket_path(partition, writer.bucket).rstrip("/") + + readers: List[RecordReader] = [] + for sorted_run in section: + suppliers: List[Callable[[], RecordReader]] = [] + for f in sorted_run.files: + suppliers.append(partial(self._kv_reader_supplier, f, bucket_path)) + readers.append(ConcatRecordReader(suppliers)) + + merge_reader: RecordReader = SortMergeReaderWithMinHeap( + readers=readers, + schema=self.table.table_schema, + merge_function=self.mf_factory.create(), + ) + if drop_delete: + merge_reader = DropDeleteRecordReader(merge_reader) + + try: + self._stream_to_writer(merge_reader, writer) + finally: + merge_reader.close() + + def _stream_to_writer( + self, + merge_reader: RecordReader, + writer: MergeTreeRollingWriter, + ) -> None: + buffer: List[tuple] = [] + while True: + iterator = merge_reader.read_batch() + if iterator is None: + break + while True: + kv = iterator.next() + if kv is None: + break + # Snapshot the row tuple — KeyValue is reused across calls. + buffer.append(tuple(kv.row_tuple)) + if len(buffer) >= self.buffer_rows: + writer.write(self._tuples_to_batch(buffer)) + buffer.clear() + if buffer: + writer.write(self._tuples_to_batch(buffer)) + buffer.clear() + + def _tuples_to_batch(self, tuples: List[tuple]) -> pa.RecordBatch: + # Transpose to columnar form, then build the RecordBatch with the + # KV-file schema we precomputed at __init__. + columns = list(zip(*tuples)) if tuples else [() for _ in self._kv_field_names] + arrays = [ + pa.array(list(col), type=self._arrow_schema.field(i).type) + for i, col in enumerate(columns) + ] + return pa.RecordBatch.from_arrays(arrays, schema=self._arrow_schema) + + def _kv_reader_supplier(self, file: DataFileMeta, bucket_path: str) -> RecordReader: + # Resolve read path locally (preferring external_path, matching + # SplitRead.file_reader_supplier) without mutating the manifest meta. + read_path = ( + file.external_path + if file.external_path + else (file.file_path if file.file_path else f"{bucket_path}/{file.file_name}") + ) + file_format = format_identifier(file.file_name) + file_io = self.table.file_io + if file_format == "avro": + file_batch_reader = FormatAvroReader( + file_io, read_path, self._kv_fields, push_down_predicate=None, + ) + else: + file_batch_reader = FormatPyArrowReader( + file_io, + file_format, + read_path, + read_fields=self._kv_fields, + push_down_predicate=None, + options=self.table.options, + ) + return KeyValueWrapReader(file_batch_reader, self._key_arity, self._value_arity) + + def _build_kv_fields(self) -> List[DataField]: + all_fields = self.table.fields + trimmed_pk = self.table.trimmed_primary_keys + out: List[DataField] = [] + for field in all_fields: + if field.name in trimmed_pk: + out.append(DataField( + field.id + KEY_FIELD_ID_START, + f"{KEY_PREFIX}{field.name}", + field.type, + )) + out.append(SpecialFields.SEQUENCE_NUMBER) + out.append(SpecialFields.VALUE_KIND) + for field in all_fields: + out.append(field) + return out + + def _build_arrow_schema(self) -> pa.Schema: + from pypaimon.schema.data_types import PyarrowFieldParser + return PyarrowFieldParser.from_paimon_schema(self._kv_fields) diff --git a/paimon-python/pypaimon/compact/rewriter/merge_tree_rolling_writer.py b/paimon-python/pypaimon/compact/rewriter/merge_tree_rolling_writer.py new file mode 100644 index 000000000000..f13306c4467e --- /dev/null +++ b/paimon-python/pypaimon/compact/rewriter/merge_tree_rolling_writer.py @@ -0,0 +1,129 @@ +################################################################################ +# 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 pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.write.writer.data_writer import DataWriter + +# DataFileMeta.file_source convention used across pypaimon's compaction path. +FILE_SOURCE_COMPACT = 1 + + +class MergeTreeRollingWriter(DataWriter): + """Writer for compaction output of primary-key (merge-tree) tables. + + Differences from KeyValueDataWriter: + - Input batches are assumed to already carry the KV system fields + (_KEY_*, _SEQUENCE_NUMBER, _VALUE_KIND) and to be sorted by + (key ASC, sequence ASC). The writer never adds system fields itself + and never advances the sequence generator. + - After the parent class writes a file at level 0, we rewrite the just- + appended DataFileMeta with the strategy-chosen output_level, the actual + min/max sequence numbers observed in the data, the count of retract + rows, and file_source=COMPACT. + """ + + def __init__(self, table, partition, bucket, output_level: int, options=None): + super().__init__( + table=table, + partition=partition, + bucket=bucket, + max_seq_number=0, # generator is not used for KV compaction output + options=options if options is not None else table.options, + write_cols=None, + ) + self.output_level = output_level + + def _process_data(self, data: pa.RecordBatch) -> pa.Table: + # Already enriched + sorted by upstream SortMergeReader/MergeFunction. + return pa.Table.from_batches([data]) + + def _merge_data(self, existing_data: pa.Table, new_data: pa.Table) -> pa.Table: + # Both halves are already in (key, seq) order and the second half's + # smallest key is guaranteed to be >= the first half's largest key + # (caller is feeding a monotonic merge stream), so concat is enough. + return pa.concat_tables([existing_data, new_data]) + + def _write_data_to_file(self, data: pa.Table): + if data.num_rows == 0: + return + + # Snapshot the file count so we can find the entry the parent appends. + before = len(self.committed_files) + super()._write_data_to_file(data) + if len(self.committed_files) <= before: + return # parent skipped (e.g. empty after processing) + + produced = self.committed_files[before:] + # The parent always writes a single file per call, but be defensive. + for idx, original in enumerate(produced): + min_seq, max_seq = self._extract_seq_bounds(data) + delete_count = self._count_retract_rows(data) + self.committed_files[before + idx] = self._patch( + original, + self.output_level, + min_seq, + max_seq, + delete_count, + ) + + @staticmethod + def _extract_seq_bounds(data: pa.Table): + seq = data.column("_SEQUENCE_NUMBER") + return pc.min(seq).as_py(), pc.max(seq).as_py() + + @staticmethod + def _count_retract_rows(data: pa.Table) -> int: + # _VALUE_KIND == 0 → INSERT; anything else (UPDATE_AFTER/DELETE/...) is + # treated as a retract for the purposes of the file-level counter. + kind = data.column("_VALUE_KIND") + return int(pc.sum(pc.cast(pc.not_equal(kind, 0), pa.int64())).as_py() or 0) + + @staticmethod + def _patch( + original: DataFileMeta, + level: int, + min_seq: int, + max_seq: int, + delete_count: int, + ) -> DataFileMeta: + return DataFileMeta( + file_name=original.file_name, + file_size=original.file_size, + row_count=original.row_count, + min_key=original.min_key, + max_key=original.max_key, + key_stats=original.key_stats, + value_stats=original.value_stats, + min_sequence_number=min_seq, + max_sequence_number=max_seq, + schema_id=original.schema_id, + level=level, + extra_files=original.extra_files, + creation_time=original.creation_time, + delete_row_count=delete_count, + embedded_index=original.embedded_index, + file_source=FILE_SOURCE_COMPACT, + value_stats_cols=original.value_stats_cols, + external_path=original.external_path, + first_row_id=original.first_row_id, + write_cols=original.write_cols, + file_path=original.file_path, + ) diff --git a/paimon-python/pypaimon/compact/strategy/__init__.py b/paimon-python/pypaimon/compact/strategy/__init__.py new file mode 100644 index 000000000000..65b48d4d79b4 --- /dev/null +++ b/paimon-python/pypaimon/compact/strategy/__init__.py @@ -0,0 +1,17 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ diff --git a/paimon-python/pypaimon/compact/strategy/compact_unit.py b/paimon-python/pypaimon/compact/strategy/compact_unit.py new file mode 100644 index 000000000000..461d542f7097 --- /dev/null +++ b/paimon-python/pypaimon/compact/strategy/compact_unit.py @@ -0,0 +1,55 @@ +################################################################################ +# 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, field +from typing import List + +from pypaimon.compact.levels import LevelSortedRun +from pypaimon.manifest.schema.data_file_meta import DataFileMeta + + +@dataclass +class CompactUnit: + """One unit of compaction work picked by a CompactStrategy. + + `output_level` is the LSM level the rewriter should write the merged + output at. `file_rewrite=False` is a hint that the rewriter may simply + upgrade files in place (no key merging needed) — used by the merge-tree + rewriter for large non-overlapping inputs. The append-only path ignores + it. + """ + + output_level: int + files: List[DataFileMeta] = field(default_factory=list) + file_rewrite: bool = False + + @classmethod + def from_level_runs(cls, output_level: int, runs: List[LevelSortedRun]) -> "CompactUnit": + files: List[DataFileMeta] = [] + for run in runs: + files.extend(run.run.files) + return cls(output_level=output_level, files=files, file_rewrite=False) + + @classmethod + def from_files( + cls, + output_level: int, + files: List[DataFileMeta], + file_rewrite: bool = False, + ) -> "CompactUnit": + return cls(output_level=output_level, files=list(files), file_rewrite=file_rewrite) diff --git a/paimon-python/pypaimon/compact/strategy/strategy.py b/paimon-python/pypaimon/compact/strategy/strategy.py new file mode 100644 index 000000000000..59cdec4c9bca --- /dev/null +++ b/paimon-python/pypaimon/compact/strategy/strategy.py @@ -0,0 +1,62 @@ +################################################################################ +# 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 List, Optional + +from pypaimon.compact.levels import LevelSortedRun +from pypaimon.compact.strategy.compact_unit import CompactUnit + + +class CompactStrategy(ABC): + """Picks which sorted runs to compact next. + + Implementations are stateful (they may track `last_full_compaction` time + or similar) but each pick() call inspects only the level snapshot it is + handed; the coordinator owns the Levels object. + """ + + @abstractmethod + def pick( + self, + num_levels: int, + runs: List[LevelSortedRun], + ) -> Optional[CompactUnit]: + """Return the next compaction unit, or None if nothing should run now.""" + + +def pick_full_compaction( + num_levels: int, + runs: List[LevelSortedRun], +) -> Optional[CompactUnit]: + """Force a single unit covering every run, output to the max level. + + Returns None when there are no runs to compact, or when the runs already + consist of a single file already at the max level (idempotent no-op). + Mirrors CompactStrategy.pickFullCompaction(int, List). + """ + if not runs: + return None + max_level = num_levels - 1 + if ( + len(runs) == 1 + and runs[0].level == max_level + and len(runs[0].run.files) == 1 + ): + return None + return CompactUnit.from_level_runs(max_level, runs) diff --git a/paimon-python/pypaimon/compact/strategy/universal_compaction.py b/paimon-python/pypaimon/compact/strategy/universal_compaction.py new file mode 100644 index 000000000000..837ac7bacb6a --- /dev/null +++ b/paimon-python/pypaimon/compact/strategy/universal_compaction.py @@ -0,0 +1,180 @@ +################################################################################ +# 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. +################################################################################ + +"""Universal Compaction strategy. + +Direct port of paimon-core/.../mergetree/compact/UniversalCompaction.java. +Reference: https://github.com/facebook/rocksdb/wiki/Universal-Compaction. + +Three-stage decision (in order): + 1. Size amplification: when (sum of non-max-level runs) * 100 > max_size_amp% + of the max-level run, full-compact everything to the max level. + 2. Size ratio: append a candidate prefix while + candidate_size * (100 + size_ratio) / 100 >= next.run.total_size, + stopping at the first run that breaks the ratio. + 3. File-num: if total runs > num_run_compaction_trigger, force-pick at least + (size - trigger + 1) runs. + +EarlyFullCompaction and OffPeakHours from the Java side are intentionally +omitted in this first cut — they are independent triggers that can be added +later without touching the core algorithm here. +""" + +from typing import List, Optional + +from pypaimon.compact.levels import LevelSortedRun +from pypaimon.compact.strategy.compact_unit import CompactUnit +from pypaimon.compact.strategy.strategy import CompactStrategy + + +class UniversalCompaction(CompactStrategy): + + def __init__( + self, + max_size_amp: int = 200, + size_ratio: int = 1, + num_run_compaction_trigger: int = 5, + ): + if max_size_amp <= 0: + raise ValueError(f"max_size_amp must be > 0, got {max_size_amp}") + if size_ratio < 0: + raise ValueError(f"size_ratio must be >= 0, got {size_ratio}") + if num_run_compaction_trigger < 1: + raise ValueError( + f"num_run_compaction_trigger must be >= 1, got {num_run_compaction_trigger}" + ) + self.max_size_amp = max_size_amp + self.size_ratio = size_ratio + self.num_run_compaction_trigger = num_run_compaction_trigger + + def pick( + self, + num_levels: int, + runs: List[LevelSortedRun], + ) -> Optional[CompactUnit]: + max_level = num_levels - 1 + + # 1. Size amplification. + unit = self._pick_for_size_amp(max_level, runs) + if unit is not None: + return unit + + # 2. Size ratio. + unit = self._pick_for_size_ratio(max_level, runs) + if unit is not None: + return unit + + # 3. File num. + if len(runs) > self.num_run_compaction_trigger: + candidate_count = len(runs) - self.num_run_compaction_trigger + 1 + return self._pick_for_size_ratio_with_count( + max_level, runs, candidate_count, force_pick=False + ) + + return None + + def force_pick_l0( + self, + num_levels: int, + runs: List[LevelSortedRun], + ) -> Optional[CompactUnit]: + """Pick all consecutive L0 runs at the head of `runs` (no-op if none).""" + candidate_count = 0 + for r in runs: + if r.level > 0: + break + candidate_count += 1 + if candidate_count == 0: + return None + return self._pick_for_size_ratio_with_count( + num_levels - 1, runs, candidate_count, force_pick=True + ) + + # ---- internal helpers -------------------------------------------------- + + def _pick_for_size_amp( + self, + max_level: int, + runs: List[LevelSortedRun], + ) -> Optional[CompactUnit]: + if len(runs) < self.num_run_compaction_trigger: + return None + candidate_size = sum(r.total_size() for r in runs[: len(runs) - 1]) + earliest_run_size = runs[-1].total_size() + # Universal compaction's amplification = non-maxLevel total / maxLevel. + if candidate_size * 100 > self.max_size_amp * earliest_run_size: + return CompactUnit.from_level_runs(max_level, runs) + return None + + def _pick_for_size_ratio( + self, + max_level: int, + runs: List[LevelSortedRun], + ) -> Optional[CompactUnit]: + if len(runs) < self.num_run_compaction_trigger: + return None + return self._pick_for_size_ratio_with_count(max_level, runs, 1, force_pick=False) + + def _pick_for_size_ratio_with_count( + self, + max_level: int, + runs: List[LevelSortedRun], + candidate_count: int, + force_pick: bool, + ) -> Optional[CompactUnit]: + candidate_size = sum(r.total_size() for r in runs[:candidate_count]) + i = candidate_count + while i < len(runs): + nxt = runs[i] + if candidate_size * (100.0 + self.size_ratio) / 100.0 < nxt.total_size(): + break + candidate_size += nxt.total_size() + candidate_count += 1 + i += 1 + + if force_pick or candidate_count > 1: + return self._create_unit(runs, max_level, candidate_count) + return None + + def _create_unit( + self, + runs: List[LevelSortedRun], + max_level: int, + run_count: int, + ) -> CompactUnit: + if run_count == len(runs): + output_level = max_level + else: + # Compact into the level just below the next, untouched run. + output_level = max(0, runs[run_count].level - 1) + + if output_level == 0: + # Output to L0 is meaningless — keep extending until we can land on + # a real level (or until we cover everything, which falls back to + # max_level below). + while run_count < len(runs): + nxt = runs[run_count] + run_count += 1 + if nxt.level != 0: + output_level = nxt.level + break + + if run_count == len(runs): + output_level = max_level + + return CompactUnit.from_level_runs(output_level, runs[:run_count]) diff --git a/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py b/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py new file mode 100644 index 000000000000..c90a6190f242 --- /dev/null +++ b/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py @@ -0,0 +1,105 @@ +################################################################################ +# 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, Tuple + +from pypaimon.compact.rewriter.merge_tree_compact_rewriter import \ + MergeTreeCompactRewriter +from pypaimon.compact.task.compact_task import CompactTask, register_compact_task +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.read.interval_partition import IntervalPartition +from pypaimon.read.reader.merge_function import \ + create_merge_function_factory +from pypaimon.write.commit_message import CommitMessage + + +@register_compact_task +class MergeTreeCompactTask(CompactTask): + """Compact a single (partition, bucket) of a primary-key table. + + Carries the picked CompactUnit's files plus the strategy-decided + output_level and drop_delete flag. The driver attaches the in-process + FileStoreTable; Phase 4 will plumb the loader fields for distributed + execution. + """ + + TYPE = "merge-tree-compact" + + def __init__( + self, + partition: Tuple, + bucket: int, + files: List[DataFileMeta], + output_level: int, + drop_delete: bool, + table=None, + ): + self.partition = tuple(partition) + self.bucket = bucket + self.files = list(files) + self.output_level = output_level + self.drop_delete = drop_delete + self._table = table + + def with_table(self, table) -> "MergeTreeCompactTask": + self._table = table + return self + + def run(self) -> CommitMessage: + if self._table is None: + raise RuntimeError( + "MergeTreeCompactTask has no table attached. The CompactJob/driver " + "must call with_table(table) before handing tasks to an executor." + ) + + # IntervalPartition reproduces split_read.MergeFileSplitRead.create_reader's + # section grouping so the rewriter sees the same "non-overlapping + # SortedRuns per section" layout it would on a normal scan. + sections = IntervalPartition(self.files).partition() + + rewriter = MergeTreeCompactRewriter( + table=self._table, + mf_factory=create_merge_function_factory(self._table.options), + ) + after = rewriter.rewrite( + partition=self.partition, + bucket=self.bucket, + output_level=self.output_level, + sections=sections, + drop_delete=self.drop_delete, + ) + + return CommitMessage( + partition=self.partition, + bucket=self.bucket, + compact_before=list(self.files), + compact_after=list(after), + ) + + def to_dict(self) -> Dict[str, Any]: + raise NotImplementedError( + "MergeTreeCompactTask.to_dict() is reserved for Phase 4 distributed " + "execution." + ) + + @classmethod + def from_dict(cls, data: Dict[str, Any]) -> "MergeTreeCompactTask": + raise NotImplementedError( + "MergeTreeCompactTask.from_dict() is reserved for Phase 4 distributed " + "execution." + ) diff --git a/paimon-python/pypaimon/read/reader/merge_function.py b/paimon-python/pypaimon/read/reader/merge_function.py new file mode 100644 index 000000000000..9f6251d20019 --- /dev/null +++ b/paimon-python/pypaimon/read/reader/merge_function.py @@ -0,0 +1,155 @@ +################################################################################ +# 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. +################################################################################ + +"""Merge functions for primary key reduction. + +A MergeFunction defines how multiple KeyValues sharing the same primary key +are reduced into one. SortMergeReader feeds a function with all KVs for a key +in ascending sequence order via reset()/add()*/get_result(), then moves on. + +Phase 3 ships the production DeduplicateMergeFunction (kept identical to the +prior in-line implementation in sort_merge_reader.py) and stubs for the other +three engines so tables tagged with those engines fail loudly instead of +silently producing wrong data. Phase 6 will fill in the stubs. +""" + +from abc import ABC, abstractmethod +from typing import Optional + +from pypaimon.common.options.core_options import CoreOptions, MergeEngine +from pypaimon.table.row.key_value import KeyValue + + +class MergeFunction(ABC): + """Reduces a sequence of KeyValues sharing the same primary key into one.""" + + @abstractmethod + def reset(self) -> None: + """Discard any state from the previous key.""" + + @abstractmethod + def add(self, kv: KeyValue) -> None: + """Accept the next KV for the current key (caller delivers in seq order).""" + + @abstractmethod + def get_result(self) -> Optional[KeyValue]: + """Return the merged value for the current key, or None to drop the row.""" + + +class MergeFunctionFactory(ABC): + """A factory exists per-engine because some engines (PartialUpdate / + Aggregate) build per-call instances bound to projected schemas.""" + + @abstractmethod + def create(self) -> MergeFunction: + """Return a fresh MergeFunction. Caller owns it for one merge pass.""" + + +class DeduplicateMergeFunction(MergeFunction): + """Keep the latest KV (highest sequence number) for each key. + + Because SortMergeReader hands KVs over in ascending sequence order, the + last one added is always the latest — no comparison needed here. + """ + + def __init__(self): + self.latest_kv: Optional[KeyValue] = None + + def reset(self) -> None: + self.latest_kv = None + + def add(self, kv: KeyValue) -> None: + self.latest_kv = kv + + def get_result(self) -> Optional[KeyValue]: + return self.latest_kv + + +class DeduplicateMergeFunctionFactory(MergeFunctionFactory): + def create(self) -> MergeFunction: + return DeduplicateMergeFunction() + + +# --- Stubs reserved for Phase 6 ---------------------------------------------- +# These exist so MergeFunctionFactory.create_for(options) can route every Java +# MergeEngine to a Python class today; tables tagged with these engines simply +# fail loudly instead of silently producing wrong results, and Phase 6 will +# fill in the bodies without changing any callers. + + +class _UnimplementedMergeFunction(MergeFunction): + engine_name = "" + + def reset(self) -> None: + raise NotImplementedError( + f"MergeEngine '{self.engine_name}' compaction is not implemented yet " + f"(planned for Phase 6)." + ) + + def add(self, kv: KeyValue) -> None: + raise NotImplementedError( + f"MergeEngine '{self.engine_name}' compaction is not implemented yet " + f"(planned for Phase 6)." + ) + + def get_result(self) -> Optional[KeyValue]: + raise NotImplementedError( + f"MergeEngine '{self.engine_name}' compaction is not implemented yet " + f"(planned for Phase 6)." + ) + + +class PartialUpdateMergeFunction(_UnimplementedMergeFunction): + engine_name = "partial-update" + + +class AggregateMergeFunction(_UnimplementedMergeFunction): + engine_name = "aggregation" + + +class FirstRowMergeFunction(_UnimplementedMergeFunction): + engine_name = "first-row" + + +class _UnimplementedFactory(MergeFunctionFactory): + def __init__(self, engine_name: str, impl_cls: type): + self.engine_name = engine_name + self.impl_cls = impl_cls + + def create(self) -> MergeFunction: + # Build the instance now so callers see the failure at the first call + # site they own, with the engine name in the traceback. + return self.impl_cls() + + +def create_merge_function_factory(options: CoreOptions) -> MergeFunctionFactory: + """Pick the correct factory for the table's configured merge engine. + + Unknown / unsupported engines raise here (rather than later inside the + rewriter) so the failure points back at the configuration directly. + """ + engine = options.merge_engine() + if engine == MergeEngine.DEDUPLICATE: + return DeduplicateMergeFunctionFactory() + if engine == MergeEngine.PARTIAL_UPDATE: + return _UnimplementedFactory("partial-update", PartialUpdateMergeFunction) + if engine == MergeEngine.AGGREGATE: + return _UnimplementedFactory("aggregation", AggregateMergeFunction) + if engine == MergeEngine.FIRST_ROW: + return _UnimplementedFactory("first-row", FirstRowMergeFunction) + raise ValueError(f"Unsupported MergeEngine: {engine!r}") diff --git a/paimon-python/pypaimon/read/reader/sort_merge_reader.py b/paimon-python/pypaimon/read/reader/sort_merge_reader.py index aedd593b702b..35d04cbd63a0 100644 --- a/paimon-python/pypaimon/read/reader/sort_merge_reader.py +++ b/paimon-python/pypaimon/read/reader/sort_merge_reader.py @@ -21,6 +21,8 @@ from pypaimon.read.reader.iface.record_iterator import RecordIterator from pypaimon.read.reader.iface.record_reader import RecordReader +from pypaimon.read.reader.merge_function import (DeduplicateMergeFunction, + MergeFunction) from pypaimon.schema.data_types import DataField, Keyword from pypaimon.schema.table_schema import TableSchema from pypaimon.table.row.internal_row import InternalRow @@ -28,11 +30,21 @@ class SortMergeReaderWithMinHeap(RecordReader): - """SortMergeReader implemented with min-heap.""" - - def __init__(self, readers: List[RecordReader[KeyValue]], schema: TableSchema): + """SortMergeReader implemented with min-heap. + + `merge_function` defaults to DeduplicateMergeFunction so the existing read + path is unchanged; compaction passes a factory-built instance to honor the + table's configured merge engine. + """ + + def __init__( + self, + readers: List[RecordReader[KeyValue]], + schema: TableSchema, + merge_function: Optional[MergeFunction] = None, + ): self.next_batch_readers = list(readers) - self.merge_function = DeduplicateMergeFunction() + self.merge_function = merge_function if merge_function is not None else DeduplicateMergeFunction() if schema.partition_keys: trimmed_primary_keys = [pk for pk in schema.primary_keys if pk not in schema.partition_keys] @@ -124,22 +136,6 @@ def _next_impl(self): 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 diff --git a/paimon-python/pypaimon/table/row/key_value.py b/paimon-python/pypaimon/table/row/key_value.py index 22647c4b6d6b..41e32386882a 100644 --- a/paimon-python/pypaimon/table/row/key_value.py +++ b/paimon-python/pypaimon/table/row/key_value.py @@ -55,3 +55,12 @@ def sequence_number(self) -> int: @property def value_row_kind_byte(self) -> int: return self._row_tuple[self.key_arity + 1] + + @property + def row_tuple(self) -> tuple: + """The underlying physical row tuple (key_cols, seq, kind, value_cols). + + Compaction writers consume this verbatim when buffering KVs back into + a RecordBatch — the column order matches the on-disk KV file schema. + """ + return self._row_tuple diff --git a/paimon-python/pypaimon/tests/compact/levels_test.py b/paimon-python/pypaimon/tests/compact/levels_test.py new file mode 100644 index 000000000000..a29b6a5c52b5 --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/levels_test.py @@ -0,0 +1,127 @@ +################################################################################ +# 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 unittest +from datetime import datetime +from typing import List + +from pypaimon.compact.levels import Levels +from pypaimon.data.timestamp import Timestamp +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.manifest.schema.simple_stats import SimpleStats +from pypaimon.schema.data_types import AtomicType, DataField +from pypaimon.table.row.generic_row import GenericRow + +PK_FIELDS = [DataField(0, "id", AtomicType("BIGINT"))] + + +def _key(v: int) -> GenericRow: + return GenericRow([v], PK_FIELDS) + + +def _file(name: str, level: int, *, min_k: int, max_k: int, + min_seq: int, max_seq: int, file_size: int = 1024, + ts_ms: int = 1_700_000_000_000) -> DataFileMeta: + return DataFileMeta.create( + file_name=name, + file_size=file_size, + row_count=10, + min_key=_key(min_k), + max_key=_key(max_k), + key_stats=SimpleStats.empty_stats(), + value_stats=SimpleStats.empty_stats(), + min_sequence_number=min_seq, + max_sequence_number=max_seq, + schema_id=0, + level=level, + extra_files=[], + creation_time=Timestamp.from_epoch_millis(ts_ms), + ) + + +def _key_cmp(a: GenericRow, b: GenericRow) -> int: + av = a.values[0] + bv = b.values[0] + return -1 if av < bv else (1 if av > bv else 0) + + +class LevelsTest(unittest.TestCase): + + def test_level0_orders_newest_first(self): + files: List[DataFileMeta] = [ + _file("f1", 0, min_k=1, max_k=2, min_seq=10, max_seq=20), + _file("f2", 0, min_k=3, max_k=4, min_seq=30, max_seq=40), + _file("f3", 0, min_k=5, max_k=6, min_seq=50, max_seq=60), + ] + levels = Levels(_key_cmp, files, num_levels=3) + + ordered = levels.level0 + self.assertEqual(["f3", "f2", "f1"], [f.file_name for f in ordered]) + + def test_number_of_sorted_runs_counts_l0_files_plus_nonempty_levels(self): + files = [ + _file("a", 0, min_k=1, max_k=2, min_seq=10, max_seq=20), + _file("b", 0, min_k=3, max_k=4, min_seq=30, max_seq=40), + _file("c", 1, min_k=5, max_k=8, min_seq=50, max_seq=60), + _file("d", 1, min_k=9, max_k=12, min_seq=70, max_seq=80), + _file("e", 3, min_k=20, max_k=30, min_seq=90, max_seq=100), + ] + levels = Levels(_key_cmp, files, num_levels=5) + + # L0 has 2 files (=2 runs), L1 has 1 SortedRun, L3 has 1 SortedRun → 4 + self.assertEqual(4, levels.number_of_sorted_runs()) + self.assertEqual(3, levels.non_empty_highest_level()) + + def test_levels_grow_to_accommodate_input_above_declared_num_levels(self): + files = [_file("z", 7, min_k=1, max_k=2, min_seq=10, max_seq=20)] + # Declare 3 but the file is at level 7 — Levels must expand. + levels = Levels(_key_cmp, files, num_levels=3) + self.assertEqual(8, levels.number_of_levels()) # levels 0..7 + self.assertEqual(7, levels.non_empty_highest_level()) + + def test_update_replaces_files_at_their_levels(self): + a = _file("a", 0, min_k=1, max_k=2, min_seq=10, max_seq=20) + b = _file("b", 0, min_k=3, max_k=4, min_seq=30, max_seq=40) + c = _file("c", 2, min_k=5, max_k=6, min_seq=50, max_seq=60) + levels = Levels(_key_cmp, [a, b, c], num_levels=4) + + merged = _file("merged", 2, min_k=1, max_k=6, min_seq=10, max_seq=60) + levels.update(before=[a, b, c], after=[merged]) + + self.assertEqual(0, len(levels.level0)) + self.assertEqual(["merged"], [f.file_name for f in levels.run_of_level(2).files]) + self.assertEqual(1, levels.number_of_sorted_runs()) + + def test_update_per_level_routing(self): + a = _file("a", 0, min_k=1, max_k=2, min_seq=10, max_seq=20) + b = _file("b", 1, min_k=5, max_k=6, min_seq=30, max_seq=40) + levels = Levels(_key_cmp, [a, b], num_levels=3) + + # Move a from L0 → new file at L1; replace b at L1 with new file. + new_at_l1 = _file("new", 1, min_k=1, max_k=6, min_seq=10, max_seq=40) + levels.update(before=[a, b], after=[new_at_l1]) + self.assertEqual([], levels.level0) + self.assertEqual(["new"], [f.file_name for f in levels.run_of_level(1).files]) + + def test_invalid_num_levels_rejected(self): + with self.assertRaises(ValueError): + Levels(_key_cmp, [], num_levels=1) + + +if __name__ == "__main__": + unittest.main() diff --git a/paimon-python/pypaimon/tests/compact/merge_function_test.py b/paimon-python/pypaimon/tests/compact/merge_function_test.py new file mode 100644 index 000000000000..8a9ff1905817 --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/merge_function_test.py @@ -0,0 +1,85 @@ +################################################################################ +# 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 unittest +from unittest.mock import MagicMock + +from pypaimon.common.options import Options +from pypaimon.common.options.core_options import CoreOptions, MergeEngine +from pypaimon.read.reader.merge_function import ( + DeduplicateMergeFunction, DeduplicateMergeFunctionFactory, + create_merge_function_factory) +from pypaimon.table.row.key_value import KeyValue + + +def _kv(key: int, seq: int, value: str = "v", value_kind: int = 0) -> KeyValue: + kv = KeyValue(key_arity=1, value_arity=1) + kv.replace((key, seq, value_kind, value)) + return kv + + +class DeduplicateMergeFunctionTest(unittest.TestCase): + + def test_keeps_last_added(self): + mf = DeduplicateMergeFunction() + mf.reset() + mf.add(_kv(1, 10, "old")) + mf.add(_kv(1, 20, "new")) + result = mf.get_result() + self.assertIsNotNone(result) + self.assertEqual(20, result.sequence_number) + + def test_reset_clears_state(self): + mf = DeduplicateMergeFunction() + mf.add(_kv(1, 1)) + mf.reset() + self.assertIsNone(mf.get_result()) + + +class CreateMergeFunctionFactoryTest(unittest.TestCase): + + def _options_for(self, engine: MergeEngine) -> CoreOptions: + opts = Options({CoreOptions.MERGE_ENGINE.key(): engine.value}) + return CoreOptions(opts) + + def test_deduplicate_returns_factory(self): + factory = create_merge_function_factory(self._options_for(MergeEngine.DEDUPLICATE)) + self.assertIsInstance(factory, DeduplicateMergeFunctionFactory) + self.assertIsInstance(factory.create(), DeduplicateMergeFunction) + + def test_partial_update_factory_creates_stub_that_raises(self): + factory = create_merge_function_factory(self._options_for(MergeEngine.PARTIAL_UPDATE)) + mf = factory.create() + with self.assertRaises(NotImplementedError): + mf.add(_kv(1, 1)) + + def test_aggregate_factory_creates_stub_that_raises(self): + factory = create_merge_function_factory(self._options_for(MergeEngine.AGGREGATE)) + mf = factory.create() + with self.assertRaises(NotImplementedError): + mf.add(_kv(1, 1)) + + def test_first_row_factory_creates_stub_that_raises(self): + factory = create_merge_function_factory(self._options_for(MergeEngine.FIRST_ROW)) + mf = factory.create() + with self.assertRaises(NotImplementedError): + mf.add(_kv(1, 1)) + + +if __name__ == "__main__": + unittest.main() diff --git a/paimon-python/pypaimon/tests/compact/pk_compact_e2e_test.py b/paimon-python/pypaimon/tests/compact/pk_compact_e2e_test.py new file mode 100644 index 000000000000..0842c040b4ae --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/pk_compact_e2e_test.py @@ -0,0 +1,160 @@ +################################################################################ +# 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 shutil +import tempfile +import unittest + +import pyarrow as pa + +from pypaimon import CatalogFactory, Schema +from pypaimon.common.options.core_options import CoreOptions +from pypaimon.compact.options import CompactOptions + + +class PrimaryKeyCompactE2ETest(unittest.TestCase): + """End-to-end test for primary-key compaction. + + Writes multiple snapshots that each leave a new L0 file, runs the compact + job, and verifies: (1) the compacted files are tagged COMPACT in the + snapshot, (2) read-after-compact returns the deduplicated latest values, + and (3) the file count drops. + """ + + @classmethod + def setUpClass(cls): + cls.temp_dir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.temp_dir, "warehouse") + cls.catalog = CatalogFactory.create({"warehouse": cls.warehouse}) + cls.catalog.create_database("pk_db", False) + + @classmethod + def tearDownClass(cls): + shutil.rmtree(cls.temp_dir, ignore_errors=True) + + def _make_pk_table(self, name: str): + full = f"pk_db.{name}" + opts = { + CoreOptions.BUCKET.key(): "1", # single bucket → single (partition,bucket) key + CoreOptions.TARGET_FILE_SIZE.key(): "10mb", + CoreOptions.METADATA_STATS_MODE.key() + if hasattr(CoreOptions, "METADATA_STATS_MODE") + else "metadata.stats-mode": "truncate(16)", + } + pa_schema = pa.schema([("id", pa.int64()), ("name", pa.string())]) + schema = Schema.from_pyarrow_schema(pa_schema, primary_keys=["id"], options=opts) + self.catalog.create_table(full, schema, True) + return self.catalog.get_table(full) + + def _write_one(self, table, batch: pa.Table): + builder = table.new_batch_write_builder() + write = builder.new_write() + commit = builder.new_commit() + write.write_arrow(batch) + commit.commit(write.prepare_commit()) + write.close() + commit.close() + + def _read_sorted(self, table) -> pa.Table: + rb = table.new_read_builder() + scan = rb.new_scan() + splits = scan.plan().splits() + return rb.new_read().to_arrow(splits).sort_by("id") + + def _count_live_files(self, table) -> int: + from pypaimon.read.scanner.file_scanner import FileScanner + from pypaimon.manifest.manifest_list_manager import ManifestListManager + snapshot = table.snapshot_manager().get_latest_snapshot() + if snapshot is None: + return 0 + mlm = ManifestListManager(table) + + def manifest_scanner(): + return mlm.read_all(snapshot), snapshot + return len(FileScanner(table, manifest_scanner).plan_files()) + + def test_full_compaction_dedup_keeps_latest(self): + table = self._make_pk_table("dedup_keep_latest") + + # Write 3 generations of (id, name) for the same set of ids — the + # latest seen value should win after compaction. + for gen in range(3): + self._write_one(table, pa.Table.from_pydict({ + "id": pa.array([1, 2, 3, 4, 5], type=pa.int64()), + "name": [f"gen{gen}-{i}" for i in range(1, 6)], + })) + + table = self.catalog.get_table("pk_db.dedup_keep_latest") + before_files = self._count_live_files(table) + before_data = self._read_sorted(table) + # Read path already dedups — sanity check that the table sees latest. + self.assertEqual(["gen2-1", "gen2-2", "gen2-3", "gen2-4", "gen2-5"], + before_data.column("name").to_pylist()) + + messages = table.new_compact_job( + compact_options=CompactOptions(full_compaction=True), + ).execute() + + # Single bucket → single message expected. + self.assertEqual(1, len(messages)) + msg = messages[0] + self.assertGreaterEqual(len(msg.compact_before), 3, + "All 3 small writes should have been picked up") + self.assertGreaterEqual(len(msg.compact_after), 1, + "Compaction must produce at least one output file") + + table = self.catalog.get_table("pk_db.dedup_keep_latest") + after_files = self._count_live_files(table) + self.assertLess(after_files, before_files, + f"File count must decrease ({before_files} → {after_files})") + + after_data = self._read_sorted(table) + self.assertEqual(before_data, after_data, + "Compact must preserve the dedup result") + + latest = table.snapshot_manager().get_latest_snapshot() + self.assertEqual("COMPACT", latest.commit_kind) + + # Output files should land at a level > 0 (the strategy promotes them). + max_level = max(f.level for f in msg.compact_after) + self.assertGreater(max_level, 0, + "Compacted output should land at a level > 0") + + def test_no_op_when_below_compaction_trigger(self): + table = self._make_pk_table("noop_below_trigger") + # Only 2 writes — far below default num-sorted-run.compaction-trigger=5. + for i in range(2): + self._write_one(table, pa.Table.from_pydict({ + "id": pa.array([i], type=pa.int64()), + "name": [f"row-{i}"], + })) + table = self.catalog.get_table("pk_db.noop_below_trigger") + snapshot_before = table.snapshot_manager().get_latest_snapshot().id + + messages = table.new_compact_job().execute() + + self.assertEqual([], messages) + table = self.catalog.get_table("pk_db.noop_below_trigger") + self.assertEqual(snapshot_before, + table.snapshot_manager().get_latest_snapshot().id, + "Strategy decided no-op → no new snapshot") + + +if __name__ == "__main__": + unittest.main() diff --git a/paimon-python/pypaimon/tests/compact/universal_compaction_test.py b/paimon-python/pypaimon/tests/compact/universal_compaction_test.py new file mode 100644 index 000000000000..3b34424eba01 --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/universal_compaction_test.py @@ -0,0 +1,124 @@ +################################################################################ +# 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 unittest +from typing import List + +from pypaimon.compact.levels import LevelSortedRun, SortedRun +from pypaimon.compact.strategy.universal_compaction import UniversalCompaction +from pypaimon.data.timestamp import Timestamp +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.manifest.schema.simple_stats import SimpleStats +from pypaimon.schema.data_types import AtomicType, DataField +from pypaimon.table.row.generic_row import GenericRow + +PK_FIELDS = [DataField(0, "id", AtomicType("BIGINT"))] + + +def _key(v: int) -> GenericRow: + return GenericRow([v], PK_FIELDS) + + +def _file(level: int, size: int, name: str = None) -> DataFileMeta: + name = name or f"f-l{level}-{size}" + return DataFileMeta.create( + file_name=name, + file_size=size, + row_count=10, + min_key=_key(0), + max_key=_key(99), + key_stats=SimpleStats.empty_stats(), + value_stats=SimpleStats.empty_stats(), + min_sequence_number=0, + max_sequence_number=10, + schema_id=0, + level=level, + extra_files=[], + creation_time=Timestamp.from_epoch_millis(0), + ) + + +def _run(level: int, *sizes: int) -> LevelSortedRun: + return LevelSortedRun(level=level, run=SortedRun(files=[_file(level, s) for s in sizes])) + + +class UniversalCompactionTest(unittest.TestCase): + + def test_returns_none_below_trigger(self): + strategy = UniversalCompaction(num_run_compaction_trigger=5) + runs = [_run(0, 100), _run(0, 100)] + self.assertIsNone(strategy.pick(num_levels=3, runs=runs)) + + def test_size_amp_triggers_full_compaction(self): + strategy = UniversalCompaction(max_size_amp=200, num_run_compaction_trigger=5) + # 5 runs total. Top 4 sum to 1000; max-level run is 100. 1000*100 > 200*100 → trigger. + runs = [_run(0, 250)] * 4 + [_run(2, 100)] + unit = strategy.pick(num_levels=3, runs=runs) + self.assertIsNotNone(unit) + # output_level == max_level == num_levels - 1 == 2 + self.assertEqual(2, unit.output_level) + self.assertEqual(5, len(unit.files)) + + def test_size_ratio_picks_growing_prefix(self): + # No size-amp trigger (top-4 = 100, max-level = 1000 → 100*100 < 200*1000). + # size-ratio: candidate=100; next=100 → 100*101/100=101 >= 100, include. + # candidate=200; next=100 → 200*101/100=202 >= 100, include. + # candidate=300; next=100 → include. + # candidate=400; next=1000 → 400*101/100=404 < 1000, stop. Pick 4. + strategy = UniversalCompaction(max_size_amp=200, size_ratio=1, num_run_compaction_trigger=5) + runs = [_run(0, 100)] * 4 + [_run(2, 1000)] + unit = strategy.pick(num_levels=3, runs=runs) + self.assertIsNotNone(unit) + # 4 runs picked (the L0 chunks); since not all runs included, + # output_level = max(0, runs[4].level - 1) = max(0, 2-1) = 1. + self.assertEqual(4, len(unit.files)) + self.assertEqual(1, unit.output_level) + + def test_force_pick_l0_picks_only_consecutive_l0(self): + strategy = UniversalCompaction() + runs = [_run(0, 50), _run(0, 60), _run(2, 1000)] + unit = strategy.force_pick_l0(num_levels=3, runs=runs) + self.assertIsNotNone(unit) + self.assertEqual(2, len(unit.files)) + + def test_force_pick_l0_returns_none_when_no_l0(self): + strategy = UniversalCompaction() + runs = [_run(1, 100), _run(2, 200)] + self.assertIsNone(strategy.force_pick_l0(num_levels=3, runs=runs)) + + def test_picking_all_runs_outputs_to_max_level(self): + # Construct a scenario where the size-ratio loop swallows everything. + strategy = UniversalCompaction(num_run_compaction_trigger=2) + runs = [_run(0, 100)] * 6 + unit = strategy.pick(num_levels=4, runs=runs) + self.assertIsNotNone(unit) + # All swallowed → output_level = max_level = 3. + self.assertEqual(6, len(unit.files)) + self.assertEqual(3, unit.output_level) + + def test_invalid_options_raise(self): + with self.assertRaises(ValueError): + UniversalCompaction(max_size_amp=0) + with self.assertRaises(ValueError): + UniversalCompaction(size_ratio=-1) + with self.assertRaises(ValueError): + UniversalCompaction(num_run_compaction_trigger=0) + + +if __name__ == "__main__": + unittest.main() From ab37aeecf9c2c35aa456a5f0664671df313be65c Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Sun, 3 May 2026 10:36:38 +0800 Subject: [PATCH 06/12] [python][compact] phase-3 fixup: address review findings - Rewriter: count_retract_rows now matches RowKind.is_add_byte (only UPDATE_BEFORE=1 and DELETE=3 are retracts). The previous != 0 check wrongly inflated delete_row_count by counting UPDATE_AFTER rows, which would skew downstream size-amplification estimates and metrics. - Levels.update: reject out-of-range levels with a clear ValueError instead of letting an IndexError leak from _update_level when a buggy strategy hands back an output_level above number_of_levels(). - Extract build_kv_file_fields() to split_read.py and consume it from both SplitRead._create_key_value_fields and the merge-tree rewriter, so the on-disk KV file schema layout (key cols / seq / kind / value) cannot drift between read and compact paths. --- paimon-python/pypaimon/compact/levels.py | 16 +++++++ .../rewriter/merge_tree_compact_rewriter.py | 25 ++++------ .../rewriter/merge_tree_rolling_writer.py | 8 ++-- paimon-python/pypaimon/read/split_read.py | 47 ++++++++++++------- 4 files changed, 59 insertions(+), 37 deletions(-) diff --git a/paimon-python/pypaimon/compact/levels.py b/paimon-python/pypaimon/compact/levels.py index 2ce21cb934d0..da08a21a62b0 100644 --- a/paimon-python/pypaimon/compact/levels.py +++ b/paimon-python/pypaimon/compact/levels.py @@ -157,6 +157,22 @@ def update(self, before: List[DataFileMeta], after: List[DataFileMeta]) -> None: before_by_level[f.level].append(f) for f in after: after_by_level[f.level].append(f) + + # Reject out-of-range levels with a clear error instead of letting a + # downstream IndexError leak. Constructor handles the auto-grow case + # for files restored from manifest; runtime updates must stay within + # the levels we already know about. + max_seen = max( + (lvl for lvl in list(before_by_level.keys()) + list(after_by_level.keys())), + default=-1, + ) + if max_seen >= self.number_of_levels(): + raise ValueError( + f"Cannot update Levels with file at level {max_seen}; " + f"current number_of_levels={self.number_of_levels()}. " + f"Strategies must not select an output_level above the existing top." + ) + for level in range(self.number_of_levels()): self._update_level( level, diff --git a/paimon-python/pypaimon/compact/rewriter/merge_tree_compact_rewriter.py b/paimon-python/pypaimon/compact/rewriter/merge_tree_compact_rewriter.py index 950efca3c042..e243cacbab53 100644 --- a/paimon-python/pypaimon/compact/rewriter/merge_tree_compact_rewriter.py +++ b/paimon-python/pypaimon/compact/rewriter/merge_tree_compact_rewriter.py @@ -46,10 +46,9 @@ from pypaimon.read.reader.key_value_wrap_reader import KeyValueWrapReader from pypaimon.read.reader.merge_function import MergeFunctionFactory from pypaimon.read.reader.sort_merge_reader import SortMergeReaderWithMinHeap -from pypaimon.read.split_read import (KEY_FIELD_ID_START, KEY_PREFIX, +from pypaimon.read.split_read import (KEY_PREFIX, build_kv_file_fields, format_identifier) from pypaimon.schema.data_types import DataField -from pypaimon.table.special_fields import SpecialFields # Buffer KVs from the merge stream this many at a time before handing the # resulting RecordBatch to the writer. Sized to amortize per-row Python @@ -207,21 +206,13 @@ def _kv_reader_supplier(self, file: DataFileMeta, bucket_path: str) -> RecordRea return KeyValueWrapReader(file_batch_reader, self._key_arity, self._value_arity) def _build_kv_fields(self) -> List[DataField]: - all_fields = self.table.fields - trimmed_pk = self.table.trimmed_primary_keys - out: List[DataField] = [] - for field in all_fields: - if field.name in trimmed_pk: - out.append(DataField( - field.id + KEY_FIELD_ID_START, - f"{KEY_PREFIX}{field.name}", - field.type, - )) - out.append(SpecialFields.SEQUENCE_NUMBER) - out.append(SpecialFields.VALUE_KIND) - for field in all_fields: - out.append(field) - return out + # Same helper SplitRead._create_key_value_fields uses, so the on-disk + # KV file schema cannot drift between read and compact paths. + return build_kv_file_fields( + table_fields=self.table.fields, + trimmed_primary_keys=self.table.trimmed_primary_keys, + value_fields=self.table.fields, + ) def _build_arrow_schema(self) -> pa.Schema: from pypaimon.schema.data_types import PyarrowFieldParser diff --git a/paimon-python/pypaimon/compact/rewriter/merge_tree_rolling_writer.py b/paimon-python/pypaimon/compact/rewriter/merge_tree_rolling_writer.py index f13306c4467e..ec874e460927 100644 --- a/paimon-python/pypaimon/compact/rewriter/merge_tree_rolling_writer.py +++ b/paimon-python/pypaimon/compact/rewriter/merge_tree_rolling_writer.py @@ -91,10 +91,12 @@ def _extract_seq_bounds(data: pa.Table): @staticmethod def _count_retract_rows(data: pa.Table) -> int: - # _VALUE_KIND == 0 → INSERT; anything else (UPDATE_AFTER/DELETE/...) is - # treated as a retract for the purposes of the file-level counter. + # Match RowKind.is_add_byte: INSERT(0) and UPDATE_AFTER(2) are "add", + # UPDATE_BEFORE(1) and DELETE(3) are retracts. Counting != 0 here would + # wrongly include UPDATE_AFTER and inflate delete_row_count. kind = data.column("_VALUE_KIND") - return int(pc.sum(pc.cast(pc.not_equal(kind, 0), pa.int64())).as_py() or 0) + is_retract = pc.or_(pc.equal(kind, 1), pc.equal(kind, 3)) + return int(pc.sum(pc.cast(is_retract, pa.int64())).as_py() or 0) @staticmethod def _patch( diff --git a/paimon-python/pypaimon/read/split_read.py b/paimon-python/pypaimon/read/split_read.py index c88f49f3b065..6c604a86a17e 100644 --- a/paimon-python/pypaimon/read/split_read.py +++ b/paimon-python/pypaimon/read/split_read.py @@ -66,6 +66,31 @@ KEY_FIELD_ID_START = 1000000 NULL_FIELD_INDEX = -1 + +def build_kv_file_fields( + table_fields: List[DataField], + trimmed_primary_keys: List[str], + value_fields: List[DataField], +) -> List[DataField]: + """Build the on-disk KV file schema: [_KEY_pk*, _SEQUENCE_NUMBER, _VALUE_KIND, value_cols]. + + Centralizes the layout so that read (split_read) and write/compact paths + cannot drift. Field ids for key columns are derived from each PK field's + id offset by KEY_FIELD_ID_START (matches Java KeyValueFieldsExtractor). + """ + fields: List[DataField] = [] + for f in table_fields: + if f.name in trimmed_primary_keys: + fields.append(DataField( + f.id + KEY_FIELD_ID_START, + f"{KEY_PREFIX}{f.name}", + f.type, + )) + fields.append(SpecialFields.SEQUENCE_NUMBER) + fields.append(SpecialFields.VALUE_KIND) + fields.extend(value_fields) + return fields + _COMPRESS_EXTENSIONS = frozenset(['gz', 'bz2', 'deflate', 'snappy', 'lz4', 'zst']) @@ -281,23 +306,11 @@ def _get_read_data_fields(self): 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) - - all_data_fields.append(SpecialFields.SEQUENCE_NUMBER) - all_data_fields.append(SpecialFields.VALUE_KIND) - - for field in value_field: - all_data_fields.append(field) - - return all_data_fields + return build_kv_file_fields( + table_fields=self.table.fields, + trimmed_primary_keys=self.trimmed_primary_key, + value_fields=value_field, + ) def create_index_mapping(self): base_index_mapping = self._create_base_index_mapping(self.read_fields, self._get_read_data_fields()) From 6bbb7119291f5c69a5cc6d8fd87f429bbb22e120 Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Sun, 3 May 2026 10:44:36 +0800 Subject: [PATCH 07/12] [python][compact] phase-4: Ray distributed executor Compact jobs can now run their work on Ray. table.new_compact_job(..., executor=RayExecutor(), catalog_options=..., table_identifier=...).execute() plans on the driver, ships JSON-serialized CompactTask payloads through ray.remote, rebuilds the FileStoreTable inside each worker via the configured catalog, runs the rewriter, and returns CommitMessages back to the driver for one atomic commit. CompactTask base class: - with_table_loader(catalog_options, table_identifier) attaches the spec a worker uses to rebuild its table. - to_dict / from_dict are now concrete: a base envelope holding type + loader spec + payload, with subclasses owning _to_payload / _from_payload. CompactTask.deserialize(payload) returns the right subclass via the registry. - _resolve_table_via_loader() centralizes catalog rebuild so subclasses share a single in-process-vs-distributed branch. AppendCompactTask / MergeTreeCompactTask: - replace the Phase 3 NotImplementedError stubs with real payload encoders that round-trip files via DataFileMeta.to_dict and partition tuples via encode_value/decode_value (handles DATE / Decimal / Timestamp partition columns). - _resolve_table prefers the in-process table when LocalExecutor attached one and falls back to the loader otherwise. CompactJob: - Accepts catalog_options + table_identifier and propagates them onto every task before dispatch when present. LocalExecutor path unchanged. RayExecutor: - Top-level _run_task_payload worker so Ray pickling stays cheap and worker code can't capture driver state. - ray.init only when not already initialized; respects ray_init_args. - num_cpus_per_task + ray_remote_args expose the usual Ray knobs. DataFileMeta serialization: - Tolerate manifest-side BinaryRow (lazy-decoded) in addition to GenericRow, and pyarrow Array-like null_counts. Without this the Ray round trip fails on files that were just produced by the writer. setup.py already declared ray as an optional extra (pip install pypaimon[ray]); no packaging changes required. Tests: - compact_task_serde_test (5 tests): round-trip Append + MergeTree payloads with loader spec and non-JSON-native partitions; clear error when neither table nor loader was attached; unknown-type rejection in the registry. - ray_executor_test (1 test): end-to-end Append-only compaction via a real ray.init(local_mode=True), asserting commit_kind=COMPACT and data identity. Skipped automatically if ray isn't installed. --- .../pypaimon/compact/executor/ray_executor.py | 95 +++++++++++++ .../pypaimon/compact/job/compact_job.py | 20 ++- .../compact/task/append_compact_task.py | 36 +++-- .../pypaimon/compact/task/compact_task.py | 93 +++++++++---- .../compact/task/merge_tree_compact_task.py | 42 +++--- .../manifest/schema/data_file_meta.py | 27 +++- .../pypaimon/table/file_store_table.py | 6 + .../tests/compact/compact_task_serde_test.py | 125 ++++++++++++++++++ .../tests/compact/ray_executor_test.py | 122 +++++++++++++++++ 9 files changed, 500 insertions(+), 66 deletions(-) create mode 100644 paimon-python/pypaimon/compact/executor/ray_executor.py create mode 100644 paimon-python/pypaimon/tests/compact/compact_task_serde_test.py create mode 100644 paimon-python/pypaimon/tests/compact/ray_executor_test.py diff --git a/paimon-python/pypaimon/compact/executor/ray_executor.py b/paimon-python/pypaimon/compact/executor/ray_executor.py new file mode 100644 index 000000000000..ed7ee78de03b --- /dev/null +++ b/paimon-python/pypaimon/compact/executor/ray_executor.py @@ -0,0 +1,95 @@ +################################################################################ +# 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. +################################################################################ + +"""Ray-backed compaction executor. + +Driver-side: serialize each CompactTask to a JSON payload and dispatch +ray.remote tasks. Worker-side: a top-level `_run_task_payload` rebuilds +the task from the payload (which includes catalog options + table +identifier so the worker can rebuild its own FileStoreTable) and runs +it, then returns a serialized CommitMessage. The driver collects them +into Python CommitMessage objects. + +`ray` is an optional dependency — installation is `pip install pypaimon[ray]` +— so we import inside execute() and present a clear error if it isn't +available. +""" + +from typing import Any, Dict, List, Optional + +from pypaimon.compact.executor.executor import CompactExecutor +from pypaimon.compact.task.compact_task import CompactTask +from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.commit_message_serializer import CommitMessageSerializer + + +# Worker entry point. Defined at module scope so Ray can pickle it cheaply +# and so a misbehaving subclass cannot accidentally close over driver state. +def _run_task_payload(payload: bytes) -> bytes: + task = CompactTask.deserialize(payload) + message = task.run() + return CommitMessageSerializer.serialize(message) + + +class RayExecutor(CompactExecutor): + + def __init__( + self, + num_cpus_per_task: float = 1.0, + ray_remote_args: Optional[Dict[str, Any]] = None, + ray_init_args: Optional[Dict[str, Any]] = None, + ): + """Create a RayExecutor. + + - num_cpus_per_task: per-task CPU budget; passed to ray.remote. + - ray_remote_args: extra kwargs for the ray.remote decorator + (e.g. {"max_retries": 3, "memory": 1<<30}). + - ray_init_args: extra kwargs for ray.init when this executor needs + to bootstrap Ray itself. If a Ray runtime is already initialized + we leave it alone. + """ + self.num_cpus_per_task = num_cpus_per_task + self.ray_remote_args = dict(ray_remote_args or {}) + self.ray_init_args = dict(ray_init_args or {}) + + def execute(self, tasks: List[CompactTask]) -> List[CommitMessage]: + if not tasks: + return [] + + try: + import ray # type: ignore + except ImportError as e: + raise RuntimeError( + "RayExecutor requires the 'ray' package; install pypaimon[ray] " + "or 'pip install ray'." + ) from e + + if not ray.is_initialized(): + ray.init(**self.ray_init_args) + + remote_run = ray.remote(num_cpus=self.num_cpus_per_task, **self.ray_remote_args)( + _run_task_payload + ) + + # Drive serialization on the driver — gives a deterministic failure + # site (one bad task surfaces as a TypeError here, not lost in a + # remote traceback). + payloads = [task.serialize() for task in tasks] + futures = [remote_run.remote(p) for p in payloads] + result_bytes: List[bytes] = ray.get(futures) + return [CommitMessageSerializer.deserialize(b) for b in result_bytes] diff --git a/paimon-python/pypaimon/compact/job/compact_job.py b/paimon-python/pypaimon/compact/job/compact_job.py index 3a2eef44165e..679114628a53 100644 --- a/paimon-python/pypaimon/compact/job/compact_job.py +++ b/paimon-python/pypaimon/compact/job/compact_job.py @@ -18,7 +18,7 @@ import logging import uuid -from typing import List, Optional +from typing import Any, Dict, List, Optional from pypaimon.common.predicate import Predicate from pypaimon.compact.coordinator.append_compact_coordinator import \ @@ -54,12 +54,22 @@ def __init__( executor: Optional[CompactExecutor] = None, partition_predicate: Optional[Predicate] = None, commit_user: Optional[str] = None, + catalog_options: Optional[Dict[str, Any]] = None, + table_identifier: Optional[str] = None, ): + """Construct a CompactJob. + + catalog_options + table_identifier are required when using a + distributed executor (RayExecutor) — workers need them to rebuild + the table on the worker process. LocalExecutor never reads them. + """ self.table = table self.compact_options = compact_options or CompactOptions() self.executor = executor or LocalExecutor() self.partition_predicate = partition_predicate self.commit_user = commit_user or str(uuid.uuid4()) + self.catalog_options = dict(catalog_options) if catalog_options else None + self.table_identifier = table_identifier or str(table.identifier) def execute(self) -> List[CommitMessage]: """Run the compaction job and return the messages that were committed. @@ -75,6 +85,14 @@ def execute(self) -> List[CommitMessage]: ) return [] + # Distributed executors can't share the in-process FileStoreTable, so + # attach the loader spec when caller provided one. LocalExecutor + # ignores it and uses the in-process table the coordinator already + # baked into each task. + if self.catalog_options is not None: + for task in tasks: + task.with_table_loader(self.catalog_options, self.table_identifier) + logger.info( "Compacting table %s: %d task(s) via %s", self.table.identifier, diff --git a/paimon-python/pypaimon/compact/task/append_compact_task.py b/paimon-python/pypaimon/compact/task/append_compact_task.py index 42524b750b23..c9252eca661c 100644 --- a/paimon-python/pypaimon/compact/task/append_compact_task.py +++ b/paimon-python/pypaimon/compact/task/append_compact_task.py @@ -20,7 +20,8 @@ from pypaimon.compact.rewriter.append_compact_rewriter import AppendCompactRewriter from pypaimon.compact.task.compact_task import CompactTask, register_compact_task -from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.manifest.schema.data_file_meta import (DataFileMeta, decode_value, + encode_value) from pypaimon.write.commit_message import CommitMessage @@ -63,27 +64,22 @@ def run(self) -> CommitMessage: compact_after=list(after), ) - def to_dict(self) -> Dict[str, Any]: - # Distributed executors will be wired up in Phase 4 (Ray). At that - # point we'll replace this stub with catalog-options + identifier so - # workers can rebuild the table; until then the LocalExecutor path - # never serializes a task and this method should raise loudly. - raise NotImplementedError( - "AppendCompactTask.to_dict() is reserved for Phase 4 distributed " - "execution; LocalExecutor runs tasks in-process without serialization." - ) + def _to_payload(self) -> Dict[str, Any]: + return { + "partition": [encode_value(v) for v in self.partition], + "bucket": self.bucket, + "files": [f.to_dict() for f in self.files], + } @classmethod - def from_dict(cls, data: Dict[str, Any]) -> "AppendCompactTask": - raise NotImplementedError( - "AppendCompactTask.from_dict() is reserved for Phase 4 distributed " - "execution; LocalExecutor runs tasks in-process without serialization." + def _from_payload(cls, payload: Dict[str, Any]) -> "AppendCompactTask": + return cls( + partition=tuple(decode_value(v) for v in payload.get("partition") or []), + bucket=payload["bucket"], + files=[DataFileMeta.from_dict(f) for f in payload.get("files") or []], ) def _resolve_table(self): - if self._table is None: - raise RuntimeError( - "AppendCompactTask has no table attached. The CompactJob/driver " - "must call with_table(table) before handing tasks to an executor." - ) - return self._table + if self._table is not None: + return self._table + return self._resolve_table_via_loader() diff --git a/paimon-python/pypaimon/compact/task/compact_task.py b/paimon-python/pypaimon/compact/task/compact_task.py index 69d155be4c2a..18790e7360cf 100644 --- a/paimon-python/pypaimon/compact/task/compact_task.py +++ b/paimon-python/pypaimon/compact/task/compact_task.py @@ -18,7 +18,7 @@ import json from abc import ABC, abstractmethod -from typing import Any, Dict +from typing import Any, Dict, Optional from pypaimon.write.commit_message import CommitMessage @@ -26,48 +26,95 @@ class CompactTask(ABC): """A self-contained compaction unit dispatched to a worker. - The constructor argument list is the contract: anything captured here - is what the worker has to rebuild its execution context. - - JSON serialization (to_dict / from_dict / serialize / deserialize) is - declared on the base class so distributed executors have a single hook - to call, but concrete subclasses are free to defer the implementation - until distributed execution actually arrives — Phase 4 will fill in the - AppendCompactTask serialization once Ray is the executor. Until then - those methods may raise NotImplementedError; LocalExecutor never - serializes a task and is unaffected. + Two operating modes: + - In-process (LocalExecutor): the driver attaches the FileStoreTable + via with_table(); the worker reuses it directly. + - Distributed (RayExecutor): the driver attaches a table loader spec + via with_table_loader(catalog_options, table_identifier); to_dict + ships those + the data payload across, and from_dict on the worker + rebuilds the table from the catalog. + + Subclasses implement _to_payload / _from_payload to add their own + fields on top of this base envelope (catalog_loader + table_identifier + are handled here once). """ TYPE: str = "" + # Distributed-execution loader spec, populated by CompactJob when an + # executor that can't share the in-process table is in use. + _catalog_loader_options: Optional[Dict[str, str]] = None + _table_identifier: Optional[str] = None + @abstractmethod def run(self) -> CommitMessage: - """Execute the compaction unit on the local process and return a CommitMessage. + """Execute the compaction unit and return a CommitMessage. - The CommitMessage carries compact_before / compact_after files for the - driver to assemble into a single atomic commit. + Subclasses should obtain their FileStoreTable via self._resolve_table() + rather than poking at the cached _table directly, so distributed and + local paths share the same retrieval logic. """ - @abstractmethod + def with_table_loader( + self, + catalog_options: Dict[str, str], + table_identifier: str, + ) -> "CompactTask": + """Attach the spec a distributed worker uses to rebuild this task's table.""" + self._catalog_loader_options = dict(catalog_options) + self._table_identifier = table_identifier + return self + + def _resolve_table_via_loader(self): + if not self._catalog_loader_options or not self._table_identifier: + raise RuntimeError( + f"{type(self).__name__} has no in-process table and no catalog loader; " + "the driver must call with_table() or with_table_loader() before " + "handing this task to an executor." + ) + # Lazy import keeps base task module decoupled from catalog code. + from pypaimon.catalog.catalog_factory import CatalogFactory + catalog = CatalogFactory.create(dict(self._catalog_loader_options)) + return catalog.get_table(self._table_identifier) + def to_dict(self) -> Dict[str, Any]: - """Return a JSON-friendly payload identifying everything the worker needs.""" + """Standard envelope; subclasses override _to_payload to add fields.""" + return { + "type": self.TYPE, + "catalog_options": self._catalog_loader_options, + "table_identifier": self._table_identifier, + "payload": self._to_payload(), + } - def serialize(self) -> bytes: - return json.dumps(self.to_dict(), separators=(",", ":")).encode("utf-8") + @abstractmethod + def _to_payload(self) -> Dict[str, Any]: + """Subclass-specific data (partition / bucket / files / ...).""" @classmethod - def deserialize(cls, payload: bytes) -> "CompactTask": - data = json.loads(payload.decode("utf-8")) + def from_dict(cls, data: Dict[str, Any]) -> "CompactTask": task_type = data.get("type") impl = _TASK_REGISTRY.get(task_type) if impl is None: raise ValueError(f"Unknown CompactTask type: {task_type}") - return impl.from_dict(data) + task = impl._from_payload(data.get("payload") or {}) + loader_opts = data.get("catalog_options") + identifier = data.get("table_identifier") + if loader_opts and identifier: + task.with_table_loader(loader_opts, identifier) + return task @classmethod @abstractmethod - def from_dict(cls, data: Dict[str, Any]) -> "CompactTask": - """Rebuild a task from its to_dict() payload.""" + def _from_payload(cls, payload: Dict[str, Any]) -> "CompactTask": + """Construct a task from the subclass-specific payload only.""" + + def serialize(self) -> bytes: + return json.dumps(self.to_dict(), separators=(",", ":")).encode("utf-8") + + @classmethod + def deserialize(cls, payload: bytes) -> "CompactTask": + data = json.loads(payload.decode("utf-8")) + return cls.from_dict(data) _TASK_REGISTRY: Dict[str, type] = {} diff --git a/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py b/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py index c90a6190f242..ceccaf37af70 100644 --- a/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py +++ b/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py @@ -21,7 +21,8 @@ from pypaimon.compact.rewriter.merge_tree_compact_rewriter import \ MergeTreeCompactRewriter from pypaimon.compact.task.compact_task import CompactTask, register_compact_task -from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.manifest.schema.data_file_meta import (DataFileMeta, decode_value, + encode_value) from pypaimon.read.interval_partition import IntervalPartition from pypaimon.read.reader.merge_function import \ create_merge_function_factory @@ -61,11 +62,7 @@ def with_table(self, table) -> "MergeTreeCompactTask": return self def run(self) -> CommitMessage: - if self._table is None: - raise RuntimeError( - "MergeTreeCompactTask has no table attached. The CompactJob/driver " - "must call with_table(table) before handing tasks to an executor." - ) + table = self._resolve_table() # IntervalPartition reproduces split_read.MergeFileSplitRead.create_reader's # section grouping so the rewriter sees the same "non-overlapping @@ -73,8 +70,8 @@ def run(self) -> CommitMessage: sections = IntervalPartition(self.files).partition() rewriter = MergeTreeCompactRewriter( - table=self._table, - mf_factory=create_merge_function_factory(self._table.options), + table=table, + mf_factory=create_merge_function_factory(table.options), ) after = rewriter.rewrite( partition=self.partition, @@ -91,15 +88,26 @@ def run(self) -> CommitMessage: compact_after=list(after), ) - def to_dict(self) -> Dict[str, Any]: - raise NotImplementedError( - "MergeTreeCompactTask.to_dict() is reserved for Phase 4 distributed " - "execution." - ) + def _to_payload(self) -> Dict[str, Any]: + return { + "partition": [encode_value(v) for v in self.partition], + "bucket": self.bucket, + "files": [f.to_dict() for f in self.files], + "output_level": self.output_level, + "drop_delete": self.drop_delete, + } @classmethod - def from_dict(cls, data: Dict[str, Any]) -> "MergeTreeCompactTask": - raise NotImplementedError( - "MergeTreeCompactTask.from_dict() is reserved for Phase 4 distributed " - "execution." + def _from_payload(cls, payload: Dict[str, Any]) -> "MergeTreeCompactTask": + return cls( + partition=tuple(decode_value(v) for v in payload.get("partition") or []), + bucket=payload["bucket"], + files=[DataFileMeta.from_dict(f) for f in payload.get("files") or []], + output_level=payload["output_level"], + drop_delete=payload["drop_delete"], ) + + def _resolve_table(self): + if self._table is not None: + return self._table + return self._resolve_table_via_loader() diff --git a/paimon-python/pypaimon/manifest/schema/data_file_meta.py b/paimon-python/pypaimon/manifest/schema/data_file_meta.py index 18a5df3cd1d9..7158304e737e 100644 --- a/paimon-python/pypaimon/manifest/schema/data_file_meta.py +++ b/paimon-python/pypaimon/manifest/schema/data_file_meta.py @@ -353,13 +353,21 @@ def decode_value(value: Any) -> Any: raise ValueError(f"Unknown tagged value type: {tag}") -def _generic_row_to_dict(row: Optional[GenericRow]) -> Optional[Dict[str, Any]]: +def _generic_row_to_dict(row) -> Optional[Dict[str, Any]]: if row is None: return None + # GenericRow exposes .values directly; BinaryRow lazily decodes per field + # via get_field(i). Normalize both into a list of decoded Python values + # so the dict format stays uniform. + if hasattr(row, "values"): + values = row.values + else: + values = [row.get_field(i) for i in range(len(row))] + fields = getattr(row, "fields", None) return { - "values": [encode_value(v) for v in row.values], - "fields": [f.to_dict() for f in row.fields] if row.fields else [], - "row_kind": row.row_kind.value, + "values": [encode_value(v) for v in values], + "fields": [f.to_dict() for f in fields] if fields else [], + "row_kind": row.get_row_kind().value if hasattr(row, "get_row_kind") else 0, } @@ -375,10 +383,19 @@ def _generic_row_from_dict(data: Optional[Dict[str, Any]]) -> Optional[GenericRo def _simple_stats_to_dict(stats: Optional[SimpleStats]) -> Optional[Dict[str, Any]]: if stats is None: return None + # null_counts may be a Python list (writer path) or a pyarrow Array-like + # (manifest reader path). Normalize to a plain list of ints. + nc = stats.null_counts + if nc is None: + null_counts = [] + elif hasattr(nc, "to_pylist"): + null_counts = nc.to_pylist() + else: + null_counts = list(nc) return { "min_values": _generic_row_to_dict(stats.min_values), "max_values": _generic_row_to_dict(stats.max_values), - "null_counts": list(stats.null_counts) if stats.null_counts is not None else [], + "null_counts": null_counts, } diff --git a/paimon-python/pypaimon/table/file_store_table.py b/paimon-python/pypaimon/table/file_store_table.py index da06d35ac507..03838c539e07 100644 --- a/paimon-python/pypaimon/table/file_store_table.py +++ b/paimon-python/pypaimon/table/file_store_table.py @@ -372,11 +372,15 @@ def new_compact_job( executor=None, partition_predicate=None, commit_user: Optional[str] = None, + catalog_options=None, + table_identifier: Optional[str] = None, ): """Create a CompactJob bound to this table. Args mirror CompactJob — passed through so callers can construct coordinators/executors elsewhere when they need cross-table sharing. + Pass catalog_options + table_identifier when using a distributed + executor (RayExecutor) so workers can rebuild the table. """ from pypaimon.compact.job.compact_job import CompactJob return CompactJob( @@ -385,6 +389,8 @@ def new_compact_job( executor=executor, partition_predicate=partition_predicate, commit_user=commit_user, + catalog_options=catalog_options, + table_identifier=table_identifier, ) def new_stream_write_builder(self) -> StreamWriteBuilder: diff --git a/paimon-python/pypaimon/tests/compact/compact_task_serde_test.py b/paimon-python/pypaimon/tests/compact/compact_task_serde_test.py new file mode 100644 index 000000000000..27008053019e --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/compact_task_serde_test.py @@ -0,0 +1,125 @@ +################################################################################ +# 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 unittest +from datetime import date, datetime + +from pypaimon.compact.task.append_compact_task import AppendCompactTask +from pypaimon.compact.task.compact_task import CompactTask +from pypaimon.compact.task.merge_tree_compact_task import MergeTreeCompactTask +from pypaimon.data.timestamp import Timestamp +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.manifest.schema.simple_stats import SimpleStats +from pypaimon.schema.data_types import AtomicType, DataField +from pypaimon.table.row.generic_row import GenericRow + +PK_FIELDS = [DataField(0, "id", AtomicType("BIGINT"))] + + +def _make_file(name: str = "data-1.parquet") -> DataFileMeta: + return DataFileMeta.create( + file_name=name, + file_size=4096, + row_count=10, + min_key=GenericRow([1], PK_FIELDS), + max_key=GenericRow([99], PK_FIELDS), + key_stats=SimpleStats.empty_stats(), + value_stats=SimpleStats.empty_stats(), + min_sequence_number=10, + max_sequence_number=20, + schema_id=0, + level=0, + extra_files=[], + creation_time=Timestamp.from_epoch_millis(1_700_000_000_000), + ) + + +class AppendCompactTaskSerdeTest(unittest.TestCase): + + def test_round_trip_with_loader(self): + original = AppendCompactTask( + partition=("p1",), + bucket=2, + files=[_make_file("a.parquet"), _make_file("b.parquet")], + ).with_table_loader({"warehouse": "/tmp/wh"}, "default.t") + + rebuilt = CompactTask.deserialize(original.serialize()) + + self.assertIsInstance(rebuilt, AppendCompactTask) + self.assertEqual(("p1",), rebuilt.partition) + self.assertEqual(2, rebuilt.bucket) + self.assertEqual(2, len(rebuilt.files)) + self.assertEqual(["a.parquet", "b.parquet"], [f.file_name for f in rebuilt.files]) + self.assertEqual({"warehouse": "/tmp/wh"}, rebuilt._catalog_loader_options) + self.assertEqual("default.t", rebuilt._table_identifier) + + def test_partition_with_non_json_native_types_round_trips(self): + original = AppendCompactTask( + partition=(date(2024, 1, 2), datetime(2024, 1, 2, 3, 4)), + bucket=0, + files=[_make_file()], + ).with_table_loader({"warehouse": "/tmp/wh"}, "default.t") + + rebuilt = CompactTask.deserialize(original.serialize()) + + self.assertEqual((date(2024, 1, 2), datetime(2024, 1, 2, 3, 4)), rebuilt.partition) + + +class MergeTreeCompactTaskSerdeTest(unittest.TestCase): + + def test_round_trip_includes_output_level_and_drop_delete(self): + original = MergeTreeCompactTask( + partition=("p1",), + bucket=0, + files=[_make_file("merge-a.parquet"), _make_file("merge-b.parquet")], + output_level=3, + drop_delete=True, + ).with_table_loader({"warehouse": "/tmp/wh"}, "default.pk") + + rebuilt = CompactTask.deserialize(original.serialize()) + + self.assertIsInstance(rebuilt, MergeTreeCompactTask) + self.assertEqual(("p1",), rebuilt.partition) + self.assertEqual(0, rebuilt.bucket) + self.assertEqual(3, rebuilt.output_level) + self.assertTrue(rebuilt.drop_delete) + self.assertEqual(2, len(rebuilt.files)) + self.assertEqual({"warehouse": "/tmp/wh"}, rebuilt._catalog_loader_options) + self.assertEqual("default.pk", rebuilt._table_identifier) + + def test_run_without_table_or_loader_raises_clear_error(self): + task = MergeTreeCompactTask( + partition=("p1",), + bucket=0, + files=[_make_file()], + output_level=2, + drop_delete=False, + ) + with self.assertRaises(RuntimeError): + task.run() + + +class CompactTaskRegistryTest(unittest.TestCase): + + def test_unknown_type_rejected(self): + with self.assertRaises(ValueError): + CompactTask.from_dict({"type": "bogus", "payload": {}}) + + +if __name__ == "__main__": + unittest.main() diff --git a/paimon-python/pypaimon/tests/compact/ray_executor_test.py b/paimon-python/pypaimon/tests/compact/ray_executor_test.py new file mode 100644 index 000000000000..1914b8865be9 --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/ray_executor_test.py @@ -0,0 +1,122 @@ +################################################################################ +# 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 shutil +import tempfile +import unittest + +import pyarrow as pa + +from pypaimon import CatalogFactory, Schema +from pypaimon.common.options.core_options import CoreOptions +from pypaimon.compact.options import CompactOptions + +try: + import ray # noqa: F401 + HAS_RAY = True +except ImportError: + HAS_RAY = False + + +@unittest.skipUnless(HAS_RAY, "ray is not installed") +class RayExecutorE2ETest(unittest.TestCase): + """End-to-end Ray execution: rewrite via worker tasks, commit on driver. + + Uses a real local Ray runtime — the executor under test ships task + payloads through ray.remote, which exercises the full CompactTask + serde path AppendCompactTask / MergeTreeCompactTask were extended + with this phase. + """ + + @classmethod + def setUpClass(cls): + import ray + cls.temp_dir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.temp_dir, "warehouse") + cls.catalog_options = {"warehouse": cls.warehouse} + cls.catalog = CatalogFactory.create(cls.catalog_options) + cls.catalog.create_database("ray_db", False) + # local_mode keeps the test single-process; faster startup, no port + # races and the same code path as a real cluster. + ray.init(local_mode=True, ignore_reinit_error=True, log_to_driver=False) + + @classmethod + def tearDownClass(cls): + import ray + ray.shutdown() + shutil.rmtree(cls.temp_dir, ignore_errors=True) + + def _make_unaware_table(self, name: str): + full = f"ray_db.{name}" + opts = { + CoreOptions.BUCKET.key(): "-1", + CoreOptions.TARGET_FILE_SIZE.key(): "10mb", + } + pa_schema = pa.schema([("id", pa.int32()), ("name", pa.string())]) + schema = Schema.from_pyarrow_schema(pa_schema, options=opts) + self.catalog.create_table(full, schema, True) + return self.catalog.get_table(full) + + def _write_one(self, table, batch: pa.Table): + builder = table.new_batch_write_builder() + write = builder.new_write() + commit = builder.new_commit() + write.write_arrow(batch) + commit.commit(write.prepare_commit()) + write.close() + commit.close() + + def _read_sorted(self, table) -> pa.Table: + rb = table.new_read_builder() + scan = rb.new_scan() + splits = scan.plan().splits() + return rb.new_read().to_arrow(splits).sort_by("id") + + def test_append_compact_via_ray_executor(self): + from pypaimon.compact.executor.ray_executor import RayExecutor + + table = self._make_unaware_table("ray_append") + for i in range(5): + self._write_one(table, pa.Table.from_pydict({ + "id": pa.array([i * 2, i * 2 + 1], type=pa.int32()), + "name": [f"r-{i}-a", f"r-{i}-b"], + })) + + table = self.catalog.get_table("ray_db.ray_append") + before_data = self._read_sorted(table) + + job = table.new_compact_job( + compact_options=CompactOptions(min_file_num=5), + executor=RayExecutor(), + catalog_options=self.catalog_options, + table_identifier="ray_db.ray_append", + ) + messages = job.execute() + + self.assertEqual(1, len(messages)) + self.assertGreaterEqual(len(messages[0].compact_before), 5) + + table = self.catalog.get_table("ray_db.ray_append") + after_data = self._read_sorted(table) + self.assertEqual(before_data, after_data) + self.assertEqual("COMPACT", table.snapshot_manager().get_latest_snapshot().commit_kind) + + +if __name__ == "__main__": + unittest.main() From aaedcf024ad934976b63fd853d8bf18e692ad77a Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Sun, 3 May 2026 10:50:00 +0800 Subject: [PATCH 08/12] [python][compact] phase-4 fixup: address review findings MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - CompactJob.table_identifier default uses Identifier.get_full_name() instead of str(identifier). Identifier is a dataclass with no custom __str__, so str(...) returns its repr ("Identifier(database='db', ...)") and Identifier.from_string would refuse to parse that on the worker side. The default path was untested in Phase 4 (e2e passed only because the test explicitly passed table_identifier=...) — this fixup also drops that explicit kwarg from the e2e so the default is exercised. - RayExecutor module imports the AppendCompactTask / MergeTreeCompactTask modules at the top level so their @register_compact_task side effects populate the task registry inside Ray worker processes. Without this, a real (non-local_mode) Ray cluster would unpickle _run_task_payload in a fresh process whose registry is empty and CompactTask.deserialize would raise "Unknown CompactTask type". - MergeTreeCompactTask docstring updated — it no longer says "Phase 4 will plumb the loader fields" since Phase 4 already did. --- paimon-python/pypaimon/compact/executor/ray_executor.py | 8 ++++++++ paimon-python/pypaimon/compact/job/compact_job.py | 6 +++++- .../pypaimon/compact/task/merge_tree_compact_task.py | 6 +++--- paimon-python/pypaimon/tests/compact/ray_executor_test.py | 5 ++++- 4 files changed, 20 insertions(+), 5 deletions(-) diff --git a/paimon-python/pypaimon/compact/executor/ray_executor.py b/paimon-python/pypaimon/compact/executor/ray_executor.py index ed7ee78de03b..7e650e2c7b23 100644 --- a/paimon-python/pypaimon/compact/executor/ray_executor.py +++ b/paimon-python/pypaimon/compact/executor/ray_executor.py @@ -34,6 +34,14 @@ from pypaimon.compact.executor.executor import CompactExecutor from pypaimon.compact.task.compact_task import CompactTask +# Side-effect imports: each task subclass registers itself in the task +# registry at import time. Ray workers are fresh processes with an empty +# registry, so we must guarantee these modules get imported in the worker — +# importing them here means `import ray_executor` (which the worker does +# implicitly when unpickling _run_task_payload) brings the registrations +# along for the ride. +from pypaimon.compact.task import append_compact_task as _append_task # noqa: F401 +from pypaimon.compact.task import merge_tree_compact_task as _mt_task # noqa: F401 from pypaimon.write.commit_message import CommitMessage from pypaimon.write.commit_message_serializer import CommitMessageSerializer diff --git a/paimon-python/pypaimon/compact/job/compact_job.py b/paimon-python/pypaimon/compact/job/compact_job.py index 679114628a53..edf25d73e1b6 100644 --- a/paimon-python/pypaimon/compact/job/compact_job.py +++ b/paimon-python/pypaimon/compact/job/compact_job.py @@ -69,7 +69,11 @@ def __init__( self.partition_predicate = partition_predicate self.commit_user = commit_user or str(uuid.uuid4()) self.catalog_options = dict(catalog_options) if catalog_options else None - self.table_identifier = table_identifier or str(table.identifier) + # Identifier is a dataclass with no custom __str__; str(...) would + # return its repr ("Identifier(database=...)") and Identifier.from_string + # would refuse to parse that. Use get_full_name() so the worker can + # round-trip the identifier through CatalogFactory. + self.table_identifier = table_identifier or table.identifier.get_full_name() def execute(self) -> List[CommitMessage]: """Run the compaction job and return the messages that were committed. diff --git a/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py b/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py index ceccaf37af70..bfaee07f566c 100644 --- a/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py +++ b/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py @@ -34,9 +34,9 @@ class MergeTreeCompactTask(CompactTask): """Compact a single (partition, bucket) of a primary-key table. Carries the picked CompactUnit's files plus the strategy-decided - output_level and drop_delete flag. The driver attaches the in-process - FileStoreTable; Phase 4 will plumb the loader fields for distributed - execution. + output_level and drop_delete flag. The driver attaches either the + in-process FileStoreTable (LocalExecutor) or a catalog loader spec via + with_table_loader (RayExecutor); _resolve_table picks whichever is set. """ TYPE = "merge-tree-compact" diff --git a/paimon-python/pypaimon/tests/compact/ray_executor_test.py b/paimon-python/pypaimon/tests/compact/ray_executor_test.py index 1914b8865be9..cfb8b63c5f34 100644 --- a/paimon-python/pypaimon/tests/compact/ray_executor_test.py +++ b/paimon-python/pypaimon/tests/compact/ray_executor_test.py @@ -101,11 +101,14 @@ def test_append_compact_via_ray_executor(self): table = self.catalog.get_table("ray_db.ray_append") before_data = self._read_sorted(table) + # Note: deliberately omit table_identifier — exercises the default + # path (table.identifier.get_full_name()) which the worker uses + # via Identifier.from_string. A regression here would surface as + # "Cannot get splits from 'Identifier(...)'" inside the Ray task. job = table.new_compact_job( compact_options=CompactOptions(min_file_num=5), executor=RayExecutor(), catalog_options=self.catalog_options, - table_identifier="ray_db.ray_append", ) messages = job.execute() From 0475bf7faf5b846d005a1b45332c60f74f795f0d Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Wed, 6 May 2026 11:02:00 +0800 Subject: [PATCH 09/12] [python][compact] align CommitMessage with Java DataIncrement / CompactIncrement MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Restructure CommitMessage to mirror org.apache.paimon.table.sink.CommitMessageImpl exactly: instead of dropping new_files / compact_before / compact_after onto CommitMessage as flat fields, package them inside DataIncrement and CompactIncrement value objects that match their Java counterparts field-for-field. This makes Python and Java messages structurally identical and gives later phases a single, unambiguous slot to plug deletion vectors, changelog files, and global index deltas into without inventing parallel field names. New value objects: - DataIncrement(new_files, deleted_files, changelog_files, new_index_files, deleted_index_files) — direct port of org.apache.paimon.io.DataIncrement. - CompactIncrement(compact_before, compact_after, changelog_files, new_index_files, deleted_index_files) — direct port of org.apache.paimon.io.CompactIncrement. CommitMessage now holds (partition, bucket, total_buckets, data_increment, compact_increment, check_from_snapshot). Convenience properties (new_files, compact_before, compact_after, changelog_files, ...) keep call-sites readable without leaking the increment shape. Migration: - FileStoreWrite.prepare_commit, TableUpdate.prepare_commit, AppendCompactTask.run, MergeTreeCompactTask.run all build their CommitMessage through DataIncrement / CompactIncrement and now also populate total_buckets the way Java does. - CommitMessageSerializer wire format bumps to version=2 and round-trips the full increment shape, including index file lists. IndexFileMeta serialization covers identity fields only — dv_ranges / global_index_meta will be wired up alongside the deletion-vector and changelog phases. Tests updated to construct messages via increments. No behavior changes for the existing commit / read paths: FileStoreCommit still reads message.new_files / compact_before / compact_after through the new convenience properties. --- .../compact/task/append_compact_task.py | 8 +- .../compact/task/merge_tree_compact_task.py | 8 +- .../tests/commit_message_serializer_test.py | 29 ++++-- .../tests/file_store_commit_compact_test.py | 38 +++++--- .../pypaimon/tests/file_store_commit_test.py | 15 +-- .../pypaimon/write/commit_message.py | 59 ++++++++++-- .../write/commit_message_serializer.py | 96 +++++++++++++++++-- .../pypaimon/write/compact_increment.py | 58 +++++++++++ .../pypaimon/write/data_increment.py | 61 ++++++++++++ .../pypaimon/write/file_store_write.py | 4 +- paimon-python/pypaimon/write/table_update.py | 9 +- 11 files changed, 338 insertions(+), 47 deletions(-) create mode 100644 paimon-python/pypaimon/write/compact_increment.py create mode 100644 paimon-python/pypaimon/write/data_increment.py diff --git a/paimon-python/pypaimon/compact/task/append_compact_task.py b/paimon-python/pypaimon/compact/task/append_compact_task.py index c9252eca661c..ee4f60b60d03 100644 --- a/paimon-python/pypaimon/compact/task/append_compact_task.py +++ b/paimon-python/pypaimon/compact/task/append_compact_task.py @@ -23,6 +23,7 @@ from pypaimon.manifest.schema.data_file_meta import (DataFileMeta, decode_value, encode_value) from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.compact_increment import CompactIncrement @register_compact_task @@ -60,8 +61,11 @@ def run(self) -> CommitMessage: return CommitMessage( partition=self.partition, bucket=self.bucket, - compact_before=list(self.files), - compact_after=list(after), + total_buckets=table.total_buckets, + compact_increment=CompactIncrement( + compact_before=list(self.files), + compact_after=list(after), + ), ) def _to_payload(self) -> Dict[str, Any]: diff --git a/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py b/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py index bfaee07f566c..feba025c0d74 100644 --- a/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py +++ b/paimon-python/pypaimon/compact/task/merge_tree_compact_task.py @@ -27,6 +27,7 @@ from pypaimon.read.reader.merge_function import \ create_merge_function_factory from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.compact_increment import CompactIncrement @register_compact_task @@ -84,8 +85,11 @@ def run(self) -> CommitMessage: return CommitMessage( partition=self.partition, bucket=self.bucket, - compact_before=list(self.files), - compact_after=list(after), + total_buckets=table.total_buckets, + compact_increment=CompactIncrement( + compact_before=list(self.files), + compact_after=list(after), + ), ) def _to_payload(self) -> Dict[str, Any]: diff --git a/paimon-python/pypaimon/tests/commit_message_serializer_test.py b/paimon-python/pypaimon/tests/commit_message_serializer_test.py index 25782b50b25a..d41dcbe0520c 100644 --- a/paimon-python/pypaimon/tests/commit_message_serializer_test.py +++ b/paimon-python/pypaimon/tests/commit_message_serializer_test.py @@ -28,6 +28,8 @@ from pypaimon.table.row.internal_row import RowKind from pypaimon.write.commit_message import CommitMessage from pypaimon.write.commit_message_serializer import CommitMessageSerializer +from pypaimon.write.compact_increment import CompactIncrement +from pypaimon.write.data_increment import DataIncrement def _key_field(idx: int, name: str, type_str: str) -> DataField: @@ -132,9 +134,11 @@ def test_serialize_deserialize_roundtrip_for_compact_message(self): message = CommitMessage( partition=("2024-01-01", "us"), bucket=2, - new_files=[], - compact_before=before_files, - compact_after=after_files, + total_buckets=8, + compact_increment=CompactIncrement( + compact_before=before_files, + compact_after=after_files, + ), check_from_snapshot=42, ) @@ -144,6 +148,7 @@ def test_serialize_deserialize_roundtrip_for_compact_message(self): self.assertIsInstance(payload, bytes) self.assertEqual(message.partition, rebuilt.partition) self.assertEqual(message.bucket, rebuilt.bucket) + self.assertEqual(message.total_buckets, rebuilt.total_buckets) self.assertEqual(message.new_files, rebuilt.new_files) self.assertEqual(message.compact_before, rebuilt.compact_before) self.assertEqual(message.compact_after, rebuilt.compact_after) @@ -153,7 +158,7 @@ def test_serialize_deserialize_roundtrip_for_append_message(self): message = CommitMessage( partition=(), bucket=0, - new_files=[_build_data_file_meta("append-1.parquet")], + data_increment=DataIncrement(new_files=[_build_data_file_meta("append-1.parquet")]), ) rebuilt = CommitMessageSerializer.deserialize(CommitMessageSerializer.serialize(message)) @@ -165,7 +170,11 @@ def test_serialize_deserialize_roundtrip_for_append_message(self): self.assertEqual([], rebuilt.compact_after) def test_unsupported_version_is_rejected(self): - message = CommitMessage(partition=(), bucket=0, new_files=[_build_data_file_meta()]) + message = CommitMessage( + partition=(), + bucket=0, + data_increment=DataIncrement(new_files=[_build_data_file_meta()]), + ) payload_dict = CommitMessageSerializer.to_dict(message) payload_dict["version"] = CommitMessageSerializer.VERSION + 1 @@ -177,7 +186,7 @@ def test_serialize_supports_partition_with_non_json_native_types(self): message = CommitMessage( partition=(date(2024, 1, 2), Decimal("99.50"), b"raw"), bucket=0, - compact_after=[_build_data_file_meta()], + compact_increment=CompactIncrement(compact_after=[_build_data_file_meta()]), ) rebuilt = CommitMessageSerializer.deserialize(CommitMessageSerializer.serialize(message)) @@ -189,7 +198,7 @@ def test_serialize_supports_timestamp_partition(self): message = CommitMessage( partition=(ts,), bucket=0, - compact_after=[_build_data_file_meta()], + compact_increment=CompactIncrement(compact_after=[_build_data_file_meta()]), ) rebuilt = CommitMessageSerializer.deserialize(CommitMessageSerializer.serialize(message)) @@ -198,7 +207,11 @@ def test_serialize_supports_timestamp_partition(self): def test_serialize_list_round_trip(self): messages = [ - CommitMessage(partition=(f"p{i}",), bucket=i, new_files=[_build_data_file_meta(f"f{i}.parquet")]) + CommitMessage( + partition=(f"p{i}",), + bucket=i, + data_increment=DataIncrement(new_files=[_build_data_file_meta(f"f{i}.parquet")]), + ) for i in range(3) ] payloads = CommitMessageSerializer.serialize_list(messages) diff --git a/paimon-python/pypaimon/tests/file_store_commit_compact_test.py b/paimon-python/pypaimon/tests/file_store_commit_compact_test.py index e3643ae4ff47..07205a242008 100644 --- a/paimon-python/pypaimon/tests/file_store_commit_compact_test.py +++ b/paimon-python/pypaimon/tests/file_store_commit_compact_test.py @@ -26,6 +26,8 @@ from pypaimon.schema.data_types import AtomicType, DataField from pypaimon.table.row.generic_row import GenericRow from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.compact_increment import CompactIncrement +from pypaimon.write.data_increment import DataIncrement from pypaimon.write.file_store_commit import FileStoreCommit @@ -79,7 +81,7 @@ def test_build_entries_emits_add_for_new_files(self, *_): msg = CommitMessage( partition=('2024-01-15',), bucket=2, - new_files=[_make_file('a.parquet')], + data_increment=DataIncrement(new_files=[_make_file('a.parquet')]), ) entries = commit._build_commit_entries([msg]) @@ -95,8 +97,10 @@ def test_build_entries_emits_delete_for_compact_before_and_add_for_compact_after msg = CommitMessage( partition=('2024-01-15',), bucket=1, - compact_before=[_make_file('old-1.parquet'), _make_file('old-2.parquet')], - compact_after=[_make_file('merged.parquet')], + compact_increment=CompactIncrement( + compact_before=[_make_file('old-1.parquet'), _make_file('old-2.parquet')], + compact_after=[_make_file('merged.parquet')], + ), ) entries = commit._build_commit_entries([msg]) @@ -113,8 +117,10 @@ def test_commit_with_only_compact_messages_uses_compact_kind(self, *_): msg = CommitMessage( partition=('p1',), bucket=0, - compact_before=[_make_file('old.parquet')], - compact_after=[_make_file('new.parquet')], + compact_increment=CompactIncrement( + compact_before=[_make_file('old.parquet')], + compact_after=[_make_file('new.parquet')], + ), ) commit.commit([msg], commit_identifier=100) @@ -130,9 +136,11 @@ def test_commit_with_new_files_keeps_append_kind_even_when_compact_fields_presen msg = CommitMessage( partition=('p1',), bucket=0, - new_files=[_make_file('new.parquet')], - compact_before=[_make_file('old.parquet')], - compact_after=[_make_file('merged.parquet')], + data_increment=DataIncrement(new_files=[_make_file('new.parquet')]), + compact_increment=CompactIncrement( + compact_before=[_make_file('old.parquet')], + compact_after=[_make_file('merged.parquet')], + ), ) commit.commit([msg], commit_identifier=200) @@ -146,8 +154,10 @@ def test_commit_compact_uses_compact_kind_and_no_conflict_detection(self, *_): msg = CommitMessage( partition=('p1',), bucket=3, - compact_before=[_make_file('old.parquet')], - compact_after=[_make_file('new.parquet')], + compact_increment=CompactIncrement( + compact_before=[_make_file('old.parquet')], + compact_after=[_make_file('new.parquet')], + ), ) commit.commit_compact([msg], commit_identifier=300) @@ -164,9 +174,11 @@ def test_commit_compact_rejects_messages_with_new_files(self, *_): msg = CommitMessage( partition=('p1',), bucket=0, - new_files=[_make_file('append.parquet')], - compact_before=[_make_file('old.parquet')], - compact_after=[_make_file('new.parquet')], + data_increment=DataIncrement(new_files=[_make_file('append.parquet')]), + compact_increment=CompactIncrement( + compact_before=[_make_file('old.parquet')], + compact_after=[_make_file('new.parquet')], + ), ) with self.assertRaises(ValueError): diff --git a/paimon-python/pypaimon/tests/file_store_commit_test.py b/paimon-python/pypaimon/tests/file_store_commit_test.py index 958ea85a6b7e..a537e6d0dca4 100644 --- a/paimon-python/pypaimon/tests/file_store_commit_test.py +++ b/paimon-python/pypaimon/tests/file_store_commit_test.py @@ -25,6 +25,7 @@ from pypaimon.snapshot.snapshot_commit import PartitionStatistics from pypaimon.table.row.generic_row import GenericRow from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.data_increment import DataIncrement from pypaimon.write.file_store_commit import FileStoreCommit @@ -88,7 +89,7 @@ def test_generate_partition_statistics_single_partition_single_file( commit_message = CommitMessage( partition=('2024-01-15', 'us-east-1'), bucket=0, - new_files=[file_meta] + data_increment=DataIncrement(new_files=[file_meta]) ) # Test method @@ -153,7 +154,7 @@ def test_generate_partition_statistics_multiple_files_same_partition( commit_message = CommitMessage( partition=('2024-01-15', 'us-east-1'), bucket=0, - new_files=[file_meta_1, file_meta_2] + data_increment=DataIncrement(new_files=[file_meta_1, file_meta_2]) ) # Test method @@ -225,13 +226,13 @@ def test_generate_partition_statistics_multiple_partitions( commit_message_1 = CommitMessage( partition=('2024-01-15', 'us-east-1'), bucket=0, - new_files=[file_meta_1] + data_increment=DataIncrement(new_files=[file_meta_1]) ) commit_message_2 = CommitMessage( partition=('2024-01-15', 'us-west-2'), bucket=0, - new_files=[file_meta_2] + data_increment=DataIncrement(new_files=[file_meta_2]) ) # Test method @@ -294,7 +295,7 @@ def test_generate_partition_statistics_unpartitioned_table( commit_message = CommitMessage( partition=(), # Empty partition for unpartitioned table bucket=0, - new_files=[file_meta] + data_increment=DataIncrement(new_files=[file_meta]) ) # Test method @@ -333,7 +334,7 @@ def test_generate_partition_statistics_no_creation_time( commit_message = CommitMessage( partition=('2024-01-15', 'us-east-1'), bucket=0, - new_files=[file_meta] + data_increment=DataIncrement(new_files=[file_meta]) ) # Test method @@ -375,7 +376,7 @@ def test_generate_partition_statistics_mismatched_partition_keys( commit_message = CommitMessage( partition=('2024-01-15', 'us-east-1', 'extra-value'), # 3 values but table has 2 keys bucket=0, - new_files=[file_meta] + data_increment=DataIncrement(new_files=[file_meta]) ) # Test method diff --git a/paimon-python/pypaimon/write/commit_message.py b/paimon-python/pypaimon/write/commit_message.py index 77344058c3b4..24347ff8d001 100644 --- a/paimon-python/pypaimon/write/commit_message.py +++ b/paimon-python/pypaimon/write/commit_message.py @@ -17,19 +17,66 @@ ################################################################################ from dataclasses import dataclass, field -from typing import List, Tuple, Optional +from typing import List, Optional, Tuple from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.write.compact_increment import CompactIncrement +from pypaimon.write.data_increment import DataIncrement @dataclass class CommitMessage: + """File committable for sink. + + Direct port of org.apache.paimon.table.sink.CommitMessageImpl. Carries + everything one (partition, bucket) writer or compactor contributes to a + snapshot, packaged as a (data_increment, compact_increment) pair so the + same message type can describe both pure writes and compaction results. + + - partition / bucket: identify the (partition, bucket) the message + applies to. + - total_buckets: number of buckets the table had at write time, used by + the commit path to detect bucket-count changes. + - data_increment: ADD/DELETE/changelog/index deltas from a normal write. + - compact_increment: ADD/DELETE/changelog/index deltas from compaction. + - check_from_snapshot: row-tracking conflict-detection anchor; -1 means + "no check" (default). + """ + partition: Tuple bucket: int - new_files: List[DataFileMeta] = field(default_factory=list) - compact_before: List[DataFileMeta] = field(default_factory=list) - compact_after: List[DataFileMeta] = field(default_factory=list) + total_buckets: Optional[int] = None + data_increment: DataIncrement = field(default_factory=DataIncrement) + compact_increment: CompactIncrement = field(default_factory=CompactIncrement) check_from_snapshot: Optional[int] = -1 - def is_empty(self): - return not self.new_files and not self.compact_before and not self.compact_after + # ---- Convenience accessors --------------------------------------------- + # Mirror Java's CommitMessageImpl shape: callers usually want the + # individual file lists rather than reaching through the increment. + + @property + def new_files(self) -> List[DataFileMeta]: + return self.data_increment.new_files + + @property + def deleted_files(self) -> List[DataFileMeta]: + return self.data_increment.deleted_files + + @property + def changelog_files(self) -> List[DataFileMeta]: + return self.data_increment.changelog_files + + @property + def compact_before(self) -> List[DataFileMeta]: + return self.compact_increment.compact_before + + @property + def compact_after(self) -> List[DataFileMeta]: + return self.compact_increment.compact_after + + @property + def compact_changelog_files(self) -> List[DataFileMeta]: + return self.compact_increment.changelog_files + + def is_empty(self) -> bool: + return self.data_increment.is_empty() and self.compact_increment.is_empty() diff --git a/paimon-python/pypaimon/write/commit_message_serializer.py b/paimon-python/pypaimon/write/commit_message_serializer.py index 13b60e201316..cfa63ad990b9 100644 --- a/paimon-python/pypaimon/write/commit_message_serializer.py +++ b/paimon-python/pypaimon/write/commit_message_serializer.py @@ -19,9 +19,12 @@ import json from typing import Any, Dict, List +from pypaimon.index.index_file_meta import IndexFileMeta from pypaimon.manifest.schema.data_file_meta import (DataFileMeta, decode_value, encode_value) from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.compact_increment import CompactIncrement +from pypaimon.write.data_increment import DataIncrement class CommitMessageSerializer: @@ -30,9 +33,17 @@ class CommitMessageSerializer: JSON-based on purpose: human-debuggable, version-tolerant across worker Python versions, and avoids the security/compat pitfalls of pickle when shipping CompactTask outputs from Ray workers back to the driver. + + The wire shape mirrors org.apache.paimon.table.sink.CommitMessageImpl: + every message is (partition, bucket, total_buckets, data_increment, + compact_increment), with each increment carrying its own new/deleted/ + changelog file lists plus index file deltas. Today the index slots are + populated only by tables that opt into them; the serializer round-trips + them either way so adding deletion vectors / global index later does + not need a new payload version. """ - VERSION = 1 + VERSION = 2 @classmethod def serialize(cls, message: CommitMessage) -> bytes: @@ -49,9 +60,9 @@ def to_dict(cls, message: CommitMessage) -> Dict[str, Any]: "version": cls.VERSION, "partition": [encode_value(v) for v in partition], "bucket": message.bucket, - "new_files": [f.to_dict() for f in message.new_files], - "compact_before": [f.to_dict() for f in message.compact_before], - "compact_after": [f.to_dict() for f in message.compact_after], + "total_buckets": message.total_buckets, + "data_increment": cls._data_increment_to_dict(message.data_increment), + "compact_increment": cls._compact_increment_to_dict(message.compact_increment), "check_from_snapshot": message.check_from_snapshot, } @@ -66,9 +77,9 @@ def from_dict(cls, data: Dict[str, Any]) -> CommitMessage: return CommitMessage( partition=tuple(decode_value(v) for v in partition_values), bucket=data["bucket"], - new_files=[DataFileMeta.from_dict(f) for f in data.get("new_files", [])], - compact_before=[DataFileMeta.from_dict(f) for f in data.get("compact_before", [])], - compact_after=[DataFileMeta.from_dict(f) for f in data.get("compact_after", [])], + total_buckets=data.get("total_buckets"), + data_increment=cls._data_increment_from_dict(data.get("data_increment")), + compact_increment=cls._compact_increment_from_dict(data.get("compact_increment")), check_from_snapshot=data.get("check_from_snapshot", -1), ) @@ -79,3 +90,74 @@ def serialize_list(cls, messages: List[CommitMessage]) -> List[bytes]: @classmethod def deserialize_list(cls, payloads: List[bytes]) -> List[CommitMessage]: return [cls.deserialize(p) for p in payloads] + + # ---- Increment helpers ------------------------------------------------- + + @classmethod + def _data_increment_to_dict(cls, inc: DataIncrement) -> Dict[str, Any]: + return { + "new_files": [f.to_dict() for f in inc.new_files], + "deleted_files": [f.to_dict() for f in inc.deleted_files], + "changelog_files": [f.to_dict() for f in inc.changelog_files], + "new_index_files": [_index_file_to_dict(i) for i in inc.new_index_files], + "deleted_index_files": [_index_file_to_dict(i) for i in inc.deleted_index_files], + } + + @classmethod + def _data_increment_from_dict(cls, data) -> DataIncrement: + if not data: + return DataIncrement() + return DataIncrement( + new_files=[DataFileMeta.from_dict(f) for f in data.get("new_files") or []], + deleted_files=[DataFileMeta.from_dict(f) for f in data.get("deleted_files") or []], + changelog_files=[DataFileMeta.from_dict(f) for f in data.get("changelog_files") or []], + new_index_files=[_index_file_from_dict(i) for i in data.get("new_index_files") or []], + deleted_index_files=[_index_file_from_dict(i) for i in data.get("deleted_index_files") or []], + ) + + @classmethod + def _compact_increment_to_dict(cls, inc: CompactIncrement) -> Dict[str, Any]: + return { + "compact_before": [f.to_dict() for f in inc.compact_before], + "compact_after": [f.to_dict() for f in inc.compact_after], + "changelog_files": [f.to_dict() for f in inc.changelog_files], + "new_index_files": [_index_file_to_dict(i) for i in inc.new_index_files], + "deleted_index_files": [_index_file_to_dict(i) for i in inc.deleted_index_files], + } + + @classmethod + def _compact_increment_from_dict(cls, data) -> CompactIncrement: + if not data: + return CompactIncrement() + return CompactIncrement( + compact_before=[DataFileMeta.from_dict(f) for f in data.get("compact_before") or []], + compact_after=[DataFileMeta.from_dict(f) for f in data.get("compact_after") or []], + changelog_files=[DataFileMeta.from_dict(f) for f in data.get("changelog_files") or []], + new_index_files=[_index_file_from_dict(i) for i in data.get("new_index_files") or []], + deleted_index_files=[_index_file_from_dict(i) for i in data.get("deleted_index_files") or []], + ) + + +# IndexFileMeta has richer payloads (deletion vector ranges, global index +# meta) that aren't relevant to the basic compaction path yet — round-trip +# only the scalar identity fields here. Phase 6/7 (deletion vectors, +# changelog producer) will extend this to cover dv_ranges and +# global_index_meta as the rewriter starts producing them. +def _index_file_to_dict(idx: IndexFileMeta) -> Dict[str, Any]: + return { + "index_type": idx.index_type, + "file_name": idx.file_name, + "file_size": idx.file_size, + "row_count": idx.row_count, + "external_path": idx.external_path, + } + + +def _index_file_from_dict(data: Dict[str, Any]) -> IndexFileMeta: + return IndexFileMeta( + index_type=data["index_type"], + file_name=data["file_name"], + file_size=data["file_size"], + row_count=data["row_count"], + external_path=data.get("external_path"), + ) diff --git a/paimon-python/pypaimon/write/compact_increment.py b/paimon-python/pypaimon/write/compact_increment.py new file mode 100644 index 000000000000..ddf923ceb9c8 --- /dev/null +++ b/paimon-python/pypaimon/write/compact_increment.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 dataclasses import dataclass, field +from typing import List + +from pypaimon.index.index_file_meta import IndexFileMeta +from pypaimon.manifest.schema.data_file_meta import DataFileMeta + + +@dataclass +class CompactIncrement: + """Files changed before and after compaction, with changelog produced during compaction. + + Direct port of org.apache.paimon.io.CompactIncrement. + + - compact_before: input files consumed by compaction (DELETE entries). + - compact_after: rewritten output files (ADD entries). + - changelog_files: changelog files emitted while compacting (used by the + full-compaction changelog producer; empty in the basic dedup path). + - new_index_files / deleted_index_files: index file deltas attributable + to this compaction (deletion vectors / global index updates). Empty + lists by default. + """ + + compact_before: List[DataFileMeta] = field(default_factory=list) + compact_after: List[DataFileMeta] = field(default_factory=list) + changelog_files: List[DataFileMeta] = field(default_factory=list) + new_index_files: List[IndexFileMeta] = field(default_factory=list) + deleted_index_files: List[IndexFileMeta] = field(default_factory=list) + + def is_empty(self) -> bool: + return ( + not self.compact_before + and not self.compact_after + and not self.changelog_files + and not self.new_index_files + and not self.deleted_index_files + ) + + @classmethod + def empty(cls) -> "CompactIncrement": + return cls() diff --git a/paimon-python/pypaimon/write/data_increment.py b/paimon-python/pypaimon/write/data_increment.py new file mode 100644 index 000000000000..e0b1d16f98d1 --- /dev/null +++ b/paimon-python/pypaimon/write/data_increment.py @@ -0,0 +1,61 @@ +################################################################################ +# 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, field +from typing import List + +from pypaimon.index.index_file_meta import IndexFileMeta +from pypaimon.manifest.schema.data_file_meta import DataFileMeta + + +@dataclass +class DataIncrement: + """Increment of data files, changelog files and index files produced by a write. + + Direct port of org.apache.paimon.io.DataIncrement. Carries everything one + write attempt contributes to a snapshot, so a CommitMessage can be + constructed from a (DataIncrement, CompactIncrement) pair just like the + Java side. + + - new_files: data files this write created (ADD entries). + - deleted_files: data files this write removed without compaction + (e.g. row-level delete in data-evolution tables); ADD/DELETE asymmetry + is preserved by giving each list its own slot. + - changelog_files: changelog data files associated with this write. + - new_index_files / deleted_index_files: index file deltas (deletion + vectors, global index, ...). Empty lists by default. + """ + + new_files: List[DataFileMeta] = field(default_factory=list) + deleted_files: List[DataFileMeta] = field(default_factory=list) + changelog_files: List[DataFileMeta] = field(default_factory=list) + new_index_files: List[IndexFileMeta] = field(default_factory=list) + deleted_index_files: List[IndexFileMeta] = field(default_factory=list) + + def is_empty(self) -> bool: + return ( + not self.new_files + and not self.deleted_files + and not self.changelog_files + and not self.new_index_files + and not self.deleted_index_files + ) + + @classmethod + def empty(cls) -> "DataIncrement": + return cls() diff --git a/paimon-python/pypaimon/write/file_store_write.py b/paimon-python/pypaimon/write/file_store_write.py index 75b1d3a7d708..e96a35a926f9 100644 --- a/paimon-python/pypaimon/write/file_store_write.py +++ b/paimon-python/pypaimon/write/file_store_write.py @@ -22,6 +22,7 @@ from pypaimon.common.options.core_options import CoreOptions from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.data_increment import DataIncrement from pypaimon.write.writer.append_only_data_writer import AppendOnlyDataWriter from pypaimon.write.writer.data_blob_writer import DataBlobWriter from pypaimon.write.writer.data_writer import DataWriter @@ -109,7 +110,8 @@ def prepare_commit(self, commit_identifier) -> List[CommitMessage]: commit_message = CommitMessage( partition=partition, bucket=bucket, - new_files=committed_files + total_buckets=self.table.total_buckets, + data_increment=DataIncrement(new_files=committed_files), ) commit_messages.append(commit_message) return commit_messages diff --git a/paimon-python/pypaimon/write/table_update.py b/paimon-python/pypaimon/write/table_update.py index ec192a98a502..bd544d253bae 100644 --- a/paimon-python/pypaimon/write/table_update.py +++ b/paimon-python/pypaimon/write/table_update.py @@ -26,6 +26,7 @@ from pypaimon.manifest.schema.data_file_meta import DataFileMeta from pypaimon.read.split import DataSplit from pypaimon.write.commit_message import CommitMessage +from pypaimon.write.data_increment import DataIncrement from pypaimon.write.table_update_by_row_id import TableUpdateByRowId from pypaimon.write.table_upsert_by_key import TableUpsertByKey from pypaimon.write.writer.data_writer import DataWriter @@ -199,7 +200,13 @@ def arrow_reader(self) -> pyarrow.ipc.RecordBatchReader: def prepare_commit(self) -> List[CommitMessage]: commit_messages = [] for (partition, files) in self.dict.items(): - commit_messages.append(CommitMessage(partition, 0, files, self.snapshot_id)) + commit_messages.append(CommitMessage( + partition=partition, + bucket=0, + total_buckets=self.table.total_buckets, + data_increment=DataIncrement(new_files=files), + check_from_snapshot=self.snapshot_id, + )) return commit_messages def update_by_arrow_batch(self, data: pa.RecordBatch): From 68b6b3579c46b338db206c6b86ee812d29224d39 Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Wed, 6 May 2026 11:12:08 +0800 Subject: [PATCH 10/12] [python][compact] keep CommitMessageSerializer at VERSION=1 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR has not landed yet, so there is no on-disk / cross-process payload from a prior version to stay compatible with — VERSION still denotes "first shipped wire format". Bump it once we actually need to break compat with a released version. --- paimon-python/pypaimon/write/commit_message_serializer.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/paimon-python/pypaimon/write/commit_message_serializer.py b/paimon-python/pypaimon/write/commit_message_serializer.py index cfa63ad990b9..1b89830d6cc7 100644 --- a/paimon-python/pypaimon/write/commit_message_serializer.py +++ b/paimon-python/pypaimon/write/commit_message_serializer.py @@ -43,7 +43,7 @@ class CommitMessageSerializer: not need a new payload version. """ - VERSION = 2 + VERSION = 1 @classmethod def serialize(cls, message: CommitMessage) -> bytes: From a3bc32ec5e8fab2d7badecfcd265e0f7f3ddcf7a Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Wed, 6 May 2026 12:04:04 +0800 Subject: [PATCH 11/12] [python][compact] align AppendCompactCoordinator packing with Java MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Replace the count-based chunking in _pick_files_for_bucket with the size-based bin-packing algorithm Java's AppendCompactCoordinator .SubCoordinator.pack uses, so plans produced by the Python coordinator match Java's task shape on the same input: - Sort candidates by file_size ascending instead of by sequence number, so smaller files lead and the packer has the most room to grow each bin before overshooting. - Drain a bin as soon as it has >1 file AND its weighted size hits target_file_size * 2. The hardcoded ×2 is Java's "each task should yield ~2 target-sized output files" constant. - Account for source.split.open-file-cost in bin size, matching Java's per-file IO weight: a bucket of many tiny files now fans out into several tasks instead of being packed into one giant task. - Trailing bin emits only when it has at least min_file_num files; shorter tails wait for company on the next plan. full_compaction=True drops that minimum to 1 so a "rewrite this bucket" intent always produces at least one task. CompactOptions: - Drop max_file_num — Java has no such concept and size-based packing caps each task at ~2x target naturally. - Drop the now-irrelevant max>=min check; the only invariant left is min_file_num >= 1. Tests: - New append_compact_packing_test (9 cases) drives the algorithm directly with hand-built DataFileMeta lists, mirroring the kind of coverage Java's AppendCompactCoordinatorTest has for pack(). - E2E coordinator/rewriter/Ray tests now zero source.split.open-file-cost on their tiny test tables (default 4 MB would dominate the 1 KB parquet files and trigger spurious mid-loop drains). - Drop test_chunks_when_exceeding_max_file_num (max_file_num is gone) in favor of test_many_small_files_pack_into_single_task which documents the realistic tiny-file behavior. --- .../coordinator/append_compact_coordinator.py | 89 ++++++---- paimon-python/pypaimon/compact/options.py | 27 ++- .../append_compact_coordinator_test.py | 28 +-- .../tests/compact/append_compact_e2e_test.py | 5 + .../compact/append_compact_packing_test.py | 164 ++++++++++++++++++ .../compact/append_compact_rewriter_test.py | 1 + .../tests/compact/compact_options_test.py | 8 +- .../tests/compact/ray_executor_test.py | 2 + 8 files changed, 256 insertions(+), 68 deletions(-) create mode 100644 paimon-python/pypaimon/tests/compact/append_compact_packing_test.py diff --git a/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py b/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py index d3b709f492aa..0a9193dedca6 100644 --- a/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py +++ b/paimon-python/pypaimon/compact/coordinator/append_compact_coordinator.py @@ -30,16 +30,29 @@ class AppendCompactCoordinator(CompactCoordinator): """Plan compaction tasks for append-only tables (HASH_FIXED or BUCKET_UNAWARE). - For each (partition, bucket) we compact files smaller than the table's - target_file_size. A bucket is eligible only when it has at least - `min_file_num` such files (default 5) — matching the Java - AppendCompactCoordinator threshold and avoiding pointless rewrites of a - single small file. `full_compaction=True` overrides the threshold and - rewrites every file in every bucket regardless of size. - - The coordinator caps each task at `max_file_num` files; oversized buckets - produce multiple tasks so an executor can spread the work in parallel - instead of hot-spotting one worker on a huge bucket. + Per (partition, bucket) we filter to files smaller than the table's + target_file_size and bin-pack them into compaction tasks using the same + algorithm as Java AppendCompactCoordinator.SubCoordinator.pack: + + 1. Sort candidates by file size ascending (smaller files lead, so the + packer has the most flexibility to grow a bin without immediately + overshooting). + 2. Walk the sorted list adding each file to the current bin, accruing + (file_size + open_file_cost) — open_file_cost mirrors Java's per-file + IO weight so a bin of many tiny files drains earlier than naive + size accounting would suggest. + 3. Drain a bin as soon as it has >1 file AND its weighted size hits + target_file_size * 2. The ×2 is Java's hardcoded constant: each + task should produce roughly two target-sized output files, which + amortizes task setup cost while keeping output sizes predictable. + 4. The trailing bin is emitted only if it has at least min_file_num + files; smaller tails are dropped to avoid spending an entire task + on a couple of files that will collect company on the next plan. + + full_compaction=True relaxes both the size filter (large files also + enter packing) and the trailing-bin threshold (any non-empty tail + is emitted), matching the user-level intent of "rewrite this bucket + regardless of current shape". """ def __init__( @@ -71,10 +84,11 @@ def plan(self) -> List[AppendCompactTask]: bucket_files[key].append(entry.file) target_file_size = self.table.options.target_file_size(False) + open_file_cost = self.table.options.source_split_open_file_cost() tasks: List[AppendCompactTask] = [] for (partition, bucket), files in bucket_files.items(): - for chunk in self._pick_files_for_bucket(files, target_file_size): + for chunk in self._pick_files_for_bucket(files, target_file_size, open_file_cost): tasks.append( AppendCompactTask( partition=partition, @@ -89,36 +103,45 @@ def _pick_files_for_bucket( self, files: List[DataFileMeta], target_file_size: int, + open_file_cost: int, ) -> List[List[DataFileMeta]]: - """Choose which files in a single bucket get compacted, batching if needed. + """Bin-pack one bucket's files into compaction tasks. - Files >= target_file_size are skipped (they're already at output size and - rewriting them only spends IO). full_compaction overrides this skip. + Mirrors org.apache.paimon.append.AppendCompactCoordinator + .SubCoordinator.pack — see class docstring for the reasoning behind + the size-based packing, the open_file_cost weight, and the + target_file_size * 2 drain threshold. """ if self.options.full_compaction: candidates = list(files) - if not candidates: - return [] else: + # Files already at or above target size aren't worth rewriting — + # the output would be near-identical and we'd burn IO for it. candidates = [f for f in files if f.file_size < target_file_size] - if len(candidates) < self.options.min_file_num: - return [] - - # Stable order: oldest sequence first so rewrites preserve append order - # if the executor later relies on file ordering for something. - candidates.sort(key=lambda f: f.min_sequence_number) - - # NOTE: under non-full compaction, the trailing chunk may have fewer - # than min_file_num files and is dropped here — those files stay live - # and will be picked up next time more small files accumulate. This is - # a deliberate trade-off (vs. Java, which always picks them up); a - # future change can revisit it. + + if not candidates: + return [] + + candidates.sort(key=lambda f: f.file_size) + chunks: List[List[DataFileMeta]] = [] - max_per_task = self.options.max_file_num - for start in range(0, len(candidates), max_per_task): - chunk = candidates[start:start + max_per_task] - if len(chunk) >= self.options.min_file_num or self.options.full_compaction: - chunks.append(chunk) + bin_files: List[DataFileMeta] = [] + bin_size = 0 + drain_threshold = target_file_size * 2 + for f in candidates: + bin_files.append(f) + bin_size += f.file_size + open_file_cost + if len(bin_files) > 1 and bin_size >= drain_threshold: + chunks.append(bin_files) + bin_files = [] + bin_size = 0 + + # Trailing bin: under full_compaction any non-empty tail ships; + # otherwise we require min_file_num files so a tiny tail waits for + # company on the next plan instead of paying task overhead now. + min_tail = 1 if self.options.full_compaction else self.options.min_file_num + if len(bin_files) >= min_tail: + chunks.append(bin_files) return chunks def _scan_live_files(self): diff --git a/paimon-python/pypaimon/compact/options.py b/paimon-python/pypaimon/compact/options.py index 5f5a8f2cbf6d..08798195fec3 100644 --- a/paimon-python/pypaimon/compact/options.py +++ b/paimon-python/pypaimon/compact/options.py @@ -21,10 +21,9 @@ # Defaults mirror Java's append-only compaction options where possible. Only the # subset that drives append/PK planning is exposed here; per-table options -# (file format, compression, target_file_size) still come from CoreOptions on -# the table itself. +# (file format, compression, target_file_size, open_file_cost) still come from +# CoreOptions on the table itself. DEFAULT_MIN_FILE_NUM = 5 -DEFAULT_MAX_FILE_NUM = 50 DEFAULT_FORCE_FULL = False @@ -32,29 +31,28 @@ class CompactOptions: """Knobs that drive compaction planning. - target_file_size is intentionally absent — it is sourced from the table's - own CoreOptions (via DataWriter rolling) so a job inherits whatever the - writer would use, keeping output sizes consistent across write/compact. + target_file_size and open_file_cost are intentionally absent — they are + sourced from the table's own CoreOptions (target_file_size via DataWriter + rolling, open_file_cost as the per-file overhead added when computing + bin size). This keeps a job's output and packing decisions consistent + with what the regular write path would produce. + + The Java AppendCompactCoordinator's `compactionFileNumLimit` / + per-task max-file-count knobs aren't surfaced here: the size-based + bin-packing in `_pick_files_for_bucket` naturally caps each task at + ~2x target_file_size of input, which is the same shape Java produces. """ min_file_num: int = DEFAULT_MIN_FILE_NUM - max_file_num: int = DEFAULT_MAX_FILE_NUM full_compaction: bool = DEFAULT_FORCE_FULL def __post_init__(self): if self.min_file_num < 1: raise ValueError(f"min_file_num must be >= 1, got {self.min_file_num}") - if self.max_file_num < self.min_file_num: - raise ValueError( - f"max_file_num ({self.max_file_num}) must be >= " - f"min_file_num ({self.min_file_num}); silently raising it would " - f"hide the misconfiguration." - ) def to_dict(self) -> Dict[str, Any]: return { "min_file_num": self.min_file_num, - "max_file_num": self.max_file_num, "full_compaction": self.full_compaction, } @@ -64,6 +62,5 @@ def from_dict(cls, data: Optional[Dict[str, Any]]) -> "CompactOptions": return cls() return cls( min_file_num=data.get("min_file_num", DEFAULT_MIN_FILE_NUM), - max_file_num=data.get("max_file_num", DEFAULT_MAX_FILE_NUM), full_compaction=data.get("full_compaction", DEFAULT_FORCE_FULL), ) diff --git a/paimon-python/pypaimon/tests/compact/append_compact_coordinator_test.py b/paimon-python/pypaimon/tests/compact/append_compact_coordinator_test.py index 2c920008589c..66c1f3e8d145 100644 --- a/paimon-python/pypaimon/tests/compact/append_compact_coordinator_test.py +++ b/paimon-python/pypaimon/tests/compact/append_compact_coordinator_test.py @@ -52,10 +52,15 @@ def _create_unaware_table(self, table_name: str, options=None) -> "FileStoreTabl except Exception: pass # Force a small target_file_size so a few rows are already "small enough" - # to be candidates without writing thousands of rows per test. + # to be candidates without writing thousands of rows per test. We also + # zero out source.split.open-file-cost so the size-based packer's bin + # accounting degenerates to raw file_size — keeps test assertions + # crisp instead of having to reason about a 4 MB per-file overhead + # dwarfing the 1 KB test files. opts = { CoreOptions.BUCKET.key(): "-1", CoreOptions.TARGET_FILE_SIZE.key(): "1mb", + CoreOptions.SOURCE_SPLIT_OPEN_FILE_COST.key(): "0", } if options: opts.update(options) @@ -123,22 +128,19 @@ def test_full_compaction_overrides_threshold(self): "full_compaction should produce a task even below min_file_num") self.assertEqual(2, len(tasks[0].files)) - def test_chunks_when_exceeding_max_file_num(self): - table = self._create_unaware_table("chunked") + def test_many_small_files_pack_into_single_task(self): + # Real parquet files written here are ~1KB (well under the 1MB target + # set in setUp), so the size-based packer never reaches its drain + # threshold and emits a single trailing chunk containing every file. + table = self._create_unaware_table("packed_single") self._write_n_files(table, n=12) - table = self.catalog.get_table("compact_db.chunked") + table = self.catalog.get_table("compact_db.packed_single") - coordinator = AppendCompactCoordinator( - table, - CompactOptions(min_file_num=5, max_file_num=5), - ) + coordinator = AppendCompactCoordinator(table, CompactOptions(min_file_num=5)) tasks = coordinator.plan() - # 12 files, 5 per chunk → chunks of [5, 5, 2]; the last chunk (2) is - # below min_file_num so it should be dropped. - self.assertEqual(2, len(tasks)) - for t in tasks: - self.assertLessEqual(len(t.files), 5) + self.assertEqual(1, len(tasks)) + self.assertEqual(12, len(tasks[0].files)) def test_pk_table_rejected(self): full_name = "compact_db.pk_rejected" diff --git a/paimon-python/pypaimon/tests/compact/append_compact_e2e_test.py b/paimon-python/pypaimon/tests/compact/append_compact_e2e_test.py index 67043db5f465..80261e1f1e34 100644 --- a/paimon-python/pypaimon/tests/compact/append_compact_e2e_test.py +++ b/paimon-python/pypaimon/tests/compact/append_compact_e2e_test.py @@ -50,6 +50,11 @@ def _make_table(self, name: str, partitioned: bool = False): opts = { CoreOptions.BUCKET.key(): "-1", # unaware bucket CoreOptions.TARGET_FILE_SIZE.key(): "10mb", # plenty of headroom for small writes + # Zero open-file-cost so the size-based packer doesn't drain + # mid-loop on these tiny test files (each ~1 KB; with the 4 MB + # default cost a couple of files would already weigh more than + # 2x target and trigger a premature drain). + CoreOptions.SOURCE_SPLIT_OPEN_FILE_COST.key(): "0", } if partitioned: pa_schema = pa.schema([ diff --git a/paimon-python/pypaimon/tests/compact/append_compact_packing_test.py b/paimon-python/pypaimon/tests/compact/append_compact_packing_test.py new file mode 100644 index 000000000000..4d68e5b9f6f1 --- /dev/null +++ b/paimon-python/pypaimon/tests/compact/append_compact_packing_test.py @@ -0,0 +1,164 @@ +################################################################################ +# 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. +################################################################################ + +"""Pure-algorithm tests for AppendCompactCoordinator._pick_files_for_bucket. + +Drives the bin-packer with hand-built DataFileMeta lists so the size-based +packing logic can be verified independently of the storage layer. Mirrors +the test cases in Java AppendCompactCoordinatorTest's pack() coverage so +divergence shows up here first. +""" + +import unittest +from datetime import datetime +from typing import List + +from pypaimon.compact.coordinator.append_compact_coordinator import \ + AppendCompactCoordinator +from pypaimon.compact.options import CompactOptions +from pypaimon.data.timestamp import Timestamp +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.manifest.schema.simple_stats import SimpleStats +from pypaimon.table.row.generic_row import GenericRow + + +def _file(name: str, size: int) -> DataFileMeta: + return DataFileMeta.create( + file_name=name, + file_size=size, + row_count=10, + min_key=GenericRow([], []), + max_key=GenericRow([], []), + key_stats=SimpleStats.empty_stats(), + value_stats=SimpleStats.empty_stats(), + min_sequence_number=0, + max_sequence_number=10, + schema_id=0, + level=0, + extra_files=[], + creation_time=Timestamp.from_local_date_time(datetime(2024, 1, 1)), + ) + + +def _make_coord(min_file_num: int = 5, full: bool = False) -> AppendCompactCoordinator: + """Bypass __init__ — we never touch the table here, only call the pure helper.""" + coord = AppendCompactCoordinator.__new__(AppendCompactCoordinator) + coord.options = CompactOptions(min_file_num=min_file_num, full_compaction=full) + return coord + + +# Defaults matching Java's AppendCompactCoordinator +TARGET = 128 * 1024 * 1024 # 128 MB +OPEN_COST = 4 * 1024 * 1024 # 4 MB (Java's source.split.open-file-cost default) + + +class PickFilesAlgorithmTest(unittest.TestCase): + + def test_skips_files_at_or_above_target_size(self): + coord = _make_coord(min_file_num=2) + files = [_file("big-1", TARGET), _file("big-2", TARGET + 1)] + self.assertEqual([], coord._pick_files_for_bucket(files, TARGET, OPEN_COST)) + + def test_drains_bin_when_weighted_size_reaches_2x_target(self): + # 6 files of 50 MB each → weighted = 6*(50+4) = 324 MB; target*2 = 256 MB. + # Sorted ascending by size (all equal here). Walk: + # after 1 file: bin_size=54, count=1 → skip drain (bin must have >1) + # after 2 files: bin_size=108, count=2 → 108 < 256 → keep + # after 3 files: bin_size=162 → keep + # after 4 files: bin_size=216 → keep + # after 5 files: bin_size=270 → drain ✅ (chunk[0] = 5 files) + # after 6 files: bin_size=54, count=1 → tail < min_file_num=2 here + # → 1 chunk, 5 files; trailing 1 file dropped (< min_file_num=2) + coord = _make_coord(min_file_num=2) + files = [_file(f"f{i}", 50 * 1024 * 1024) for i in range(6)] + chunks = coord._pick_files_for_bucket(files, TARGET, OPEN_COST) + + self.assertEqual(1, len(chunks)) + self.assertEqual(5, len(chunks[0])) + + def test_trailing_bin_emitted_when_meets_min_file_num(self): + # 5 small files: each 10 MB. Weighted: 5*(10+4)=70 MB. Below 256 MB + # threshold → never drains mid-loop. Trailing bin has 5 files which + # equals min_file_num=5 → emitted as the only chunk. + coord = _make_coord(min_file_num=5) + files = [_file(f"f{i}", 10 * 1024 * 1024) for i in range(5)] + chunks = coord._pick_files_for_bucket(files, TARGET, OPEN_COST) + + self.assertEqual(1, len(chunks)) + self.assertEqual(5, len(chunks[0])) + + def test_trailing_bin_dropped_when_below_min_file_num(self): + coord = _make_coord(min_file_num=5) + files = [_file(f"f{i}", 10 * 1024 * 1024) for i in range(4)] + self.assertEqual([], coord._pick_files_for_bucket(files, TARGET, OPEN_COST)) + + def test_sort_by_size_ascending_lets_small_files_lead(self): + # The size-asc sort means small files accumulate first, and a single + # big file lands in the bin only once it would push past the threshold. + coord = _make_coord(min_file_num=2) + small = [_file(f"s{i}", 1 * 1024 * 1024) for i in range(3)] + large = [_file("L", 120 * 1024 * 1024)] + chunks = coord._pick_files_for_bucket(small + large, TARGET, OPEN_COST) + # After 3 smalls: weighted = 3*(1+4) = 15 MB → no drain. + # Add large: weighted = 15 + (120+4) = 139 MB; still < 256 → no drain. + # End of loop: trailing bin has 4 files → meets min_file_num=2 → emitted. + # First file in chunk should be a small one (lowest size). + self.assertEqual(1, len(chunks)) + self.assertEqual(4, len(chunks[0])) + self.assertLess(chunks[0][0].file_size, chunks[0][-1].file_size) + + def test_full_compaction_includes_files_at_target_size_and_emits_short_tails(self): + # Mix of two oversized files plus one tiny file. Without full_compaction + # the oversized files are filtered out; with full_compaction they count + # AND the trailing-bin minimum drops to 1 so even a single-file chunk + # is emitted. + coord = _make_coord(min_file_num=5, full=True) + files = [_file("big-1", TARGET), _file("big-2", TARGET + 1)] + chunks = coord._pick_files_for_bucket(files, TARGET, OPEN_COST) + # weighted = (TARGET+OPEN) + (TARGET+1+OPEN) > 2*TARGET → drain after 2nd file. + self.assertEqual(1, len(chunks)) + self.assertEqual(2, len(chunks[0])) + + def test_full_compaction_single_file_emits(self): + coord = _make_coord(min_file_num=5, full=True) + files = [_file("only", 1024)] + chunks = coord._pick_files_for_bucket(files, TARGET, OPEN_COST) + self.assertEqual(1, len(chunks)) + self.assertEqual(1, len(chunks[0])) + + def test_open_file_cost_pulls_drain_forward_for_many_tiny_files(self): + # 80 tiny 100KB files. Without open_file_cost weighting, weighted size + # is ~8 MB total — far below 256 MB threshold, never drains, all 80 + # land in one task. With open_file_cost=4 MB Java-style: weighted per + # file ≈ 4 MB, so drain triggers around bin size 64 (= 256/4) files. + coord = _make_coord(min_file_num=2) + files = [_file(f"t{i}", 100 * 1024) for i in range(80)] + chunks = coord._pick_files_for_bucket(files, TARGET, OPEN_COST) + self.assertGreaterEqual(len(chunks), 2, + "open_file_cost must split a 'many tiny files' bucket") + # Every emitted bin should hold > 1 file (the > 1 guard in the loop). + for c in chunks: + self.assertGreater(len(c), 1) + + def test_empty_input_returns_empty(self): + coord = _make_coord() + self.assertEqual([], coord._pick_files_for_bucket([], TARGET, OPEN_COST)) + + +if __name__ == "__main__": + unittest.main() diff --git a/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py b/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py index 0e377f69c534..3a362f76bd71 100644 --- a/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py +++ b/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py @@ -51,6 +51,7 @@ def _make_unaware_table(self, name: str): opts = { CoreOptions.BUCKET.key(): "-1", CoreOptions.TARGET_FILE_SIZE.key(): "10mb", + CoreOptions.SOURCE_SPLIT_OPEN_FILE_COST.key(): "0", } pa_schema = pa.schema([("id", pa.int32()), ("name", pa.string())]) schema = Schema.from_pyarrow_schema(pa_schema, options=opts) diff --git a/paimon-python/pypaimon/tests/compact/compact_options_test.py b/paimon-python/pypaimon/tests/compact/compact_options_test.py index 6217c93871ea..32ae7b8422d7 100644 --- a/paimon-python/pypaimon/tests/compact/compact_options_test.py +++ b/paimon-python/pypaimon/tests/compact/compact_options_test.py @@ -26,20 +26,14 @@ class CompactOptionsTest(unittest.TestCase): def test_defaults(self): opts = CompactOptions() self.assertEqual(5, opts.min_file_num) - self.assertEqual(50, opts.max_file_num) self.assertFalse(opts.full_compaction) - def test_max_below_min_rejected(self): - # Silently raising max to min would mask user misconfiguration. - with self.assertRaises(ValueError): - CompactOptions(min_file_num=5, max_file_num=3) - def test_min_zero_rejected(self): with self.assertRaises(ValueError): CompactOptions(min_file_num=0) def test_to_from_dict_roundtrip(self): - opts = CompactOptions(min_file_num=2, max_file_num=10, full_compaction=True) + opts = CompactOptions(min_file_num=2, full_compaction=True) rebuilt = CompactOptions.from_dict(opts.to_dict()) self.assertEqual(opts, rebuilt) diff --git a/paimon-python/pypaimon/tests/compact/ray_executor_test.py b/paimon-python/pypaimon/tests/compact/ray_executor_test.py index cfb8b63c5f34..912bfa02323e 100644 --- a/paimon-python/pypaimon/tests/compact/ray_executor_test.py +++ b/paimon-python/pypaimon/tests/compact/ray_executor_test.py @@ -67,6 +67,8 @@ def _make_unaware_table(self, name: str): opts = { CoreOptions.BUCKET.key(): "-1", CoreOptions.TARGET_FILE_SIZE.key(): "10mb", + # See AppendCompactE2ETest for why open-file-cost is zeroed here. + CoreOptions.SOURCE_SPLIT_OPEN_FILE_COST.key(): "0", } pa_schema = pa.schema([("id", pa.int32()), ("name", pa.string())]) schema = Schema.from_pyarrow_schema(pa_schema, options=opts) From 808f6426c740e5fb6bd29d5f2d2ad29e51a9e4bc Mon Sep 17 00:00:00 2001 From: TheR1sing3un Date: Wed, 6 May 2026 12:29:13 +0800 Subject: [PATCH 12/12] [python][compact] align AppendCompactRewriter sequence semantics with Java MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Java BaseAppendFileStoreWrite.compactRewrite seeds its RowDataRollingFileWriter with LongCounter(toCompact[0].minSequenceNumber()) and increments per row written. Each rolled output file therefore carries a precise [first_row_seq, last_row_seq] range and the union across all output files is contiguous: [seed, seed + total_input_rows - 1]. The previous Python rewriter: - seeded the writer with bucket-mode-dependent values (UNAWARE→0, HASH_FIXED→max(input.max_seq)) which had no Java analog; - never advanced sequence_generator.current — so every committed file ended up with min_seq == max_seq, i.e. compact output threw away the per-row seq information Java preserves. This commit introduces AppendCompactRollingWriter, an AppendOnlyDataWriter subclass that: - treats sequence_generator.start as Java's "next-to-assign" counter, so a slice of N rows is laid out as [seq_start, seq_start + N - 1]; - works around the base SequenceGenerator's off-by-one quirk by setting current = seq_end before super()._write_data_to_file (so the parent reads min/max as seq_start/seq_end exactly) and bumping both fields to seq_end + 1 afterwards (so the next slice picks up where this one ended); - stamps file_source = COMPACT on the just-appended DataFileMeta, the same shape MergeTreeRollingWriter uses on the PK side, instead of mutating it back in the rewriter. Rewriter: - seed_seq = files[0].min_sequence_number, matching Java's toCompact.get(0).minSequenceNumber(); - drops the bucket-mode-dependent _initial_max_seq helper. Tests: - new test_output_seq_range_starts_at_input0_min_seq_and_spans_total_rows enforces the Java contract: per-file (max - min + 1 == row_count) and cross-file (no gaps/overlaps, range = [seed, seed+total-1]); - new test_output_files_tagged_compact_source verifies file_source is set by the writer, not the rewriter. Out of scope (still NOTEd in the rewriter docstring): Java's reader path runs through ReadForCompact for schema-evolution + DV awareness; pypaimon still reads parquet directly. Both will be wired up alongside the broader schema-evolution / deletion-vector phases. --- .../rewriter/append_compact_rewriter.py | 119 +++++++++++++----- .../compact/append_compact_rewriter_test.py | 55 ++++++++ 2 files changed, 140 insertions(+), 34 deletions(-) diff --git a/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py b/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py index 38642a536756..290e1cc5869e 100644 --- a/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py +++ b/paimon-python/pypaimon/compact/rewriter/append_compact_rewriter.py @@ -21,29 +21,90 @@ import pyarrow as pa import pyarrow.dataset as ds +from pypaimon.compact.rewriter.merge_tree_rolling_writer import \ + FILE_SOURCE_COMPACT from pypaimon.compact.rewriter.rewriter import CompactRewriter from pypaimon.manifest.schema.data_file_meta import DataFileMeta from pypaimon.read.split_read import format_identifier -from pypaimon.table.bucket_mode import BucketMode from pypaimon.write.writer.append_only_data_writer import AppendOnlyDataWriter +class AppendCompactRollingWriter(AppendOnlyDataWriter): + """AppendOnlyDataWriter variant that mirrors Java RowDataRollingFileWriter + used by BaseAppendFileStoreWrite.compactRewrite. + + Two behavior tweaks vs. the regular append writer: + + 1. Sequence number bookkeeping. The base AppendOnlyDataWriter never + advances `sequence_generator.current`, which is fine for plain + writes (where every committed file ends up with min_seq == max_seq + == the batch's seed) but wrong for compaction output. Java's + LongCounter is incremented per row written, so each rolled output + file ends up with precisely [first_row_seq, last_row_seq] in its + metadata. We replicate that here, treating the seed value as the + seq for the first row written: file N covering R rows gets + [next_seq, next_seq + R - 1] in its meta. The base writer's + SequenceGenerator has an off-by-one quirk (current is "+1 after + last assigned"), so we set up the generator each call to make + super()._write_data_to_file emit the exact bounds we want without + touching the parent class. + + 2. Provenance. Java passes FileSource.COMPACT into the rewriter + constructor. The base writer hardcodes file_source=APPEND in + _write_data_to_file, so we patch the just-appended DataFileMeta + afterwards (same shape MergeTreeRollingWriter uses on the PK side). + """ + + def _write_data_to_file(self, data: pa.Table) -> None: + n = data.num_rows + if n == 0: + return + # `start` is treated as "next-to-assign" seq, matching Java's + # LongCounter semantics. The slice we're about to write covers + # [seq_start, seq_end]. + seq_start = self.sequence_generator.start + seq_end = seq_start + n - 1 + # Drive the parent's metadata accounting: parent reads + # min_seq = start, max_seq = current + # then sets start = current. Putting current at seq_end yields + # exactly the [seq_start, seq_end] range Java would have written. + self.sequence_generator.current = seq_end + + before = len(self.committed_files) + super()._write_data_to_file(data) + # Advance both fields past this batch so the next slice starts at + # seq_end + 1 (the parent only moved start to current, which is + # also seq_end now — we want next_to_assign, not last_assigned). + self.sequence_generator.start = seq_end + 1 + self.sequence_generator.current = seq_end + 1 + + # Stamp provenance on whatever the parent appended (rolling may emit + # multiple files in one super call in principle; loop defensively). + for i in range(before, len(self.committed_files)): + self.committed_files[i].file_source = FILE_SOURCE_COMPACT + + class AppendCompactRewriter(CompactRewriter): - """Reads input append-only files and re-writes them via the table's normal - AppendOnlyDataWriter, leveraging its built-in target_file_size rolling so - output files are sized consistently with what the table would produce on a - fresh INSERT. - - Notes on sequence numbers: append-only writers do not advance the per-row - sequence generator (only PK writers do). To stay consistent with the - write path (FileStoreWrite._create_data_writer), we seed the generator - with 0 for BUCKET_UNAWARE tables and with max(input.max_sequence_number) - for HASH_FIXED tables. - - NOTE (deferred): schema evolution across input files is not handled — we - feed batches at their on-disk schema directly into the writer. Inputs - spanning a schema change will be addressed alongside the broader schema - evolution support in a later phase. + """Reads input append-only files and re-writes them via a Java-aligned + rolling writer. Direct port of org.apache.paimon.operation + .BaseAppendFileStoreWrite.compactRewrite. + + Sequence numbers: the rolling writer's counter is seeded with + files[0].min_sequence_number (matching Java + `LongCounter(toCompact.get(0).minSequenceNumber())`) and bumped per + row written. Output files therefore carry contiguous, non-overlapping + seq ranges starting from that seed — exactly what Java produces. + + NOTE (deferred): + - Schema evolution: input batches are read straight off disk via + pyarrow.dataset, not through table-aware ReadForCompact. Inputs + spanning a schema change need the read path to evolve them; that's + part of the schema-evolution work scheduled later. + - Deletion vectors: Java's compactRewrite accepts a dvFactory that + applies per-file DV during read and persists the resulting index + delta into CompactIncrement.{newIndexFiles, deletedIndexFiles}. + Pypaimon's compaction path will plug DV in alongside the broader + DV support phase. """ def __init__(self, table): @@ -58,12 +119,17 @@ def rewrite( if not files: return [] - max_seq = self._initial_max_seq(files) - writer = AppendOnlyDataWriter( + # Java seeds the rolling writer's counter from the first file in the + # (size-sorted) input. files comes from AppendCompactCoordinator + # which sorts by size ascending — same shape Java's pack() produces — + # so files[0] is the smallest input file's min_seq. + seed_seq = files[0].min_sequence_number + + writer = AppendCompactRollingWriter( table=self.table, partition=partition, bucket=bucket, - max_seq_number=max_seq, + max_seq_number=seed_seq, options=self.table.options, write_cols=None, ) @@ -81,23 +147,8 @@ def rewrite( finally: writer.close() - for f in new_files: - # Mark provenance — Java's CompactWriter also tags compact output. - f.file_source = 1 # 1 = COMPACT in pypaimon DataFileMeta convention - return new_files - def _initial_max_seq(self, files: List[DataFileMeta]) -> int: - """Pick the writer's seed sequence number consistent with the write path. - - Mirrors FileStoreWrite._create_data_writer: BUCKET_UNAWARE always seeds - with 0; bucketed append seeds with max(input.max_seq) so subsequent - writes / reads observe a monotonically advancing seq. - """ - if self.table.bucket_mode() == BucketMode.BUCKET_UNAWARE: - return 0 - return max(f.max_sequence_number for f in files) - def _read_input_batches( self, partition: Tuple, diff --git a/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py b/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py index 3a362f76bd71..e2c012d9c6ec 100644 --- a/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py +++ b/paimon-python/pypaimon/tests/compact/append_compact_rewriter_test.py @@ -91,6 +91,61 @@ def test_does_not_mutate_input_metadata(self): # Rewriter must not write file_path back onto manifest-owned objects. self.assertEqual(original_paths, [f.file_path for f in files]) + def test_output_seq_range_starts_at_input0_min_seq_and_spans_total_rows(self): + # Mirrors Java BaseAppendFileStoreWrite.compactRewrite seeding the + # rolling writer's counter with toCompact.get(0).minSequenceNumber() + # and bumping it once per row written. After compact, the union of + # output [min_seq, max_seq] ranges must be: + # [files[0].min_seq, files[0].min_seq + total_input_rows - 1] + # contiguous, no gaps, no overlap. + table = self._make_unaware_table("seq_range") + self._write_n(table, n=5) # 5 files, 1 row each → 5 rows total + table = self.catalog.get_table("rw_db.seq_range") + coord = AppendCompactCoordinator(table, CompactOptions(min_file_num=5)) + tasks = coord.plan() + self.assertEqual(1, len(tasks)) + + # files arrive size-asc-sorted; in unaware tables every write seeded + # at seq=0, so files[0].min_sequence_number is also 0 here. + seed = tasks[0].files[0].min_sequence_number + total_rows = sum(f.row_count for f in tasks[0].files) + + rewriter = AppendCompactRewriter(table) + new_files = rewriter.rewrite(tasks[0].partition, tasks[0].bucket, list(tasks[0].files)) + + self.assertGreater(len(new_files), 0) + self.assertEqual(total_rows, sum(f.row_count for f in new_files), + "Compact must preserve total row count") + # Per-file invariant: max - min + 1 == row_count (the rolling writer + # advanced exactly once per row in this file's slice). + for f in new_files: + self.assertEqual(f.row_count, f.max_sequence_number - f.min_sequence_number + 1, + f"file {f.file_name} seq range must match its row_count") + # Cross-file invariant: starts at seed, no gaps/overlaps when sorted. + sorted_out = sorted(new_files, key=lambda f: f.min_sequence_number) + self.assertEqual(seed, sorted_out[0].min_sequence_number) + for prev, curr in zip(sorted_out, sorted_out[1:]): + self.assertEqual(prev.max_sequence_number + 1, curr.min_sequence_number, + "Adjacent output files must form a contiguous seq range") + # Total upper bound matches Java: seed + total_rows - 1. + self.assertEqual(seed + total_rows - 1, sorted_out[-1].max_sequence_number) + + def test_output_files_tagged_compact_source(self): + from pypaimon.compact.rewriter.merge_tree_rolling_writer import \ + FILE_SOURCE_COMPACT + table = self._make_unaware_table("source_tag") + self._write_n(table, n=5) + table = self.catalog.get_table("rw_db.source_tag") + coord = AppendCompactCoordinator(table, CompactOptions(min_file_num=5)) + tasks = coord.plan() + + new_files = AppendCompactRewriter(table).rewrite( + tasks[0].partition, tasks[0].bucket, list(tasks[0].files), + ) + self.assertGreater(len(new_files), 0) + for f in new_files: + self.assertEqual(FILE_SOURCE_COMPACT, f.file_source) + def test_aborts_partial_output_on_failure(self): table = self._make_unaware_table("abort_on_failure") self._write_n(table, n=5)