diff --git a/.github/workflows/paimon-python-checks.yml b/.github/workflows/paimon-python-checks.yml old mode 100644 new mode 100755 index 789e3dd9d619..80277e701cfc --- a/.github/workflows/paimon-python-checks.yml +++ b/.github/workflows/paimon-python-checks.yml @@ -32,6 +32,9 @@ on: env: PYTHON_VERSIONS: "['3.6.15', '3.10']" + JDK_VERSION: 8 + MAVEN_OPTS: -Dmaven.wagon.httpconnectionManager.ttlSeconds=30 -Dmaven.wagon.http.retryHandler.requestSentEnabled=true + concurrency: group: ${{ github.workflow }}-${{ github.event_name }}-${{ github.event.number || github.run_id }} @@ -49,6 +52,17 @@ jobs: - name: Checkout code uses: actions/checkout@v2 + - name: Set up JDK ${{ env.JDK_VERSION }} + uses: actions/setup-java@v4 + with: + java-version: ${{ env.JDK_VERSION }} + distribution: 'temurin' + + - name: Set up Maven + uses: stCarolas/setup-maven@v4.5 + with: + maven-version: 3.8.8 + - name: Install system dependencies shell: bash run: | @@ -58,19 +72,29 @@ jobs: curl \ && rm -rf /var/lib/apt/lists/* + - name: Verify Java and Maven installation + run: | + java -version + mvn -version + - name: Verify Python version run: python --version + - name: Build Java + run: | + echo "Start compiling modules" + mvn -T 2C -B clean install -DskipTests + - name: Install Python dependencies shell: bash run: | if [[ "${{ matrix.python-version }}" == "3.6.15" ]]; then python -m pip install --upgrade pip==21.3.1 python --version - python -m pip install -q readerwriterlock==1.0.9 'fsspec==2021.10.1' 'cachetools==4.2.4' 'ossfs==2021.8.0' pyarrow==6.0.1 pandas==1.1.5 'polars==0.9.12' 'fastavro==1.4.7' zstandard==0.19.0 dataclasses==0.8.0 flake8 pytest py4j==0.10.9.9 requests 2>&1 >/dev/null + python -m pip install -q readerwriterlock==1.0.9 'fsspec==2021.10.1' 'cachetools==4.2.4' 'ossfs==2021.8.0' pyarrow==6.0.1 pandas==1.1.5 'polars==0.9.12' 'fastavro==1.4.7' zstandard==0.19.0 dataclasses==0.8.0 flake8 pytest py4j==0.10.9.9 requests parameterized==0.8.1 2>&1 >/dev/null else python -m pip install --upgrade pip - python -m pip install -q readerwriterlock==1.0.9 fsspec==2024.3.1 cachetools==5.3.3 ossfs==2023.12.0 ray==2.48.0 fastavro==1.11.1 pyarrow==16.0.0 zstandard==0.24.0 polars==1.32.0 duckdb==1.3.2 numpy==1.24.3 pandas==2.0.3 flake8==4.0.1 pytest~=7.0 py4j==0.10.9.9 requests 2>&1 >/dev/null + python -m pip install -q readerwriterlock==1.0.9 fsspec==2024.3.1 cachetools==5.3.3 ossfs==2023.12.0 ray==2.48.0 fastavro==1.11.1 pyarrow==16.0.0 zstandard==0.24.0 polars==1.32.0 duckdb==1.3.2 numpy==1.24.3 pandas==2.0.3 flake8==4.0.1 pytest~=7.0 py4j==0.10.9.9 requests parameterized==0.9.0 2>&1 >/dev/null fi - name: Run lint-python.sh shell: bash diff --git a/docs/config.toml b/docs/config.toml index 41efc7993013..32f67796d868 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -baseURL = '//paimon.apache.org/docs/master' +baseURL = '//paimon.apache.org/docs/1.3' languageCode = 'en-us' title = 'Apache Paimon' enableGitInfo = false @@ -24,7 +24,7 @@ pygmentsUseClasses = true [params] # Flag whether this is a stable version or not. # Used for the quickstart page. - IsStable = false + IsStable = true # Flag to indicate whether an outdated warning should be shown. ShowOutDatedWarning = false @@ -34,14 +34,14 @@ pygmentsUseClasses = true # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) - Version = "1.3-SNAPSHOT" + Version = "1.3.0" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version - VersionTitle = "1.3-SNAPSHOT" + VersionTitle = "1.3" # The branch for this version of Apache Paimon - Branch = "master" + Branch = "1.3" # The most recent supported Apache Flink version FlinkVersion = "1.20" @@ -67,14 +67,14 @@ pygmentsUseClasses = true ["JavaDocs", "//paimon.apache.org/docs/master/api/java/"], ] - StableDocs = "https://paimon.apache.org/docs/1.0" + StableDocs = "https://paimon.apache.org/docs/1.3" PreviousDocs = [ ["master", "https://paimon.apache.org/docs/master"], - ["stable", "https://paimon.apache.org/docs/1.2"], + ["stable", "https://paimon.apache.org/docs/1.3"], + ["1.3", "https://paimon.apache.org/docs/1.3"], ["1.2", "https://paimon.apache.org/docs/1.2"], ["1.1", "https://paimon.apache.org/docs/1.1"], - ["1.0", "https://paimon.apache.org/docs/1.0"], ] BookSection = '/' diff --git a/docs/content/append-table/incremental-clustering.md b/docs/content/append-table/incremental-clustering.md new file mode 100644 index 000000000000..aa72a348fce5 --- /dev/null +++ b/docs/content/append-table/incremental-clustering.md @@ -0,0 +1,175 @@ +--- +title: "Incremental Clustering" +weight: 4 +type: docs +aliases: +- /append-table/incremental-clustering.html +--- + + +# Incremental Clustering + +Paimon currently supports ordering append tables using SFC (Space-Filling Curve)(see [sort compact]({{< ref "maintenance/dedicated-compaction#sort-compact" >}}) for more info). +The resulting data layout typically delivers better performance for queries that target clustering keys. +However, with the current SortCompaction, even when neither the data nor the clustering keys have changed, +each run still rewrites the entire dataset, which is extremely costly. + +To address this, Paimon introduced a more flexible, incremental clustering mechanism—Incremental Clustering. +On each run, it selects only a specific subset of files to cluster, avoiding a full rewrite. This enables low-cost, +sort-based optimization of the data layout and improves query performance. In addition, with Incremental Clustering, +you can adjust clustering keys without rewriting existing data, the layout evolves dynamically as cluster runs and +gradually converges to an optimal state, significantly reducing the decision-making complexity around data layout. + + +Incremental Clustering supports: +- Support incremental clustering; minimizing write amplification as possible. +- Support small-file compaction; during rewrites, respect target-file-size. +- Support changing clustering keys; newly ingested data is clustered according to the latest clustering keys. +- Provide a full mode; when selected, the entire dataset will be reclustered. + +**Only append unaware-bucket table supports Incremental Clustering.** + +## Enable Incremental Clustering + +To enable Incremental Clustering, the following configuration needs to be set for the table: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
OptionValueRequiredTypeDescription
clustering.incremental
trueYesBooleanMust be set to true to enable incremental clustering. Default is false.
clustering.columns
'clustering-columns'YesStringThe clustering columns, in the format 'columnName1,columnName2'. It is not recommended to use partition keys as clustering keys.
clustering.strategy
'zorder' or 'hilbert' or 'order'NoStringThe ordering algorithm used for clustering. If not set, It'll decided from the number of clustering columns. 'order' is used for 1 column, 'zorder' for less than 5 columns, and 'hilbert' for 5 or more columns.
+ +Once Incremental Clustering for a table is enabled, you can run Incremental Clustering in batch mode periodically +to continuously optimizes data layout of the table and deliver better query performance. + +**Note**: Since common compaction also rewrites files, it may disrupt the ordered data layout built by Incremental Clustering. +Therefore, when Incremental Clustering is enabled, the table no longer supports write-time compaction or dedicated compaction; +clustering and small-file merging must be performed exclusively via Incremental Clustering runs. + +## Run Incremental Clustering +{{< hint info >}} + +only support running Incremental Clustering in batch mode. + +{{< /hint >}} + +To run a Incremental Clustering job, follow these instructions. + +You don’t need to specify any clustering-related parameters when running Incremental Clustering, +these options are already defined as table options. If you need to change clustering settings, please update the corresponding table options. + +{{< tabs "incremental-clustering" >}} + +{{< tab "Spark SQL" >}} + +Run the following sql: + +```sql +--set the write parallelism, if too big, may generate a large number of small files. +SET spark.sql.shuffle.partitions=10; + +-- run incremental clustering +CALL sys.compact(table => 'T') + +-- run incremental clustering with full mode, this will recluster all data +CALL sys.compact(table => 'T', compact_strategy => 'full') +``` +{{< /tab >}} + +{{< tab "Flink Action" >}} + +Run the following command to submit a incremental clustering job for the table. + +```bash +/bin/flink run \ + /path/to/paimon-flink-action-{{< version >}}.jar \ + compact \ + --warehouse \ + --database \ + --table \ + [--compact_strategy ] \ + [--table_conf ] \ + [--catalog_conf [--catalog_conf ...]] +``` + +Example: run incremental clustering + +```bash +/bin/flink run \ + /path/to/paimon-flink-action-{{< version >}}.jar \ + compact \ + --warehouse s3:///path/to/warehouse \ + --database test_db \ + --table test_table \ + --table_conf sink.parallelism=2 \ + --compact_strategy minor \ + --catalog_conf s3.endpoint=https://****.com \ + --catalog_conf s3.access-key=***** \ + --catalog_conf s3.secret-key=***** +``` +* `--compact_strategy` Determines how to pick files to be cluster, the default is `minor`. + * `full` : All files will be selected for clustered. + * `minor` : Pick the set of files that need to be clustered based on specified conditions. + +Note: write parallelism is set by `sink.parallelism`, if too big, may generate a large number of small files. + +You can use `-D execution.runtime-mode=batch` or `-yD execution.runtime-mode=batch` (for the ON-YARN scenario) to use batch mode. +{{< /tab >}} + +{{< /tabs >}} + +## Implement +To balance write amplification and sorting effectiveness, Paimon leverages the LSM Tree notion of levels to stratify data files +and uses the Universal Compaction strategy to select files for clustering. +- Newly written data lands in level-0; files in level-0 are unclustered. +- All files in level-i are produced by sorting within the same sorting set. +- By analogy with Universal Compaction: in level-0, each file is a sorted run; in level-i, all files together constitute a single sorted run. During clustering, the sorted run is the basic unit of work. + +By introducing more levels, we can control the amount of data processed in each clustering run. +Data at higher levels is more stably clustered and less likely to be rewritten, thereby mitigating write amplification while maintaining good sorting effectiveness. diff --git a/docs/content/append-table/row-tracking.md b/docs/content/append-table/row-tracking.md index 04aec8bc5881..fff59f5edfeb 100644 --- a/docs/content/append-table/row-tracking.md +++ b/docs/content/append-table/row-tracking.md @@ -26,9 +26,9 @@ under the License. # Row tracking -Row tracking allows Paimon to track row-level lineage in a Paimon append table. Once enabled on a Paimon table, two more hidden columns will be added to the table schema: -- `_ROW_ID`: BIGINT, this is a unique identifier for each row in the table. It is used to track the lineage of the row and can be used to identify the row in case of update, merge into or delete. -- `_SEQUENCE_NUMBER`: BIGINT, this is field indicates which `version` of this record is. It actually is the snapshot-id of the snapshot that this row belongs to. It is used to track the lineage of the row version. +Row tracking allows Paimon to track row-level tracking in a Paimon append table. Once enabled on a Paimon table, two more hidden columns will be added to the table schema: +- `_ROW_ID`: BIGINT, this is a unique identifier for each row in the table. It is used to track the update of the row and can be used to identify the row in case of update, merge into or delete. +- `_SEQUENCE_NUMBER`: BIGINT, this is field indicates which `version` of this record is. It actually is the snapshot-id of the snapshot that this row belongs to. It is used to track the update of the row version. Hidden columns follows the following rules: - Whenever we read from one table with row tracking enabled, the `_ROW_ID` and `_SEQUENCE_NUMBER` will be `NOT NULL`. @@ -57,7 +57,7 @@ CREATE TABLE t (id INT, data STRING) TBLPROPERTIES ('row-tracking.enabled' = 'tr INSERT INTO t VALUES (11, 'a'), (22, 'b') ``` -You can select the row lineage meta column with the following sql in spark: +You can select the row tracking meta column with the following sql in spark: ```sql SELECT id, data, _ROW_ID, _SEQUENCE_NUMBER FROM t; ``` diff --git a/docs/content/concepts/functions.md b/docs/content/concepts/functions.md index e66cd07762b4..4430e37c3af0 100644 --- a/docs/content/concepts/functions.md +++ b/docs/content/concepts/functions.md @@ -86,4 +86,4 @@ This statement deletes the existing `parse_str` function from the `mydb` databas ## Functions in Spark -see [SQL Functions]({{< ref "spark/sql-functions#user-defined-function" >}}) \ No newline at end of file +see [SQL Functions]({{< ref "spark/sql-functions#user-defined-function" >}}) diff --git a/docs/content/concepts/rest/dlf.md b/docs/content/concepts/rest/dlf.md index 9510fa2431c8..d093ad07dfed 100644 --- a/docs/content/concepts/rest/dlf.md +++ b/docs/content/concepts/rest/dlf.md @@ -3,8 +3,9 @@ title: "DLF Token" weight: 3 type: docs aliases: -- /concepts/rest/dlf.html + - /concepts/rest/dlf.html --- + + +# Views + +A view is a logical table that encapsulates business logic and domain-specific semantics. +While most compute engines support views natively, each engine stores view metadata in proprietary formats, creating interoperability challenges across different platforms. +Paimon views abstracting engine-specific query dialects and establishing unified metadata standards. +View metadata could enable centralized view management that facilitates cross-engine sharing and reduces maintenance complexity in heterogeneous computing environments. + +## Catalog support + +View metadata is persisted only when the catalog implementation supports it: + +- **Hive metastore catalog** – view metadata is stored together with table metadata inside the + metastore warehouse. +- **REST catalog** – view metadata is kept in the REST backend and exposed through the catalog API. + +File-system catalogs do not currently support views because they lack persistent metadata storage. + + +### Representation structure + +| Field | Type | Description | +|-----------|------|-------------| +| `query` | `string` | Canonical SQL `SELECT` statement that defines the view. | +| `dialect` | `string` | SQL dialect identifier (for example, `spark` or `flink`). | + +Multiple representations can be stored for the same version so that different engines can access the +view using their native dialect. + +## Operations + +### Create or replace view + +Use `CREATE VIEW` or `CREATE OR REPLACE VIEW` to register a view. Paimon assigns a UUID, writes the +first metadata file, and records version `1`. + +```sql +CREATE VIEW sales_view AS +SELECT region, SUM(amount) AS total_amount +FROM sales +GROUP BY region; +``` + +### Alter view dialect via procedure + +Paimon provides the `sys.alter_view_dialect` procedure so that engines can manage multiple SQL +representations for the same view version. + +#### Flink example + +```sql +-- Add a Flink dialect +CALL [catalog.]sys.alter_view_dialect('view_identifier', 'add', 'flink', 'SELECT ...'); + +-- Update the stored Flink dialect +CALL [catalog.]sys.alter_view_dialect('view_identifier', 'update', 'flink', 'SELECT ...'); + +-- Drop the Flink dialect representation +CALL [catalog.]sys.alter_view_dialect('view_identifier', 'drop', 'flink'); +``` + +#### Spark example + +```sql +-- Add a Spark dialect +CALL sys.alter_view_dialect('view_identifier', 'add', 'spark', 'SELECT ...'); + +-- Update the Spark dialect +CALL sys.alter_view_dialect('view_identifier', 'update', 'spark', 'SELECT ...'); + +-- Drop the Spark dialect +CALL sys.alter_view_dialect('view_identifier', 'drop', 'spark'); +``` + +### Drop view + +`DROP VIEW view_name;` + +## See also + +- [Spark SQL DDL – Views]({{< ref "spark/sql-ddl#view" >}}) +- [REST Catalog Overview]({{< ref "concepts/rest/overview" >}}) +- [REST Catalog View API]({{< ref "concepts/rest/rest-api" >}}) diff --git a/docs/content/flink/sql-ddl.md b/docs/content/flink/sql-ddl.md index 462670427076..21845a59bec3 100644 --- a/docs/content/flink/sql-ddl.md +++ b/docs/content/flink/sql-ddl.md @@ -213,7 +213,7 @@ note that partition fields and primary key fields can not be specified. ## Create Table As Select -Table can be created and populated by the results of a query, for example, we have a sql like this: `CREATE TABLE table_b AS SELECT id, name FORM table_a`, +Table can be created and populated by the results of a query, for example, we have a sql like this: `CREATE TABLE table_b AS SELECT id, name FROM table_a`, The resulting table `table_b` will be equivalent to create the table and insert the data with the following statement: `CREATE TABLE table_b (id INT, name STRING); INSERT INTO table_b SELECT id, name FROM table_a;` diff --git a/docs/content/iceberg/overview.md b/docs/content/iceberg/overview.md index 8a953aa4a0df..952bab40083e 100644 --- a/docs/content/iceberg/overview.md +++ b/docs/content/iceberg/overview.md @@ -70,7 +70,7 @@ Set the following table options, so that Paimon tables can generate Iceberg comp -For most SQL users, we recommend setting `'metadata.iceberg.storage' = 'hadoop-catalog'` +For most SQL users, we recommend setting `'metadata.iceberg.storage' = 'hadoop-catalog' or `'metadata.iceberg.storage' = 'hive-catalog'`, so that all tables can be visited as an Iceberg warehouse. For Iceberg Java API users, you might consider setting `'metadata.iceberg.storage' = 'table-location'`, @@ -84,22 +84,28 @@ If not set, the default behavior depends on the storage type. Paimon Iceberg compatibility currently supports the following data types. | Paimon Data Type | Iceberg Data Type | -|------------------|-------------------| -| `BOOLEAN` | `boolean` | -| `INT` | `int` | -| `BIGINT` | `long` | -| `FLOAT` | `float` | -| `DOUBLE` | `double` | -| `DECIMAL` | `decimal` | -| `CHAR` | `string` | -| `VARCHAR` | `string` | -| `BINARY` | `binary` | -| `VARBINARY` | `binary` | -| `DATE` | `date` | -| `TIMESTAMP`* | `timestamp` | -| `TIMESTAMP_LTZ`* | `timestamptz` | -| `ARRAY` | `list` | -| `MAP` | `map` | -| `ROW` | `struct` | +|----------------|-------------------| +| `BOOLEAN` | `boolean` | +| `INT` | `int` | +| `BIGINT` | `long` | +| `FLOAT` | `float` | +| `DOUBLE` | `double` | +| `DECIMAL` | `decimal` | +| `CHAR` | `string` | +| `VARCHAR` | `string` | +| `BINARY` | `binary` | +| `VARBINARY` | `binary` | +| `DATE` | `date` | +| `TIMESTAMP` (precision 4-6) | `timestamp` | +| `TIMESTAMP_LTZ` (precision 4-6) | `timestamptz` | +| `TIMESTAMP` (precision 7-9) | `timestamp_ns` | +| `TIMESTAMP_LTZ` (precision 7-9) | `timestamptz_ns` | +| `ARRAY` | `list` | +| `MAP` | `map` | +| `ROW` | `struct` | -*: `TIMESTAMP` and `TIMESTAMP_LTZ` type only support precision from 4 to 6 +{{< hint info >}} +**Note on Timestamp Types:** +- `TIMESTAMP` and `TIMESTAMP_LTZ` types with precision from 4 to 6 are mapped to standard Iceberg timestamp types +- `TIMESTAMP` and `TIMESTAMP_LTZ` types with precision from 7 to 9 use nanosecond precision and require Iceberg v3 format +{{< /hint >}} \ No newline at end of file diff --git a/docs/content/maintenance/manage-partitions.md b/docs/content/maintenance/manage-partitions.md index 7b851fe0554a..42d6f4601b38 100644 --- a/docs/content/maintenance/manage-partitions.md +++ b/docs/content/maintenance/manage-partitions.md @@ -166,5 +166,5 @@ enable this by setting `'partition.mark-done-action.mode' = 'watermark'`. ### Batch Mark Done -For batch mode, you can trigger partition mark done when end input by setting `'partition.end-input-to-done'='true'`. - +For batch mode, you can trigger partition mark done when end input by setting `'partition.end-input-to-done'='true'`, +and all partitions written in this batch will be marked done. diff --git a/docs/content/primary-key-table/data-distribution.md b/docs/content/primary-key-table/data-distribution.md index 1ae880786b96..6dfae0fe9adb 100644 --- a/docs/content/primary-key-table/data-distribution.md +++ b/docs/content/primary-key-table/data-distribution.md @@ -55,8 +55,6 @@ Dynamic Bucket only support single write job. Please do not start multiple jobs (this can lead to duplicate data). Even if you enable `'write-only'` and start a dedicated compaction job, it won't work. {{< /hint >}} -### Normal Dynamic Bucket Mode - When your updates do not cross partitions (no partitions, or primary keys contain all partition fields), Dynamic Bucket mode uses HASH index to maintain mapping from key to bucket, it requires more memory than fixed bucket mode. @@ -66,27 +64,6 @@ Performance: entries in a partition takes up **1 GB** more memory, partitions that are no longer active do not take up memory. 2. For tables with low update rates, this mode is recommended to significantly improve performance. -`Normal Dynamic Bucket Mode` supports sort-compact to speed up queries. See [Sort Compact]({{< ref "maintenance/dedicated-compaction#sort-compact" >}}). - -### Cross Partitions Upsert Dynamic Bucket Mode - -When you need cross partition upsert (primary keys not contain all partition fields), Dynamic Bucket mode directly -maintains the mapping of keys to partition and bucket, uses local disks, and initializes indexes by reading all -existing keys in the table when starting stream write job. Different merge engines have different behaviors: - -1. Deduplicate: Delete data from the old partition and insert new data into the new partition. -2. PartialUpdate & Aggregation: Insert new data into the old partition. -3. FirstRow: Ignore new data if there is old value. - -Performance: For tables with a large amount of data, there will be a significant loss in performance. Moreover, -initialization takes a long time. - -If your upsert does not rely on too old data, you can consider configuring index TTL to reduce Index and initialization time: -- `'cross-partition-upsert.index-ttl'`: The TTL in rocksdb index and initialization, this can avoid maintaining too many - indexes and lead to worse and worse performance. - -But please note that this may also cause data duplication. - ## Postpone Bucket Postpone bucket mode is configured by `'bucket' = '-2'`. @@ -101,12 +78,33 @@ To move the records into the correct bucket and make them readable, you need to run a compaction job. See `compact` [procedure]({{< ref "flink/procedures" >}}). The bucket number for the partitions compacted for the first time -is configured by the option `postpone.default-bucket-num`, whose default value is `4`. +is configured by the option `postpone.default-bucket-num`, whose default value is `1`. Finally, when you feel that the bucket number of some partition is too small, you can also run a rescale job. See `rescale` [procedure]({{< ref "flink/procedures" >}}). +## Cross Partitions Upsert + +When you need cross partition upsert (primary keys not contain all partition fields), recommend using the '-1' bucket. +Key Dynamic mode directly maintains the mapping of keys to partition and bucket, uses local disks, and initializes +indexes by reading all existing keys in the table when starting stream write job. Different merge engines have different behaviors: + +1. Deduplicate: Delete data from the old partition and insert new data into the new partition. +2. PartialUpdate & Aggregation: Insert new data into the old partition. +3. FirstRow: Ignore new data if there is old value. + +Performance: For tables with a large amount of data, there will be a significant loss in performance. Moreover, +initialization takes a long time. + +If your upsert does not rely on too old data, you can consider configuring index TTL to reduce Index and initialization time: +- `'cross-partition-upsert.index-ttl'`: The TTL in rocksdb index and initialization, this can avoid maintaining too many + indexes and lead to worse and worse performance. + +You can also use Cross Partitions Upsert with bucket (N > 0) or bucket (-2), in these modes, there is no global index to +ensure that your data undergoes reasonable deduplication, so relying on your input to have a complete changelog can +ensure the uniqueness of the data. + ## Pick Partition Fields The following three types of fields may be defined as partition fields in the warehouse: diff --git a/docs/content/primary-key-table/merge-engine/aggregation.md b/docs/content/primary-key-table/merge-engine/aggregation.md index 92e0f79f6427..a575e487388a 100644 --- a/docs/content/primary-key-table/merge-engine/aggregation.md +++ b/docs/content/primary-key-table/merge-engine/aggregation.md @@ -307,6 +307,10 @@ public static class BitmapContainsUDF extends ScalarFunction { Use `fields..nested-key=pk0,pk1,...` to specify the primary keys of the nested table. If no keys, row will be appended to array. + Use `fields..count-limit=` to specify the maximum number of rows in the nested table. When no nested-key, it will select data + sequentially up to limit; but if nested-key is specified, it cannot guarantee the correctness of the aggregation result. This option can be used to + avoid abnormal input. + An example: {{< tabs "nested_update-example" >}} diff --git a/docs/content/program-api/python-api.md b/docs/content/program-api/python-api.md index 6f7d8bb69759..2a01dab5bc84 100644 --- a/docs/content/program-api/python-api.md +++ b/docs/content/program-api/python-api.md @@ -32,8 +32,6 @@ implementation of the brand new PyPaimon does not require JDK installation. ## Environment Settings -### SDK Installing - SDK is published at [pypaimon](https://pypi.org/project/pypaimon/). You can install by ```shell @@ -44,6 +42,8 @@ pip install pypaimon Before coming into contact with the Table, you need to create a Catalog. +{{< tabs "create-catalog" >}} +{{< tab "filesystem" >}} ```python from pypaimon import CatalogFactory @@ -53,14 +53,33 @@ catalog_options = { } catalog = CatalogFactory.create(catalog_options) ``` +{{< /tab >}} +{{< tab "rest catalog" >}} +The sample code is as follows. The detailed meaning of option can be found in [DLF Token](../concepts/rest/dlf.md). -Currently, PyPaimon only support filesystem catalog and rest catalog. See [Catalog]({{< ref "concepts/catalog" >}}). +```python +from pypaimon import CatalogFactory -## Create Database & Table +# Note that keys and values are all string +catalog_options = { + 'metastore': 'rest', + 'warehouse': 'xxx', + 'uri': 'xxx', + 'dlf.region': 'xxx', + 'token.provider': 'xxx', + 'dlf.access-key-id': 'xxx', + 'dlf.access-key-secret': 'xxx' +} +catalog = CatalogFactory.create(catalog_options) +``` +{{< /tab >}} +{{< /tabs >}} + +Currently, PyPaimon only support filesystem catalog and rest catalog. See [Catalog]({{< ref "concepts/catalog" >}}). You can use the catalog to create table for writing data. -### Create Database (optional) +## Create Database Table is located in a database. If you want to create table in a new database, you should create it. @@ -72,7 +91,7 @@ catalog.create_database( ) ``` -### Create Schema +## Create Table Table schema contains fields definition, partition keys, primary keys, table options and comment. The field definition is described by `pyarrow.Schema`. All arguments except fields definition are optional. @@ -131,8 +150,6 @@ schema = Schema.from_pyarrow_schema( ) ``` -### Create Table - After building table schema, you can create corresponding table: ```python @@ -142,13 +159,8 @@ catalog.create_table( schema=schema, ignore_if_exists=True # To raise error if the table exists, set False ) -``` - -## Get Table -The Table interface provides tools to read and write table. - -```python +# Get Table table = catalog.get_table('database_name.table_name') ``` @@ -203,7 +215,7 @@ write_builder = table.new_batch_write_builder().overwrite({'dt': '2024-01-01'}) ## Batch Read -### Get ReadBuilder and Perform pushdown +### Predicate pushdown A `ReadBuilder` is used to build reading utils and perform filter and projection pushdown. @@ -238,7 +250,7 @@ You can also pushdown projection by `ReadBuilder`: read_builder = read_builder.with_projection(['f3', 'f2']) ``` -### Scan Plan +### Generate Splits Then you can step into Scan Plan stage to get `splits`: @@ -247,11 +259,9 @@ table_scan = read_builder.new_scan() splits = table_scan.plan().splits() ``` -### Read Splits - Finally, you can read data from the `splits` to various data format. -#### Apache Arrow +### Read Apache Arrow This requires `pyarrow` to be installed. @@ -285,8 +295,9 @@ for batch in table_read.to_arrow_batch_reader(splits): # f1: ["a","b","c"] ``` -#### Python Iterator -You can read the data row by row into a native Python iterator. +### Read Python Iterator + +You can read the data row by row into a native Python iterator. This is convenient for custom row-based processing logic. ```python @@ -298,7 +309,7 @@ for row in table_read.to_iterator(splits): # ["a","b","c"] ``` -#### Pandas +### Read Pandas This requires `pandas` to be installed. @@ -317,7 +328,7 @@ print(df) # ... ``` -#### DuckDB +### Read DuckDB This requires `duckdb` to be installed. @@ -340,7 +351,7 @@ print(duckdb_con.query("SELECT * FROM duckdb_table WHERE f0 = 1").fetchdf()) # 0 1 a ``` -#### Ray +### Read Ray This requires `ray` to be installed. @@ -365,23 +376,177 @@ print(ray_dataset.to_pandas()) # ... ``` +### Incremental Read + +This API allows reading data committed between two snapshot timestamps. The steps are as follows. + +- Set the option `CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP` on a copied table via `table.copy({...})`. The value must + be a string: `"startMillis,endMillis"`, where `startMillis` is exclusive and `endMillis` is inclusive. +- Use `SnapshotManager` to obtain snapshot timestamps or you can determine them by yourself. +- Read the data as above. + +Example: + +```python +from pypaimon import CatalogFactory +from pypaimon.common.core_options import CoreOptions +from pypaimon.snapshot.snapshot_manager import SnapshotManager + +# Prepare catalog and obtain a table +catalog = CatalogFactory.create({'warehouse': '/path/to/warehouse'}) +table = catalog.get_table('default.your_table_name') + +# Assume the table has at least two snapshots (1 and 2) +snapshot_manager = SnapshotManager(table) +t1 = snapshot_manager.get_snapshot_by_id(1).time_millis +t2 = snapshot_manager.get_snapshot_by_id(2).time_millis + +# Read records committed between [t1, t2] +table_inc = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: f"{t1},{t2}"}) + +read_builder = table_inc.new_read_builder() +table_scan = read_builder.new_scan() +table_read = read_builder.new_read() +splits = table_scan.plan().splits() + +# To Arrow +arrow_table = table_read.to_arrow(splits) + +# Or to pandas +pandas_df = table_read.to_pandas(splits) +``` + +### Shard Read + +Shard Read allows you to read data in parallel by dividing the table into multiple shards. This is useful for +distributed processing and parallel computation. + +You can specify the shard index and total number of shards to read a specific portion of the data: + +```python +# Prepare read builder +table = catalog.get_table('database_name.table_name') +read_builder = table.new_read_builder() +table_read = read_builder.new_read() + +# Read the second shard (index 1) out of 3 total shards +splits = read_builder.new_scan().with_shard(1, 3).plan().splits() + +# Read all shards and concatenate results +splits1 = read_builder.new_scan().with_shard(0, 3).plan().splits() +splits2 = read_builder.new_scan().with_shard(1, 3).plan().splits() +splits3 = read_builder.new_scan().with_shard(2, 3).plan().splits() + +# Combine results from all shards + +all_splits = splits1 + splits2 + splits3 +pa_table = table_read.to_arrow(all_splits) +``` + +Example with shard read: + +```python +import pyarrow as pa +from pypaimon import CatalogFactory, Schema + +# Create catalog +catalog_options = {'warehouse': 'file:///path/to/warehouse'} +catalog = CatalogFactory.create(catalog_options) +catalog.create_database("default", False) +# Define schema +pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('behavior', pa.string()), + ('dt', pa.string()), +]) + +# Create table and write data +schema = Schema.from_pyarrow_schema(pa_schema, partition_keys=['dt']) +catalog.create_table('default.test_table', schema, False) +table = catalog.get_table('default.test_table') + +# Write data in two batches +write_builder = table.new_batch_write_builder() + +# First write +table_write = write_builder.new_write() +table_commit = write_builder.new_commit() +data1 = { + 'user_id': [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010, 1011, 1012, 1013, 1014], + 'behavior': ['a', 'b', 'c', None, 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm'], + 'dt': ['p1', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1'], +} +pa_table = pa.Table.from_pydict(data1, schema=pa_schema) +table_write.write_arrow(pa_table) +table_commit.commit(table_write.prepare_commit()) +table_write.close() +table_commit.close() + +# Second write +table_write = write_builder.new_write() +table_commit = write_builder.new_commit() +data2 = { + 'user_id': [5, 6, 7, 8, 18], + 'item_id': [1005, 1006, 1007, 1008, 1018], + 'behavior': ['e', 'f', 'g', 'h', 'z'], + 'dt': ['p2', 'p1', 'p2', 'p2', 'p1'], +} +pa_table = pa.Table.from_pydict(data2, schema=pa_schema) +table_write.write_arrow(pa_table) +table_commit.commit(table_write.prepare_commit()) +table_write.close() +table_commit.close() + +# Read specific shard +read_builder = table.new_read_builder() +table_read = read_builder.new_read() + +# Read shard 2 out of 3 total shards +splits = read_builder.new_scan().with_shard(2, 3).plan().splits() +shard_data = table_read.to_arrow(splits) + +# Verify shard distribution by reading all shards +splits1 = read_builder.new_scan().with_shard(0, 3).plan().splits() +splits2 = read_builder.new_scan().with_shard(1, 3).plan().splits() +splits3 = read_builder.new_scan().with_shard(2, 3).plan().splits() + +# Combine all shards should equal full table read +all_shards_data = pa.concat_tables([ + table_read.to_arrow(splits1), + table_read.to_arrow(splits2), + table_read.to_arrow(splits3), +]) +full_table_data = table_read.to_arrow(read_builder.new_scan().plan().splits()) +``` + +Key points about shard read: + +- **Shard Index**: Zero-based index of the shard to read (0 to total_shards-1) +- **Total Shards**: Total number of shards to divide the data into +- **Data Distribution**: Data is distributed evenly across shards, with remainder rows going to the last shard +- **Parallel Processing**: Each shard can be processed independently for better performance +- **Consistency**: Combining all shards should produce the complete table data + ## Data Types -| Python Native Type | PyArrow Type | Paimon Type | -| :--- | :--- | :--- | -| `int` | `pyarrow.int8()` | `TINYINT` | -| `int` | `pyarrow.int16()` | `SMALLINT` | -| `int` | `pyarrow.int32()` | `INT` | -| `int` | `pyarrow.int64()` | `BIGINT` | -| `float` | `pyarrow.float32()` | `FLOAT` | -| `float` | `pyarrow.float64()` | `DOUBLE` | -| `bool` | `pyarrow.bool_()` | `BOOLEAN` | -| `str` | `pyarrow.string()` | `STRING`, `CHAR(n)`, `VARCHAR(n)` | -| `bytes` | `pyarrow.binary()` | `BYTES`, `VARBINARY(n)` | -| `bytes` | `pyarrow.binary(length)` | `BINARY(length)` | -| `decimal.Decimal` | `pyarrow.decimal128(precision, scale)` | `DECIMAL(precision, scale)` | -| `datetime.datetime` | `pyarrow.timestamp(unit, tz=None)` | `TIMESTAMP(p)` | -| `datetime.date` | `pyarrow.date32()` | `DATE` | -| `datetime.time` | `pyarrow.time32(unit)` or `pyarrow.time64(unit)` | `TIME(p)` | + +| Python Native Type | PyArrow Type | Paimon Type | +|:--------------------|:-------------------------------------------------|:----------------------------------| +| `int` | `pyarrow.int8()` | `TINYINT` | +| `int` | `pyarrow.int16()` | `SMALLINT` | +| `int` | `pyarrow.int32()` | `INT` | +| `int` | `pyarrow.int64()` | `BIGINT` | +| `float` | `pyarrow.float32()` | `FLOAT` | +| `float` | `pyarrow.float64()` | `DOUBLE` | +| `bool` | `pyarrow.bool_()` | `BOOLEAN` | +| `str` | `pyarrow.string()` | `STRING`, `CHAR(n)`, `VARCHAR(n)` | +| `bytes` | `pyarrow.binary()` | `BYTES`, `VARBINARY(n)` | +| `bytes` | `pyarrow.binary(length)` | `BINARY(length)` | +| `decimal.Decimal` | `pyarrow.decimal128(precision, scale)` | `DECIMAL(precision, scale)` | +| `datetime.datetime` | `pyarrow.timestamp(unit, tz=None)` | `TIMESTAMP(p)` | +| `datetime.date` | `pyarrow.date32()` | `DATE` | +| `datetime.time` | `pyarrow.time32(unit)` or `pyarrow.time64(unit)` | `TIME(p)` | ## Predicate diff --git a/docs/content/spark/procedures.md b/docs/content/spark/procedures.md index f3cff7016001..84da6821a4cc 100644 --- a/docs/content/spark/procedures.md +++ b/docs/content/spark/procedures.md @@ -370,9 +370,9 @@ This section introduce all available spark procedures about paimon. -- mark single partition done
- CALL sys.mark_partition_done(table => 'default.T', parititions => 'day=2024-07-01')

+ CALL sys.mark_partition_done(table => 'default.T', partitions => 'day=2024-07-01')

-- mark multiple partitions done
- CALL sys.mark_partition_done(table => 'default.T', parititions => 'day=2024-07-01;day=2024-07-02') + CALL sys.mark_partition_done(table => 'default.T', partitions => 'day=2024-07-01;day=2024-07-02') diff --git a/docs/content/spark/sql-functions.md b/docs/content/spark/sql-functions.md index 65019ac5fd9a..99f4dcb153ca 100644 --- a/docs/content/spark/sql-functions.md +++ b/docs/content/spark/sql-functions.md @@ -28,12 +28,11 @@ under the License. This section introduce all available Paimon Spark functions. - ## Built-in Function ### max_pt -`max_pt($table_name)` +`sys.max_pt($table_name)` It accepts a string type literal to specify the table name and return a max-valid-toplevel partition value. - **valid**: the partition which contains data files @@ -47,15 +46,13 @@ It would throw exception when: **Example** ```sql -SELECT max_pt('t'); +SELECT sys.max_pt('t'); -- 20250101 -SELECT * FROM t where pt = max_pt('t'); +SELECT * FROM t where pt = sys.max_pt('t'); -- a, 20250101 ``` -**Since: 1.1.0** - ## User-defined Function Paimon Spark supports two types of user-defined functions: lambda functions and file-based functions. diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 2ca6b6c7bc3f..2d4abf978718 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -146,6 +146,24 @@ String Specifies the column name(s) used for comparison during range partitioning, in the format 'columnName1,columnName2'. If not set or set to an empty string, it indicates that the range partitioning feature is not enabled. This option will be effective only for append table without primary keys and batch execution mode. + +
clustering.history-partition.idle-to-full-sort
+ (none) + Duration + The duration after which a partition without new updates is considered a historical partition. Historical partitions will be automatically fully clustered during the cluster operation.This option takes effects when 'clustering.history-partition.auto.enabled' is true. + + +
clustering.history-partition.limit
+ 5 + Integer + The limit of history partition number for automatically performing full clustering. + + +
clustering.incremental
+ false + Boolean + Whether enable incremental clustering. +
clustering.strategy
"auto" @@ -362,12 +380,6 @@ Boolean Enable data file thin mode to avoid duplicate columns storage. - -
delete-file.thread-num
- (none) - Integer - The maximum number of concurrent deleting files. By default is the number of processors available to the Java virtual machine. -
delete.force-produce-changelog
false @@ -452,6 +464,12 @@ Boolean Whether enabled read file index. + +
file-operation.thread-num
+ (none) + Integer + The maximum number of concurrent file operations. By default is the number of processors available to the Java virtual machine. +
file-reader-async-threshold
10 mb diff --git a/docs/static/rest-catalog-open-api.yaml b/docs/static/rest-catalog-open-api.yaml index 0b85c56fcbc0..9bfc7cbdfb1d 100644 --- a/docs/static/rest-catalog-open-api.yaml +++ b/docs/static/rest-catalog-open-api.yaml @@ -350,6 +350,11 @@ paths: in: query schema: type: string + - name: tableType + description: Filter tables by table type. All table types will be returned if not set or empty. + in: query + schema: + type: string responses: "200": description: OK diff --git a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java index e2e141f141af..a3da8ef5f1cb 100644 --- a/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-api/src/main/java/org/apache/paimon/CoreOptions.java @@ -81,6 +81,8 @@ public class CoreOptions implements Serializable { public static final String NESTED_KEY = "nested-key"; + public static final String COUNT_LIMIT = "count-limit"; + public static final String DISTINCT = "distinct"; public static final String LIST_AGG_DELIMITER = "list-agg-delimiter"; @@ -1767,12 +1769,13 @@ public InlineElement getDescription() { + "a forced lookup compaction will be performed to flush L0 files to higher level. " + "This option is only valid when lookup-compact mode is gentle."); - public static final ConfigOption DELETE_FILE_THREAD_NUM = - key("delete-file.thread-num") + public static final ConfigOption FILE_OPERATION_THREAD_NUM = + key("file-operation.thread-num") .intType() .noDefaultValue() + .withFallbackKeys("delete-file.thread-num") .withDescription( - "The maximum number of concurrent deleting files. " + "The maximum number of concurrent file operations. " + "By default is the number of processors available to the Java virtual machine."); public static final ConfigOption SCAN_FALLBACK_BRANCH = @@ -1930,7 +1933,28 @@ public InlineElement getDescription() { + "in 'clustering.by-columns'. 'order' is used for 1 column, 'zorder' for less than 5 columns, " + "and 'hilbert' for 5 or more columns."); - @Immutable + public static final ConfigOption CLUSTERING_INCREMENTAL = + key("clustering.incremental") + .booleanType() + .defaultValue(false) + .withDescription("Whether enable incremental clustering."); + + public static final ConfigOption CLUSTERING_HISTORY_PARTITION_LIMIT = + key("clustering.history-partition.limit") + .intType() + .defaultValue(5) + .withDescription( + "The limit of history partition number for automatically performing full clustering."); + + public static final ConfigOption CLUSTERING_HISTORY_PARTITION_IDLE_TIME = + key("clustering.history-partition.idle-to-full-sort") + .durationType() + .noDefaultValue() + .withDescription( + "The duration after which a partition without new updates is considered a historical partition. " + + "Historical partitions will be automatically fully clustered during the cluster operation." + + "This option takes effects when 'clustering.history-partition.auto.enabled' is true."); + public static final ConfigOption ROW_TRACKING_ENABLED = key("row-tracking.enabled") .booleanType() @@ -2189,6 +2213,13 @@ public List fieldNestedUpdateAggNestedKey(String fieldName) { return Arrays.stream(keyString.split(",")).map(String::trim).collect(Collectors.toList()); } + public int fieldNestedUpdateAggCountLimit(String fieldName) { + return options.get( + key(FIELDS_PREFIX + "." + fieldName + "." + COUNT_LIMIT) + .intType() + .defaultValue(Integer.MAX_VALUE)); + } + public boolean fieldCollectAggDistinct(String fieldName) { return options.get( key(FIELDS_PREFIX + "." + fieldName + "." + DISTINCT) @@ -2257,8 +2288,8 @@ public boolean cleanEmptyDirectories() { return options.get(SNAPSHOT_CLEAN_EMPTY_DIRECTORIES); } - public int deleteFileThreadNum() { - return options.getOptional(DELETE_FILE_THREAD_NUM) + public int fileOperationThreadNum() { + return options.getOptional(FILE_OPERATION_THREAD_NUM) .orElseGet(() -> Runtime.getRuntime().availableProcessors()); } @@ -2987,6 +3018,18 @@ public List clusteringColumns() { return clusteringColumns(options.get(CLUSTERING_COLUMNS)); } + public boolean clusteringIncrementalEnabled() { + return options.get(CLUSTERING_INCREMENTAL); + } + + public Duration clusteringHistoryPartitionIdleTime() { + return options.get(CLUSTERING_HISTORY_PARTITION_IDLE_TIME); + } + + public int clusteringHistoryPartitionLimit() { + return options.get(CLUSTERING_HISTORY_PARTITION_LIMIT); + } + public OrderType clusteringStrategy(int columnSize) { return clusteringStrategy(options.get(CLUSTERING_STRATEGY), columnSize); } diff --git a/paimon-api/src/main/java/org/apache/paimon/Snapshot.java b/paimon-api/src/main/java/org/apache/paimon/Snapshot.java index 31defa3bfb7a..a318a72e40a3 100644 --- a/paimon-api/src/main/java/org/apache/paimon/Snapshot.java +++ b/paimon-api/src/main/java/org/apache/paimon/Snapshot.java @@ -496,13 +496,16 @@ public boolean equals(Object o) { /** Type of changes in this snapshot. */ public enum CommitKind { - /** Changes flushed from the mem table. */ + /** New data files are appended to the table and no data file is deleted. */ APPEND, /** Changes by compacting existing data files. */ COMPACT, - /** Changes that clear up the whole partition and then add new records. */ + /** + * New data files are added to overwrite existing data files or just delete existing data + * files. + */ OVERWRITE, /** Collect statistics. */ diff --git a/paimon-api/src/main/java/org/apache/paimon/rest/RESTApi.java b/paimon-api/src/main/java/org/apache/paimon/rest/RESTApi.java index 08ed9f93d2f1..0a9989bdd5bf 100644 --- a/paimon-api/src/main/java/org/apache/paimon/rest/RESTApi.java +++ b/paimon-api/src/main/java/org/apache/paimon/rest/RESTApi.java @@ -141,6 +141,7 @@ public class RESTApi { public static final String DATABASE_NAME_PATTERN = "databaseNamePattern"; public static final String TABLE_NAME_PATTERN = "tableNamePattern"; + public static final String TABLE_TYPE = "tableType"; public static final String VIEW_NAME_PATTERN = "viewNamePattern"; public static final String FUNCTION_NAME_PATTERN = "functionNamePattern"; public static final String PARTITION_NAME_PATTERN = "partitionNamePattern"; @@ -354,14 +355,16 @@ public PagedList listTablesPaged( String databaseName, @Nullable Integer maxResults, @Nullable String pageToken, - @Nullable String tableNamePattern) { + @Nullable String tableNamePattern, + @Nullable String tableType) { ListTablesResponse response = client.get( resourcePaths.tables(databaseName), buildPagedQueryParams( maxResults, pageToken, - Pair.of(TABLE_NAME_PATTERN, tableNamePattern)), + Pair.of(TABLE_NAME_PATTERN, tableNamePattern), + Pair.of(TABLE_TYPE, tableType)), ListTablesResponse.class, restAuthFunction); List tables = response.getTables(); @@ -385,6 +388,8 @@ public PagedList listTablesPaged( * from a specific point. * @param tableNamePattern A sql LIKE pattern (%) for table names. All tables will be returned * if not set or empty. Currently, only prefix matching is supported. + * @param tableType Optional parameter to filter tables by table type. All table types will be + * returned if not set or empty. * @return {@link PagedList}: elements and nextPageToken. * @throws NoSuchResourceException Exception thrown on HTTP 404 means the database not exists * @throws ForbiddenException Exception thrown on HTTP 403 means don't have the permission for @@ -394,14 +399,16 @@ public PagedList listTableDetailsPaged( String databaseName, @Nullable Integer maxResults, @Nullable String pageToken, - @Nullable String tableNamePattern) { + @Nullable String tableNamePattern, + @Nullable String tableType) { ListTableDetailsResponse response = client.get( resourcePaths.tableDetails(databaseName), buildPagedQueryParams( maxResults, pageToken, - Pair.of(TABLE_NAME_PATTERN, tableNamePattern)), + Pair.of(TABLE_NAME_PATTERN, tableNamePattern), + Pair.of(TABLE_TYPE, tableType)), ListTableDetailsResponse.class, restAuthFunction); List tables = response.getTableDetails(); diff --git a/paimon-api/src/main/java/org/apache/paimon/table/SpecialFields.java b/paimon-api/src/main/java/org/apache/paimon/table/SpecialFields.java index 042f1d1d53be..d89776747bb9 100644 --- a/paimon-api/src/main/java/org/apache/paimon/table/SpecialFields.java +++ b/paimon-api/src/main/java/org/apache/paimon/table/SpecialFields.java @@ -139,33 +139,33 @@ public static int getMapValueFieldId(int mapFieldId, int depth) { + depth; } - public static RowType rowTypeWithRowLineage(RowType rowType) { - return rowTypeWithRowLineage(rowType, false); + public static RowType rowTypeWithRowTracking(RowType rowType) { + return rowTypeWithRowTracking(rowType, false); } /** - * Add row lineage fields to rowType. + * Add row tracking fields to rowType. * * @param sequenceNumberNullable sequence number is not null for user, but is nullable when read * and write */ - public static RowType rowTypeWithRowLineage(RowType rowType, boolean sequenceNumberNullable) { - List fieldsWithRowLineage = new ArrayList<>(rowType.getFields()); + public static RowType rowTypeWithRowTracking(RowType rowType, boolean sequenceNumberNullable) { + List fieldsWithRowTracking = new ArrayList<>(rowType.getFields()); - fieldsWithRowLineage.forEach( + fieldsWithRowTracking.forEach( f -> { if (ROW_ID.name().equals(f.name()) || SEQUENCE_NUMBER.name().equals(f.name())) { throw new IllegalArgumentException( - "Row lineage field name '" + "Row tracking field name '" + f.name() + "' conflicts with existing field names."); } }); - fieldsWithRowLineage.add(SpecialFields.ROW_ID); - fieldsWithRowLineage.add( + fieldsWithRowTracking.add(SpecialFields.ROW_ID); + fieldsWithRowTracking.add( sequenceNumberNullable ? SpecialFields.SEQUENCE_NUMBER.copy(true) : SpecialFields.SEQUENCE_NUMBER); - return new RowType(fieldsWithRowLineage); + return new RowType(fieldsWithRowTracking); } } diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java index 7e487e54fb20..cf80b9d21f49 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/ArrowUtils.java @@ -208,8 +208,10 @@ public static long timestampToEpoch( } public static ArrowCStruct serializeToCStruct( - VectorSchemaRoot vsr, ArrowArray array, ArrowSchema schema) { - BufferAllocator bufferAllocator = vsr.getVector(0).getAllocator(); + VectorSchemaRoot vsr, + ArrowArray array, + ArrowSchema schema, + BufferAllocator bufferAllocator) { Data.exportVectorSchemaRoot(bufferAllocator, vsr, null, array, schema); return ArrowCStruct.of(array, schema); } diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowBatchConverter.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowBatchConverter.java index 5747a39d2157..b38edc68135b 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowBatchConverter.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowBatchConverter.java @@ -62,6 +62,9 @@ public VectorSchemaRoot next(int maxBatchRows) { return root; } + public abstract ArrowBatchConverter copy( + VectorSchemaRoot root, ArrowFieldWriter[] fieldWriters); + protected abstract void doWrite(int maxBatchRows); public void close() { diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowPerRowBatchConverter.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowPerRowBatchConverter.java index 9ee495c84939..16e0bfbd1dd1 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowPerRowBatchConverter.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowPerRowBatchConverter.java @@ -36,6 +36,14 @@ public ArrowPerRowBatchConverter(VectorSchemaRoot root, ArrowFieldWriter[] field super(root, fieldWriters); } + @Override + public ArrowPerRowBatchConverter copy(VectorSchemaRoot root, ArrowFieldWriter[] fieldWriters) { + ArrowPerRowBatchConverter newConverter = new ArrowPerRowBatchConverter(root, fieldWriters); + newConverter.iterator = this.iterator; + newConverter.currentRow = this.currentRow; + return newConverter; + } + @Override public void doWrite(int maxBatchRows) { int rowIndex = 0; diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverter.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverter.java index 06dc8d6467c8..a7da552ed39b 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverter.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/converter/ArrowVectorizedBatchConverter.java @@ -46,6 +46,19 @@ public ArrowVectorizedBatchConverter(VectorSchemaRoot root, ArrowFieldWriter[] f super(root, fieldWriters); } + @Override + public ArrowVectorizedBatchConverter copy( + VectorSchemaRoot root, ArrowFieldWriter[] fieldWriters) { + ArrowVectorizedBatchConverter newConverter = + new ArrowVectorizedBatchConverter(root, fieldWriters); + newConverter.iterator = this.iterator; + newConverter.batch = this.batch; + newConverter.pickedInColumn = this.pickedInColumn; + newConverter.totalNumRows = this.totalNumRows; + newConverter.startIndex = this.startIndex; + return newConverter; + } + @Override public void doWrite(int maxBatchRows) { int batchRows = Math.min(maxBatchRows, totalNumRows - startIndex); diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java index afa58250c336..09fe6e712bb9 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/vector/ArrowFormatCWriter.java @@ -69,7 +69,8 @@ public boolean write(InternalRow currentRow) { public ArrowCStruct toCStruct() { VectorSchemaRoot vectorSchemaRoot = realWriter.getVectorSchemaRoot(); - return ArrowUtils.serializeToCStruct(vectorSchemaRoot, array, schema); + return ArrowUtils.serializeToCStruct( + vectorSchemaRoot, array, schema, realWriter.getAllocator()); } public void flush() { diff --git a/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriters.java b/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriters.java index 1c7bb742f529..9e4f371a793d 100644 --- a/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriters.java +++ b/paimon-arrow/src/main/java/org/apache/paimon/arrow/writer/ArrowFieldWriters.java @@ -526,7 +526,8 @@ public ArrayWriter( @Override public void reset() { - fieldVector.reset(); + super.reset(); + elementWriter.reset(); offset = 0; } @@ -613,7 +614,9 @@ public MapWriter( @Override public void reset() { - fieldVector.reset(); + super.reset(); + keyWriter.reset(); + valueWriter.reset(); offset = 0; } @@ -769,6 +772,14 @@ public RowWriter( this.fieldWriters = fieldWriters; } + @Override + public void reset() { + super.reset(); + for (ArrowFieldWriter fieldWriter : fieldWriters) { + fieldWriter.reset(); + } + } + @Override protected void doWrite( ColumnVector columnVector, diff --git a/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/ArrowFormatWriterTest.java b/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/ArrowFormatWriterTest.java index d7ee8ca1a566..6436be955dbf 100644 --- a/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/ArrowFormatWriterTest.java +++ b/paimon-arrow/src/test/java/org/apache/paimon/arrow/vector/ArrowFormatWriterTest.java @@ -22,6 +22,8 @@ import org.apache.paimon.arrow.reader.ArrowBatchReader; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.GenericArray; +import org.apache.paimon.data.GenericMap; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.Timestamp; @@ -34,8 +36,12 @@ import org.apache.arrow.memory.OutOfMemoryException; import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarCharVector; import org.apache.arrow.vector.VectorSchemaRoot; -import org.assertj.core.api.Assertions; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.arrow.vector.complex.MapVector; +import org.apache.arrow.vector.complex.StructVector; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -43,11 +49,15 @@ import java.math.BigDecimal; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.concurrent.ThreadLocalRandom; +import static org.assertj.core.api.Assertions.assertThat; + /** Test for {@link org.apache.paimon.arrow.vector.ArrowFormatWriter}. */ public class ArrowFormatWriterTest { @@ -115,7 +125,7 @@ public void testWrite() { InternalRow expectec = list.get(i); for (InternalRow.FieldGetter fieldGetter : fieldGetters) { - Assertions.assertThat(fieldGetter.getFieldOrNull(actual)) + assertThat(fieldGetter.getFieldOrNull(actual)) .isEqualTo(fieldGetter.getFieldOrNull(expectec)); } } @@ -158,7 +168,7 @@ public void testReadWithSchemaMessUp() { InternalRow expectec = list.get(i); for (InternalRow.FieldGetter fieldGetter : fieldGetters) { - Assertions.assertThat(fieldGetter.getFieldOrNull(actual)) + assertThat(fieldGetter.getFieldOrNull(actual)) .isEqualTo(fieldGetter.getFieldOrNull(expectec)); } } @@ -194,9 +204,9 @@ public void testWriteWithMemoryLimit(boolean limitMemory) { if (limitMemory) { for (int i = 0; i < 64; i++) { - Assertions.assertThat(writer.write(genericRow)).isTrue(); + assertThat(writer.write(genericRow)).isTrue(); } - Assertions.assertThat(writer.write(genericRow)).isFalse(); + assertThat(writer.write(genericRow)).isFalse(); } writer.reset(); @@ -211,8 +221,8 @@ public void testWriteWithMemoryLimit(boolean limitMemory) { } if (limitMemory) { - Assertions.assertThat(writer.memoryUsed()).isLessThan(memoryLimit); - Assertions.assertThat(writer.getAllocator().getAllocatedMemory()) + assertThat(writer.memoryUsed()).isLessThan(memoryLimit); + assertThat(writer.getAllocator().getAllocatedMemory()) .isGreaterThan(memoryLimit) .isLessThan(2 * memoryLimit); } @@ -244,7 +254,7 @@ public void testArrowBundleRecords() { InternalRow expectec = list.get(i); for (InternalRow.FieldGetter fieldGetter : fieldGetters) { - Assertions.assertThat(fieldGetter.getFieldOrNull(actual)) + assertThat(fieldGetter.getFieldOrNull(actual)) .isEqualTo(fieldGetter.getFieldOrNull(expectec)); } } @@ -276,6 +286,140 @@ public void testWriteWithExternalAllocator(boolean allocationFailed) { } } + @Test + public void testArrowFormatCWriterWithEmptySchema() { + RowType emptyschema = new RowType(new ArrayList<>()); + + try (RootAllocator rootAllocator = new RootAllocator(); + BufferAllocator allocator = + rootAllocator.newChildAllocator("paimonWriter", 0, Long.MAX_VALUE); + ArrowFormatCWriter writer = + new ArrowFormatCWriter(emptyschema, 4096, true, allocator)) { + for (int i = 0; i < 100; i++) { + writer.write(GenericRow.of()); + } + writer.flush(); + ArrowCStruct cStruct = writer.toCStruct(); + assertThat(cStruct).isNotNull(); + writer.release(); + } + } + + @Test + public void testWriteArrayMapTwice() { + try (ArrowFormatWriter arrowFormatWriter = + new ArrowFormatWriter( + RowType.of( + DataTypes.ARRAY( + DataTypes.MAP(DataTypes.STRING(), DataTypes.STRING()))), + 1, + true)) { + writeAndCheckArrayMap(arrowFormatWriter); + writeAndCheckArrayMap(arrowFormatWriter); + } + } + + private void writeAndCheckArrayMap(ArrowFormatWriter arrowFormatWriter) { + GenericRow genericRow = new GenericRow(1); + Map map = new HashMap<>(); + map.put(BinaryString.fromString("a"), BinaryString.fromString("b")); + map.put(BinaryString.fromString("c"), BinaryString.fromString("d")); + GenericArray array = new GenericArray(new Object[] {new GenericMap(map)}); + genericRow.setField(0, array); + arrowFormatWriter.write(genericRow); + arrowFormatWriter.flush(); + + VectorSchemaRoot vsr = arrowFormatWriter.getVectorSchemaRoot(); + ListVector listVector = (ListVector) vsr.getVector(0); + MapVector mapVector = (MapVector) listVector.getDataVector(); + assertThat(mapVector.getValueCount()).isEqualTo(1); + VarCharVector keyVector = + (VarCharVector) mapVector.getDataVector().getChildrenFromFields().get(0); + assertThat(keyVector.getValueCount()).isEqualTo(2); + assertThat(new String(keyVector.get(0))).isEqualTo("a"); + assertThat(new String(keyVector.get(1))).isEqualTo("c"); + VarCharVector valueVector = + (VarCharVector) mapVector.getDataVector().getChildrenFromFields().get(1); + assertThat(valueVector.getValueCount()).isEqualTo(2); + assertThat(new String(valueVector.get(0))).isEqualTo("b"); + assertThat(new String(valueVector.get(1))).isEqualTo("d"); + arrowFormatWriter.reset(); + } + + @Test + public void testWriteMapArrayTwice() { + try (ArrowFormatWriter arrowFormatWriter = + new ArrowFormatWriter( + RowType.of( + DataTypes.MAP(DataTypes.INT(), DataTypes.ARRAY(DataTypes.INT()))), + 1, + true)) { + writeAndCheckMapArray(arrowFormatWriter); + writeAndCheckMapArray(arrowFormatWriter); + } + } + + private void writeAndCheckMapArray(ArrowFormatWriter arrowFormatWriter) { + GenericRow genericRow = new GenericRow(1); + GenericArray array1 = new GenericArray(new Object[] {1, 2}); + GenericArray array2 = new GenericArray(new Object[] {3, 4}); + Map map = new HashMap<>(); + map.put(1, array1); + map.put(2, array2); + GenericMap genericMap = new GenericMap(map); + genericRow.setField(0, genericMap); + arrowFormatWriter.write(genericRow); + arrowFormatWriter.flush(); + + VectorSchemaRoot vsr = arrowFormatWriter.getVectorSchemaRoot(); + MapVector mapVector = (MapVector) vsr.getVector(0); + IntVector keyVector = (IntVector) mapVector.getDataVector().getChildrenFromFields().get(0); + assertThat(keyVector.getValueCount()).isEqualTo(2); + assertThat(keyVector.get(0)).isEqualTo(1); + assertThat(keyVector.get(1)).isEqualTo(2); + ListVector valueVector = + (ListVector) mapVector.getDataVector().getChildrenFromFields().get(1); + assertThat(valueVector.getValueCount()).isEqualTo(2); + IntVector innerValueVector = (IntVector) valueVector.getDataVector(); + assertThat(innerValueVector.getValueCount()).isEqualTo(4); + assertThat(innerValueVector.get(0)).isEqualTo(1); + assertThat(innerValueVector.get(1)).isEqualTo(2); + assertThat(innerValueVector.get(2)).isEqualTo(3); + assertThat(innerValueVector.get(3)).isEqualTo(4); + arrowFormatWriter.reset(); + } + + @Test + public void testWriteRowArrayTwice() { + try (ArrowFormatWriter arrowFormatWriter = + new ArrowFormatWriter( + RowType.of(DataTypes.ROW(DataTypes.ARRAY(DataTypes.INT()))), 1, true)) { + writeAndCheckRowArray(arrowFormatWriter); + writeAndCheckRowArray(arrowFormatWriter); + } + } + + private void writeAndCheckRowArray(ArrowFormatWriter arrowFormatWriter) { + GenericRow genericRow = new GenericRow(1); + GenericRow innerRow = new GenericRow(1); + GenericArray array = new GenericArray(new Object[] {1, 2}); + innerRow.setField(0, array); + genericRow.setField(0, innerRow); + arrowFormatWriter.write(genericRow); + arrowFormatWriter.flush(); + + VectorSchemaRoot vsr = arrowFormatWriter.getVectorSchemaRoot(); + assertThat(vsr.getRowCount()).isEqualTo(1); + StructVector structVector = (StructVector) vsr.getVector(0); + ListVector listVector = (ListVector) structVector.getChildrenFromFields().get(0); + assertThat(listVector.getValueCount()).isEqualTo(1); + IntVector dataVector = (IntVector) listVector.getDataVector(); + assertThat(dataVector.getValueCount()).isEqualTo(2); + assertThat(dataVector.get(0)).isEqualTo(1); + assertThat(dataVector.get(1)).isEqualTo(2); + arrowFormatWriter.reset(); + } + private void writeAndCheck(ArrowFormatCWriter writer) { List list = new ArrayList<>(); List fieldGetters = new ArrayList<>(); @@ -301,7 +445,7 @@ private void writeAndCheck(ArrowFormatCWriter writer) { InternalRow expectec = list.get(i); for (InternalRow.FieldGetter fieldGetter : fieldGetters) { - Assertions.assertThat(fieldGetter.getFieldOrNull(actual)) + assertThat(fieldGetter.getFieldOrNull(actual)) .isEqualTo(fieldGetter.getFieldOrNull(expectec)); } } diff --git a/paimon-common/src/main/java/org/apache/paimon/casting/FallbackMappingRow.java b/paimon-common/src/main/java/org/apache/paimon/casting/FallbackMappingRow.java index 364c09d90f5f..a83aba874c2d 100644 --- a/paimon-common/src/main/java/org/apache/paimon/casting/FallbackMappingRow.java +++ b/paimon-common/src/main/java/org/apache/paimon/casting/FallbackMappingRow.java @@ -27,7 +27,7 @@ import org.apache.paimon.data.variant.Variant; import org.apache.paimon.types.RowKind; -/** Row with row lineage inject in. */ +/** Row with fallback mapping row inject in. */ public class FallbackMappingRow implements InternalRow { private InternalRow main; @@ -181,9 +181,9 @@ public InternalRow getRow(int pos, int numFields) { return main.getRow(pos, numFields); } - public FallbackMappingRow replace(InternalRow main, InternalRow rowLineage) { + public FallbackMappingRow replace(InternalRow main, InternalRow fallbackRow) { this.main = main; - this.fallbackRow = rowLineage; + this.fallbackRow = fallbackRow; return this; } } diff --git a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java index 37f8d8edf235..8c641660ba2c 100644 --- a/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java +++ b/paimon-common/src/main/java/org/apache/paimon/data/columnar/ColumnarRowIterator.java @@ -124,7 +124,7 @@ public ColumnarRowIterator mapping( return this; } - public ColumnarRowIterator assignRowLineage( + public ColumnarRowIterator assignRowTracking( Long firstRowId, Long snapshotId, Map meta) { VectorizedColumnBatch vectorizedColumnBatch = row.batch(); ColumnVector[] vectors = vectorizedColumnBatch.columns; diff --git a/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndex.java b/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndex.java index 3c9dcadba3ec..83d6b64b64b1 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndex.java +++ b/paimon-common/src/main/java/org/apache/paimon/fileindex/bloomfilter/BloomFilterFileIndex.java @@ -117,7 +117,7 @@ private static class Reader extends FileIndexReader { private final FastHash hashFunction; public Reader(DataType type, byte[] serializedBytes) { - // little endian + // big endian int numHashFunctions = ((serializedBytes[0] << 24) + (serializedBytes[1] << 16) diff --git a/paimon-common/src/main/java/org/apache/paimon/rest/RESTTokenFileIO.java b/paimon-common/src/main/java/org/apache/paimon/rest/RESTTokenFileIO.java index f5de31afb891..e3b321ff2e15 100644 --- a/paimon-common/src/main/java/org/apache/paimon/rest/RESTTokenFileIO.java +++ b/paimon-common/src/main/java/org/apache/paimon/rest/RESTTokenFileIO.java @@ -45,7 +45,6 @@ import java.io.UncheckedIOException; import java.util.Map; import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import static org.apache.paimon.options.CatalogOptions.FILE_IO_ALLOW_CACHE; import static org.apache.paimon.rest.RESTApi.TOKEN_EXPIRATION_SAFE_TIME_MILLIS; @@ -64,7 +63,6 @@ public class RESTTokenFileIO implements FileIO { private static final Cache FILE_IO_CACHE = Caffeine.newBuilder() - .expireAfterAccess(30, TimeUnit.MINUTES) .maximumSize(1000) .removalListener( (ignored, value, cause) -> IOUtils.closeQuietly((FileIO) value)) diff --git a/paimon-common/src/main/java/org/apache/paimon/table/BucketMode.java b/paimon-common/src/main/java/org/apache/paimon/table/BucketMode.java index 277c4740c18a..4740ba8a0080 100644 --- a/paimon-common/src/main/java/org/apache/paimon/table/BucketMode.java +++ b/paimon-common/src/main/java/org/apache/paimon/table/BucketMode.java @@ -38,20 +38,21 @@ public enum BucketMode { HASH_FIXED, /** - * The dynamic bucket mode records which bucket the key corresponds to through the index files. - * The index records the correspondence between the hash value of the primary-key and the - * bucket. This mode cannot support multiple concurrent writes or bucket skipping for reading - * filter conditions. This mode only works for changelog table. + * Hash-Dynamic mode records the correspondence between the hash of the primary key and the + * bucket number. It is used to simplify the distribution of primary keys to buckets, but cannot + * support large amounts of data. It cannot support multiple concurrent writes or bucket + * skipping for reading filter. This mode only works for primary key table. */ HASH_DYNAMIC, /** - * The cross partition mode is for cross partition upsert (primary keys not contain all - * partition fields). It directly maintains the mapping of primary keys to partition and bucket, - * uses local disks, and initializes indexes by reading all existing keys in the table when - * starting stream write job. + * Key-Dynamic mode records the correspondence between the primary key and the partition + + * bucket number. It is used to cross partition upsert (primary keys not contain all partition + * fields). It directly maintains the mapping of primary keys to partition and bucket using + * local disks, and initializes indexes by reading all existing keys in the table when starting + * write job. */ - CROSS_PARTITION, + KEY_DYNAMIC, /** * Ignoring bucket concept, although all data is written to bucket-0, the parallelism of reads diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/FileDeletionThreadPool.java b/paimon-common/src/main/java/org/apache/paimon/utils/FileOperationThreadPool.java similarity index 89% rename from paimon-common/src/main/java/org/apache/paimon/utils/FileDeletionThreadPool.java rename to paimon-common/src/main/java/org/apache/paimon/utils/FileOperationThreadPool.java index 638d6f9a4cbf..87c096c50d53 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/FileDeletionThreadPool.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/FileOperationThreadPool.java @@ -24,10 +24,10 @@ import static org.apache.paimon.utils.ThreadPoolUtils.createCachedThreadPool; -/** Thread pool to delete files using {@link FileIO}. */ -public class FileDeletionThreadPool { +/** Thread pool to operate files using {@link FileIO}. */ +public class FileOperationThreadPool { - private static final String THREAD_NAME = "DELETE-FILE-THREAD-POOL"; + private static final String THREAD_NAME = "FILE-OPERATION-THREAD-POOL"; private static ThreadPoolExecutor executorService = createCachedThreadPool(Runtime.getRuntime().availableProcessors(), THREAD_NAME); diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowPartitionComputer.java b/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowPartitionComputer.java index edefce318986..2382647aa9f7 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowPartitionComputer.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/InternalRowPartitionComputer.java @@ -107,7 +107,11 @@ public static Map convertSpecToInternal( public static GenericRow convertSpecToInternalRow( Map spec, RowType partType, String defaultPartValue) { - checkArgument(spec.size() == partType.getFieldCount()); + checkArgument( + spec.size() == partType.getFieldCount(), + "Partition spec %s size not match partition type %s", + spec, + partType); GenericRow partRow = new GenericRow(spec.size()); List fieldNames = partType.getFieldNames(); for (Map.Entry entry : spec.entrySet()) { diff --git a/paimon-common/src/main/java/org/apache/paimon/utils/TypeUtils.java b/paimon-common/src/main/java/org/apache/paimon/utils/TypeUtils.java index 7b6d030104d2..e80b6621e9b1 100644 --- a/paimon-common/src/main/java/org/apache/paimon/utils/TypeUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/utils/TypeUtils.java @@ -144,12 +144,7 @@ public static Object castFromStringInternal(String s, DataType type, boolean isC } else { // Compatible canal-cdc Float f = Float.valueOf(s); - String floatStr = f.toString(); - if (s.contains(".") && !s.contains("E")) { - int decimal = s.length() - s.indexOf(".") - 1; - floatStr = String.format("%." + decimal + "f", f); - } - if (!floatStr.equals(s)) { + if (!f.toString().equals(Double.toString(d))) { throw new NumberFormatException( s + " cannot be cast to float due to precision loss"); } else { diff --git a/paimon-common/src/test/java/org/apache/paimon/utils/TypeUtilsTest.java b/paimon-common/src/test/java/org/apache/paimon/utils/TypeUtilsTest.java index 4c204e55198b..d7dab1614a48 100644 --- a/paimon-common/src/test/java/org/apache/paimon/utils/TypeUtilsTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/utils/TypeUtilsTest.java @@ -198,8 +198,8 @@ public void testDoubleCastFromString() { @Test public void testFloatCastFromString() { - String[] values = {"123.456", "0.00042", "1.00001"}; - Float[] expected = {123.456f, 0.00042f, 1.00001f}; + String[] values = {"123.456", "0.00042", "1.00001", "175.26562", "0.00046", "6.1042607E-4"}; + Float[] expected = {123.456f, 0.00042f, 1.00001f, 175.26562f, 0.00046f, 0.00061042607f}; for (int i = 0; i < values.length; i++) { Object result = TypeUtils.castFromCdcValueString(values[i], DataTypes.FLOAT()); assertThat(result).isEqualTo(expected[i]); diff --git a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java index 2d71f61580db..bbeb0aa150f3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java @@ -298,7 +298,10 @@ public FileStoreCommitImpl newCommit(String commitUser, FileStoreTable table) { options.commitMinRetryWait(), options.commitMaxRetryWait(), options.commitStrictModeLastSafeSnapshot().orElse(null), - options.rowTrackingEnabled()); + options.rowTrackingEnabled(), + !schema.primaryKeys().isEmpty(), + options.deletionVectorsEnabled(), + newIndexFileHandler()); } @Override @@ -312,7 +315,7 @@ public SnapshotDeletion newSnapshotDeletion() { newStatsFileHandler(), options.changelogProducer() != CoreOptions.ChangelogProducer.NONE, options.cleanEmptyDirectories(), - options.deleteFileThreadNum()); + options.fileOperationThreadNum()); } @Override @@ -325,7 +328,7 @@ public ChangelogDeletion newChangelogDeletion() { newIndexFileHandler(), newStatsFileHandler(), options.cleanEmptyDirectories(), - options.deleteFileThreadNum()); + options.fileOperationThreadNum()); } @Override @@ -343,7 +346,7 @@ public TagDeletion newTagDeletion() { newIndexFileHandler(), newStatsFileHandler(), options.cleanEmptyDirectories(), - options.deleteFileThreadNum()); + options.fileOperationThreadNum()); } public abstract Comparator newKeyComparator(); diff --git a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java index 2f552d799bb1..9fb9694e5625 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java @@ -27,6 +27,7 @@ import org.apache.paimon.operation.BaseAppendFileStoreWrite; import org.apache.paimon.operation.BucketSelectConverter; import org.apache.paimon.operation.BucketedAppendFileStoreWrite; +import org.apache.paimon.operation.DataEvolutionFileStoreScan; import org.apache.paimon.operation.DataEvolutionSplitRead; import org.apache.paimon.operation.RawFileSplitRead; import org.apache.paimon.predicate.Predicate; @@ -110,7 +111,7 @@ public BaseAppendFileStoreWrite newWrite(String commitUser, @Nullable Integer wr if (bucketMode() == BucketMode.BUCKET_UNAWARE) { RawFileSplitRead readForCompact = newRead(); if (options.rowTrackingEnabled()) { - readForCompact.withReadType(SpecialFields.rowTypeWithRowLineage(rowType)); + readForCompact.withReadType(SpecialFields.rowTypeWithRowTracking(rowType)); } return new AppendFileStoreWrite( fileIO, @@ -168,6 +169,17 @@ public AppendOnlyFileStoreScan newScan() { return Optional.empty(); }; + if (options().dataEvolutionEnabled()) { + return new DataEvolutionFileStoreScan( + newManifestsReader(), + bucketSelectConverter, + snapshotManager(), + schemaManager, + schema, + manifestFileFactory(), + options.scanManifestParallelism()); + } + return new AppendOnlyFileStoreScan( newManifestsReader(), bucketSelectConverter, diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java index 5e5b354e6593..077068df73ff 100644 --- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java @@ -54,7 +54,6 @@ import static org.apache.paimon.predicate.PredicateBuilder.and; import static org.apache.paimon.predicate.PredicateBuilder.pickTransformFieldMapping; import static org.apache.paimon.predicate.PredicateBuilder.splitAnd; -import static org.apache.paimon.utils.Preconditions.checkArgument; /** {@link FileStore} for querying and updating {@link KeyValue}s. */ public class KeyValueFileStore extends AbstractFileStore { @@ -104,9 +103,8 @@ public BucketMode bucketMode() { case -2: return BucketMode.POSTPONE_MODE; case -1: - return crossPartitionUpdate ? BucketMode.CROSS_PARTITION : BucketMode.HASH_DYNAMIC; + return crossPartitionUpdate ? BucketMode.KEY_DYNAMIC : BucketMode.HASH_DYNAMIC; default: - checkArgument(!crossPartitionUpdate); return BucketMode.HASH_FIXED; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/append/AppendCompactTask.java b/paimon-core/src/main/java/org/apache/paimon/append/AppendCompactTask.java index af8997f981b3..42c45488424e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/AppendCompactTask.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/AppendCompactTask.java @@ -25,7 +25,6 @@ import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataIncrement; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.IndexManifestEntry; import org.apache.paimon.operation.BaseAppendFileStoreWrite; @@ -38,7 +37,6 @@ import java.util.Collections; import java.util.List; import java.util.Objects; -import java.util.stream.Collectors; import static org.apache.paimon.table.BucketMode.UNAWARE_BUCKET; @@ -74,7 +72,10 @@ public CommitMessage doCompact(FileStoreTable table, BaseAppendFileStoreWrite wr Preconditions.checkArgument( dvEnabled || compactBefore.size() > 1, "AppendOnlyCompactionTask need more than one file input."); - IndexIncrement indexIncrement; + // If compact task didn't compact all files, the remain deletion files will be written into + // new deletion files. + List newIndexFiles = new ArrayList<>(); + List deletedIndexFiles = new ArrayList<>(); if (dvEnabled) { AppendDeleteFileMaintainer dvIndexFileMaintainer = BaseAppendDeleteFileMaintainer.forUnawareAppend( @@ -91,21 +92,25 @@ public CommitMessage doCompact(FileStoreTable table, BaseAppendFileStoreWrite wr compactBefore.forEach( f -> dvIndexFileMaintainer.notifyRemovedDeletionVector(f.fileName())); List indexEntries = dvIndexFileMaintainer.persist(); - Preconditions.checkArgument( - indexEntries.stream().noneMatch(i -> i.kind() == FileKind.ADD)); - List removed = - indexEntries.stream() - .map(IndexManifestEntry::indexFile) - .collect(Collectors.toList()); - indexIncrement = new IndexIncrement(Collections.emptyList(), removed); + for (IndexManifestEntry entry : indexEntries) { + if (entry.kind() == FileKind.ADD) { + newIndexFiles.add(entry.indexFile()); + } else { + deletedIndexFiles.add(entry.indexFile()); + } + } } else { compactAfter.addAll( write.compactRewrite(partition, UNAWARE_BUCKET, null, compactBefore)); - indexIncrement = new IndexIncrement(Collections.emptyList()); } CompactIncrement compactIncrement = - new CompactIncrement(compactBefore, compactAfter, Collections.emptyList()); + new CompactIncrement( + compactBefore, + compactAfter, + Collections.emptyList(), + newIndexFiles, + deletedIndexFiles); return new CommitMessageImpl( partition, // bucket 0 is bucket for unaware-bucket table @@ -113,8 +118,7 @@ public CommitMessage doCompact(FileStoreTable table, BaseAppendFileStoreWrite wr 0, table.coreOptions().bucket(), DataIncrement.emptyIncrement(), - compactIncrement, - indexIncrement); + compactIncrement); } public int hashCode() { diff --git a/paimon-core/src/main/java/org/apache/paimon/append/cluster/HistoryPartitionCluster.java b/paimon-core/src/main/java/org/apache/paimon/append/cluster/HistoryPartitionCluster.java new file mode 100644 index 000000000000..8acbb8e7785a --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/append/cluster/HistoryPartitionCluster.java @@ -0,0 +1,192 @@ +/* + * 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. + */ + +package org.apache.paimon.append.cluster; + +import org.apache.paimon.Snapshot; +import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.compact.CompactUnit; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.mergetree.LevelSortedRun; +import org.apache.paimon.partition.PartitionPredicate; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.utils.InternalRowPartitionComputer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.paimon.append.cluster.IncrementalClusterManager.constructPartitionLevels; +import static org.apache.paimon.append.cluster.IncrementalClusterManager.logForPartitionLevel; + +/** Handle historical partition for full clustering. */ +public class HistoryPartitionCluster { + + private static final Logger LOG = LoggerFactory.getLogger(HistoryPartitionCluster.class); + + private final FileStoreTable table; + private final Snapshot snapshot; + private final IncrementalClusterStrategy incrementalClusterStrategy; + private final InternalRowPartitionComputer partitionComputer; + private final PartitionPredicate specifiedPartitions; + private final Duration historyPartitionIdleTime; + private final int historyPartitionLimit; + private final int maxLevel; + + public HistoryPartitionCluster( + FileStoreTable table, + Snapshot snapshot, + IncrementalClusterStrategy incrementalClusterStrategy, + InternalRowPartitionComputer partitionComputer, + PartitionPredicate specifiedPartitions, + Duration historyPartitionIdleTime, + int historyPartitionLimit) { + this.table = table; + this.snapshot = snapshot; + this.incrementalClusterStrategy = incrementalClusterStrategy; + this.partitionComputer = partitionComputer; + this.specifiedPartitions = specifiedPartitions; + this.historyPartitionIdleTime = historyPartitionIdleTime; + this.historyPartitionLimit = historyPartitionLimit; + this.maxLevel = table.coreOptions().numLevels() - 1; + } + + @Nullable + public static HistoryPartitionCluster create( + FileStoreTable table, + Snapshot snapshot, + IncrementalClusterStrategy incrementalClusterStrategy, + InternalRowPartitionComputer partitionComputer, + @Nullable PartitionPredicate specifiedPartitions) { + if (table.schema().partitionKeys().isEmpty()) { + return null; + } + if (specifiedPartitions == null) { + return null; + } + + Duration idleTime = table.coreOptions().clusteringHistoryPartitionIdleTime(); + if (idleTime == null) { + return null; + } + + int limit = table.coreOptions().clusteringHistoryPartitionLimit(); + return new HistoryPartitionCluster( + table, + snapshot, + incrementalClusterStrategy, + partitionComputer, + specifiedPartitions, + idleTime, + limit); + } + + public Map pickForHistoryPartitions() { + Map> partitionLevels = + constructLevelsForHistoryPartitions(); + logForPartitionLevel(partitionLevels, partitionComputer); + + Map units = new HashMap<>(); + partitionLevels.forEach( + (k, v) -> { + Optional pick = + incrementalClusterStrategy.pick(maxLevel + 1, v, true); + pick.ifPresent(compactUnit -> units.put(k, compactUnit)); + }); + return units; + } + + @VisibleForTesting + public Map> constructLevelsForHistoryPartitions() { + long historyMilli = + LocalDateTime.now() + .minus(historyPartitionIdleTime) + .atZone(ZoneId.systemDefault()) + .toInstant() + .toEpochMilli(); + + List historyPartitions = + table.newSnapshotReader().withSnapshot(snapshot) + .withLevelMinMaxFilter((min, max) -> min < maxLevel) + .withLevelFilter(level -> level < maxLevel).partitionEntries().stream() + .filter(entry -> entry.lastFileCreationTime() < historyMilli) + .sorted(Comparator.comparingLong(PartitionEntry::lastFileCreationTime)) + .map(PartitionEntry::partition) + .collect(Collectors.toList()); + + // read dataFileMeta for history partitions + List historyDataSplits = + table.newSnapshotReader() + .withSnapshot(snapshot) + .withPartitionFilter(historyPartitions) + .read() + .dataSplits(); + + Map> historyPartitionFiles = new HashMap<>(); + for (DataSplit dataSplit : historyDataSplits) { + historyPartitionFiles + .computeIfAbsent(dataSplit.partition(), k -> new ArrayList<>()) + .addAll(dataSplit.dataFiles()); + } + + return filterPartitions(historyPartitionFiles).entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> constructPartitionLevels(entry.getValue()))); + } + + private Map> filterPartitions( + Map> partitionFiles) { + Map> result = new HashMap<>(); + partitionFiles.forEach( + (part, files) -> { + if (specifiedPartitions.test(part)) { + // already contain in specified partitions + return; + } + + if (result.size() < historyPartitionLimit) { + // in limit, can be picked + result.put(part, files); + } + }); + LOG.info( + "Find {} history partitions for full clustering, the history partitions are {}", + result.size(), + result.keySet().stream() + .map(partitionComputer::generatePartValues) + .collect(Collectors.toSet())); + return result; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/append/cluster/IncrementalClusterManager.java b/paimon-core/src/main/java/org/apache/paimon/append/cluster/IncrementalClusterManager.java new file mode 100644 index 000000000000..20dc2a49d21b --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/append/cluster/IncrementalClusterManager.java @@ -0,0 +1,334 @@ +/* + * 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. + */ + +package org.apache.paimon.append.cluster; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.Snapshot; +import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.compact.CompactUnit; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.deletionvectors.append.AppendDeleteFileMaintainer; +import org.apache.paimon.deletionvectors.append.BaseAppendDeleteFileMaintainer; +import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.io.CompactIncrement; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.io.DataIncrement; +import org.apache.paimon.manifest.FileKind; +import org.apache.paimon.manifest.IndexManifestEntry; +import org.apache.paimon.mergetree.LevelSortedRun; +import org.apache.paimon.mergetree.SortedRun; +import org.apache.paimon.partition.PartitionPredicate; +import org.apache.paimon.table.BucketMode; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.DeletionFile; +import org.apache.paimon.table.source.SplitGenerator; +import org.apache.paimon.table.source.snapshot.SnapshotReader; +import org.apache.paimon.utils.InternalRowPartitionComputer; +import org.apache.paimon.utils.Pair; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +import static org.apache.paimon.CoreOptions.CLUSTERING_INCREMENTAL; +import static org.apache.paimon.utils.Preconditions.checkArgument; + +/** Manager for Incremental Clustering. */ +public class IncrementalClusterManager { + + private static final Logger LOG = LoggerFactory.getLogger(IncrementalClusterManager.class); + + private final FileStoreTable table; + private final InternalRowPartitionComputer partitionComputer; + private final Snapshot snapshot; + private final SnapshotReader snapshotReader; + private final IncrementalClusterStrategy incrementalClusterStrategy; + private final CoreOptions.OrderType clusterCurve; + private final List clusterKeys; + private final int numLevels; + private final @Nullable HistoryPartitionCluster historyPartitionCluster; + + public IncrementalClusterManager(FileStoreTable table) { + this(table, null); + } + + public IncrementalClusterManager( + FileStoreTable table, @Nullable PartitionPredicate specifiedPartitions) { + checkArgument( + table.bucketMode() == BucketMode.BUCKET_UNAWARE, + "only append unaware-bucket table support incremental clustering."); + this.table = table; + CoreOptions options = table.coreOptions(); + checkArgument( + options.clusteringIncrementalEnabled(), + "Only support incremental clustering when '%s' is true.", + CLUSTERING_INCREMENTAL.key()); + this.numLevels = options.numLevels(); + this.partitionComputer = + new InternalRowPartitionComputer( + table.coreOptions().partitionDefaultName(), + table.store().partitionType(), + table.partitionKeys().toArray(new String[0]), + table.coreOptions().legacyPartitionName()); + this.snapshot = table.snapshotManager().latestSnapshot(); + this.snapshotReader = + table.newSnapshotReader() + .dropStats() + .withPartitionFilter(specifiedPartitions) + .withSnapshot(snapshot); + this.incrementalClusterStrategy = + new IncrementalClusterStrategy( + table.schemaManager(), + options.clusteringColumns(), + options.maxSizeAmplificationPercent(), + options.sortedRunSizeRatio(), + options.numSortedRunCompactionTrigger()); + this.clusterCurve = options.clusteringStrategy(options.clusteringColumns().size()); + this.clusterKeys = options.clusteringColumns(); + this.historyPartitionCluster = + HistoryPartitionCluster.create( + table, + snapshot, + incrementalClusterStrategy, + partitionComputer, + specifiedPartitions); + } + + public Map prepareForCluster(boolean fullCompaction) { + // 1. construct LSM structure for each partition + Map> partitionLevels = constructLevels(); + logForPartitionLevel(partitionLevels, partitionComputer); + + // 2. pick files to be clustered for each partition + Map units = new HashMap<>(); + partitionLevels.forEach( + (k, v) -> { + Optional pick = + incrementalClusterStrategy.pick(numLevels, v, fullCompaction); + pick.ifPresent(compactUnit -> units.put(k, compactUnit)); + }); + + if (historyPartitionCluster != null) { + units.putAll(historyPartitionCluster.pickForHistoryPartitions()); + } + + if (LOG.isDebugEnabled()) { + units.forEach( + (partition, compactUnit) -> { + String filesInfo = + compactUnit.files().stream() + .map( + file -> + String.format( + "%s,%s,%s", + file.fileName(), + file.level(), + file.fileSize())) + .collect(Collectors.joining(", ")); + LOG.debug( + "Partition {}, outputLevel:{}, clustered with {} files: [{}]", + partitionComputer.generatePartValues(partition), + compactUnit.outputLevel(), + compactUnit.files().size(), + filesInfo); + }); + } + return units; + } + + public Map> constructLevels() { + List dataSplits = snapshotReader.read().dataSplits(); + Map> partitionFiles = new HashMap<>(); + for (DataSplit dataSplit : dataSplits) { + partitionFiles + .computeIfAbsent(dataSplit.partition(), k -> new ArrayList<>()) + .addAll(dataSplit.dataFiles()); + } + return partitionFiles.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + entry -> constructPartitionLevels(entry.getValue()))); + } + + public static List constructPartitionLevels(List partitionFiles) { + List partitionLevels = new ArrayList<>(); + Map> levelMap = + partitionFiles.stream().collect(Collectors.groupingBy(DataFileMeta::level)); + + for (Map.Entry> entry : levelMap.entrySet()) { + int level = entry.getKey(); + if (level == 0) { + for (DataFileMeta level0File : entry.getValue()) { + partitionLevels.add( + new LevelSortedRun(level, SortedRun.fromSingle(level0File))); + } + } else { + // don't need to guarantee that the files within the same sorted run are + // non-overlapping here, so we call SortedRun.fromSorted() to avoid sorting and + // validation + partitionLevels.add( + new LevelSortedRun(level, SortedRun.fromSorted(entry.getValue()))); + } + } + + // sort by level + partitionLevels.sort(Comparator.comparing(LevelSortedRun::level)); + return partitionLevels; + } + + public Map, CommitMessage>> toSplitsAndRewriteDvFiles( + Map compactUnits) { + Map, CommitMessage>> result = new HashMap<>(); + boolean dvEnabled = table.coreOptions().deletionVectorsEnabled(); + for (BinaryRow partition : compactUnits.keySet()) { + CompactUnit unit = compactUnits.get(partition); + AppendDeleteFileMaintainer dvMaintainer = + dvEnabled + ? BaseAppendDeleteFileMaintainer.forUnawareAppend( + table.store().newIndexFileHandler(), snapshot, partition) + : null; + List splits = new ArrayList<>(); + + DataSplit.Builder builder = + DataSplit.builder() + .withPartition(partition) + .withBucket(0) + .withTotalBuckets(1) + .isStreaming(false); + + SplitGenerator splitGenerator = snapshotReader.splitGenerator(); + List splitGroups = + splitGenerator.splitForBatch(unit.files()); + + for (SplitGenerator.SplitGroup splitGroup : splitGroups) { + List dataFiles = splitGroup.files; + + String bucketPath = + snapshotReader.pathFactory().bucketPath(partition, 0).toString(); + builder.withDataFiles(dataFiles) + .rawConvertible(splitGroup.rawConvertible) + .withBucketPath(bucketPath); + + if (dvMaintainer != null) { + List dataDeletionFiles = new ArrayList<>(); + for (DataFileMeta file : dataFiles) { + DeletionFile deletionFile = + dvMaintainer.notifyRemovedDeletionVector(file.fileName()); + dataDeletionFiles.add(deletionFile); + } + builder.withDataDeletionFiles(dataDeletionFiles); + } + splits.add(builder.build()); + } + + // generate delete dv index meta + CommitMessage dvCommitMessage = null; + if (dvMaintainer != null) { + List newIndexFiles = new ArrayList<>(); + List deletedIndexFiles = new ArrayList<>(); + List indexEntries = dvMaintainer.persist(); + for (IndexManifestEntry entry : indexEntries) { + if (entry.kind() == FileKind.ADD) { + newIndexFiles.add(entry.indexFile()); + } else { + deletedIndexFiles.add(entry.indexFile()); + } + } + dvCommitMessage = + new CommitMessageImpl( + dvMaintainer.getPartition(), + 0, + table.coreOptions().bucket(), + DataIncrement.emptyIncrement(), + new CompactIncrement( + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + newIndexFiles, + deletedIndexFiles)); + } + + result.put(partition, Pair.of(splits, dvCommitMessage)); + } + + return result; + } + + public static List upgrade( + List filesAfterCluster, int outputLevel) { + return filesAfterCluster.stream() + .map(file -> file.upgrade(outputLevel)) + .collect(Collectors.toList()); + } + + public static void logForPartitionLevel( + Map> partitionLevels, + InternalRowPartitionComputer partitionComputer) { + if (LOG.isDebugEnabled()) { + partitionLevels.forEach( + (partition, levelSortedRuns) -> { + String runsInfo = + levelSortedRuns.stream() + .map( + lsr -> + String.format( + "level-%s:%s", + lsr.level(), + lsr.run().files().size())) + .collect(Collectors.joining(",")); + LOG.debug( + "Partition {} has {} runs: [{}]", + partitionComputer.generatePartValues(partition), + levelSortedRuns.size(), + runsInfo); + }); + } + } + + public CoreOptions.OrderType clusterCurve() { + return clusterCurve; + } + + public List clusterKeys() { + return clusterKeys; + } + + public Snapshot snapshot() { + return snapshot; + } + + @VisibleForTesting + HistoryPartitionCluster historyPartitionCluster() { + return historyPartitionCluster; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/append/cluster/IncrementalClusterStrategy.java b/paimon-core/src/main/java/org/apache/paimon/append/cluster/IncrementalClusterStrategy.java new file mode 100644 index 000000000000..1a90c16ce382 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/append/cluster/IncrementalClusterStrategy.java @@ -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. + */ + +package org.apache.paimon.append.cluster; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.compact.CompactUnit; +import org.apache.paimon.mergetree.LevelSortedRun; +import org.apache.paimon.mergetree.compact.UniversalCompaction; +import org.apache.paimon.schema.SchemaManager; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Optional; + +/** Cluster strategy to decide which files to select for cluster. */ +public class IncrementalClusterStrategy { + + private static final Logger LOG = LoggerFactory.getLogger(IncrementalClusterStrategy.class); + + private final List clusterKeys; + private final SchemaManager schemaManager; + + private final UniversalCompaction universalCompaction; + + public IncrementalClusterStrategy( + SchemaManager schemaManager, + List clusterKeys, + int maxSizeAmp, + int sizeRatio, + int numRunCompactionTrigger) { + this.universalCompaction = + new UniversalCompaction(maxSizeAmp, sizeRatio, numRunCompactionTrigger, null, null); + this.clusterKeys = clusterKeys; + this.schemaManager = schemaManager; + } + + public Optional pick( + int numLevels, List runs, boolean fullCompaction) { + if (fullCompaction) { + return pickFullCompaction(numLevels, runs); + } + return universalCompaction.pick(numLevels, runs); + } + + public Optional pickFullCompaction(int numLevels, List runs) { + int maxLevel = numLevels - 1; + if (runs.isEmpty()) { + // no sorted run, no need to compact + if (LOG.isDebugEnabled()) { + LOG.debug("no sorted run, no need to compact"); + } + return Optional.empty(); + } + + if (runs.size() == 1 && runs.get(0).level() == maxLevel) { + long schemaId = runs.get(0).run().files().get(0).schemaId(); + CoreOptions coreOptions = CoreOptions.fromMap(schemaManager.schema(schemaId).options()); + // only one sorted run in the maxLevel with the same cluster key + if (coreOptions.clusteringColumns().equals(clusterKeys)) { + if (LOG.isDebugEnabled()) { + LOG.debug( + "only one sorted run in the maxLevel with the same cluster key, no need to compact"); + } + return Optional.empty(); + } + } + + // full compaction + return Optional.of(CompactUnit.fromLevelRuns(maxLevel, runs)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index 841643383cd8..5df78212f1ec 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -250,17 +250,27 @@ public List listTables(String databaseName) throws DatabaseNotExistExcep @Override public PagedList listTablesPaged( - String databaseName, Integer maxResults, String pageToken, String tableNamePattern) + String databaseName, + Integer maxResults, + String pageToken, + String tableNamePattern, + String tableType) throws DatabaseNotExistException { CatalogUtils.validateNamePattern(this, tableNamePattern); + CatalogUtils.validateTableType(this, tableType); return new PagedList<>(listTables(databaseName), null); } @Override public PagedList listTableDetailsPaged( - String databaseName, Integer maxResults, String pageToken, String tableNamePattern) + String databaseName, + @Nullable Integer maxResults, + @Nullable String pageToken, + @Nullable String tableNamePattern, + @Nullable String tableType) throws DatabaseNotExistException { CatalogUtils.validateNamePattern(this, tableNamePattern); + CatalogUtils.validateTableType(this, tableType); if (isSystemDatabase(databaseName)) { List
systemTables = SystemTableLoader.loadGlobalTableNames().stream() @@ -285,16 +295,21 @@ public PagedList
listTableDetailsPaged( // check db exists getDatabase(databaseName); - return listTableDetailsPagedImpl(databaseName, maxResults, pageToken); + return listTableDetailsPagedImpl( + databaseName, maxResults, pageToken, tableNamePattern, tableType); } protected abstract List listTablesImpl(String databaseName); protected PagedList
listTableDetailsPagedImpl( - String databaseName, Integer maxResults, String pageToken) + String databaseName, + @Nullable Integer maxResults, + @Nullable String pageToken, + @Nullable String tableNamePattern, + @Nullable String tableType) throws DatabaseNotExistException { PagedList pagedTableNames = - listTablesPaged(databaseName, maxResults, pageToken, null); + listTablesPaged(databaseName, maxResults, pageToken, tableNamePattern, tableType); return new PagedList<>( pagedTableNames.getElements().stream() .map( diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index 702dc155a5e8..a3510777d729 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -186,7 +186,8 @@ PagedList listTablesPaged( String databaseName, @Nullable Integer maxResults, @Nullable String pageToken, - @Nullable String tableNamePattern) + @Nullable String tableNamePattern, + @Nullable String tableType) throws DatabaseNotExistException; /** @@ -203,6 +204,8 @@ PagedList listTablesPaged( * from a specific point. * @param tableNamePattern A sql LIKE pattern (%) for table names. All table details will be * returned if not set or empty. Currently, only prefix matching is supported. + * @param tableType Optional parameter to filter tables by table type. All table types will be + * returned if not set or empty. * @return a list of the table details with provided page size in this database and next page * token, or a list of the details of all tables in this database if the catalog does not * {@link #supportsListObjectsPaged()}. @@ -213,7 +216,8 @@ PagedList
listTableDetailsPaged( String databaseName, @Nullable Integer maxResults, @Nullable String pageToken, - @Nullable String tableNamePattern) + @Nullable String tableNamePattern, + @Nullable String tableType) throws DatabaseNotExistException; /** @@ -576,12 +580,12 @@ default void registerTable(Identifier identifier, String path) /** * Whether this catalog supports list objects paged. If not, corresponding methods will fall * back to listing all objects. For example, {@link #listTablesPaged(String, Integer, String, - * String)} would fall back to {@link #listTables(String)}. + * String, String)} would fall back to {@link #listTables(String)}. * *
    *
  • {@link #listDatabasesPaged(Integer, String, String)}. - *
  • {@link #listTablesPaged(String, Integer, String, String)}. - *
  • {@link #listTableDetailsPaged(String, Integer, String, String)}. + *
  • {@link #listTablesPaged(String, Integer, String, String, String)}. + *
  • {@link #listTableDetailsPaged(String, Integer, String, String, String)}. *
  • {@link #listViewsPaged(String, Integer, String, String)}. *
  • {@link #listViewDetailsPaged(String, Integer, String, String)}. *
  • {@link #listPartitionsPaged(Identifier, Integer, String, String)}. @@ -595,8 +599,8 @@ default void registerTable(Identifier identifier, String path) * *
      *
    • {@link #listDatabasesPaged(Integer, String, String)}. - *
    • {@link #listTablesPaged(String, Integer, String, String)}. - *
    • {@link #listTableDetailsPaged(String, Integer, String, String)}. + *
    • {@link #listTablesPaged(String, Integer, String, String, String)}. + *
    • {@link #listTableDetailsPaged(String, Integer, String, String, String)}. *
    • {@link #listViewsPaged(String, Integer, String, String)}. *
    • {@link #listViewDetailsPaged(String, Integer, String, String)}. *
    • {@link #listPartitionsPaged(Identifier, Integer, String, String)}. @@ -606,6 +610,10 @@ default boolean supportsListByPattern() { return false; } + default boolean supportsListTableByType() { + return false; + } + // ==================== Version management methods ========================== /** diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java index fe204bfc8767..c36993ff0af4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CatalogUtils.java @@ -163,6 +163,15 @@ public static void validateNamePattern(Catalog catalog, String namePattern) { } } + public static void validateTableType(Catalog catalog, String tableType) { + if (Objects.nonNull(tableType) && !catalog.supportsListTableByType()) { + throw new UnsupportedOperationException( + String.format( + "Current catalog %s does not support table type filter.", + catalog.getClass().getSimpleName())); + } + } + public static List listPartitionsFromFileSystem(Table table) { Options options = Options.fromMap(table.options()); InternalRowPartitionComputer computer = diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java index 08776714a83f..a4e90e50499c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/DelegateCatalog.java @@ -102,16 +102,26 @@ public List listTables(String databaseName) throws DatabaseNotExistExcep @Override public PagedList listTablesPaged( - String databaseName, Integer maxResults, String pageToken, String tableNamePattern) + String databaseName, + Integer maxResults, + String pageToken, + String tableNamePattern, + String tableType) throws DatabaseNotExistException { - return wrapped.listTablesPaged(databaseName, maxResults, pageToken, tableNamePattern); + return wrapped.listTablesPaged( + databaseName, maxResults, pageToken, tableNamePattern, tableType); } @Override public PagedList
listTableDetailsPaged( - String databaseName, Integer maxResults, String pageToken, String tableNamePattern) + String databaseName, + Integer maxResults, + String pageToken, + String tableNamePattern, + String tableType) throws DatabaseNotExistException { - return wrapped.listTableDetailsPaged(databaseName, maxResults, pageToken, tableNamePattern); + return wrapped.listTableDetailsPaged( + databaseName, maxResults, pageToken, tableNamePattern, tableType); } @Override @@ -165,6 +175,11 @@ public boolean supportsListByPattern() { return wrapped.supportsListByPattern(); } + @Override + public boolean supportsListTableByType() { + return wrapped.supportsListTableByType(); + } + @Override public boolean supportsVersionManagement() { return wrapped.supportsVersionManagement(); diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java index 67b97389175b..6ce9e1c3875e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java @@ -52,7 +52,6 @@ public DeletionVectorIndexFileWriter( *

TODO: We can consider sending a message to delete the deletion file in the future. */ public IndexFileMeta writeSingleFile(Map input) throws IOException { - DeletionFileWriter writer = new DeletionFileWriter(indexPathFactory, fileIO); try { for (Map.Entry entry : input.entrySet()) { diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index 1a392b0ae9e5..3714d59bfb24 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -174,15 +174,15 @@ private static Object toTypeObject(DataType dataType, int fieldId, int depth) { case TIMESTAMP_WITHOUT_TIME_ZONE: int timestampPrecision = ((TimestampType) dataType).getPrecision(); Preconditions.checkArgument( - timestampPrecision > 3 && timestampPrecision <= 6, - "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6."); - return "timestamp"; + timestampPrecision > 3 && timestampPrecision <= 9, + "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 9."); + return timestampPrecision >= 7 ? "timestamp_ns" : "timestamp"; case TIMESTAMP_WITH_LOCAL_TIME_ZONE: int timestampLtzPrecision = ((LocalZonedTimestampType) dataType).getPrecision(); Preconditions.checkArgument( - timestampLtzPrecision > 3 && timestampLtzPrecision <= 6, - "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6."); - return "timestamptz"; + timestampLtzPrecision > 3 && timestampLtzPrecision <= 9, + "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 9."); + return timestampLtzPrecision >= 7 ? "timestamptz_ns" : "timestamptz"; case ARRAY: ArrayType arrayType = (ArrayType) dataType; return new IcebergListType( diff --git a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java index 4496ec50f4b2..51d46f62e9b4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java @@ -85,10 +85,11 @@ public static LinkedHashMap rowArrayDataToDvMetas( LinkedHashMap dvMetas = new LinkedHashMap<>(arrayData.size()); for (int i = 0; i < arrayData.size(); i++) { InternalRow row = arrayData.getRow(i, DeletionVectorMeta.SCHEMA.getFieldCount()); + String dataFileName = row.getString(0).toString(); dvMetas.put( - row.getString(0).toString(), + dataFileName, new DeletionVectorMeta( - row.getString(0).toString(), + dataFileName, row.getInt(1), row.getInt(2), row.isNullAt(3) ? null : row.getLong(3))); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/CompactIncrement.java b/paimon-core/src/main/java/org/apache/paimon/io/CompactIncrement.java index ad51642a2a4b..6454c99ef8ab 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/CompactIncrement.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/CompactIncrement.java @@ -18,6 +18,9 @@ package org.apache.paimon.io; +import org.apache.paimon.index.IndexFileMeta; + +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; @@ -29,14 +32,27 @@ public class CompactIncrement { private final List compactBefore; private final List compactAfter; private final List changelogFiles; + private final List newIndexFiles; + private final List deletedIndexFiles; public CompactIncrement( List compactBefore, List compactAfter, List changelogFiles) { + this(compactBefore, compactAfter, changelogFiles, new ArrayList<>(), new ArrayList<>()); + } + + public CompactIncrement( + List compactBefore, + List compactAfter, + List changelogFiles, + List newIndexFiles, + List deletedIndexFiles) { this.compactBefore = compactBefore; this.compactAfter = compactAfter; this.changelogFiles = changelogFiles; + this.newIndexFiles = newIndexFiles; + this.deletedIndexFiles = deletedIndexFiles; } public List compactBefore() { @@ -51,8 +67,20 @@ public List changelogFiles() { return changelogFiles; } + public List newIndexFiles() { + return newIndexFiles; + } + + public List deletedIndexFiles() { + return deletedIndexFiles; + } + public boolean isEmpty() { - return compactBefore.isEmpty() && compactAfter.isEmpty() && changelogFiles.isEmpty(); + return compactBefore.isEmpty() + && compactAfter.isEmpty() + && changelogFiles.isEmpty() + && newIndexFiles.isEmpty() + && deletedIndexFiles.isEmpty(); } @Override @@ -67,7 +95,9 @@ public boolean equals(Object o) { CompactIncrement that = (CompactIncrement) o; return Objects.equals(compactBefore, that.compactBefore) && Objects.equals(compactAfter, that.compactAfter) - && Objects.equals(changelogFiles, that.changelogFiles); + && Objects.equals(changelogFiles, that.changelogFiles) + && Objects.equals(newIndexFiles, that.newIndexFiles) + && Objects.equals(deletedIndexFiles, that.deletedIndexFiles); } @Override @@ -78,10 +108,14 @@ public int hashCode() { @Override public String toString() { return String.format( - "CompactIncrement {compactBefore = %s, compactAfter = %s, changelogFiles = %s}", + "CompactIncrement {compactBefore = %s, compactAfter = %s, changelogFiles = %s, newIndexFiles = %s, deletedIndexFiles = %s}", compactBefore.stream().map(DataFileMeta::fileName).collect(Collectors.toList()), compactAfter.stream().map(DataFileMeta::fileName).collect(Collectors.toList()), - changelogFiles.stream().map(DataFileMeta::fileName).collect(Collectors.toList())); + changelogFiles.stream().map(DataFileMeta::fileName).collect(Collectors.toList()), + newIndexFiles.stream().map(IndexFileMeta::fileName).collect(Collectors.toList()), + deletedIndexFiles.stream() + .map(IndexFileMeta::fileName) + .collect(Collectors.toList())); } public static CompactIncrement emptyIncrement() { diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java index f7b283daf013..8f7c9d6dad3b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java @@ -48,7 +48,7 @@ public class DataFileRecordReader implements FileRecordReader { @Nullable private final int[] indexMapping; @Nullable private final PartitionInfo partitionInfo; @Nullable private final CastFieldGetter[] castMapping; - private final boolean rowLineageEnabled; + private final boolean rowTrackingEnabled; @Nullable private final Long firstRowId; private final long maxSequenceNumber; private final Map systemFields; @@ -60,7 +60,7 @@ public DataFileRecordReader( @Nullable int[] indexMapping, @Nullable CastFieldGetter[] castMapping, @Nullable PartitionInfo partitionInfo, - boolean rowLineageEnabled, + boolean rowTrackingEnabled, @Nullable Long firstRowId, long maxSequenceNumber, Map systemFields) @@ -75,7 +75,7 @@ public DataFileRecordReader( this.indexMapping = indexMapping; this.partitionInfo = partitionInfo; this.castMapping = castMapping; - this.rowLineageEnabled = rowLineageEnabled; + this.rowTrackingEnabled = rowTrackingEnabled; this.firstRowId = firstRowId; this.maxSequenceNumber = maxSequenceNumber; this.systemFields = systemFields; @@ -91,10 +91,10 @@ public FileRecordIterator readBatch() throws IOException { if (iterator instanceof ColumnarRowIterator) { iterator = ((ColumnarRowIterator) iterator).mapping(partitionInfo, indexMapping); - if (rowLineageEnabled) { + if (rowTrackingEnabled) { iterator = ((ColumnarRowIterator) iterator) - .assignRowLineage(firstRowId, maxSequenceNumber, systemFields); + .assignRowTracking(firstRowId, maxSequenceNumber, systemFields); } } else { if (partitionInfo != null) { @@ -108,33 +108,33 @@ public FileRecordIterator readBatch() throws IOException { iterator = iterator.transform(projectedRow::replaceRow); } - if (rowLineageEnabled && !systemFields.isEmpty()) { - GenericRow lineageRow = new GenericRow(2); + if (rowTrackingEnabled && !systemFields.isEmpty()) { + GenericRow trackingRow = new GenericRow(2); - int[] fallbackToLineageMappings = new int[tableRowType.getFieldCount()]; - Arrays.fill(fallbackToLineageMappings, -1); + int[] fallbackToTrackingMappings = new int[tableRowType.getFieldCount()]; + Arrays.fill(fallbackToTrackingMappings, -1); if (systemFields.containsKey(SpecialFields.ROW_ID.name())) { - fallbackToLineageMappings[systemFields.get(SpecialFields.ROW_ID.name())] = 0; + fallbackToTrackingMappings[systemFields.get(SpecialFields.ROW_ID.name())] = 0; } if (systemFields.containsKey(SpecialFields.SEQUENCE_NUMBER.name())) { - fallbackToLineageMappings[ + fallbackToTrackingMappings[ systemFields.get(SpecialFields.SEQUENCE_NUMBER.name())] = 1; } FallbackMappingRow fallbackMappingRow = - new FallbackMappingRow(fallbackToLineageMappings); + new FallbackMappingRow(fallbackToTrackingMappings); final FileRecordIterator iteratorInner = iterator; iterator = iterator.transform( row -> { if (firstRowId != null) { - lineageRow.setField( + trackingRow.setField( 0, iteratorInner.returnedPosition() + firstRowId); } - lineageRow.setField(1, maxSequenceNumber); - return fallbackMappingRow.replace(row, lineageRow); + trackingRow.setField(1, maxSequenceNumber); + return fallbackMappingRow.replace(row, trackingRow); }); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataIncrement.java b/paimon-core/src/main/java/org/apache/paimon/io/DataIncrement.java index b7860fb15e79..6049c4dbbb0c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataIncrement.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataIncrement.java @@ -18,25 +18,41 @@ package org.apache.paimon.io; +import org.apache.paimon.index.IndexFileMeta; + +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.stream.Collectors; -/** Newly created data files and changelog files. */ +/** Increment of data files, changelog files and index files. */ public class DataIncrement { private final List newFiles; private final List deletedFiles; private final List changelogFiles; + private final List newIndexFiles; + private final List deletedIndexFiles; public DataIncrement( List newFiles, List deletedFiles, List changelogFiles) { + this(newFiles, deletedFiles, changelogFiles, new ArrayList<>(), new ArrayList<>()); + } + + public DataIncrement( + List newFiles, + List deletedFiles, + List changelogFiles, + List newIndexFiles, + List deletedIndexFiles) { this.newFiles = newFiles; this.deletedFiles = deletedFiles; this.changelogFiles = changelogFiles; + this.newIndexFiles = newIndexFiles; + this.deletedIndexFiles = deletedIndexFiles; } public static DataIncrement emptyIncrement() { @@ -56,8 +72,20 @@ public List changelogFiles() { return changelogFiles; } + public List newIndexFiles() { + return newIndexFiles; + } + + public List deletedIndexFiles() { + return deletedIndexFiles; + } + public boolean isEmpty() { - return newFiles.isEmpty() && changelogFiles.isEmpty(); + return newFiles.isEmpty() + && deletedFiles.isEmpty() + && changelogFiles.isEmpty() + && newIndexFiles.isEmpty() + && deletedIndexFiles.isEmpty(); } @Override @@ -71,20 +99,28 @@ public boolean equals(Object o) { DataIncrement that = (DataIncrement) o; return Objects.equals(newFiles, that.newFiles) - && Objects.equals(changelogFiles, that.changelogFiles); + && Objects.equals(deletedFiles, that.deletedFiles) + && Objects.equals(changelogFiles, that.changelogFiles) + && Objects.equals(newIndexFiles, that.newIndexFiles) + && Objects.equals(deletedIndexFiles, that.deletedIndexFiles); } @Override public int hashCode() { - return Objects.hash(newFiles, changelogFiles); + return Objects.hash( + newFiles, deletedFiles, changelogFiles, newIndexFiles, deletedIndexFiles); } @Override public String toString() { return String.format( - "DataIncrement {newFiles = %s, deletedFiles = %s, changelogFiles = %s}", + "DataIncrement {newFiles = %s, deletedFiles = %s, changelogFiles = %s, newIndexFiles = %s, deletedIndexFiles = %s}", newFiles.stream().map(DataFileMeta::fileName).collect(Collectors.toList()), deletedFiles.stream().map(DataFileMeta::fileName).collect(Collectors.toList()), - changelogFiles.stream().map(DataFileMeta::fileName).collect(Collectors.toList())); + changelogFiles.stream().map(DataFileMeta::fileName).collect(Collectors.toList()), + newIndexFiles.stream().map(IndexFileMeta::fileName).collect(Collectors.toList()), + deletedIndexFiles.stream() + .map(IndexFileMeta::fileName) + .collect(Collectors.toList())); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/IndexIncrement.java b/paimon-core/src/main/java/org/apache/paimon/io/IndexIncrement.java deleted file mode 100644 index 9f985f54ed41..000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/io/IndexIncrement.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.io; - -import org.apache.paimon.index.IndexFileMeta; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; - -/** Incremental index files. */ -public class IndexIncrement { - - private final List newIndexFiles; - - private final List deletedIndexFiles; - - public IndexIncrement(List newIndexFiles) { - this.newIndexFiles = newIndexFiles; - this.deletedIndexFiles = Collections.emptyList(); - } - - public IndexIncrement( - List newIndexFiles, List deletedIndexFiles) { - this.newIndexFiles = newIndexFiles; - this.deletedIndexFiles = deletedIndexFiles; - } - - public List newIndexFiles() { - return newIndexFiles; - } - - public List deletedIndexFiles() { - return deletedIndexFiles; - } - - public boolean isEmpty() { - return newIndexFiles.isEmpty() && deletedIndexFiles.isEmpty(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - IndexIncrement that = (IndexIncrement) o; - return Objects.equals(newIndexFiles, that.newIndexFiles) - && Objects.equals(deletedIndexFiles, that.deletedIndexFiles); - } - - @Override - public int hashCode() { - List all = new ArrayList<>(newIndexFiles); - all.addAll(deletedIndexFiles); - return Objects.hash(all); - } - - @Override - public String toString() { - return String.format( - "IndexIncrement {newIndexFiles = %s, deletedIndexFiles = %s}", - newIndexFiles.stream().map(IndexFileMeta::fileName).collect(Collectors.toList()), - deletedIndexFiles.stream() - .map(IndexFileMeta::fileName) - .collect(Collectors.toList())); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/io/PojoDataFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/io/PojoDataFileMeta.java index 6e858fbd2286..88f1a740edea 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/PojoDataFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/PojoDataFileMeta.java @@ -50,7 +50,7 @@ public class PojoDataFileMeta implements DataFileMeta { private final SimpleStats keyStats; private final SimpleStats valueStats; - // As for row-lineage table, this will be reassigned while committing + // As for row-tracking table, this will be reassigned while committing private final long minSequenceNumber; private final long maxSequenceNumber; private final long schemaId; diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java index 52b029563e9c..b400258b8439 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java @@ -22,7 +22,6 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.Filter; -import org.apache.paimon.utils.Preconditions; import javax.annotation.Nullable; @@ -40,6 +39,7 @@ import static org.apache.paimon.utils.ManifestReadThreadPool.randomlyExecuteSequentialReturn; import static org.apache.paimon.utils.ManifestReadThreadPool.sequentialBatchedExecute; +import static org.apache.paimon.utils.Preconditions.checkState; /** Entry representing a file. */ public interface FileEntry { @@ -77,7 +77,7 @@ class Identifier { public final int level; public final String fileName; public final List extraFiles; - @Nullable private final byte[] embeddedIndex; + @Nullable public final byte[] embeddedIndex; @Nullable public final String externalPath; /* Cache the hash code for the string */ @@ -190,7 +190,7 @@ static void mergeEntries(Iterable entries, Map from(List entries) { return entries.stream().map(SimpleFileEntry::from).collect(Collectors.toList()); } @@ -115,6 +130,11 @@ public String fileName() { return fileName; } + @Nullable + public byte[] embeddedIndex() { + return embeddedIndex; + } + @Nullable @Override public String externalPath() { diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntryWithDV.java b/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntryWithDV.java new file mode 100644 index 000000000000..75d73f345f21 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntryWithDV.java @@ -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. + */ + +package org.apache.paimon.manifest; + +import javax.annotation.Nullable; + +import java.util.Objects; + +/** A {@link FileEntry} contains {@link SimpleFileEntry} and dv file name. */ +public class SimpleFileEntryWithDV extends SimpleFileEntry { + + @Nullable private final String dvFileName; + + public SimpleFileEntryWithDV(SimpleFileEntry entry, @Nullable String dvFileName) { + super( + entry.kind(), + entry.partition(), + entry.bucket(), + entry.totalBuckets(), + entry.level(), + entry.fileName(), + entry.extraFiles(), + entry.embeddedIndex(), + entry.minKey(), + entry.maxKey(), + entry.externalPath()); + this.dvFileName = dvFileName; + } + + public Identifier identifier() { + return new IdentifierWithDv(super.identifier(), dvFileName); + } + + @Nullable + public String dvFileName() { + return dvFileName; + } + + public SimpleFileEntry toDelete() { + return new SimpleFileEntryWithDV(super.toDelete(), dvFileName); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + SimpleFileEntryWithDV that = (SimpleFileEntryWithDV) o; + return Objects.equals(dvFileName, that.dvFileName); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), dvFileName); + } + + @Override + public String toString() { + return super.toString() + ", {dvFileName=" + dvFileName + '}'; + } + + /** + * The same {@link Identifier} indicates that the {@link ManifestEntry} refers to the same data + * file. + */ + static class IdentifierWithDv extends Identifier { + + private final String dvFileName; + + public IdentifierWithDv(Identifier identifier, String dvFileName) { + super( + identifier.partition, + identifier.bucket, + identifier.level, + identifier.fileName, + identifier.extraFiles, + identifier.embeddedIndex, + identifier.externalPath); + this.dvFileName = dvFileName; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + IdentifierWithDv that = (IdentifierWithDv) o; + return Objects.equals(dvFileName, that.dvFileName); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), dvFileName); + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/KeyValueBuffer.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/KeyValueBuffer.java index 597e2847b55b..7e40ddaa8660 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/KeyValueBuffer.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/KeyValueBuffer.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.KeyValue; +import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.disk.ExternalBuffer; import org.apache.paimon.disk.IOManager; @@ -40,6 +41,7 @@ import java.util.Comparator; import java.util.List; import java.util.NoSuchElementException; +import java.util.function.Supplier; /** A buffer to cache {@link KeyValue}s. */ public interface KeyValueBuffer { @@ -65,6 +67,12 @@ public HybridBuffer(int threshold, LazyField lazyBinaryBuffer) { this.lazyBinaryBuffer = lazyBinaryBuffer; } + @Nullable + @VisibleForTesting + BinaryBuffer binaryBuffer() { + return binaryBuffer; + } + @Override public void reset() { listBuffer.reset(); @@ -89,7 +97,9 @@ public void put(KeyValue kv) { private void spillToBinary() { BinaryBuffer binaryBuffer = lazyBinaryBuffer.get(); try (CloseableIterator iterator = listBuffer.iterator()) { - binaryBuffer.put(iterator.next()); + while (iterator.hasNext()) { + binaryBuffer.put(iterator.next()); + } } catch (Exception e) { throw new RuntimeException(e); } @@ -162,20 +172,20 @@ public CloseableIterator iterator() { return new CloseableIterator() { private boolean hasNextWasCalled = false; - private boolean hasNext = false; + private boolean nextResult = false; @Override public boolean hasNext() { if (!hasNextWasCalled) { - hasNext = iterator.advanceNext(); + nextResult = iterator.advanceNext(); hasNextWasCalled = true; } - return hasNext; + return nextResult; } @Override public KeyValue next() { - if (!hasNext) { + if (!hasNext()) { throw new NoSuchElementException(); } hasNextWasCalled = false; @@ -215,6 +225,17 @@ static BinaryBuffer createBinaryBuffer( return new BinaryBuffer(buffer, kvSerializer); } + static HybridBuffer createHybridBuffer( + CoreOptions options, + RowType keyType, + RowType valueType, + @Nullable IOManager ioManager) { + Supplier binarySupplier = + () -> createBinaryBuffer(options, keyType, valueType, ioManager); + int threshold = options == null ? 1024 : options.lookupMergeRecordsThreshold(); + return new HybridBuffer(threshold, new LazyField<>(binarySupplier)); + } + static void insertInto( KeyValueBuffer buffer, KeyValue highLevel, Comparator comparator) { List newCandidates = new ArrayList<>(); diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/LookupMergeFunction.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/LookupMergeFunction.java index bcd0692757b4..1bd9aaa84339 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/LookupMergeFunction.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/LookupMergeFunction.java @@ -22,17 +22,12 @@ import org.apache.paimon.KeyValue; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; -import org.apache.paimon.mergetree.compact.KeyValueBuffer.BinaryBuffer; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.CloseableIterator; -import org.apache.paimon.utils.LazyField; import javax.annotation.Nullable; import java.util.Comparator; -import java.util.function.Supplier; - -import static org.apache.paimon.mergetree.compact.KeyValueBuffer.createBinaryBuffer; /** * A {@link MergeFunction} for lookup, this wrapper only considers the latest high level record, @@ -54,11 +49,7 @@ public LookupMergeFunction( RowType valueType, @Nullable IOManager ioManager) { this.mergeFunction = mergeFunction; - Supplier binarySupplier = - () -> createBinaryBuffer(options, keyType, valueType, ioManager); - int threshold = options == null ? 1024 : options.lookupMergeRecordsThreshold(); - this.candidates = - new KeyValueBuffer.HybridBuffer(threshold, new LazyField<>(binarySupplier)); + this.candidates = KeyValueBuffer.createHybridBuffer(options, keyType, valueType, ioManager); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCollectAgg.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCollectAgg.java index d9e706f6e853..6fbd30508568 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCollectAgg.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldCollectAgg.java @@ -86,7 +86,7 @@ public FieldCollectAgg(String name, ArrayType dataType, boolean distinct) { public Object aggReversed(Object accumulator, Object inputField) { // we don't need to actually do the reverse here for this agg // because accumulator has been distinct, just let accumulator be accumulator will speed up - // dinstinct process + // distinct process return agg(accumulator, inputField); } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldNestedUpdateAgg.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldNestedUpdateAgg.java index 005bf7b17f1f..b2848b35b410 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldNestedUpdateAgg.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/FieldNestedUpdateAgg.java @@ -36,6 +36,7 @@ import static org.apache.paimon.codegen.CodeGenUtils.newProjection; import static org.apache.paimon.codegen.CodeGenUtils.newRecordEqualiser; +import static org.apache.paimon.options.ConfigOptions.key; import static org.apache.paimon.utils.Preconditions.checkNotNull; /** @@ -51,7 +52,10 @@ public class FieldNestedUpdateAgg extends FieldAggregator { @Nullable private final Projection keyProjection; @Nullable private final RecordEqualiser elementEqualiser; - public FieldNestedUpdateAgg(String name, ArrayType dataType, List nestedKey) { + private final int countLimit; + + public FieldNestedUpdateAgg( + String name, ArrayType dataType, List nestedKey, int countLimit) { super(name, dataType); RowType nestedType = (RowType) dataType.getElementType(); this.nestedFields = nestedType.getFieldCount(); @@ -62,6 +66,9 @@ public FieldNestedUpdateAgg(String name, ArrayType dataType, List nested this.keyProjection = newProjection(nestedType, nestedKey); this.elementEqualiser = null; } + + // If deduplicate key is set, we don't guarantee that the result is exactly right + this.countLimit = countLimit; } @Override @@ -73,9 +80,15 @@ public Object agg(Object accumulator, Object inputField) { InternalArray acc = (InternalArray) accumulator; InternalArray input = (InternalArray) inputField; + if (acc.size() >= countLimit) { + return accumulator; + } + + int remainCount = countLimit - acc.size(); + List rows = new ArrayList<>(acc.size() + input.size()); addNonNullRows(acc, rows); - addNonNullRows(input, rows); + addNonNullRows(input, rows, remainCount); if (keyProjection != null) { Map map = new HashMap<>(); @@ -141,4 +154,18 @@ private void addNonNullRows(InternalArray array, List rows) { rows.add(array.getRow(i, nestedFields)); } } + + private void addNonNullRows(InternalArray array, List rows, int remainSize) { + int count = 0; + for (int i = 0; i < array.size(); i++) { + if (count >= remainSize) { + return; + } + if (array.isNullAt(i)) { + continue; + } + rows.add(array.getRow(i, nestedFields)); + count++; + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/factory/FieldNestedUpdateAggFactory.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/factory/FieldNestedUpdateAggFactory.java index 43d8eb429f07..070931e01135 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/factory/FieldNestedUpdateAggFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/aggregate/factory/FieldNestedUpdateAggFactory.java @@ -36,7 +36,10 @@ public class FieldNestedUpdateAggFactory implements FieldAggregatorFactory { @Override public FieldNestedUpdateAgg create(DataType fieldType, CoreOptions options, String field) { - return createFieldNestedUpdateAgg(fieldType, options.fieldNestedUpdateAggNestedKey(field)); + return createFieldNestedUpdateAgg( + fieldType, + options.fieldNestedUpdateAggNestedKey(field), + options.fieldNestedUpdateAggCountLimit(field)); } @Override @@ -45,7 +48,7 @@ public String identifier() { } private FieldNestedUpdateAgg createFieldNestedUpdateAgg( - DataType fieldType, List nestedKey) { + DataType fieldType, List nestedKey, int countLimit) { if (nestedKey == null) { nestedKey = Collections.emptyList(); } @@ -56,6 +59,6 @@ private FieldNestedUpdateAgg createFieldNestedUpdateAgg( ArrayType arrayType = (ArrayType) fieldType; checkArgument(arrayType.getElementType() instanceof RowType, typeErrorMsg, fieldType); - return new FieldNestedUpdateAgg(identifier(), arrayType, nestedKey); + return new FieldNestedUpdateAgg(identifier(), arrayType, nestedKey, countLimit); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java index b21c77db1d01..7ed77406c777 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java @@ -196,6 +196,12 @@ public FileStoreScan withLevelFilter(Filter levelFilter) { return this; } + @Override + public FileStoreScan withLevelMinMaxFilter(BiFilter minMaxFilter) { + manifestsReader.withLevelMinMaxFilter(minMaxFilter); + return this; + } + @Override public FileStoreScan enableValueFilter() { return this; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java index 7b77fb179f6d..ddf2addf5313 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java @@ -28,9 +28,9 @@ import org.apache.paimon.disk.IOManager; import org.apache.paimon.index.DynamicBucketIndexMaintainer; import org.apache.paimon.index.IndexFileHandler; -import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; -import org.apache.paimon.io.IndexIncrement; +import org.apache.paimon.io.DataIncrement; import org.apache.paimon.memory.MemoryPoolFactory; import org.apache.paimon.metrics.MetricRegistry; import org.apache.paimon.operation.metrics.CompactionMetrics; @@ -213,22 +213,26 @@ public List prepareCommit(boolean waitCompaction, long commitIden WriterContainer writerContainer = entry.getValue(); CommitIncrement increment = writerContainer.writer.prepareCommit(waitCompaction); - List newIndexFiles = new ArrayList<>(); + DataIncrement newFilesIncrement = increment.newFilesIncrement(); + CompactIncrement compactIncrement = increment.compactIncrement(); if (writerContainer.dynamicBucketMaintainer != null) { - newIndexFiles.addAll(writerContainer.dynamicBucketMaintainer.prepareCommit()); + newFilesIncrement + .newIndexFiles() + .addAll(writerContainer.dynamicBucketMaintainer.prepareCommit()); } CompactDeletionFile compactDeletionFile = increment.compactDeletionFile(); if (compactDeletionFile != null) { - compactDeletionFile.getOrCompute().ifPresent(newIndexFiles::add); + compactDeletionFile + .getOrCompute() + .ifPresent(compactIncrement.newIndexFiles()::add); } CommitMessageImpl committable = new CommitMessageImpl( partition, bucket, writerContainer.totalBuckets, - increment.newFilesIncrement(), - increment.compactIncrement(), - new IndexIncrement(newIndexFiles)); + newFilesIncrement, + compactIncrement); result.add(committable); if (committable.isEmpty()) { diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java new file mode 100644 index 000000000000..fad67e318320 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionFileStoreScan.java @@ -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. + */ + +package org.apache.paimon.operation; + +import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.manifest.ManifestFile; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.utils.SnapshotManager; + +/** {@link FileStoreScan} for data-evolution enabled table. */ +public class DataEvolutionFileStoreScan extends AppendOnlyFileStoreScan { + + public DataEvolutionFileStoreScan( + ManifestsReader manifestsReader, + BucketSelectConverter bucketSelectConverter, + SnapshotManager snapshotManager, + SchemaManager schemaManager, + TableSchema schema, + ManifestFile.Factory manifestFileFactory, + Integer scanManifestParallelism) { + super( + manifestsReader, + bucketSelectConverter, + snapshotManager, + schemaManager, + schema, + manifestFileFactory, + scanManifestParallelism, + false); + } + + public DataEvolutionFileStoreScan withFilter(Predicate predicate) { + return this; + } + + /** Note: Keep this thread-safe. */ + @Override + protected boolean filterByStats(ManifestEntry entry) { + return true; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java index c2a6ebd18fe3..f4c426e86f12 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/DataEvolutionSplitRead.java @@ -37,6 +37,7 @@ import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.table.source.DataEvolutionSplitGenerator; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.types.DataField; @@ -56,7 +57,7 @@ import java.util.stream.Collectors; import static java.lang.String.format; -import static org.apache.paimon.table.SpecialFields.rowTypeWithRowLineage; +import static org.apache.paimon.table.SpecialFields.rowTypeWithRowTracking; import static org.apache.paimon.utils.Preconditions.checkArgument; /** @@ -124,7 +125,7 @@ public RecordReader createReader(DataSplit split) throws IOExceptio new Builder( formatDiscover, readRowType.getFields(), - schema -> rowTypeWithRowLineage(schema.logicalRowType(), true).getFields(), + schema -> rowTypeWithRowTracking(schema.logicalRowType(), true).getFields(), null, null, null); @@ -189,7 +190,8 @@ private DataEvolutionFileReader createUnionReader( long schemaId = file.schemaId(); TableSchema dataSchema = schemaManager.schema(schemaId).project(file.writeCols()); int[] fieldIds = - rowTypeWithRowLineage(dataSchema.logicalRowType()).getFields().stream() + SpecialFields.rowTypeWithRowTracking(dataSchema.logicalRowType()).getFields() + .stream() .mapToInt(DataField::id) .toArray(); List readFields = new ArrayList<>(); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java index dae3d5a0f29c..98a761b47166 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java @@ -35,7 +35,7 @@ import org.apache.paimon.manifest.ManifestList; import org.apache.paimon.stats.StatsFileHandler; import org.apache.paimon.utils.DataFilePathFactories; -import org.apache.paimon.utils.FileDeletionThreadPool; +import org.apache.paimon.utils.FileOperationThreadPool; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.SnapshotManager; @@ -102,7 +102,7 @@ public FileDeletionBase( this.statsFileHandler = statsFileHandler; this.cleanEmptyDirectories = cleanEmptyDirectories; this.deletionBuckets = new HashMap<>(); - this.deleteFileExecutor = FileDeletionThreadPool.getExecutorService(deleteFileThreadNum); + this.deleteFileExecutor = FileOperationThreadPool.getExecutorService(deleteFileThreadNum); } /** diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommit.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommit.java index 4156ce0a834c..6a00db6f0ee1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommit.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommit.java @@ -50,7 +50,7 @@ public interface FileStoreCommit extends AutoCloseable { * note that this partition does not necessarily equal to the partitions of the newly added * key-values. This is just the partition to be cleaned up. */ - int overwrite( + int overwritePartition( Map partition, ManifestCommittable committable, Map properties); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java index 56fde4e30e43..7df158978188 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java @@ -20,11 +20,13 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; +import org.apache.paimon.Snapshot.CommitKind; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.catalog.SnapshotCommit; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.fs.FileIO; +import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFilePathFactory; import org.apache.paimon.manifest.FileEntry; @@ -60,7 +62,6 @@ import org.apache.paimon.utils.IOUtils; import org.apache.paimon.utils.InternalRowPartitionComputer; import org.apache.paimon.utils.Pair; -import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.SnapshotManager; import org.slf4j.Logger; @@ -69,7 +70,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -87,14 +87,16 @@ import static java.util.Collections.emptyList; import static org.apache.paimon.deletionvectors.DeletionVectorsIndexFile.DELETION_VECTORS_INDEX; -import static org.apache.paimon.index.HashIndexFile.HASH_INDEX; import static org.apache.paimon.manifest.ManifestEntry.recordCount; import static org.apache.paimon.manifest.ManifestEntry.recordCountAdd; import static org.apache.paimon.manifest.ManifestEntry.recordCountDelete; import static org.apache.paimon.partition.PartitionPredicate.createBinaryPartitions; import static org.apache.paimon.partition.PartitionPredicate.createPartitionPredicate; +import static org.apache.paimon.utils.ConflictDeletionUtils.buildBaseEntriesWithDV; +import static org.apache.paimon.utils.ConflictDeletionUtils.buildDeltaEntriesWithDV; import static org.apache.paimon.utils.InternalRowPartitionComputer.partToSimpleString; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.Preconditions.checkState; /** * Default implementation of {@link FileStoreCommit}. @@ -151,6 +153,9 @@ public class FileStoreCommitImpl implements FileStoreCommit { @Nullable private Long strictModeLastSafeSnapshot; private final InternalRowPartitionComputer partitionComputer; private final boolean rowTrackingEnabled; + private final boolean isPkTable; + private final boolean deletionVectorsEnabled; + private final IndexFileHandler indexFileHandler; private boolean ignoreEmptyCommit; private CommitMetrics commitMetrics; @@ -187,7 +192,10 @@ public FileStoreCommitImpl( long commitMinRetryWait, long commitMaxRetryWait, @Nullable Long strictModeLastSafeSnapshot, - boolean rowTrackingEnabled) { + boolean rowTrackingEnabled, + boolean isPkTable, + boolean deletionVectorsEnabled, + IndexFileHandler indexFileHandler) { this.snapshotCommit = snapshotCommit; this.fileIO = fileIO; this.schemaManager = schemaManager; @@ -231,6 +239,9 @@ public FileStoreCommitImpl( this.statsFileHandler = statsFileHandler; this.bucketMode = bucketMode; this.rowTrackingEnabled = rowTrackingEnabled; + this.isPkTable = isPkTable; + this.deletionVectorsEnabled = deletionVectorsEnabled; + this.indexFileHandler = indexFileHandler; } @Override @@ -295,24 +306,24 @@ public int commit(ManifestCommittable committable, boolean checkAppendFiles) { List appendTableFiles = new ArrayList<>(); List appendChangelog = new ArrayList<>(); + List appendIndexFiles = new ArrayList<>(); List compactTableFiles = new ArrayList<>(); List compactChangelog = new ArrayList<>(); - List appendHashIndexFiles = new ArrayList<>(); - List compactDvIndexFiles = new ArrayList<>(); + List compactIndexFiles = new ArrayList<>(); collectChanges( committable.fileCommittables(), appendTableFiles, appendChangelog, + appendIndexFiles, compactTableFiles, compactChangelog, - appendHashIndexFiles, - compactDvIndexFiles); + compactIndexFiles); try { List appendSimpleEntries = SimpleFileEntry.from(appendTableFiles); if (!ignoreEmptyCommit || !appendTableFiles.isEmpty() || !appendChangelog.isEmpty() - || !appendHashIndexFiles.isEmpty()) { + || !appendIndexFiles.isEmpty()) { // Optimization for common path. // Step 1: // Read manifest entries from changed partitions here and check for conflicts. @@ -321,17 +332,29 @@ public int commit(ManifestCommittable committable, boolean checkAppendFiles) { // This optimization is mainly used to decrease the number of times we read from // files. latestSnapshot = snapshotManager.latestSnapshot(); + CommitKind commitKind = CommitKind.APPEND; + ConflictCheck conflictCheck = noConflictCheck(); + if (containsFileDeletionOrDeletionVectors(appendSimpleEntries, appendIndexFiles)) { + commitKind = CommitKind.OVERWRITE; + conflictCheck = mustConflictCheck(); + } + if (latestSnapshot != null && checkAppendFiles) { // it is possible that some partitions only have compact changes, // so we need to contain all changes baseEntries.addAll( readAllEntriesFromChangedPartitions( - latestSnapshot, appendTableFiles, compactTableFiles)); + latestSnapshot, + changedPartitions( + appendTableFiles, + compactTableFiles, + appendIndexFiles))); noConflictsOrFail( - latestSnapshot.commitUser(), + latestSnapshot, baseEntries, appendSimpleEntries, - Snapshot.CommitKind.APPEND); + appendIndexFiles, + commitKind); safeLatestSnapshotId = latestSnapshot.id(); } @@ -339,20 +362,20 @@ public int commit(ManifestCommittable committable, boolean checkAppendFiles) { tryCommit( appendTableFiles, appendChangelog, - appendHashIndexFiles, + appendIndexFiles, committable.identifier(), committable.watermark(), committable.logOffsets(), committable.properties(), - Snapshot.CommitKind.APPEND, - noConflictCheck(), + commitKind, + conflictCheck, null); generatedSnapshot += 1; } if (!compactTableFiles.isEmpty() || !compactChangelog.isEmpty() - || !compactDvIndexFiles.isEmpty()) { + || !compactIndexFiles.isEmpty()) { // Optimization for common path. // Step 2: // Add appendChanges to the manifest entries read above and check for conflicts. @@ -363,10 +386,11 @@ public int commit(ManifestCommittable committable, boolean checkAppendFiles) { if (safeLatestSnapshotId != null) { baseEntries.addAll(appendSimpleEntries); noConflictsOrFail( - latestSnapshot.commitUser(), + latestSnapshot, baseEntries, SimpleFileEntry.from(compactTableFiles), - Snapshot.CommitKind.COMPACT); + compactIndexFiles, + CommitKind.COMPACT); // assume this compact commit follows just after the append commit created above safeLatestSnapshotId += 1; } @@ -375,12 +399,12 @@ public int commit(ManifestCommittable committable, boolean checkAppendFiles) { tryCommit( compactTableFiles, compactChangelog, - compactDvIndexFiles, + compactIndexFiles, committable.identifier(), committable.watermark(), committable.logOffsets(), committable.properties(), - Snapshot.CommitKind.COMPACT, + CommitKind.COMPACT, hasConflictChecked(safeLatestSnapshotId), null); generatedSnapshot += 1; @@ -425,8 +449,23 @@ private void reportCommit( commitMetrics.reportCommit(commitStats); } + private boolean containsFileDeletionOrDeletionVectors( + List appendSimpleEntries, List appendIndexFiles) { + for (SimpleFileEntry appendSimpleEntry : appendSimpleEntries) { + if (appendSimpleEntry.kind().equals(FileKind.DELETE)) { + return true; + } + } + for (IndexManifestEntry appendIndexFile : appendIndexFiles) { + if (appendIndexFile.indexFile().indexType().equals(DELETION_VECTORS_INDEX)) { + return true; + } + } + return false; + } + @Override - public int overwrite( + public int overwritePartition( Map partition, ManifestCommittable committable, Map properties) { @@ -447,18 +486,18 @@ public int overwrite( int attempts = 0; List appendTableFiles = new ArrayList<>(); List appendChangelog = new ArrayList<>(); + List appendIndexFiles = new ArrayList<>(); List compactTableFiles = new ArrayList<>(); List compactChangelog = new ArrayList<>(); - List appendHashIndexFiles = new ArrayList<>(); - List compactDvIndexFiles = new ArrayList<>(); + List compactIndexFiles = new ArrayList<>(); collectChanges( committable.fileCommittables(), appendTableFiles, appendChangelog, + appendIndexFiles, compactTableFiles, compactChangelog, - appendHashIndexFiles, - compactDvIndexFiles); + compactIndexFiles); if (!appendChangelog.isEmpty() || !compactChangelog.isEmpty()) { StringBuilder warnMessage = @@ -492,7 +531,7 @@ public int overwrite( partitionFilter = PartitionPredicate.fromMultiple(partitionType, partitions); } } else { - // partition may be partial partition fields, so here must to use predicate way. + // partition may be partial partition fields, so here must use predicate way. Predicate partitionPredicate = createPartitionPredicate(partition, partitionType, partitionDefaultName); partitionFilter = @@ -515,10 +554,10 @@ public int overwrite( // overwrite new files if (!skipOverwrite) { attempts += - tryOverwrite( + tryOverwritePartition( partitionFilter, appendTableFiles, - appendHashIndexFiles, + appendIndexFiles, committable.identifier(), committable.watermark(), committable.logOffsets(), @@ -526,17 +565,17 @@ public int overwrite( generatedSnapshot += 1; } - if (!compactTableFiles.isEmpty() || !compactDvIndexFiles.isEmpty()) { + if (!compactTableFiles.isEmpty() || !compactIndexFiles.isEmpty()) { attempts += tryCommit( compactTableFiles, emptyList(), - compactDvIndexFiles, + compactIndexFiles, committable.identifier(), committable.watermark(), committable.logOffsets(), committable.properties(), - Snapshot.CommitKind.COMPACT, + CommitKind.COMPACT, mustConflictCheck(), null); generatedSnapshot += 1; @@ -589,7 +628,7 @@ public void dropPartitions(List> partitions, long commitIden partitionFilter = PartitionPredicate.fromPredicate(partitionType, predicate); } - tryOverwrite( + tryOverwritePartition( partitionFilter, emptyList(), emptyList(), @@ -601,7 +640,7 @@ public void dropPartitions(List> partitions, long commitIden @Override public void truncateTable(long commitIdentifier) { - tryOverwrite( + tryOverwritePartition( null, emptyList(), emptyList(), @@ -646,7 +685,7 @@ public void commitStatistics(Statistics stats, long commitIdentifier) { null, Collections.emptyMap(), Collections.emptyMap(), - Snapshot.CommitKind.ANALYZE, + CommitKind.ANALYZE, noConflictCheck(), statsFileName); } @@ -665,10 +704,10 @@ private void collectChanges( List commitMessages, List appendTableFiles, List appendChangelog, + List appendIndexFiles, List compactTableFiles, List compactChangelog, - List appendHashIndexFiles, - List compactDvIndexFiles) { + List compactIndexFiles) { for (CommitMessage message : commitMessages) { CommitMessageImpl commitMessage = (CommitMessageImpl) message; commitMessage @@ -686,6 +725,29 @@ private void collectChanges( .newFilesIncrement() .changelogFiles() .forEach(m -> appendChangelog.add(makeEntry(FileKind.ADD, commitMessage, m))); + commitMessage + .newFilesIncrement() + .newIndexFiles() + .forEach( + m -> + appendIndexFiles.add( + new IndexManifestEntry( + FileKind.ADD, + commitMessage.partition(), + commitMessage.bucket(), + m))); + commitMessage + .newFilesIncrement() + .deletedIndexFiles() + .forEach( + m -> + appendIndexFiles.add( + new IndexManifestEntry( + FileKind.DELETE, + commitMessage.partition(), + commitMessage.bucket(), + m))); + commitMessage .compactIncrement() .compactBefore() @@ -702,50 +764,27 @@ private void collectChanges( .changelogFiles() .forEach(m -> compactChangelog.add(makeEntry(FileKind.ADD, commitMessage, m))); commitMessage - .indexIncrement() + .compactIncrement() .newIndexFiles() .forEach( - f -> { - switch (f.indexType()) { - case HASH_INDEX: - appendHashIndexFiles.add( - new IndexManifestEntry( - FileKind.ADD, - commitMessage.partition(), - commitMessage.bucket(), - f)); - break; - case DELETION_VECTORS_INDEX: - compactDvIndexFiles.add( - new IndexManifestEntry( - FileKind.ADD, - commitMessage.partition(), - commitMessage.bucket(), - f)); - break; - default: - throw new RuntimeException( - "Unknown index type: " + f.indexType()); - } - }); + m -> + compactIndexFiles.add( + new IndexManifestEntry( + FileKind.ADD, + commitMessage.partition(), + commitMessage.bucket(), + m))); commitMessage - .indexIncrement() + .compactIncrement() .deletedIndexFiles() .forEach( - f -> { - if (f.indexType().equals(DELETION_VECTORS_INDEX)) { - compactDvIndexFiles.add( + m -> + compactIndexFiles.add( new IndexManifestEntry( FileKind.DELETE, commitMessage.partition(), commitMessage.bucket(), - f)); - } else { - throw new RuntimeException( - "This index type is not supported to delete: " - + f.indexType()); - } - }); + m))); } if (!commitMessages.isEmpty()) { List msg = new ArrayList<>(); @@ -755,17 +794,17 @@ private void collectChanges( if (!appendChangelog.isEmpty()) { msg.add(appendChangelog.size() + " append Changelogs"); } + if (!appendIndexFiles.isEmpty()) { + msg.add(appendIndexFiles.size() + " append index files"); + } if (!compactTableFiles.isEmpty()) { msg.add(compactTableFiles.size() + " compact table files"); } if (!compactChangelog.isEmpty()) { msg.add(compactChangelog.size() + " compact Changelogs"); } - if (!appendHashIndexFiles.isEmpty()) { - msg.add(appendHashIndexFiles.size() + " append hash index files"); - } - if (!compactDvIndexFiles.isEmpty()) { - msg.add(compactDvIndexFiles.size() + " compact dv index files"); + if (!compactIndexFiles.isEmpty()) { + msg.add(compactIndexFiles.size() + " compact index files"); } LOG.info("Finished collecting changes, including: {}", String.join(", ", msg)); } @@ -789,7 +828,7 @@ private int tryCommit( @Nullable Long watermark, Map logOffsets, Map properties, - Snapshot.CommitKind commitKind, + CommitKind commitKind, ConflictCheck conflictCheck, @Nullable String statsFileName) { int retryCount = 0; @@ -833,7 +872,13 @@ private int tryCommit( return retryCount + 1; } - private int tryOverwrite( + /** + * Try to overwrite partition. + * + * @param partitionFilter Partition filter indicating which partitions to overwrite, if {@code + * null}, overwrites the entire table. + */ + private int tryOverwritePartition( @Nullable PartitionPredicate partitionFilter, List changes, List indexFiles, @@ -886,7 +931,7 @@ private int tryOverwrite( watermark, logOffsets, properties, - Snapshot.CommitKind.OVERWRITE, + CommitKind.OVERWRITE, mustConflictCheck(), null); } @@ -901,7 +946,7 @@ CommitResult tryCommitOnce( @Nullable Long watermark, Map logOffsets, Map properties, - Snapshot.CommitKind commitKind, + CommitKind commitKind, @Nullable Snapshot latestSnapshot, ConflictCheck conflictCheck, @Nullable String newStatsFileName) { @@ -940,8 +985,8 @@ CommitResult tryCommitOnce( if (strictModeLastSafeSnapshot != null && strictModeLastSafeSnapshot >= 0) { for (long id = strictModeLastSafeSnapshot + 1; id < newSnapshotId; id++) { Snapshot snapshot = snapshotManager.snapshot(id); - if ((snapshot.commitKind() == Snapshot.CommitKind.COMPACT - || snapshot.commitKind() == Snapshot.CommitKind.OVERWRITE) + if ((snapshot.commitKind() == CommitKind.COMPACT + || snapshot.commitKind() == CommitKind.OVERWRITE) && !snapshot.commitUser().equals(commitUser)) { throw new RuntimeException( String.format( @@ -975,10 +1020,7 @@ CommitResult tryCommitOnce( // latestSnapshotId is different from the snapshot id we've checked for conflicts, // so we have to check again List changedPartitions = - deltaFiles.stream() - .map(ManifestEntry::partition) - .distinct() - .collect(Collectors.toList()); + changedPartitions(deltaFiles, Collections.emptyList(), indexFiles); if (retryResult != null && retryResult.latestSnapshot != null) { baseDataFiles = new ArrayList<>(retryResult.baseDataFiles); List incremental = @@ -993,9 +1035,10 @@ CommitResult tryCommitOnce( readAllEntriesFromChangedPartitions(latestSnapshot, changedPartitions); } noConflictsOrFail( - latestSnapshot.commitUser(), + latestSnapshot, baseDataFiles, SimpleFileEntry.from(deltaFiles), + indexFiles, commitKind); } @@ -1051,7 +1094,7 @@ CommitResult tryCommitOnce( // assign row id for new files List rowIdAssigned = new ArrayList<>(); nextRowIdStart = - assignRowLineageMeta(firstRowIdStart, snapshotAssigned, rowIdAssigned); + assignRowTrackingMeta(firstRowIdStart, snapshotAssigned, rowIdAssigned); deltaFiles = rowIdAssigned; } @@ -1173,7 +1216,7 @@ CommitResult tryCommitOnce( return new SuccessResult(); } - private long assignRowLineageMeta( + private long assignRowTrackingMeta( long firstRowIdStart, List deltaFiles, List rowIdAssigned) { @@ -1272,7 +1315,7 @@ private boolean compactManifestOnce() { latestSnapshot.indexManifest(), commitUser, Long.MAX_VALUE, - Snapshot.CommitKind.COMPACT, + CommitKind.COMPACT, System.currentTimeMillis(), latestSnapshot.logOffsets(), latestSnapshot.totalRecordCount(), @@ -1323,16 +1366,23 @@ private List readIncrementalChanges( return entries; } - @SafeVarargs - private final List readAllEntriesFromChangedPartitions( - Snapshot snapshot, List... changes) { - List changedPartitions = - Arrays.stream(changes) - .flatMap(Collection::stream) - .map(ManifestEntry::partition) - .distinct() - .collect(Collectors.toList()); - return readAllEntriesFromChangedPartitions(snapshot, changedPartitions); + private List changedPartitions( + List appendTableFiles, + List compactTableFiles, + List appendIndexFiles) { + Set changedPartitions = new HashSet<>(); + for (ManifestEntry appendTableFile : appendTableFiles) { + changedPartitions.add(appendTableFile.partition()); + } + for (ManifestEntry compactTableFile : compactTableFiles) { + changedPartitions.add(compactTableFile.partition()); + } + for (IndexManifestEntry appendIndexFile : appendIndexFiles) { + if (appendIndexFile.indexFile().indexType().equals(DELETION_VECTORS_INDEX)) { + changedPartitions.add(appendIndexFile.partition()); + } + } + return new ArrayList<>(changedPartitions); } private List readAllEntriesFromChangedPartitions( @@ -1348,14 +1398,37 @@ private List readAllEntriesFromChangedPartitions( } private void noConflictsOrFail( - String baseCommitUser, + Snapshot snapshot, List baseEntries, - List changes, - Snapshot.CommitKind commitKind) { + List deltaEntries, + List deltaIndexEntries, + CommitKind commitKind) { + String baseCommitUser = snapshot.commitUser(); + if (checkForDeletionVector(commitKind)) { + // Enrich dvName in fileEntry to checker for base ADD dv and delta DELETE dv. + // For example: + // If the base file is , + // then the delta file must be ; and vice versa, + // If the delta file is , + // then the base file must be . + try { + baseEntries = + buildBaseEntriesWithDV( + baseEntries, + snapshot.indexManifest() == null + ? Collections.emptyList() + : indexFileHandler.readManifest(snapshot.indexManifest())); + deltaEntries = + buildDeltaEntriesWithDV(baseEntries, deltaEntries, deltaIndexEntries); + } catch (Throwable e) { + throw conflictException(commitUser, baseEntries, deltaEntries).apply(e); + } + } + List allEntries = new ArrayList<>(baseEntries); - allEntries.addAll(changes); + allEntries.addAll(deltaEntries); - if (commitKind != Snapshot.CommitKind.OVERWRITE) { + if (commitKind != CommitKind.OVERWRITE) { // total buckets within the same partition should remain the same Map totalBuckets = new HashMap<>(); for (SimpleFileEntry entry : allEntries) { @@ -1384,35 +1457,22 @@ private void noConflictsOrFail( + " without overwrite. Give up committing.", baseCommitUser, baseEntries, - changes, + deltaEntries, null); LOG.warn("", conflictException.getLeft()); throw conflictException.getRight(); } } - Function exceptionFunction = - e -> { - Pair conflictException = - createConflictException( - "File deletion conflicts detected! Give up committing.", - baseCommitUser, - baseEntries, - changes, - e); - LOG.warn("", conflictException.getLeft()); - return conflictException.getRight(); - }; - Collection mergedEntries; try { // merge manifest entries and also check if the files we want to delete are still there mergedEntries = FileEntry.mergeEntries(allEntries); } catch (Throwable e) { - throw exceptionFunction.apply(e); + throw conflictException(commitUser, baseEntries, deltaEntries).apply(e); } - assertNoDelete(mergedEntries, exceptionFunction); + assertNoDelete(mergedEntries, conflictException(commitUser, baseEntries, deltaEntries)); // fast exit for file store without keys if (keyComparator == null) { @@ -1446,7 +1506,7 @@ private void noConflictsOrFail( + b.identifier().toString(pathFactory), baseCommitUser, baseEntries, - changes, + deltaEntries, null); LOG.warn("", conflictException.getLeft()); @@ -1456,12 +1516,48 @@ private void noConflictsOrFail( } } + private Function conflictException( + String baseCommitUser, + List baseEntries, + List deltaEntries) { + return e -> { + Pair conflictException = + createConflictException( + "File deletion conflicts detected! Give up committing.", + baseCommitUser, + baseEntries, + deltaEntries, + e); + LOG.warn("", conflictException.getLeft()); + return conflictException.getRight(); + }; + } + + private boolean checkForDeletionVector(CommitKind commitKind) { + if (!deletionVectorsEnabled) { + return false; + } + + // todo: Add them once contains DELETE type. + // PK table's compact dv index only contains ADD type, skip conflict detection. + if (isPkTable && commitKind == CommitKind.COMPACT) { + return false; + } + + // Non-PK table's hash fixed bucket mode only contains ADD type, skip conflict detection. + if (!isPkTable && bucketMode.equals(BucketMode.HASH_FIXED)) { + return false; + } + + return true; + } + private void assertNoDelete( Collection mergedEntries, Function exceptionFunction) { try { for (SimpleFileEntry entry : mergedEntries) { - Preconditions.checkState( + checkState( entry.kind() != FileKind.DELETE, "Trying to delete file %s for table %s which is not previously added.", entry.fileName(), diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java index 5fa312723753..047b1c3f5dde 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java @@ -75,6 +75,8 @@ public interface FileStoreScan { FileStoreScan withLevelFilter(Filter levelFilter); + FileStoreScan withLevelMinMaxFilter(BiFilter minMaxFilter); + FileStoreScan enableValueFilter(); FileStoreScan withManifestEntryFilter(Filter filter); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreWrite.java index b268f7b7a55a..ce71b8cd2a29 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreWrite.java @@ -25,7 +25,6 @@ import org.apache.paimon.index.DynamicBucketIndexMaintainer; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.memory.MemoryPoolFactory; -import org.apache.paimon.memory.MemorySegmentPool; import org.apache.paimon.metrics.MetricRegistry; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.SinkRecord; @@ -58,15 +57,6 @@ default void withWriteType(RowType writeType) { throw new UnsupportedOperationException(); } - /** - * With memory pool for the current file store write. - * - * @param memoryPool the given memory pool. - */ - default FileStoreWrite withMemoryPool(MemorySegmentPool memoryPool) { - return withMemoryPoolFactory(new MemoryPoolFactory(memoryPool)); - } - /** * With memory pool factory for the current file store write. * diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/ListUnexistingFiles.java b/paimon-core/src/main/java/org/apache/paimon/operation/ListUnexistingFiles.java index a704a329c71a..c47916871a5c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/ListUnexistingFiles.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/ListUnexistingFiles.java @@ -25,6 +25,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; +import org.apache.paimon.utils.FileOperationThreadPool; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.ThreadPoolUtils; @@ -38,8 +39,6 @@ import java.util.Map; import java.util.concurrent.ThreadPoolExecutor; -import static org.apache.paimon.utils.ThreadPoolUtils.createCachedThreadPool; - /** List what data files recorded in manifests are missing from the filesystem. */ public class ListUnexistingFiles { @@ -51,8 +50,8 @@ public ListUnexistingFiles(FileStoreTable table) { this.table = table; this.pathFactory = table.store().pathFactory(); this.executor = - createCachedThreadPool( - table.coreOptions().deleteFileThreadNum(), "LIST_UNEXISTING_FILES"); + FileOperationThreadPool.getExecutorService( + table.coreOptions().fileOperationThreadNum()); } public Map> list(BinaryRow partition) throws Exception { diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java b/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java index 9456e6c9aae3..34ff3fb72143 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java @@ -89,7 +89,7 @@ public LocalOrphanFilesClean(FileStoreTable table, long olderThanMillis, boolean this.deleteFiles = new ArrayList<>(); this.executor = createCachedThreadPool( - table.coreOptions().deleteFileThreadNum(), "ORPHAN_FILES_CLEAN"); + table.coreOptions().fileOperationThreadNum(), "ORPHAN_FILES_CLEAN"); this.dryRun = dryRun; } @@ -276,7 +276,7 @@ public static List createOrphanFilesCleans( : new HashMap() { { put( - CoreOptions.DELETE_FILE_THREAD_NUM.key(), + CoreOptions.FILE_OPERATION_THREAD_NUM.key(), parallelism.toString()); } }; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/ManifestsReader.java b/paimon-core/src/main/java/org/apache/paimon/operation/ManifestsReader.java index d58bb797e340..b3b89e72aafb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/ManifestsReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/ManifestsReader.java @@ -27,6 +27,7 @@ import org.apache.paimon.stats.SimpleStats; import org.apache.paimon.table.source.ScanMode; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BiFilter; import org.apache.paimon.utils.SnapshotManager; import javax.annotation.Nullable; @@ -52,6 +53,7 @@ public class ManifestsReader { @Nullable private Integer specifiedBucket = null; @Nullable private Integer specifiedLevel = null; @Nullable private PartitionPredicate partitionFilter = null; + @Nullable private BiFilter levelMinMaxFilter = null; public ManifestsReader( RowType partitionType, @@ -79,6 +81,11 @@ public ManifestsReader withLevel(int level) { return this; } + public ManifestsReader withLevelMinMaxFilter(BiFilter minMaxFilter) { + this.levelMinMaxFilter = minMaxFilter; + return this; + } + public ManifestsReader withPartitionFilter(Predicate predicate) { this.partitionFilter = PartitionPredicate.fromPredicate(partitionType, predicate); return this; @@ -160,6 +167,9 @@ private boolean filterManifestFileMeta(ManifestFileMeta manifest) { && (specifiedLevel < minLevel || specifiedLevel > maxLevel)) { return false; } + if (levelMinMaxFilter != null && !levelMinMaxFilter.test(minLevel, maxLevel)) { + return false; + } } if (partitionFilter == null) { diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/MergeFileSplitRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/MergeFileSplitRead.java index 08344c08b50e..563c3b57be5a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/MergeFileSplitRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/MergeFileSplitRead.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.KeyValue; +import org.apache.paimon.KeyValueFileStore; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.deletionvectors.DeletionVector; @@ -64,7 +65,12 @@ import static org.apache.paimon.predicate.PredicateBuilder.containsFields; import static org.apache.paimon.predicate.PredicateBuilder.splitAnd; -/** A {@link SplitRead} to read row lineage table which need field merge. */ +/** + * An implementation for {@link KeyValueFileStore}, this class handle LSM merging and changelog row + * kind things, it will force reading fields such as sequence and row_kind. + * + * @see RawFileSplitRead If in batch mode and reading raw files, it is recommended to use this read. + */ public class MergeFileSplitRead implements SplitRead { private final TableSchema tableSchema; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java index 610a0c0a2ede..5dc8522a8fc4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java @@ -64,7 +64,7 @@ import java.util.Map; import static org.apache.paimon.predicate.PredicateBuilder.splitAnd; -import static org.apache.paimon.table.SpecialFields.rowTypeWithRowLineage; +import static org.apache.paimon.table.SpecialFields.rowTypeWithRowTracking; /** A {@link SplitRead} to read raw file directly from {@link DataSplit}. */ public class RawFileSplitRead implements SplitRead { @@ -173,7 +173,7 @@ public RecordReader createReader( readRowType.getFields(), schema -> { if (rowTrackingEnabled) { - return rowTypeWithRowLineage(schema.logicalRowType(), true) + return rowTypeWithRowTracking(schema.logicalRowType(), true) .getFields(); } return schema.fields(); diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java index 6cdb23b5a54d..09f4f5b5edaa 100644 --- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -224,10 +224,12 @@ public PagedList listTablesPaged( String databaseName, @Nullable Integer maxResults, @Nullable String pageToken, - @Nullable String tableNamePattern) + @Nullable String tableNamePattern, + @Nullable String tableType) throws DatabaseNotExistException { try { - return api.listTablesPaged(databaseName, maxResults, pageToken, tableNamePattern); + return api.listTablesPaged( + databaseName, maxResults, pageToken, tableNamePattern, tableType); } catch (NoSuchResourceException e) { throw new DatabaseNotExistException(databaseName); } @@ -238,11 +240,13 @@ public PagedList

listTableDetailsPaged( String db, @Nullable Integer maxResults, @Nullable String pageToken, - @Nullable String tableNamePattern) + @Nullable String tableNamePattern, + @Nullable String tableType) throws DatabaseNotExistException { try { PagedList tables = - api.listTableDetailsPaged(db, maxResults, pageToken, tableNamePattern); + api.listTableDetailsPaged( + db, maxResults, pageToken, tableNamePattern, tableType); return new PagedList<>( tables.getElements().stream() .map(t -> toTable(db, t)) @@ -330,6 +334,11 @@ public boolean supportsListByPattern() { return true; } + @Override + public boolean supportsListTableByType() { + return true; + } + @Override public boolean supportsVersionManagement() { return true; @@ -345,7 +354,7 @@ public boolean commitSnapshot( try { return api.commitSnapshot(identifier, tableUuid, snapshot, statistics); } catch (NoSuchResourceException e) { - throw new TableNotExistException(identifier); + throw new TableNotExistException(identifier, e); } catch (ForbiddenException e) { throw new TableNoPermissionException(identifier, e); } catch (BadRequestException e) { diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java index f2f3d89064ee..cbfe81b3a9a0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java @@ -372,7 +372,7 @@ protected void updateLastColumn( }.updateIntermediateColumn(newFields, 0); } else if (change instanceof RenameColumn) { RenameColumn rename = (RenameColumn) change; - assertNotUpdatingPrimaryKeys(oldTableSchema, rename.fieldNames(), "rename"); + assertNotUpdatingPartitionKeys(oldTableSchema, rename.fieldNames(), "rename"); new NestedColumnModifier(rename.fieldNames(), lazyIdentifier) { @Override protected void updateLastColumn( @@ -416,6 +416,7 @@ protected void updateLastColumn( }.updateIntermediateColumn(newFields, 0); } else if (change instanceof UpdateColumnType) { UpdateColumnType update = (UpdateColumnType) change; + assertNotUpdatingPartitionKeys(oldTableSchema, update.fieldNames(), "update"); assertNotUpdatingPrimaryKeys(oldTableSchema, update.fieldNames(), "update"); updateNestedColumn( newFields, @@ -458,11 +459,9 @@ protected void updateLastColumn( lazyIdentifier); } else if (change instanceof UpdateColumnNullability) { UpdateColumnNullability update = (UpdateColumnNullability) change; - if (update.fieldNames().length == 1 - && update.newNullability() - && oldTableSchema.primaryKeys().contains(update.fieldNames()[0])) { - throw new UnsupportedOperationException( - "Cannot change nullability of primary key"); + if (update.newNullability()) { + assertNotUpdatingPrimaryKeys( + oldTableSchema, update.fieldNames(), "change nullability of"); } updateNestedColumn( newFields, @@ -839,17 +838,29 @@ private static void dropColumnValidation(TableSchema schema, DropColumn change) } } - private static void assertNotUpdatingPrimaryKeys( + private static void assertNotUpdatingPartitionKeys( TableSchema schema, String[] fieldNames, String operation) { // partition keys can't be nested columns if (fieldNames.length > 1) { return; } - String columnToRename = fieldNames[0]; - if (schema.partitionKeys().contains(columnToRename)) { + String fieldName = fieldNames[0]; + if (schema.partitionKeys().contains(fieldName)) { throw new UnsupportedOperationException( - String.format( - "Cannot " + operation + " partition column: [%s]", columnToRename)); + String.format("Cannot %s partition column: [%s]", operation, fieldName)); + } + } + + private static void assertNotUpdatingPrimaryKeys( + TableSchema schema, String[] fieldNames, String operation) { + // primary keys can't be nested columns + if (fieldNames.length > 1) { + return; + } + String fieldName = fieldNames[0]; + if (schema.primaryKeys().contains(fieldName)) { + throw new UnsupportedOperationException( + String.format("Cannot %s primary key", operation)); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java index 064630023d95..09de891a37c2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaValidation.java @@ -239,6 +239,8 @@ public static void validateTableSchema(TableSchema schema) { validateMergeFunctionFactory(schema); validateRowTracking(schema, options); + + validateIncrementalClustering(schema, options); } public static void validateFallbackBranch(SchemaManager schemaManager, TableSchema schema) { @@ -567,14 +569,6 @@ private static void validateBucket(TableSchema schema, CoreOptions options) { } else if (bucket < 1 && !isPostponeBucketTable(schema, bucket)) { throw new RuntimeException("The number of buckets needs to be greater than 0."); } else { - if (schema.crossPartitionUpdate()) { - throw new IllegalArgumentException( - String.format( - "You should use dynamic bucket (bucket = -1) mode in cross partition update case " - + "(Primary key constraint %s not include all partition fields %s).", - schema.primaryKeys(), schema.partitionKeys())); - } - if (schema.primaryKeys().isEmpty() && schema.bucketKeys().isEmpty()) { throw new RuntimeException( "You should define a 'bucket-key' for bucketed append mode."); @@ -639,11 +633,11 @@ private static void validateRowTracking(TableSchema schema, CoreOptions options) if (rowTrackingEnabled) { checkArgument( options.bucket() == -1, - "Cannot define %s for row lineage table, it only support bucket = -1", + "Cannot define %s for row tracking table, it only support bucket = -1", CoreOptions.BUCKET.key()); checkArgument( schema.primaryKeys().isEmpty(), - "Cannot define %s for row lineage table.", + "Cannot define %s for row tracking table.", PRIMARY_KEY.key()); } @@ -656,4 +650,17 @@ private static void validateRowTracking(TableSchema schema, CoreOptions options) "Data evolution config must disabled with deletion-vectors.enabled"); } } + + private static void validateIncrementalClustering(TableSchema schema, CoreOptions options) { + if (options.clusteringIncrementalEnabled()) { + checkArgument( + options.bucket() == -1, + "Cannot define %s for incremental clustering table, it only support bucket = -1", + CoreOptions.BUCKET.key()); + checkArgument( + schema.primaryKeys().isEmpty(), + "Cannot define %s for incremental clustering table.", + PRIMARY_KEY.key()); + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index db0ca5e5ffc1..aa5b8bb70d79 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -238,7 +238,7 @@ public RowKeyExtractor createRowKeyExtractor() { case HASH_FIXED: return new FixedBucketRowKeyExtractor(schema()); case HASH_DYNAMIC: - case CROSS_PARTITION: + case KEY_DYNAMIC: return new DynamicBucketRowKeyExtractor(schema()); case BUCKET_UNAWARE: return new AppendTableRowKeyExtractor(schema()); @@ -455,7 +455,8 @@ public TableCommitImpl newCommit(String commitUser) { new ConsumerManager(fileIO, path, snapshotManager().branch()), options.snapshotExpireExecutionMode(), name(), - options.forceCreatingSnapshot()); + options.forceCreatingSnapshot(), + options.fileOperationThreadNum()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageImpl.java index 5831066816e1..8e715462f582 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageImpl.java @@ -18,20 +18,17 @@ package org.apache.paimon.table.sink; -import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataIncrement; import org.apache.paimon.io.DataInputViewStreamWrapper; import org.apache.paimon.io.DataOutputViewStreamWrapper; -import org.apache.paimon.io.IndexIncrement; import javax.annotation.Nullable; import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; -import java.util.Collections; import java.util.Objects; import static org.apache.paimon.utils.SerializationUtils.deserializedBytes; @@ -50,37 +47,18 @@ public class CommitMessageImpl implements CommitMessage { private transient @Nullable Integer totalBuckets; private transient DataIncrement dataIncrement; private transient CompactIncrement compactIncrement; - private transient IndexIncrement indexIncrement; - @VisibleForTesting public CommitMessageImpl( BinaryRow partition, int bucket, @Nullable Integer totalBuckets, DataIncrement dataIncrement, CompactIncrement compactIncrement) { - this( - partition, - bucket, - totalBuckets, - dataIncrement, - compactIncrement, - new IndexIncrement(Collections.emptyList())); - } - - public CommitMessageImpl( - BinaryRow partition, - int bucket, - @Nullable Integer totalBuckets, - DataIncrement dataIncrement, - CompactIncrement compactIncrement, - IndexIncrement indexIncrement) { this.partition = partition; this.bucket = bucket; this.totalBuckets = totalBuckets; this.dataIncrement = dataIncrement; this.compactIncrement = compactIncrement; - this.indexIncrement = indexIncrement; } @Override @@ -106,12 +84,8 @@ public CompactIncrement compactIncrement() { return compactIncrement; } - public IndexIncrement indexIncrement() { - return indexIncrement; - } - public boolean isEmpty() { - return dataIncrement.isEmpty() && compactIncrement.isEmpty() && indexIncrement.isEmpty(); + return dataIncrement.isEmpty() && compactIncrement.isEmpty(); } private void writeObject(ObjectOutputStream out) throws IOException { @@ -131,7 +105,6 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE this.totalBuckets = message.totalBuckets; this.dataIncrement = message.dataIncrement; this.compactIncrement = message.compactIncrement; - this.indexIncrement = message.indexIncrement; } @Override @@ -148,14 +121,12 @@ public boolean equals(Object o) { && Objects.equals(partition, that.partition) && Objects.equals(totalBuckets, that.totalBuckets) && Objects.equals(dataIncrement, that.dataIncrement) - && Objects.equals(compactIncrement, that.compactIncrement) - && Objects.equals(indexIncrement, that.indexIncrement); + && Objects.equals(compactIncrement, that.compactIncrement); } @Override public int hashCode() { - return Objects.hash( - partition, bucket, totalBuckets, dataIncrement, compactIncrement, indexIncrement); + return Objects.hash(partition, bucket, totalBuckets, dataIncrement, compactIncrement); } @Override @@ -166,8 +137,7 @@ public String toString() { + "bucket = %d, " + "totalBuckets = %s, " + "newFilesIncrement = %s, " - + "compactIncrement = %s, " - + "indexIncrement = %s}", - partition, bucket, totalBuckets, dataIncrement, compactIncrement, indexIncrement); + + "compactIncrement = %s}", + partition, bucket, totalBuckets, dataIncrement, compactIncrement); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageLegacyV2Serializer.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageLegacyV2Serializer.java index 88e9e3513dce..5c3976af81f9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageLegacyV2Serializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageLegacyV2Serializer.java @@ -18,6 +18,7 @@ package org.apache.paimon.table.sink; +import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; @@ -26,7 +27,6 @@ import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataIncrement; import org.apache.paimon.io.DataInputView; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.stats.SimpleStats; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; @@ -64,23 +64,28 @@ public List deserializeList(DataInputView view) throws IOExceptio } public CommitMessage deserialize(DataInputView view) throws IOException { + BinaryRow partition = deserializeBinaryRow(view); + int bucket = view.readInt(); if (dataFileSerializer == null) { dataFileSerializer = new DataFileMetaLegacyV2Serializer(); indexEntrySerializer = new IndexFileMetaLegacyV2Serializer(); } - return new CommitMessageImpl( - deserializeBinaryRow(view), - view.readInt(), - null, + DataIncrement dataIncrement = new DataIncrement( dataFileSerializer.deserializeList(view), Collections.emptyList(), - dataFileSerializer.deserializeList(view)), + dataFileSerializer.deserializeList(view)); + CompactIncrement compactIncrement = new CompactIncrement( dataFileSerializer.deserializeList(view), dataFileSerializer.deserializeList(view), - dataFileSerializer.deserializeList(view)), - new IndexIncrement(indexEntrySerializer.deserializeList(view))); + dataFileSerializer.deserializeList(view)); + if (compactIncrement.isEmpty()) { + dataIncrement.newIndexFiles().addAll(indexEntrySerializer.deserializeList(view)); + } else { + compactIncrement.newIndexFiles().addAll(indexEntrySerializer.deserializeList(view)); + } + return new CommitMessageImpl(partition, bucket, null, dataIncrement, compactIncrement); } private static RowType legacyDataFileSchema() { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java index 529b0cb369bf..0a34434028d9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java @@ -18,6 +18,7 @@ package org.apache.paimon.table.sink; +import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.serializer.VersionedSerializer; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.index.IndexFileMetaSerializer; @@ -36,13 +37,11 @@ import org.apache.paimon.io.DataInputView; import org.apache.paimon.io.DataOutputView; import org.apache.paimon.io.DataOutputViewStreamWrapper; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.utils.IOExceptionSupplier; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import static org.apache.paimon.utils.SerializationUtils.deserializeBinaryRow; @@ -51,7 +50,7 @@ /** {@link VersionedSerializer} for {@link CommitMessage}. */ public class CommitMessageSerializer implements VersionedSerializer { - private static final int CURRENT_VERSION = 9; + public static final int CURRENT_VERSION = 10; private final DataFileMetaSerializer dataFileSerializer; private final IndexFileMetaSerializer indexEntrySerializer; @@ -103,14 +102,19 @@ private void serialize(CommitMessage obj, DataOutputView view) throws IOExceptio view.writeBoolean(false); } + // data increment dataFileSerializer.serializeList(message.newFilesIncrement().newFiles(), view); dataFileSerializer.serializeList(message.newFilesIncrement().deletedFiles(), view); dataFileSerializer.serializeList(message.newFilesIncrement().changelogFiles(), view); + indexEntrySerializer.serializeList(message.newFilesIncrement().newIndexFiles(), view); + indexEntrySerializer.serializeList(message.newFilesIncrement().deletedIndexFiles(), view); + + // compact increment dataFileSerializer.serializeList(message.compactIncrement().compactBefore(), view); dataFileSerializer.serializeList(message.compactIncrement().compactAfter(), view); dataFileSerializer.serializeList(message.compactIncrement().changelogFiles(), view); - indexEntrySerializer.serializeList(message.indexIncrement().newIndexFiles(), view); - indexEntrySerializer.serializeList(message.indexIncrement().deletedIndexFiles(), view); + indexEntrySerializer.serializeList(message.compactIncrement().newIndexFiles(), view); + indexEntrySerializer.serializeList(message.compactIncrement().deletedIndexFiles(), view); } @Override @@ -132,18 +136,48 @@ private CommitMessage deserialize(int version, DataInputView view) throws IOExce IOExceptionSupplier> fileDeserializer = fileDeserializer(version, view); IOExceptionSupplier> indexEntryDeserializer = indexEntryDeserializer(version, view); - - return new CommitMessageImpl( - deserializeBinaryRow(view), - view.readInt(), - version >= 7 && view.readBoolean() ? view.readInt() : null, - new DataIncrement( - fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()), - new CompactIncrement( - fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()), - new IndexIncrement( - indexEntryDeserializer.get(), - version <= 2 ? Collections.emptyList() : indexEntryDeserializer.get())); + if (version >= 10) { + return new CommitMessageImpl( + deserializeBinaryRow(view), + view.readInt(), + view.readBoolean() ? view.readInt() : null, + new DataIncrement( + fileDeserializer.get(), + fileDeserializer.get(), + fileDeserializer.get(), + indexEntryDeserializer.get(), + indexEntryDeserializer.get()), + new CompactIncrement( + fileDeserializer.get(), + fileDeserializer.get(), + fileDeserializer.get(), + indexEntryDeserializer.get(), + indexEntryDeserializer.get())); + } else { + BinaryRow partition = deserializeBinaryRow(view); + int bucket = view.readInt(); + Integer totalBuckets = version >= 7 && view.readBoolean() ? view.readInt() : null; + DataIncrement dataIncrement = + new DataIncrement( + fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()); + CompactIncrement compactIncrement = + new CompactIncrement( + fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()); + if (compactIncrement.isEmpty()) { + dataIncrement.newIndexFiles().addAll(indexEntryDeserializer.get()); + } else { + compactIncrement.newIndexFiles().addAll(indexEntryDeserializer.get()); + } + if (version > 2) { + if (compactIncrement.isEmpty()) { + dataIncrement.deletedIndexFiles().addAll(indexEntryDeserializer.get()); + } else { + compactIncrement.deletedIndexFiles().addAll(indexEntryDeserializer.get()); + } + } + return new CommitMessageImpl( + partition, bucket, totalBuckets, dataIncrement, compactIncrement); + } } private IOExceptionSupplier> fileDeserializer( diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java index 95b2df34f513..4f3bb5b71b09 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java @@ -36,6 +36,7 @@ import org.apache.paimon.utils.CompactedChangelogPathResolver; import org.apache.paimon.utils.DataFilePathFactories; import org.apache.paimon.utils.ExecutorThreadFactory; +import org.apache.paimon.utils.FileOperationThreadPool; import org.apache.paimon.utils.IndexFilePathFactories; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; @@ -57,6 +58,7 @@ import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.function.Predicate; @@ -66,7 +68,6 @@ import static java.util.Collections.singletonList; import static org.apache.paimon.CoreOptions.ExpireExecutionMode; import static org.apache.paimon.table.sink.BatchWriteBuilder.COMMIT_IDENTIFIER; -import static org.apache.paimon.utils.ManifestReadThreadPool.getExecutorService; import static org.apache.paimon.utils.Preconditions.checkState; import static org.apache.paimon.utils.ThreadPoolUtils.randomlyExecuteSequentialReturn; @@ -79,18 +80,16 @@ public class TableCommitImpl implements InnerTableCommit { @Nullable private final Runnable expireSnapshots; @Nullable private final PartitionExpire partitionExpire; @Nullable private final TagAutoManager tagAutoManager; - @Nullable private final Duration consumerExpireTime; private final ConsumerManager consumerManager; - private final ExecutorService maintainExecutor; private final AtomicReference maintainError; - private final String tableName; + private final boolean forceCreatingSnapshot; + private final ThreadPoolExecutor fileCheckExecutor; @Nullable private Map overwritePartition = null; private boolean batchCommitted = false; - private final boolean forceCreatingSnapshot; private boolean expireForEmptyCommit = true; public TableCommitImpl( @@ -102,7 +101,8 @@ public TableCommitImpl( ConsumerManager consumerManager, ExpireExecutionMode expireExecutionMode, String tableName, - boolean forceCreatingSnapshot) { + boolean forceCreatingSnapshot, + int threadNum) { if (partitionExpire != null) { commit.withPartitionExpire(partitionExpire); } @@ -125,6 +125,7 @@ public TableCommitImpl( this.tableName = tableName; this.forceCreatingSnapshot = forceCreatingSnapshot; + this.fileCheckExecutor = FileOperationThreadPool.getExecutorService(threadNum); } public boolean forceCreatingSnapshot() { @@ -248,7 +249,8 @@ public void commitMultiple(List committables, boolean check committable = new ManifestCommittable(Long.MAX_VALUE); } int newSnapshots = - commit.overwrite(overwritePartition, committable, Collections.emptyMap()); + commit.overwritePartition( + overwritePartition, committable, Collections.emptyMap()); maintain( committable.identifier(), maintainExecutor, @@ -290,14 +292,15 @@ private void checkFilesExistence(List committables) { Consumer collector = f -> files.addAll(f.collectFiles(pathFactory)); msg.newFilesIncrement().newFiles().forEach(collector); msg.newFilesIncrement().changelogFiles().forEach(collector); - msg.compactIncrement().compactBefore().forEach(collector); - msg.compactIncrement().compactAfter().forEach(collector); - msg.indexIncrement().newIndexFiles().stream() + msg.newFilesIncrement().newIndexFiles().stream() .map(indexFileFactory::toPath) .forEach(files::add); - msg.indexIncrement().deletedIndexFiles().stream() + msg.compactIncrement().compactAfter().forEach(collector); + msg.compactIncrement().newIndexFiles().stream() .map(indexFileFactory::toPath) .forEach(files::add); + + // skip compact before files, deleted index files } } @@ -322,7 +325,7 @@ private void checkFilesExistence(List committables) { List nonExistFiles = Lists.newArrayList( randomlyExecuteSequentialReturn( - getExecutorService(null), + fileCheckExecutor, f -> nonExists.test(f) ? singletonList(f) : emptyList(), resolvedFiles)); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWrite.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWrite.java index 73956c30e00f..a36f314cd696 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWrite.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; import org.apache.paimon.io.BundleRecords; +import org.apache.paimon.memory.MemoryPoolFactory; import org.apache.paimon.memory.MemorySegmentPool; import org.apache.paimon.metrics.MetricRegistry; import org.apache.paimon.table.Table; @@ -43,7 +44,11 @@ public interface TableWrite extends AutoCloseable { TableWrite withWriteType(RowType writeType); /** With {@link MemorySegmentPool} for the current table write. */ - TableWrite withMemoryPool(MemorySegmentPool memoryPool); + default TableWrite withMemoryPool(MemorySegmentPool memoryPool) { + return withMemoryPoolFactory(new MemoryPoolFactory(memoryPool)); + } + + TableWrite withMemoryPoolFactory(MemoryPoolFactory memoryPoolFactory); /** Calculate which partition {@code row} belongs to. */ BinaryRow getPartition(InternalRow row); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWriteImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWriteImpl.java index 99e30bcccf8d..493a2b4ffe0b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWriteImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableWriteImpl.java @@ -27,7 +27,6 @@ import org.apache.paimon.io.BundleRecords; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.memory.MemoryPoolFactory; -import org.apache.paimon.memory.MemorySegmentPool; import org.apache.paimon.metrics.MetricRegistry; import org.apache.paimon.operation.BundleFileStoreWriter; import org.apache.paimon.operation.FileStoreWrite; @@ -55,7 +54,6 @@ */ public class TableWriteImpl implements InnerTableWrite, Restorable>> { - private final RowType rowType; private final FileStoreWrite write; private final KeyAndBucketExtractor keyAndBucketExtractor; private final RecordExtractor recordExtractor; @@ -64,8 +62,9 @@ public class TableWriteImpl implements InnerTableWrite, Restorable recordExtractor, @Nullable RowKindGenerator rowKindGenerator, @Nullable RowKindFilter rowKindFilter) { - this.rowType = rowType; + this.writeType = rowType; this.write = write; this.keyAndBucketExtractor = keyAndBucketExtractor; this.recordExtractor = recordExtractor; @@ -116,15 +115,17 @@ public TableWriteImpl withIOManager(IOManager ioManager) { @Override public TableWriteImpl withWriteType(RowType writeType) { write.withWriteType(writeType); + this.writeType = writeType; + List notNullColumnNames = + writeType.getFields().stream() + .filter(field -> !field.type().isNullable()) + .map(DataField::name) + .collect(Collectors.toList()); + this.notNullFieldIndex = writeType.getFieldIndices(notNullColumnNames); return this; } @Override - public TableWriteImpl withMemoryPool(MemorySegmentPool memoryPool) { - write.withMemoryPool(memoryPool); - return this; - } - public TableWriteImpl withMemoryPoolFactory(MemoryPoolFactory memoryPoolFactory) { write.withMemoryPoolFactory(memoryPoolFactory); return this; @@ -195,7 +196,7 @@ public SinkRecord writeAndReturn(InternalRow row, int bucket) throws Exception { private void checkNullability(InternalRow row) { for (int idx : notNullFieldIndex) { if (row.isNullAt(idx)) { - String columnName = rowType.getFields().get(idx).name(); + String columnName = writeType.getFields().get(idx).name(); throw new RuntimeException( String.format("Cannot write null to non-null column(%s)", columnName)); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java b/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java index 079477b00188..3a4c112a95ee 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java @@ -328,6 +328,22 @@ public int hashCode() { rawConvertible); } + @Override + public String toString() { + return "{" + + "snapshotId=" + + snapshotId + + ", partition=hash-" + + partition.hashCode() + + ", bucket=" + + bucket + + ", rawConvertible=" + + rawConvertible + + '}' + + "@" + + Integer.toHexString(hashCode()); + } + private void writeObject(ObjectOutputStream out) throws IOException { serialize(new DataOutputViewStreamWrapper(out)); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java index 82112f5fd4a1..ed80f3f92a2b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java @@ -34,6 +34,7 @@ import org.apache.paimon.table.source.Split; import org.apache.paimon.table.source.SplitGenerator; import org.apache.paimon.table.source.TableScan; +import org.apache.paimon.utils.BiFilter; import org.apache.paimon.utils.ChangelogManager; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.Filter; @@ -87,6 +88,8 @@ public interface SnapshotReader { SnapshotReader withLevelFilter(Filter levelFilter); + SnapshotReader withLevelMinMaxFilter(BiFilter minMaxFilter); + SnapshotReader enableValueFilter(); SnapshotReader withManifestEntryFilter(Filter filter); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java index 81c015a9d47b..477ce34e9322 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java @@ -47,6 +47,7 @@ import org.apache.paimon.table.source.PlanImpl; import org.apache.paimon.table.source.ScanMode; import org.apache.paimon.table.source.SplitGenerator; +import org.apache.paimon.utils.BiFilter; import org.apache.paimon.utils.ChangelogManager; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.Filter; @@ -254,6 +255,12 @@ public SnapshotReader withLevelFilter(Filter levelFilter) { return this; } + @Override + public SnapshotReader withLevelMinMaxFilter(BiFilter minMaxFilter) { + scan.withLevelMinMaxFilter(minMaxFilter); + return this; + } + @Override public SnapshotReader enableValueFilter() { scan.enableValueFilter(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java index ab3b13a5abbd..3fd6a8d8af3d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java @@ -59,6 +59,7 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BiFilter; import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.ChangelogManager; import org.apache.paimon.utils.FileStorePathFactory; @@ -359,6 +360,12 @@ public SnapshotReader withLevelFilter(Filter levelFilter) { return this; } + @Override + public SnapshotReader withLevelMinMaxFilter(BiFilter minMaxFilter) { + wrapped.withLevelMinMaxFilter(minMaxFilter); + return this; + } + @Override public SnapshotReader enableValueFilter() { wrapped.enableValueFilter(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java index 8488e3ff9ba2..cc8d1621a3fa 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java @@ -131,7 +131,7 @@ public RecordReader createReader(Split split) throws IOException { (row1, row2) -> new AuditLogRow( readProjection, convertToArray(row1, row2, fieldGetters)), - wrapped.rowType()); + this.wrappedReadType); } else { return dataRead.createReader(split) .transform( diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java index eb18e679f503..caa8315eb9ff 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java @@ -274,6 +274,9 @@ private TableScan.Plan tablePlan(FileStoreTable storeTable) { return levelPredicate.test( GenericRow.of(null, null, null, null, null, level)); }); + } else { + // avoid that batchScanSkipLevel0 is true + scan.withLevelFilter(level -> true); } return scan.plan(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java index 07efaf85a5b7..04d3490df81d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java @@ -182,7 +182,8 @@ public RecordReader createReader(Split split) throws IOException { throw new IllegalArgumentException("Unsupported split: " + split.getClass()); } - List partitions = fileStoreTable.newScan().listPartitionEntries(); + List partitions = + fileStoreTable.newScan().withLevelFilter(level -> true).listPartitionEntries(); @SuppressWarnings("unchecked") CastExecutor partitionCastExecutor = diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/RowLineageTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/RowTrackingTable.java similarity index 90% rename from paimon-core/src/main/java/org/apache/paimon/table/system/RowLineageTable.java rename to paimon-core/src/main/java/org/apache/paimon/table/system/RowTrackingTable.java index 75e4319bc113..519bf62d183c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/RowLineageTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/RowTrackingTable.java @@ -30,6 +30,7 @@ import org.apache.paimon.table.DataTable; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.ReadonlyTable; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.DataTableScan; import org.apache.paimon.table.source.InnerTableRead; @@ -48,16 +49,15 @@ import java.util.Optional; import static org.apache.paimon.catalog.Identifier.SYSTEM_TABLE_SPLITTER; -import static org.apache.paimon.table.SpecialFields.rowTypeWithRowLineage; /** A {@link Table} for reading row id of table. */ -public class RowLineageTable implements DataTable, ReadonlyTable { +public class RowTrackingTable implements DataTable, ReadonlyTable { - public static final String ROW_LINEAGE = "row_lineage"; + public static final String ROW_TRACKING = "row_tracking"; private final FileStoreTable wrapped; - public RowLineageTable(FileStoreTable wrapped) { + public RowTrackingTable(FileStoreTable wrapped) { this.wrapped = wrapped; if (!coreOptions().rowTrackingEnabled()) { @@ -96,12 +96,12 @@ public SimpleFileReader indexManifestFileReader() { @Override public String name() { - return wrapped.name() + SYSTEM_TABLE_SPLITTER + ROW_LINEAGE; + return wrapped.name() + SYSTEM_TABLE_SPLITTER + ROW_TRACKING; } @Override public RowType rowType() { - return rowTypeWithRowLineage(wrapped.rowType()); + return SpecialFields.rowTypeWithRowTracking(wrapped.rowType()); } @Override @@ -176,7 +176,7 @@ public BranchManager branchManager() { @Override public DataTable switchToBranch(String branchName) { - return new RowLineageTable(wrapped.switchToBranch(branchName)); + return new RowTrackingTable(wrapped.switchToBranch(branchName)); } @Override @@ -186,7 +186,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new RowLineageTable(wrapped.copy(dynamicOptions)); + return new RowTrackingTable(wrapped.copy(dynamicOptions)); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java index 69f7a65ddf8c..d79f542f7d38 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java @@ -45,7 +45,7 @@ import static org.apache.paimon.table.system.OptionsTable.OPTIONS; import static org.apache.paimon.table.system.PartitionsTable.PARTITIONS; import static org.apache.paimon.table.system.ReadOptimizedTable.READ_OPTIMIZED; -import static org.apache.paimon.table.system.RowLineageTable.ROW_LINEAGE; +import static org.apache.paimon.table.system.RowTrackingTable.ROW_TRACKING; import static org.apache.paimon.table.system.SchemasTable.SCHEMAS; import static org.apache.paimon.table.system.SnapshotsTable.SNAPSHOTS; import static org.apache.paimon.table.system.StatisticTable.STATISTICS; @@ -73,7 +73,7 @@ public class SystemTableLoader { .put(STATISTICS, StatisticTable::new) .put(BINLOG, BinlogTable::new) .put(TABLE_INDEXES, TableIndexesTable::new) - .put(ROW_LINEAGE, RowLineageTable::new) + .put(ROW_TRACKING, RowTrackingTable::new) .build(); public static final List SYSTEM_TABLES = new ArrayList<>(SYSTEM_TABLE_LOADERS.keySet()); diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ConflictDeletionUtils.java b/paimon-core/src/main/java/org/apache/paimon/utils/ConflictDeletionUtils.java new file mode 100644 index 000000000000..00942ea0489a --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/utils/ConflictDeletionUtils.java @@ -0,0 +1,149 @@ +/* + * 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. + */ + +package org.apache.paimon.utils; + +import org.apache.paimon.index.DeletionVectorMeta; +import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.manifest.FileKind; +import org.apache.paimon.manifest.IndexManifestEntry; +import org.apache.paimon.manifest.SimpleFileEntry; +import org.apache.paimon.manifest.SimpleFileEntryWithDV; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.paimon.utils.Preconditions.checkState; + +/** Utils for conflict deletion. */ +public class ConflictDeletionUtils { + + public static List buildBaseEntriesWithDV( + List baseEntries, List baseIndexEntries) { + if (baseEntries.isEmpty()) { + return Collections.emptyList(); + } + + Map fileNameToDVFileName = new HashMap<>(); + for (IndexManifestEntry indexManifestEntry : baseIndexEntries) { + // Should not attach DELETE type dv index for base file. + if (!indexManifestEntry.kind().equals(FileKind.DELETE)) { + IndexFileMeta indexFile = indexManifestEntry.indexFile(); + if (indexFile.dvRanges() != null) { + for (DeletionVectorMeta value : indexFile.dvRanges().values()) { + checkState( + !fileNameToDVFileName.containsKey(value.dataFileName()), + "One file should correspond to only one dv entry."); + fileNameToDVFileName.put(value.dataFileName(), indexFile.fileName()); + } + } + } + } + + // Attach dv name to file entries. + List entriesWithDV = new ArrayList<>(baseEntries.size()); + for (SimpleFileEntry fileEntry : baseEntries) { + entriesWithDV.add( + new SimpleFileEntryWithDV( + fileEntry, fileNameToDVFileName.get(fileEntry.fileName()))); + } + return entriesWithDV; + } + + public static List buildDeltaEntriesWithDV( + List baseEntries, + List deltaEntries, + List deltaIndexEntries) { + if (deltaEntries.isEmpty() && deltaIndexEntries.isEmpty()) { + return Collections.emptyList(); + } + + List entriesWithDV = new ArrayList<>(deltaEntries.size()); + + // One file may correspond to more than one dv entries, for example, delete the old dv, and + // create a new one. + Map> fileNameToDVEntry = new HashMap<>(); + for (IndexManifestEntry deltaIndexEntry : deltaIndexEntries) { + if (deltaIndexEntry.indexFile().dvRanges() != null) { + for (DeletionVectorMeta meta : deltaIndexEntry.indexFile().dvRanges().values()) { + fileNameToDVEntry.putIfAbsent(meta.dataFileName(), new ArrayList<>()); + fileNameToDVEntry.get(meta.dataFileName()).add(deltaIndexEntry); + } + } + } + + Set fileNotInDeltaEntries = new HashSet<>(fileNameToDVEntry.keySet()); + // 1. Attach dv name to delta file entries. + for (SimpleFileEntry fileEntry : deltaEntries) { + if (fileNameToDVEntry.containsKey(fileEntry.fileName())) { + List dvs = fileNameToDVEntry.get(fileEntry.fileName()); + checkState(dvs.size() == 1, "Delta entry only can have one dv file"); + entriesWithDV.add( + new SimpleFileEntryWithDV(fileEntry, dvs.get(0).indexFile().fileName())); + fileNotInDeltaEntries.remove(fileEntry.fileName()); + } else { + entriesWithDV.add(new SimpleFileEntryWithDV(fileEntry, null)); + } + } + + // 2. For file not in delta entries, build entry with dv with baseEntries. + if (!fileNotInDeltaEntries.isEmpty()) { + Map fileNameToFileEntry = new HashMap<>(); + for (SimpleFileEntry baseEntry : baseEntries) { + if (baseEntry.kind().equals(FileKind.ADD)) { + fileNameToFileEntry.put(baseEntry.fileName(), baseEntry); + } + } + + for (String fileName : fileNotInDeltaEntries) { + SimpleFileEntryWithDV simpleFileEntry = + (SimpleFileEntryWithDV) fileNameToFileEntry.get(fileName); + checkState( + simpleFileEntry != null, + String.format( + "Trying to create deletion vector on file %s which is not previously added.", + fileName)); + List dvEntries = fileNameToDVEntry.get(fileName); + // If dv entry's type id DELETE, add DELETE + // If dv entry's type id ADD, add ADD + for (IndexManifestEntry dvEntry : dvEntries) { + entriesWithDV.add( + new SimpleFileEntryWithDV( + dvEntry.kind().equals(FileKind.ADD) + ? simpleFileEntry + : simpleFileEntry.toDelete(), + dvEntry.indexFile().fileName())); + } + + // If one file correspond to only one dv entry and the type is ADD, + // we need to add a DELETE. + // This happens when create a dv for a file that doesn't have dv before. + if (dvEntries.size() == 1 && dvEntries.get(0).kind().equals(FileKind.ADD)) { + entriesWithDV.add(new SimpleFileEntryWithDV(simpleFileEntry.toDelete(), null)); + } + } + } + + return entriesWithDV; + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/JavaPyE2ETest.java b/paimon-core/src/test/java/org/apache/paimon/JavaPyE2ETest.java new file mode 100644 index 000000000000..3759bc87570d --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/JavaPyE2ETest.java @@ -0,0 +1,159 @@ +/* + * 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. + */ + +package org.apache.paimon; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.DataFormatTestUtil; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fs.Path; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.sink.InnerTableCommit; +import org.apache.paimon.table.sink.StreamTableWrite; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.table.source.TableRead; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.utils.TraceableFileIO; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledIfSystemProperty; + +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +import static org.apache.paimon.table.SimpleTableTestBase.getResult; +import static org.assertj.core.api.Assertions.assertThat; + +/** Mixed language overwrite test for Java and Python interoperability. */ +public class JavaPyE2ETest { + + java.nio.file.Path tempDir = Paths.get("../paimon-python/pypaimon/tests/e2e").toAbsolutePath(); + + // Fields from TableTestBase that we need + protected final String commitUser = UUID.randomUUID().toString(); + protected Path warehouse; + protected Catalog catalog; + protected String database; + + @BeforeEach + public void before() throws Exception { + database = "default"; + + // Create warehouse directory if it doesn't exist + if (!Files.exists(tempDir.resolve("warehouse"))) { + Files.createDirectories(tempDir.resolve("warehouse")); + } + + warehouse = new Path(TraceableFileIO.SCHEME + "://" + tempDir.resolve("warehouse")); + catalog = CatalogFactory.createCatalog(CatalogContext.create(warehouse)); + + // Create database if it doesn't exist + try { + catalog.createDatabase(database, false); + } catch (Catalog.DatabaseAlreadyExistException e) { + // Database already exists, ignore + } + } + + @Test + @EnabledIfSystemProperty(named = "run.e2e.tests", matches = "true") + public void testJavaWriteRead() throws Exception { + Identifier identifier = identifier("mixed_test_tablej"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .column("category", DataTypes.STRING()) + .column("value", DataTypes.DOUBLE()) + .partitionKeys("category") + .option("dynamic-partition-overwrite", "false") + .build(); + + catalog.createTable(identifier, schema, true); + Table table = catalog.getTable(identifier); + FileStoreTable fileStoreTable = (FileStoreTable) table; + + try (StreamTableWrite write = fileStoreTable.newWrite(commitUser); + InnerTableCommit commit = fileStoreTable.newCommit(commitUser)) { + + write.write(createRow(1, "Apple", "Fruit", 1.5)); + write.write(createRow(2, "Banana", "Fruit", 0.8)); + write.write(createRow(3, "Carrot", "Vegetable", 0.6)); + write.write(createRow(4, "Broccoli", "Vegetable", 1.2)); + write.write(createRow(5, "Chicken", "Meat", 5.0)); + write.write(createRow(6, "Beef", "Meat", 8.0)); + + commit.commit(0, write.prepareCommit(true, 0)); + } + + List splits = + new ArrayList<>(fileStoreTable.newSnapshotReader().read().dataSplits()); + TableRead read = fileStoreTable.newRead(); + List res = + getResult( + read, + splits, + row -> DataFormatTestUtil.toStringNoRowKind(row, table.rowType())); + assertThat(res) + .containsExactlyInAnyOrder( + "1, Apple, Fruit, 1.5", + "2, Banana, Fruit, 0.8", + "3, Carrot, Vegetable, 0.6", + "4, Broccoli, Vegetable, 1.2", + "5, Chicken, Meat, 5.0", + "6, Beef, Meat, 8.0"); + } + + @Test + @EnabledIfSystemProperty(named = "run.e2e.tests", matches = "true") + public void testRead() throws Exception { + Identifier identifier = identifier("mixed_test_tablep"); + Table table = catalog.getTable(identifier); + FileStoreTable fileStoreTable = (FileStoreTable) table; + List splits = + new ArrayList<>(fileStoreTable.newSnapshotReader().read().dataSplits()); + TableRead read = fileStoreTable.newRead(); + List res = + getResult( + read, + splits, + row -> DataFormatTestUtil.toStringNoRowKind(row, table.rowType())); + System.out.println(res); + } + + // Helper method from TableTestBase + protected Identifier identifier(String tableName) { + return new Identifier(database, tableName); + } + + private static InternalRow createRow(int id, String name, String category, double value) { + return GenericRow.of( + id, BinaryString.fromString(name), BinaryString.fromString(category), value); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java index 9caa379884f1..2754022dd77a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java @@ -29,14 +29,15 @@ import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.io.CompactIncrement; +import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataIncrement; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.operation.FileStoreCommitImpl; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.SchemaUtils; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.stats.SimpleStats; import org.apache.paimon.table.CatalogEnvironment; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.CommitMessageImpl; @@ -44,6 +45,7 @@ import org.apache.paimon.types.RowType; import org.apache.paimon.utils.TraceableFileIO; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -93,6 +95,10 @@ public FileIO fileIO() { return this.fileIO; } + public TableSchema schema() { + return schema; + } + public FileStoreCommitImpl newCommit() { return super.newCommit(commitUser, null); } @@ -111,9 +117,13 @@ public CommitMessage removeIndexFiles( partition, bucket, options().bucket(), - DataIncrement.emptyIncrement(), - CompactIncrement.emptyIncrement(), - new IndexIncrement(Collections.emptyList(), indexFileMetas)); + new DataIncrement( + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + indexFileMetas), + CompactIncrement.emptyIncrement()); } public List scanDVIndexFiles(BinaryRow partition, int bucket) { @@ -135,6 +145,38 @@ public BucketedDvMaintainer createOrRestoreDVMaintainer(BinaryRow partition, int return factory.create(partition, bucket, indexFiles); } + public CommitMessageImpl writeDataFiles( + BinaryRow partition, int bucket, List dataFileNames) throws IOException { + List fileMetas = new ArrayList<>(); + Path bucketPath = pathFactory().bucketPath(partition, bucket); + for (String dataFileName : dataFileNames) { + Path path = new Path(bucketPath, dataFileName); + fileIO.newOutputStream(path, false).close(); + fileMetas.add( + DataFileMeta.forAppend( + path.getName(), + 10L, + 10L, + SimpleStats.EMPTY_STATS, + 0L, + 0L, + schema.id(), + Collections.emptyList(), + null, + null, + null, + null, + null, + null)); + } + return new CommitMessageImpl( + partition, + bucket, + options().bucket(), + new DataIncrement(fileMetas, Collections.emptyList(), Collections.emptyList()), + CompactIncrement.emptyIncrement()); + } + public CommitMessageImpl writeDVIndexFiles( BinaryRow partition, int bucket, Map> dataFileToPositions) { BucketedDvMaintainer dvMaintainer = createOrRestoreDVMaintainer(partition, bucket); @@ -149,9 +191,13 @@ public CommitMessageImpl writeDVIndexFiles( partition, bucket, options().bucket(), - DataIncrement.emptyIncrement(), - CompactIncrement.emptyIncrement(), - new IndexIncrement(indexFiles)); + new DataIncrement( + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + indexFiles, + Collections.emptyList()), + CompactIncrement.emptyIncrement()); } public static TestAppendFileStore createAppendStore( diff --git a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java index 8e4cafc7de60..aa405060f422 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java @@ -26,7 +26,7 @@ import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFilePathFactory; -import org.apache.paimon.io.IndexIncrement; +import org.apache.paimon.io.DataIncrement; import org.apache.paimon.manifest.FileEntry; import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.FileSource; @@ -263,7 +263,7 @@ public List overwriteData( null, Collections.emptyList(), (commit, committable) -> - commit.overwrite(partition, committable, Collections.emptyMap())); + commit.overwritePartition(partition, committable, Collections.emptyMap())); } public Snapshot dropPartitions(List> partitions) { @@ -346,14 +346,15 @@ public List commitDataImpl( entryWithPartition.getValue().entrySet()) { CommitIncrement increment = entryWithBucket.getValue().prepareCommit(ignorePreviousFiles); + DataIncrement dataIncrement = increment.newFilesIncrement(); + dataIncrement.newIndexFiles().addAll(indexFiles); committable.addFileCommittable( new CommitMessageImpl( entryWithPartition.getKey(), entryWithBucket.getKey(), options().bucket(), - increment.newFilesIncrement(), - increment.compactIncrement(), - new IndexIncrement(indexFiles))); + dataIncrement, + increment.compactIncrement())); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/append/AppendCompactTaskTest.java b/paimon-core/src/test/java/org/apache/paimon/append/AppendCompactTaskTest.java new file mode 100644 index 000000000000..0e95135873b0 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/append/AppendCompactTaskTest.java @@ -0,0 +1,212 @@ +/* + * 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. + */ + +package org.apache.paimon.append; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.TestAppendFileStore; +import org.apache.paimon.TestKeyValueGenerator; +import org.apache.paimon.compact.CompactManager; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.deletionvectors.BucketedDvMaintainer; +import org.apache.paimon.deletionvectors.DeletionVector; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.FileIOFinder; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.operation.BaseAppendFileStoreWrite; +import org.apache.paimon.operation.FileStoreScan; +import org.apache.paimon.operation.RawFileSplitRead; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.SchemaUtils; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.FileStorePathFactory; +import org.apache.paimon.utils.SnapshotManager; +import org.apache.paimon.utils.TraceableFileIO; + +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.function.Function; + +import static org.apache.paimon.io.DataFileTestUtils.newFile; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link AppendCompactTask}. */ +public class AppendCompactTaskTest { + + @TempDir java.nio.file.Path tempDir; + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testAppendCompactionWithDeletionVectors(boolean compactBeforeAllFiles) + throws Exception { + TestAppendFileStore store = + createAppendStore( + tempDir, + Collections.singletonMap( + CoreOptions.DELETION_VECTORS_ENABLED.key(), "true")); + + // Create deletion vectors for two files + // Each file has some deleted rows + Map> dvs = new HashMap<>(); + dvs.put("data-0.orc", Arrays.asList(1, 3, 5)); + dvs.put("data-1.orc", Arrays.asList(2, 4, 6)); + + // Write deletion vectors for all files to simulate existing deletion vectors + CommitMessageImpl commitMessage = store.writeDVIndexFiles(BinaryRow.EMPTY_ROW, 0, dvs); + store.commit(commitMessage); + + List allFiles = + Arrays.asList( + newFile("data-0.orc", 0, 0, 100, 100), + newFile("data-1.orc", 0, 101, 200, 200)); + + List beforeFiles = + compactBeforeAllFiles ? allFiles : Collections.singletonList(allFiles.get(0)); + AppendCompactTask compactTask = new AppendCompactTask(BinaryRow.EMPTY_ROW, beforeFiles); + + FileStoreTable table = + FileStoreTableFactory.create( + store.fileIO(), store.options().path(), store.schema()); + BaseAppendFileStoreWrite write = new NoopAppendWrite(store); + + CommitMessageImpl compactMessage = (CommitMessageImpl) compactTask.doCompact(table, write); + + assertThat(compactMessage.compactIncrement().deletedIndexFiles()).isNotEmpty(); + if (compactBeforeAllFiles) { + assertThat(compactMessage.compactIncrement().newIndexFiles()).isEmpty(); + } else { + assertThat(compactMessage.compactIncrement().newIndexFiles()).isNotEmpty(); + } + } + + private static class NoopAppendWrite extends BaseAppendFileStoreWrite { + + public NoopAppendWrite(TestAppendFileStore store) { + this( + store.fileIO(), + store.newRead(), + 0L, + store.schema().logicalRowType(), + store.schema().logicalPartitionType(), + store.pathFactory(), + store.snapshotManager(), + store.newScan(), + store.options(), + BucketedDvMaintainer.factory(store.newIndexFileHandler()), + "test"); + } + + private NoopAppendWrite( + FileIO fileIO, + RawFileSplitRead readForCompact, + long schemaId, + RowType rowType, + RowType partitionType, + FileStorePathFactory pathFactory, + SnapshotManager snapshotManager, + FileStoreScan scan, + CoreOptions options, + @Nullable BucketedDvMaintainer.Factory dvMaintainerFactory, + String tableName) { + super( + fileIO, + readForCompact, + schemaId, + rowType, + partitionType, + pathFactory, + snapshotManager, + scan, + options, + dvMaintainerFactory, + tableName); + } + + @Override + public List compactRewrite( + BinaryRow partition, + int bucket, + @Nullable Function dvFactory, + List toCompact) + throws Exception { + return Collections.emptyList(); + } + + @Override + protected CompactManager getCompactManager( + BinaryRow partition, + int bucket, + List restoredFiles, + ExecutorService compactExecutor, + @Nullable BucketedDvMaintainer dvMaintainer) { + return null; + } + + @Override + protected Function, Boolean> createWriterCleanChecker() { + return null; + } + } + + private TestAppendFileStore createAppendStore( + java.nio.file.Path tempDir, Map dynamicOptions) throws Exception { + String root = TraceableFileIO.SCHEME + "://" + tempDir.toString(); + Path path = new Path(tempDir.toUri()); + FileIO fileIO = FileIOFinder.find(new Path(root)); + SchemaManager schemaManage = new SchemaManager(new LocalFileIO(), path); + + Map options = new HashMap<>(dynamicOptions); + options.put(CoreOptions.PATH.key(), root); + TableSchema tableSchema = + SchemaUtils.forceCommit( + schemaManage, + new Schema( + TestKeyValueGenerator.DEFAULT_ROW_TYPE.getFields(), + Collections.emptyList(), + Collections.emptyList(), + options, + null)); + return new TestAppendFileStore( + fileIO, + schemaManage, + new CoreOptions(options), + tableSchema, + RowType.of(), + RowType.of(), + TestKeyValueGenerator.DEFAULT_ROW_TYPE, + (new Path(root)).getName()); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/append/cluster/HistoryPartitionClusterTest.java b/paimon-core/src/test/java/org/apache/paimon/append/cluster/HistoryPartitionClusterTest.java new file mode 100644 index 000000000000..f1f93ca989b2 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/append/cluster/HistoryPartitionClusterTest.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.append.cluster; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.mergetree.LevelSortedRun; +import org.apache.paimon.partition.PartitionPredicate; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.assertj.core.util.Lists; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.append.cluster.IncrementalClusterManagerTest.writeOnce; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link HistoryPartitionCluster}. */ +public class HistoryPartitionClusterTest { + + @TempDir java.nio.file.Path tempDir; + + @Test + public void testHistoryPartitionAutoClustering() throws Exception { + FileStoreTable table = createTable(Collections.emptyMap(), Collections.singletonList("f2")); + writeOnce( + table, + GenericRow.of( + 1, 1, BinaryString.fromString("pt1"), BinaryString.fromString("test"))); + writeOnce( + table, + GenericRow.of( + 1, 1, BinaryString.fromString("pt2"), BinaryString.fromString("test"))); + + Thread.sleep(2000); + writeOnce( + table, + GenericRow.of( + 1, 1, BinaryString.fromString("pt3"), BinaryString.fromString("test"))); + writeOnce( + table, + GenericRow.of( + 1, 1, BinaryString.fromString("pt4"), BinaryString.fromString("test"))); + + // test specify history partition and enable history partition auto clustering + HistoryPartitionCluster historyPartitionCluster = + new IncrementalClusterManager( + table, + PartitionPredicate.fromMultiple( + RowType.of(DataTypes.INT()), + Lists.newArrayList(BinaryRow.singleColumn("pt1")))) + .historyPartitionCluster(); + Map> partitionLevels = + historyPartitionCluster.constructLevelsForHistoryPartitions(); + assertThat(partitionLevels.size()).isEqualTo(1); + assertThat(partitionLevels.get(BinaryRow.singleColumn("pt2"))).isNotEmpty(); + + // test specify non-history partition and enable history partition auto clustering + historyPartitionCluster = + new IncrementalClusterManager( + table, + PartitionPredicate.fromMultiple( + RowType.of(DataTypes.INT()), + Lists.newArrayList(BinaryRow.singleColumn("pt3")))) + .historyPartitionCluster(); + partitionLevels = historyPartitionCluster.constructLevelsForHistoryPartitions(); + assertThat(partitionLevels.size()).isEqualTo(1); + assertThat(partitionLevels.get(BinaryRow.singleColumn("pt1"))).isNotEmpty(); + + // test not specify partition and disable history partition auto clustering + historyPartitionCluster = new IncrementalClusterManager(table).historyPartitionCluster(); + assertThat(historyPartitionCluster).isNull(); + } + + protected FileStoreTable createTable( + Map customOptions, List partitionKeys) throws Exception { + Map options = new HashMap<>(); + options.put(CoreOptions.BUCKET.key(), "-1"); + options.put(CoreOptions.CLUSTERING_COLUMNS.key(), "f0,f1"); + options.put(CoreOptions.CLUSTERING_INCREMENTAL.key(), "true"); + options.put(CoreOptions.CLUSTERING_HISTORY_PARTITION_IDLE_TIME.key(), "2s"); + options.put(CoreOptions.CLUSTERING_HISTORY_PARTITION_LIMIT.key(), "1"); + options.putAll(customOptions); + + Schema schema = + new Schema( + RowType.of( + DataTypes.INT(), + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.STRING()) + .getFields(), + partitionKeys, + Collections.emptyList(), + options, + ""); + + SchemaManager schemaManager = + new SchemaManager(LocalFileIO.create(), new Path(tempDir.toString())); + return FileStoreTableFactory.create( + LocalFileIO.create(), + new Path(tempDir.toString()), + schemaManager.createTable(schema)); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/append/cluster/IncrementalClusterManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/append/cluster/IncrementalClusterManagerTest.java new file mode 100644 index 000000000000..a4d1071092e8 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/append/cluster/IncrementalClusterManagerTest.java @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.append.cluster; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.compact.CompactUnit; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.manifest.FileSource; +import org.apache.paimon.mergetree.LevelSortedRun; +import org.apache.paimon.partition.PartitionPredicate; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.sink.BatchTableCommit; +import org.apache.paimon.table.sink.BatchTableWrite; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.assertj.core.util.Lists; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link IncrementalClusterManager}. */ +public class IncrementalClusterManagerTest { + + @TempDir java.nio.file.Path tempDir; + + @Test + public void testNonUnAwareBucketTable() { + Map options = new HashMap<>(); + options.put(CoreOptions.BUCKET.key(), "1"); + options.put(CoreOptions.BUCKET_KEY.key(), "f0"); + + assertThatThrownBy(() -> createTable(options, Collections.emptyList())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot define bucket for incremental clustering table, it only support bucket = -1"); + } + + @Test + public void testNonClusterIncremental() throws Exception { + Map options = new HashMap<>(); + options.put(CoreOptions.BUCKET.key(), "-1"); + options.put(CoreOptions.CLUSTERING_INCREMENTAL.key(), "false"); + FileStoreTable table = createTable(options, Collections.emptyList()); + assertThatThrownBy(() -> new IncrementalClusterManager(table)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Only support incremental clustering when 'clustering.incremental' is true."); + } + + @Test + public void testConstructPartitionLevels() throws Exception { + // Create a valid table for IncrementalClusterManager + Map options = new HashMap<>(); + FileStoreTable table = createTable(options, Collections.emptyList()); + IncrementalClusterManager incrementalClusterManager = new IncrementalClusterManager(table); + + // Create test files with different levels + List partitionFiles = new ArrayList<>(); + + // Level 0 files (should be individual LevelSortedRuns) + DataFileMeta level0File1 = createFile(100, 1, 0); + DataFileMeta level0File2 = createFile(200, 1, 0); + partitionFiles.add(level0File1); + partitionFiles.add(level0File2); + + // Level 1 files (should be grouped into one LevelSortedRun) + DataFileMeta level1File1 = createFile(300, 1, 1); + DataFileMeta level1File2 = createFile(400, 1, 1); + partitionFiles.add(level1File1); + partitionFiles.add(level1File2); + + // Level 2 files (should be grouped into one LevelSortedRun) + DataFileMeta level2File1 = createFile(500, 1, 2); + partitionFiles.add(level2File1); + + // Call the method under test + List result = + incrementalClusterManager.constructPartitionLevels(partitionFiles); + + // Verify the results + assertThat(result).hasSize(4); // 2 level-0 runs + 1 level-1 run + 1 level-2 run + + // Verify sorting by level + assertThat(result.get(0).level()).isEqualTo(0); + assertThat(result.get(1).level()).isEqualTo(0); + assertThat(result.get(2).level()).isEqualTo(1); + assertThat(result.get(3).level()).isEqualTo(2); + + // Verify level 0 files are individual runs + assertThat(result.get(0).run().files()).hasSize(1); + assertThat(result.get(1).run().files()).hasSize(1); + + // Verify level 1 files are grouped together + assertThat(result.get(2).run().files()).hasSize(2); + assertThat(result.get(2).run().files()).containsExactlyInAnyOrder(level1File1, level1File2); + + // Verify level 2 file + assertThat(result.get(3).run().files()).hasSize(1); + assertThat(result.get(3).run().files()).containsExactly(level2File1); + } + + @Test + public void testUpgrade() { + // Create test files with different levels + List filesAfterCluster = new ArrayList<>(); + DataFileMeta file1 = createFile(100, 1, 0); + DataFileMeta file2 = createFile(200, 1, 1); + DataFileMeta file3 = createFile(300, 1, 2); + filesAfterCluster.add(file1); + filesAfterCluster.add(file2); + filesAfterCluster.add(file3); + + // Test upgrading to level 3 + int outputLevel = 3; + List upgradedFiles = + IncrementalClusterManager.upgrade(filesAfterCluster, outputLevel); + + // Verify the results + assertThat(upgradedFiles).hasSize(3); + + // Verify all files are upgraded to the specified output level + for (DataFileMeta upgradedFile : upgradedFiles) { + assertThat(upgradedFile.level()).isEqualTo(outputLevel); + } + } + + @Test + public void testHistoryPartitionAutoClustering() throws Exception { + Map options = new HashMap<>(); + options.put(CoreOptions.CLUSTERING_HISTORY_PARTITION_IDLE_TIME.key(), "2s"); + options.put(CoreOptions.CLUSTERING_HISTORY_PARTITION_LIMIT.key(), "1"); + + FileStoreTable table = createTable(options, Collections.singletonList("f2")); + writeOnce( + table, + GenericRow.of( + 1, 1, BinaryString.fromString("pt1"), BinaryString.fromString("test"))); + writeOnce( + table, + GenericRow.of( + 1, 1, BinaryString.fromString("pt2"), BinaryString.fromString("test"))); + + Thread.sleep(2000); + writeOnce( + table, + GenericRow.of( + 1, 1, BinaryString.fromString("pt3"), BinaryString.fromString("test"))); + writeOnce( + table, + GenericRow.of( + 1, 1, BinaryString.fromString("pt4"), BinaryString.fromString("test"))); + + // test specify partition and enable history partition auto clustering + IncrementalClusterManager incrementalClusterManager = + new IncrementalClusterManager( + table, + PartitionPredicate.fromMultiple( + RowType.of(DataTypes.INT()), + Lists.newArrayList(BinaryRow.singleColumn("pt3")))); + Map partitionLevels = + incrementalClusterManager.prepareForCluster(true); + assertThat(partitionLevels.size()).isEqualTo(2); + assertThat(partitionLevels.get(BinaryRow.singleColumn("pt1"))).isNotNull(); + assertThat(partitionLevels.get(BinaryRow.singleColumn("pt3"))).isNotNull(); + + // test don't specify partition and enable history partition auto clustering + incrementalClusterManager = new IncrementalClusterManager(table); + partitionLevels = incrementalClusterManager.prepareForCluster(true); + assertThat(partitionLevels.size()).isEqualTo(4); + + // test specify partition and disable history partition auto clustering + SchemaChange schemaChange = + SchemaChange.removeOption(CoreOptions.CLUSTERING_HISTORY_PARTITION_IDLE_TIME.key()); + incrementalClusterManager = + new IncrementalClusterManager( + table.copy( + table.schemaManager() + .commitChanges(Collections.singletonList(schemaChange))), + PartitionPredicate.fromMultiple( + RowType.of(DataTypes.INT()), + Lists.newArrayList(BinaryRow.singleColumn("pt3")))); + partitionLevels = incrementalClusterManager.prepareForCluster(true); + assertThat(partitionLevels.size()).isEqualTo(1); + assertThat(partitionLevels.get(BinaryRow.singleColumn("pt3"))).isNotNull(); + } + + protected FileStoreTable createTable( + Map customOptions, List partitionKeys) throws Exception { + Map options = new HashMap<>(); + options.put(CoreOptions.BUCKET.key(), "-1"); + options.put(CoreOptions.CLUSTERING_COLUMNS.key(), "f0,f1"); + options.put(CoreOptions.CLUSTERING_INCREMENTAL.key(), "true"); + options.putAll(customOptions); + + Schema schema = + new Schema( + RowType.of( + DataTypes.INT(), + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.STRING()) + .getFields(), + partitionKeys, + Collections.emptyList(), + options, + ""); + + SchemaManager schemaManager = + new SchemaManager(LocalFileIO.create(), new Path(tempDir.toString())); + return FileStoreTableFactory.create( + LocalFileIO.create(), + new Path(tempDir.toString()), + schemaManager.createTable(schema)); + } + + protected static void writeOnce(FileStoreTable table, GenericRow... rows) { + String commitUser = "test_user"; + try (BatchTableWrite write = table.newWrite(commitUser); + BatchTableCommit commit = table.newCommit(commitUser)) { + for (GenericRow row : rows) { + write.write(row); + } + commit.commit(write.prepareCommit()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected static DataFileMeta createFile(long size, long schemaId, int level) { + return DataFileMeta.create( + "", + size, + 1, + null, + null, + null, + null, + 0, + 0, + schemaId, + level, + null, + null, + FileSource.APPEND, + null, + null, + null); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/append/cluster/IncrementalClusterStrategyTest.java b/paimon-core/src/test/java/org/apache/paimon/append/cluster/IncrementalClusterStrategyTest.java new file mode 100644 index 000000000000..a00f2442ae48 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/append/cluster/IncrementalClusterStrategyTest.java @@ -0,0 +1,222 @@ +/* + * 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. + */ + +package org.apache.paimon.append.cluster; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.compact.CompactUnit; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.manifest.FileSource; +import org.apache.paimon.mergetree.LevelSortedRun; +import org.apache.paimon.mergetree.SortedRun; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VarCharType; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static java.util.Collections.emptyList; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link IncrementalClusterStrategy}. */ +public class IncrementalClusterStrategyTest { + + @TempDir static java.nio.file.Path tempDir; + + private static SchemaManager schemaManager; + private static IncrementalClusterStrategy incrementalClusterStrategy; + + @BeforeAll + public static void setUp() throws Exception { + schemaManager = new SchemaManager(LocalFileIO.create(), new Path(tempDir.toString())); + prepareSchema(); + incrementalClusterStrategy = + new IncrementalClusterStrategy(schemaManager, Arrays.asList("f0", "f1"), 25, 1, 3); + } + + @Test + public void testPickFullCompactionWithEmptyRuns() { + // Test case: empty runs should return empty + Optional result = + incrementalClusterStrategy.pickFullCompaction(3, Collections.emptyList()); + assertThat(result.isPresent()).isFalse(); + } + + @Test + public void testPickFullCompactionWithSingleRunSameClusterKey() { + // Test case: single run at max level with same cluster key should return empty + // Using schema-0 which has clustering columns "f0,f1" (same as clusterKeys) + int maxLevel = 2; + DataFileMeta file = createFile(1, 0L, maxLevel); + LevelSortedRun run = new LevelSortedRun(maxLevel, SortedRun.fromSingle(file)); + List runs = Collections.singletonList(run); + + Optional result = incrementalClusterStrategy.pickFullCompaction(3, runs); + assertThat(result.isPresent()).isFalse(); + } + + @Test + public void testPickFullCompactionWithSingleRunDifferentClusterKey() { + // Test case: single run at max level with different cluster key should return compaction + // Using schema-1 which has clustering columns "f2,f3" (different from clusterKeys "f0,f1") + int maxLevel = 2; + DataFileMeta file = createFile(1, 1L, maxLevel); // Use schema-1 + LevelSortedRun run = new LevelSortedRun(maxLevel, SortedRun.fromSingle(file)); + List runs = Collections.singletonList(run); + + Optional result = incrementalClusterStrategy.pickFullCompaction(3, runs); + assertThat(result.isPresent()).isTrue(); + assertThat(result.get().outputLevel()).isEqualTo(maxLevel); + assertThat(result.get().files()).hasSize(1); + assertThat(result.get().files().get(0).fileSize()).isEqualTo(1); + } + + @Test + public void testPickFullCompactionWithSingleRunNotAtMaxLevel() { + // Test case: single run not at max level should return compaction + int maxLevel = 2; + int runLevel = 1; + DataFileMeta file = createFile(1, 0L, runLevel); + LevelSortedRun run = new LevelSortedRun(runLevel, SortedRun.fromSingle(file)); + List runs = Collections.singletonList(run); + + Optional result = incrementalClusterStrategy.pickFullCompaction(3, runs); + assertThat(result.isPresent()).isTrue(); + assertThat(result.get().outputLevel()).isEqualTo(maxLevel); + assertThat(result.get().files()).hasSize(1); + assertThat(result.get().files().get(0).fileSize()).isEqualTo(1); + } + + @Test + public void testPickFullCompactionWithMultipleRuns() { + // Test case: multiple runs should return compaction + int maxLevel = 2; + DataFileMeta file1 = createFile(1, 0L, 0); + DataFileMeta file2 = createFile(2, 1L, 1); + DataFileMeta file3 = createFile(3, 0L, maxLevel); + + LevelSortedRun run1 = new LevelSortedRun(0, SortedRun.fromSingle(file1)); + LevelSortedRun run2 = new LevelSortedRun(1, SortedRun.fromSingle(file2)); + LevelSortedRun run3 = new LevelSortedRun(maxLevel, SortedRun.fromSingle(file3)); + + List runs = Arrays.asList(run1, run2, run3); + + Optional result = incrementalClusterStrategy.pickFullCompaction(3, runs); + assertThat(result.isPresent()).isTrue(); + assertThat(result.get().outputLevel()).isEqualTo(maxLevel); + assertThat(result.get().files()).hasSize(3); + + long[] fileSizes = + result.get().files().stream().mapToLong(DataFileMeta::fileSize).toArray(); + assertThat(fileSizes).isEqualTo(new long[] {1, 2, 3}); + } + + @Test + public void testPickFullCompactionWithDifferentNumLevels() { + // Test case: different number of levels + DataFileMeta file1 = createFile(1, 0L, 0); + DataFileMeta file2 = createFile(2, 1L, 1); + + LevelSortedRun run1 = new LevelSortedRun(0, SortedRun.fromSingle(file1)); + LevelSortedRun run2 = new LevelSortedRun(1, SortedRun.fromSingle(file2)); + + List runs = Arrays.asList(run1, run2); + + // Test with numLevels = 5, maxLevel should be 4 + Optional result = incrementalClusterStrategy.pickFullCompaction(5, runs); + assertThat(result.isPresent()).isTrue(); + assertThat(result.get().outputLevel()).isEqualTo(4); // maxLevel = numLevels - 1 + assertThat(result.get().files()).hasSize(2); + } + + @Test + public void testPickFullCompactionWithMixedSchemas() { + // Test case: runs with mixed schemas (some same, some different cluster keys) + int maxLevel = 2; + DataFileMeta file1 = createFile(1, 0L, 0); // schema-0: f0,f1 (same as clusterKeys) + DataFileMeta file2 = createFile(2, 1L, 1); // schema-1: f2,f3 (different from clusterKeys) + DataFileMeta file3 = createFile(3, 0L, maxLevel); // schema-0: f0,f1 (same as clusterKeys) + + LevelSortedRun run1 = new LevelSortedRun(0, SortedRun.fromSingle(file1)); + LevelSortedRun run2 = new LevelSortedRun(1, SortedRun.fromSingle(file2)); + LevelSortedRun run3 = new LevelSortedRun(maxLevel, SortedRun.fromSingle(file3)); + + List runs = Arrays.asList(run1, run2, run3); + + Optional result = incrementalClusterStrategy.pickFullCompaction(3, runs); + assertThat(result.isPresent()).isTrue(); + assertThat(result.get().outputLevel()).isEqualTo(maxLevel); + assertThat(result.get().files()).hasSize(3); + } + + private static void prepareSchema() throws Exception { + // schema-0 + Map options = new HashMap<>(); + options.put(CoreOptions.CLUSTERING_COLUMNS.key(), "f0,f1"); + Schema schema = + new Schema( + RowType.of( + VarCharType.STRING_TYPE, + VarCharType.STRING_TYPE, + VarCharType.STRING_TYPE, + VarCharType.STRING_TYPE) + .getFields(), + emptyList(), + emptyList(), + options, + ""); + schemaManager.createTable(schema); + // schema-1 + schemaManager.commitChanges( + SchemaChange.setOption(CoreOptions.CLUSTERING_COLUMNS.key(), "f2,f3")); + } + + protected static DataFileMeta createFile(long size, long schemaId, int level) { + return DataFileMeta.create( + "", + size, + 1, + null, + null, + null, + null, + 0, + 0, + schemaId, + level, + null, + null, + FileSource.APPEND, + null, + null, + null); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java index 7222f0df6c6b..7c69928e3649 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java @@ -305,7 +305,8 @@ public void testListTablesPaged() throws Exception { // List tables paged returns an empty list when there are no tables in the database String databaseName = "tables_paged_db"; catalog.createDatabase(databaseName, false); - PagedList pagedTables = catalog.listTablesPaged(databaseName, null, null, null); + PagedList pagedTables = + catalog.listTablesPaged(databaseName, null, null, null, null); assertThat(pagedTables.getElements()).isEmpty(); assertNull(pagedTables.getNextPageToken()); @@ -318,22 +319,22 @@ public void testListTablesPaged() throws Exception { // List tables paged returns a list with the names of all tables in the database in all // catalogs except RestCatalog // even if the maxResults or pageToken is not null - pagedTables = catalog.listTablesPaged(databaseName, null, null, null); + pagedTables = catalog.listTablesPaged(databaseName, null, null, null, null); assertPagedTables(pagedTables, tableNames); int maxResults = 2; - pagedTables = catalog.listTablesPaged(databaseName, maxResults, null, null); + pagedTables = catalog.listTablesPaged(databaseName, maxResults, null, null, null); assertPagedTables(pagedTables, tableNames); String pageToken = "table1"; - pagedTables = catalog.listTablesPaged(databaseName, maxResults, pageToken, null); + pagedTables = catalog.listTablesPaged(databaseName, maxResults, pageToken, null, null); assertPagedTables(pagedTables, tableNames); maxResults = 8; - pagedTables = catalog.listTablesPaged(databaseName, maxResults, null, null); + pagedTables = catalog.listTablesPaged(databaseName, maxResults, null, null, null); assertPagedTables(pagedTables, tableNames); - pagedTables = catalog.listTablesPaged(databaseName, maxResults, pageToken, null); + pagedTables = catalog.listTablesPaged(databaseName, maxResults, pageToken, null, null); assertPagedTables(pagedTables, tableNames); // List tables throws DatabaseNotExistException when the database does not exist @@ -342,7 +343,7 @@ public void testListTablesPaged() throws Exception { .isThrownBy( () -> catalog.listTablesPaged( - "non_existing_db", finalMaxResults, pageToken, null)); + "non_existing_db", finalMaxResults, pageToken, null, null)); } @Test @@ -351,7 +352,7 @@ public void testListTableDetailsPaged() throws Exception { String databaseName = "table_details_paged_db"; catalog.createDatabase(databaseName, false); PagedList
pagedTableDetails = - catalog.listTableDetailsPaged(databaseName, null, null, null); + catalog.listTableDetailsPaged(databaseName, null, null, null, null); assertThat(pagedTableDetails.getElements()).isEmpty(); assertNull(pagedTableDetails.getNextPageToken()); @@ -364,28 +365,30 @@ public void testListTableDetailsPaged() throws Exception { Identifier.create(databaseName, tableName), DEFAULT_TABLE_SCHEMA, false); } - pagedTableDetails = catalog.listTableDetailsPaged(databaseName, null, null, null); + pagedTableDetails = catalog.listTableDetailsPaged(databaseName, null, null, null, null); assertPagedTableDetails(pagedTableDetails, tableNames.length, tableNames); assertNull(pagedTableDetails.getNextPageToken()); int maxResults = 2; - pagedTableDetails = catalog.listTableDetailsPaged(databaseName, maxResults, null, null); + pagedTableDetails = + catalog.listTableDetailsPaged(databaseName, maxResults, null, null, null); assertPagedTableDetails(pagedTableDetails, tableNames.length, tableNames); assertNull(pagedTableDetails.getNextPageToken()); String pageToken = "table1"; pagedTableDetails = - catalog.listTableDetailsPaged(databaseName, maxResults, pageToken, null); + catalog.listTableDetailsPaged(databaseName, maxResults, pageToken, null, null); assertPagedTableDetails(pagedTableDetails, tableNames.length, tableNames); assertNull(pagedTableDetails.getNextPageToken()); maxResults = 8; - pagedTableDetails = catalog.listTableDetailsPaged(databaseName, maxResults, null, null); + pagedTableDetails = + catalog.listTableDetailsPaged(databaseName, maxResults, null, null, null); assertPagedTableDetails(pagedTableDetails, tableNames.length, tableNames); assertNull(pagedTableDetails.getNextPageToken()); pagedTableDetails = - catalog.listTableDetailsPaged(databaseName, maxResults, pageToken, null); + catalog.listTableDetailsPaged(databaseName, maxResults, pageToken, null, null); assertPagedTableDetails(pagedTableDetails, tableNames.length, tableNames); assertNull(pagedTableDetails.getNextPageToken()); @@ -395,7 +398,7 @@ public void testListTableDetailsPaged() throws Exception { .isThrownBy( () -> catalog.listTableDetailsPaged( - "non_existing_db", finalMaxResults, pageToken, null)); + "non_existing_db", finalMaxResults, pageToken, null, null)); } @Test diff --git a/paimon-core/src/test/java/org/apache/paimon/deletionvectors/BucketedDvMaintainerTest.java b/paimon-core/src/test/java/org/apache/paimon/deletionvectors/BucketedDvMaintainerTest.java index c195517731f2..018d55e93304 100644 --- a/paimon-core/src/test/java/org/apache/paimon/deletionvectors/BucketedDvMaintainerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/deletionvectors/BucketedDvMaintainerTest.java @@ -23,17 +23,20 @@ import org.apache.paimon.catalog.PrimaryKeyTableTestBase; import org.apache.paimon.compact.CompactDeletionFile; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.fs.Path; import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.io.CompactIncrement; +import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataIncrement; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.manifest.IndexManifestEntry; +import org.apache.paimon.stats.SimpleStats; import org.apache.paimon.table.sink.BatchTableCommit; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.utils.FileIOUtils; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -41,6 +44,8 @@ import java.io.File; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -48,13 +53,22 @@ import java.util.stream.Collectors; import static java.util.Collections.emptyList; -import static org.apache.paimon.data.BinaryRow.EMPTY_ROW; import static org.apache.paimon.deletionvectors.DeletionVectorsIndexFile.DELETION_VECTORS_INDEX; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link BucketedDvMaintainer}. */ public class BucketedDvMaintainerTest extends PrimaryKeyTableTestBase { private IndexFileHandler fileHandler; + private final BinaryRow partition = BinaryRow.singleColumn(1); + + @BeforeEach + public void setUp() throws Exception { + // write files + CommitMessageImpl commitMessage = + writeDataFiles(partition, 0, Arrays.asList("f1", "f2", "f3")); + BatchTableCommit commit = table.newBatchWriteBuilder().newCommit(); + commit.commit(Collections.singletonList(commitMessage)); + } @ParameterizedTest @ValueSource(booleans = {true, false}) @@ -62,7 +76,7 @@ public void test0(boolean bitmap64) { initIndexHandler(bitmap64); BucketedDvMaintainer.Factory factory = BucketedDvMaintainer.factory(fileHandler); - BucketedDvMaintainer dvMaintainer = factory.create(EMPTY_ROW, 0, emptyList()); + BucketedDvMaintainer dvMaintainer = factory.create(partition, 0, emptyList()); assertThat(dvMaintainer.bitmap64).isEqualTo(bitmap64); dvMaintainer.notifyNewDeletion("f1", 1); @@ -75,7 +89,7 @@ public void test0(boolean bitmap64) { IndexFileMeta file = dvMaintainer.writeDeletionVectorsIndex().get(); Map deletionVectors = - fileHandler.readAllDeletionVectors(EMPTY_ROW, 0, Collections.singletonList(file)); + fileHandler.readAllDeletionVectors(partition, 0, Collections.singletonList(file)); assertThat(deletionVectors.get("f1").isDeleted(1)).isTrue(); assertThat(deletionVectors.get("f1").isDeleted(2)).isFalse(); assertThat(deletionVectors.get("f2").isDeleted(1)).isFalse(); @@ -90,7 +104,7 @@ public void test1(boolean bitmap64) { BucketedDvMaintainer.Factory factory = BucketedDvMaintainer.factory(fileHandler); - BucketedDvMaintainer dvMaintainer = factory.create(EMPTY_ROW, 0, new HashMap<>()); + BucketedDvMaintainer dvMaintainer = factory.create(partition, 0, new HashMap<>()); DeletionVector deletionVector1 = createDeletionVector(bitmap64); deletionVector1.delete(1); deletionVector1.delete(3); @@ -101,19 +115,23 @@ public void test1(boolean bitmap64) { IndexFileMeta file = dvMaintainer.writeDeletionVectorsIndex().get(); CommitMessage commitMessage = new CommitMessageImpl( - EMPTY_ROW, + partition, 0, 1, - DataIncrement.emptyIncrement(), - CompactIncrement.emptyIncrement(), - new IndexIncrement(Collections.singletonList(file))); + new DataIncrement( + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.singletonList(file), + Collections.emptyList()), + CompactIncrement.emptyIncrement()); BatchTableCommit commit = table.newBatchWriteBuilder().newCommit(); commit.commit(Collections.singletonList(commitMessage)); Snapshot latestSnapshot = table.snapshotManager().latestSnapshot(); List indexFiles = - fileHandler.scan(latestSnapshot, DELETION_VECTORS_INDEX, EMPTY_ROW, 0); - dvMaintainer = factory.create(EMPTY_ROW, 0, indexFiles); + fileHandler.scan(latestSnapshot, DELETION_VECTORS_INDEX, partition, 0); + dvMaintainer = factory.create(partition, 0, indexFiles); DeletionVector deletionVector2 = dvMaintainer.deletionVectorOf("f1").get(); assertThat(deletionVector2.isDeleted(1)).isTrue(); assertThat(deletionVector2.isDeleted(2)).isFalse(); @@ -124,18 +142,22 @@ public void test1(boolean bitmap64) { file = dvMaintainer.writeDeletionVectorsIndex().get(); commitMessage = new CommitMessageImpl( - EMPTY_ROW, + partition, 0, 1, - DataIncrement.emptyIncrement(), - CompactIncrement.emptyIncrement(), - new IndexIncrement(Collections.singletonList(file))); + new DataIncrement( + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.singletonList(file), + Collections.emptyList()), + CompactIncrement.emptyIncrement()); commit = table.newBatchWriteBuilder().newCommit(); commit.commit(Collections.singletonList(commitMessage)); latestSnapshot = table.snapshotManager().latestSnapshot(); - indexFiles = fileHandler.scan(latestSnapshot, DELETION_VECTORS_INDEX, EMPTY_ROW, 0); - dvMaintainer = factory.create(EMPTY_ROW, 0, indexFiles); + indexFiles = fileHandler.scan(latestSnapshot, DELETION_VECTORS_INDEX, partition, 0); + dvMaintainer = factory.create(partition, 0, indexFiles); DeletionVector deletionVector3 = dvMaintainer.deletionVectorOf("f1").get(); assertThat(deletionVector3.isDeleted(1)).isTrue(); assertThat(deletionVector3.isDeleted(2)).isTrue(); @@ -147,7 +169,7 @@ public void testCompactDeletion(boolean bitmap64) throws IOException { initIndexHandler(bitmap64); BucketedDvMaintainer.Factory factory = BucketedDvMaintainer.factory(fileHandler); - BucketedDvMaintainer dvMaintainer = factory.create(EMPTY_ROW, 0, emptyList()); + BucketedDvMaintainer dvMaintainer = factory.create(partition, 0, emptyList()); File indexDir = new File(tempPath.toFile(), "/default.db/T/index"); @@ -188,7 +210,7 @@ public void testReadAndWriteMixedDv(boolean bitmap64) { // write first kind dv initIndexHandler(bitmap64); BucketedDvMaintainer.Factory factory1 = BucketedDvMaintainer.factory(fileHandler); - BucketedDvMaintainer dvMaintainer1 = factory1.create(EMPTY_ROW, 0, new HashMap<>()); + BucketedDvMaintainer dvMaintainer1 = factory1.create(partition, 0, new HashMap<>()); dvMaintainer1.notifyNewDeletion("f1", 1); dvMaintainer1.notifyNewDeletion("f1", 3); dvMaintainer1.notifyNewDeletion("f2", 1); @@ -198,12 +220,16 @@ public void testReadAndWriteMixedDv(boolean bitmap64) { IndexFileMeta file = dvMaintainer1.writeDeletionVectorsIndex().get(); CommitMessage commitMessage1 = new CommitMessageImpl( - EMPTY_ROW, + partition, 0, 1, DataIncrement.emptyIncrement(), - CompactIncrement.emptyIncrement(), - new IndexIncrement(Collections.singletonList(file))); + new CompactIncrement( + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.singletonList(file), + Collections.emptyList())); BatchTableCommit commit1 = table.newBatchWriteBuilder().newCommit(); commit1.commit(Collections.singletonList(commitMessage1)); @@ -212,8 +238,8 @@ public void testReadAndWriteMixedDv(boolean bitmap64) { BucketedDvMaintainer.Factory factory2 = BucketedDvMaintainer.factory(fileHandler); List indexFiles = fileHandler.scan( - table.latestSnapshot().get(), DELETION_VECTORS_INDEX, EMPTY_ROW, 0); - BucketedDvMaintainer dvMaintainer2 = factory2.create(EMPTY_ROW, 0, indexFiles); + table.latestSnapshot().get(), DELETION_VECTORS_INDEX, partition, 0); + BucketedDvMaintainer dvMaintainer2 = factory2.create(partition, 0, indexFiles); dvMaintainer2.notifyNewDeletion("f1", 10); dvMaintainer2.notifyNewDeletion("f3", 1); dvMaintainer2.notifyNewDeletion("f3", 3); @@ -231,22 +257,26 @@ public void testReadAndWriteMixedDv(boolean bitmap64) { file = dvMaintainer2.writeDeletionVectorsIndex().get(); CommitMessage commitMessage2 = new CommitMessageImpl( - EMPTY_ROW, + partition, 0, 1, DataIncrement.emptyIncrement(), - CompactIncrement.emptyIncrement(), - new IndexIncrement(Collections.singletonList(file))); + new CompactIncrement( + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.singletonList(file), + Collections.emptyList())); BatchTableCommit commit2 = table.newBatchWriteBuilder().newCommit(); commit2.commit(Collections.singletonList(commitMessage2)); // test read dv index file which contains two kinds of dv Map readDvs = fileHandler.readAllDeletionVectors( - EMPTY_ROW, + partition, 0, fileHandler.scan( - table.latestSnapshot().get(), "DELETION_VECTORS", EMPTY_ROW, 0)); + table.latestSnapshot().get(), "DELETION_VECTORS", partition, 0)); assertThat(readDvs.size()).isEqualTo(3); assertThat(dvs.get("f1").getCardinality()).isEqualTo(3); assertThat(dvs.get("f2").getCardinality()).isEqualTo(2); @@ -278,7 +308,39 @@ public static BucketedDvMaintainer createOrRestore( .map(IndexManifestEntry::indexFile) .collect(Collectors.toList()); Map deletionVectors = - new HashMap<>(handler.readAllDeletionVectors(EMPTY_ROW, 0, indexFiles)); - return factory.create(EMPTY_ROW, 0, deletionVectors); + new HashMap<>(handler.readAllDeletionVectors(partition, 0, indexFiles)); + return factory.create(partition, 0, deletionVectors); + } + + private CommitMessageImpl writeDataFiles( + BinaryRow partition, int bucket, List dataFileNames) throws IOException { + List fileMetas = new ArrayList<>(); + Path bucketPath = table.store().pathFactory().bucketPath(partition, bucket); + for (String dataFileName : dataFileNames) { + Path path = new Path(bucketPath, dataFileName); + table.fileIO().newOutputStream(path, false).close(); + fileMetas.add( + DataFileMeta.forAppend( + path.getName(), + 10L, + 10L, + SimpleStats.EMPTY_STATS, + 0L, + 0L, + table.schema().id(), + Collections.emptyList(), + null, + null, + null, + null, + null, + null)); + } + return new CommitMessageImpl( + partition, + bucket, + null, + new DataIncrement(fileMetas, Collections.emptyList(), Collections.emptyList()), + CompactIncrement.emptyIncrement()); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java b/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java index 3cd9308d79c3..3a03985c885d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java @@ -71,10 +71,10 @@ public void test(boolean bitmap64) throws Exception { Map dataFileToDeletionFiles = new HashMap<>(); dataFileToDeletionFiles.putAll( createDeletionFileMapFromIndexFileMetas( - indexPathFactory, commitMessage1.indexIncrement().newIndexFiles())); + indexPathFactory, commitMessage1.newFilesIncrement().newIndexFiles())); dataFileToDeletionFiles.putAll( createDeletionFileMapFromIndexFileMetas( - indexPathFactory, commitMessage2.indexIncrement().newIndexFiles())); + indexPathFactory, commitMessage2.newFilesIncrement().newIndexFiles())); AppendDeleteFileMaintainer dvIFMaintainer = store.createDVIFMaintainer(BinaryRow.EMPTY_ROW, dataFileToDeletionFiles); @@ -108,7 +108,7 @@ public void test(boolean bitmap64) throws Exception { .findAny() .get(); assertThat(entry.indexFile()) - .isEqualTo(commitMessage1.indexIncrement().newIndexFiles().get(0)); + .isEqualTo(commitMessage1.newFilesIncrement().newIndexFiles().get(0)); entry = res.stream() .filter(file -> file.kind() == FileKind.DELETE) @@ -116,7 +116,7 @@ public void test(boolean bitmap64) throws Exception { .findAny() .get(); assertThat(entry.indexFile()) - .isEqualTo(commitMessage2.indexIncrement().newIndexFiles().get(0)); + .isEqualTo(commitMessage2.newFilesIncrement().newIndexFiles().get(0)); } private Map createDeletionFileMapFromIndexFileMetas( diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/metadata/IcebergDataFieldTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/metadata/IcebergDataFieldTest.java index ce18ed2da147..f96a0634598e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/metadata/IcebergDataFieldTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/metadata/IcebergDataFieldTest.java @@ -171,6 +171,39 @@ void testTimestampTypeConversions() { new DataField(2, "timestamptz", new LocalZonedTimestampType(false, 6)); IcebergDataField icebergTimestampLtz = new IcebergDataField(timestampLtzField); assertThat(icebergTimestampLtz.type()).isEqualTo("timestamptz"); + + // Test timestamp_ns (precision 7) + DataField timestampNs7Field = new DataField(3, "timestamp_ns", new TimestampType(false, 7)); + IcebergDataField icebergTimestampNs7 = new IcebergDataField(timestampNs7Field); + assertThat(icebergTimestampNs7.type()).isEqualTo("timestamp_ns"); + + // Test timestamp_ns (precision 8) + DataField timestampNs8Field = new DataField(4, "timestamp_ns", new TimestampType(false, 8)); + IcebergDataField icebergTimestampNs8 = new IcebergDataField(timestampNs8Field); + assertThat(icebergTimestampNs8.type()).isEqualTo("timestamp_ns"); + + // Test timestamp_ns (precision 9) + DataField timestampNs9Field = new DataField(5, "timestamp_ns", new TimestampType(false, 9)); + IcebergDataField icebergTimestampNs9 = new IcebergDataField(timestampNs9Field); + assertThat(icebergTimestampNs9.type()).isEqualTo("timestamp_ns"); + + // Test timestamptz_ns (precision 7) + DataField timestampLtzNs7Field = + new DataField(6, "timestamptz_ns", new LocalZonedTimestampType(false, 7)); + IcebergDataField icebergTimestampLtzNs7 = new IcebergDataField(timestampLtzNs7Field); + assertThat(icebergTimestampLtzNs7.type()).isEqualTo("timestamptz_ns"); + + // Test timestamptz_ns (precision 8) + DataField timestampLtzNs8Field = + new DataField(7, "timestamptz_ns", new LocalZonedTimestampType(false, 8)); + IcebergDataField icebergTimestampLtzNs8 = new IcebergDataField(timestampLtzNs8Field); + assertThat(icebergTimestampLtzNs8.type()).isEqualTo("timestamptz_ns"); + + // Test timestamptz_ns (precision 9) + DataField timestampLtzNs9Field = + new DataField(8, "timestamptz_ns", new LocalZonedTimestampType(false, 9)); + IcebergDataField icebergTimestampLtzNs9 = new IcebergDataField(timestampLtzNs9Field); + assertThat(icebergTimestampLtzNs9.type()).isEqualTo("timestamptz_ns"); } @Test @@ -182,15 +215,15 @@ void testTimestampPrecisionValidation() { assertThatThrownBy(() -> new IcebergDataField(invalidTimestampField)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( - "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6"); + "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 9"); - // Test invalid precision (> 6) + // Test invalid precision (<= 3) DataField invalidTimestampField2 = - new DataField(2, "timestamp", new TimestampType(false, 7)); + new DataField(2, "timestamp", new TimestampType(false, 3)); assertThatThrownBy(() -> new IcebergDataField(invalidTimestampField2)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( - "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6"); + "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 9"); // Test invalid local timezone timestamp precision (<= 3) DataField invalidTimestampLtzField = @@ -198,15 +231,26 @@ void testTimestampPrecisionValidation() { assertThatThrownBy(() -> new IcebergDataField(invalidTimestampLtzField)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( - "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6"); - - // Test invalid local timezone timestamp precision (> 6) - DataField invalidTimestampLtzField2 = - new DataField(4, "timestamptz", new LocalZonedTimestampType(false, 7)); - assertThatThrownBy(() -> new IcebergDataField(invalidTimestampLtzField2)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining( - "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6"); + "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 9"); + + // Test valid precision boundaries + DataField validTimestamp4 = new DataField(4, "timestamp", new TimestampType(false, 4)); + IcebergDataField icebergTimestamp4 = new IcebergDataField(validTimestamp4); + assertThat(icebergTimestamp4.type()).isEqualTo("timestamp"); + + DataField validTimestamp9 = new DataField(5, "timestamp", new TimestampType(false, 9)); + IcebergDataField icebergTimestamp9 = new IcebergDataField(validTimestamp9); + assertThat(icebergTimestamp9.type()).isEqualTo("timestamp_ns"); + + DataField validTimestampLtz4 = + new DataField(6, "timestamptz", new LocalZonedTimestampType(false, 4)); + IcebergDataField icebergTimestampLtz4 = new IcebergDataField(validTimestampLtz4); + assertThat(icebergTimestampLtz4.type()).isEqualTo("timestamptz"); + + DataField validTimestampLtz9 = + new DataField(7, "timestamptz", new LocalZonedTimestampType(false, 9)); + IcebergDataField icebergTimestampLtz9 = new IcebergDataField(validTimestampLtz9); + assertThat(icebergTimestampLtz9.type()).isEqualTo("timestamptz_ns"); } @Test diff --git a/paimon-core/src/test/java/org/apache/paimon/index/DynamicBucketIndexMaintainerTest.java b/paimon-core/src/test/java/org/apache/paimon/index/DynamicBucketIndexMaintainerTest.java index 6f5e0d2ecd3d..b4f5363a06fe 100644 --- a/paimon-core/src/test/java/org/apache/paimon/index/DynamicBucketIndexMaintainerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/index/DynamicBucketIndexMaintainerTest.java @@ -75,7 +75,7 @@ private Map> readIndex(List messag Map> index = new HashMap<>(); for (CommitMessage commitMessage : messages) { CommitMessageImpl message = (CommitMessageImpl) commitMessage; - List files = message.indexIncrement().newIndexFiles(); + List files = message.newFilesIncrement().newIndexFiles(); if (files.isEmpty()) { continue; } diff --git a/paimon-core/src/test/java/org/apache/paimon/index/HashBucketAssignerTest.java b/paimon-core/src/test/java/org/apache/paimon/index/HashBucketAssignerTest.java index 01eca82c2ed7..4c5415d45de6 100644 --- a/paimon-core/src/test/java/org/apache/paimon/index/HashBucketAssignerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/index/HashBucketAssignerTest.java @@ -22,7 +22,6 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataIncrement; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.table.sink.StreamTableCommit; @@ -216,10 +215,13 @@ private CommitMessage createCommitMessage( bucket, totalBuckets, new DataIncrement( - Collections.emptyList(), Collections.emptyList(), Collections.emptyList()), + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + Collections.singletonList(file), + Collections.emptyList()), new CompactIncrement( - Collections.emptyList(), Collections.emptyList(), Collections.emptyList()), - new IndexIncrement(Collections.singletonList(file))); + Collections.emptyList(), Collections.emptyList(), Collections.emptyList())); } @Test diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java index 927da8e97d20..ccc8f6c57e22 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java @@ -24,7 +24,6 @@ import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataIncrement; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.stats.SimpleStats; import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.utils.IOUtils; @@ -45,6 +44,100 @@ /** Compatibility Test for {@link ManifestCommittableSerializer}. */ public class ManifestCommittableSerializerCompatibilityTest { + @Test + public void testCompatibilityToV4CommitV10() throws IOException { + String fileName = "manifest-committable-v10"; + + SimpleStats keyStats = + new SimpleStats( + singleColumn("min_key"), + singleColumn("max_key"), + fromLongArray(new Long[] {0L})); + SimpleStats valueStats = + new SimpleStats( + singleColumn("min_value"), + singleColumn("max_value"), + fromLongArray(new Long[] {0L})); + DataFileMeta dataFile = + DataFileMeta.create( + "my_file", + 1024 * 1024, + 1024, + singleColumn("min_key"), + singleColumn("max_key"), + keyStats, + valueStats, + 15, + 200, + 5, + 3, + Arrays.asList("extra1", "extra2"), + Timestamp.fromLocalDateTime(LocalDateTime.parse("2022-03-02T20:20:12")), + 11L, + new byte[] {1, 2, 4}, + FileSource.COMPACT, + Arrays.asList("field1", "field2", "field3"), + "hdfs://localhost:9000/path/to/file", + 1L, + Arrays.asList("asdf", "qwer", "zxcv")); + List dataFiles = Collections.singletonList(dataFile); + IndexFileMeta hashIndexFile = + new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, null, null); + + LinkedHashMap dvRanges = new LinkedHashMap<>(); + dvRanges.put("dv_key1", new DeletionVectorMeta("dv_key1", 1, 2, 3L)); + dvRanges.put("dv_key2", new DeletionVectorMeta("dv_key2", 3, 4, 5L)); + IndexFileMeta devIndexFile = + new IndexFileMeta( + "my_index_type", + "my_index_file", + 1024 * 100, + 1002, + dvRanges, + "external_path"); + + CommitMessageImpl commitMessage = + new CommitMessageImpl( + singleColumn("my_partition"), + 11, + 16, + new DataIncrement( + dataFiles, + dataFiles, + dataFiles, + Collections.singletonList(hashIndexFile), + Collections.singletonList(hashIndexFile)), + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + Collections.singletonList(devIndexFile), + Collections.emptyList())); + + ManifestCommittable manifestCommittable = + new ManifestCommittable( + 5, + 202020L, + Collections.singletonMap(5, 555L), + Collections.singletonList(commitMessage)); + manifestCommittable.addProperty("k1", "v1"); + manifestCommittable.addProperty("k2", "v2"); + + ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); + byte[] bytes = serializer.serialize(manifestCommittable); + ManifestCommittable deserialized = serializer.deserialize(serializer.getVersion(), bytes); + assertThat(deserialized).isEqualTo(manifestCommittable); + + byte[] oldBytes = + IOUtils.readFully( + ManifestCommittableSerializerCompatibilityTest.class + .getClassLoader() + .getResourceAsStream("compatibility/" + fileName), + true); + deserialized = serializer.deserialize(4, oldBytes); + assertThat(deserialized).isEqualTo(manifestCommittable); + } + @Test public void testCompatibilityToV4CommitV9() throws IOException { SimpleStats keyStats = @@ -100,8 +193,12 @@ public void testCompatibilityToV4CommitV9() throws IOException { 11, 16, new DataIncrement(dataFiles, dataFiles, dataFiles), - new CompactIncrement(dataFiles, dataFiles, dataFiles), - new IndexIncrement(indexFiles)); + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + indexFiles, + Collections.emptyList())); ManifestCommittable manifestCommittable = new ManifestCommittable( @@ -178,8 +275,12 @@ public void testCompatibilityToV4CommitV8() throws IOException { 11, 16, new DataIncrement(dataFiles, dataFiles, dataFiles), - new CompactIncrement(dataFiles, dataFiles, dataFiles), - new IndexIncrement(indexFiles)); + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + indexFiles, + Collections.emptyList())); ManifestCommittable manifestCommittable = new ManifestCommittable( @@ -256,8 +357,12 @@ public void testCompatibilityToV4CommitV7() throws IOException { 11, 16, new DataIncrement(dataFiles, dataFiles, dataFiles), - new CompactIncrement(dataFiles, dataFiles, dataFiles), - new IndexIncrement(indexFiles)); + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + indexFiles, + Collections.emptyList())); ManifestCommittable manifestCommittable = new ManifestCommittable( @@ -333,8 +438,12 @@ public void testCompatibilityToV3CommitV7() throws IOException { 11, 16, new DataIncrement(dataFiles, dataFiles, dataFiles), - new CompactIncrement(dataFiles, dataFiles, dataFiles), - new IndexIncrement(indexFiles)); + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + indexFiles, + Collections.emptyList())); ManifestCommittable manifestCommittable = new ManifestCommittable( @@ -408,8 +517,12 @@ public void testCompatibilityToV3CommitV6() throws IOException { 11, null, new DataIncrement(dataFiles, dataFiles, dataFiles), - new CompactIncrement(dataFiles, dataFiles, dataFiles), - new IndexIncrement(indexFiles)); + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + indexFiles, + Collections.emptyList())); ManifestCommittable manifestCommittable = new ManifestCommittable( @@ -483,8 +596,12 @@ public void testCompatibilityToV3CommitV5() throws IOException { 11, null, new DataIncrement(dataFiles, dataFiles, dataFiles), - new CompactIncrement(dataFiles, dataFiles, dataFiles), - new IndexIncrement(indexFiles)); + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + indexFiles, + Collections.emptyList())); ManifestCommittable manifestCommittable = new ManifestCommittable( @@ -557,8 +674,12 @@ public void testCompatibilityToV3CommitV4() throws IOException { 11, null, new DataIncrement(dataFiles, dataFiles, dataFiles), - new CompactIncrement(dataFiles, dataFiles, dataFiles), - new IndexIncrement(indexFiles)); + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + indexFiles, + Collections.emptyList())); ManifestCommittable manifestCommittable = new ManifestCommittable( @@ -632,8 +753,12 @@ public void testCompatibilityToV3CommitV3() throws IOException { 11, null, new DataIncrement(dataFiles, dataFiles, dataFiles), - new CompactIncrement(dataFiles, dataFiles, dataFiles), - new IndexIncrement(indexFiles)); + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + indexFiles, + Collections.emptyList())); ManifestCommittable manifestCommittable = new ManifestCommittable( @@ -707,8 +832,12 @@ public void testCompatibilityToV2CommitV2() throws IOException { 11, null, new DataIncrement(dataFiles, dataFiles, dataFiles), - new CompactIncrement(dataFiles, dataFiles, dataFiles), - new IndexIncrement(indexFiles)); + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + indexFiles, + Collections.emptyList())); ManifestCommittable manifestCommittable = new ManifestCommittable( @@ -778,8 +907,12 @@ public void testCompatibilityToVersion2PaimonV07() throws IOException { 11, null, new DataIncrement(dataFiles, Collections.emptyList(), dataFiles), - new CompactIncrement(dataFiles, dataFiles, dataFiles), - new IndexIncrement(indexFiles)); + new CompactIncrement( + dataFiles, + dataFiles, + dataFiles, + indexFiles, + Collections.emptyList())); ManifestCommittable manifestCommittable = new ManifestCommittable( diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/KeyValueBufferTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/KeyValueBufferTest.java index e3f485f765ff..d04015b3704a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/KeyValueBufferTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/KeyValueBufferTest.java @@ -25,10 +25,11 @@ import org.apache.paimon.disk.IOManager; import org.apache.paimon.disk.IOManagerImpl; import org.apache.paimon.mergetree.compact.KeyValueBuffer.BinaryBuffer; +import org.apache.paimon.mergetree.compact.KeyValueBuffer.HybridBuffer; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; import org.apache.paimon.types.DataField; -import org.apache.paimon.types.IntType; +import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.CloseableIterator; @@ -40,7 +41,10 @@ import java.util.ArrayList; import java.util.List; +import java.util.NoSuchElementException; +import static java.util.Collections.singletonList; +import static org.apache.paimon.CoreOptions.LOOKUP_MERGE_RECORDS_THRESHOLD; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link KeyValueBuffer}. */ @@ -55,30 +59,14 @@ public class KeyValueBufferTest { @BeforeEach public void beforeEach() { this.ioManager = new IOManagerImpl(tempDir.toString()); - this.keyType = - new RowType( - new ArrayList() { - { - add(new DataField(0, "key", new IntType())); - } - }); - this.valueType = - new RowType( - new ArrayList() { - { - add(new DataField(0, "value", new IntType())); - } - }); + this.keyType = new RowType(singletonList(new DataField(0, "key", DataTypes.INT()))); + this.valueType = new RowType(singletonList(new DataField(0, "value", DataTypes.INT()))); } @AfterEach - public void afterEach() { + public void afterEach() throws Exception { if (ioManager != null) { - try { - ioManager.close(); - } catch (Exception e) { - // Ignore exception during close - } + ioManager.close(); } } @@ -114,10 +102,35 @@ public void testBinaryBufferPutAndIterator() throws Exception { BinaryBuffer binaryBuffer = KeyValueBuffer.createBinaryBuffer( new CoreOptions(options), keyType, valueType, ioManager); + innerTestBuffer(binaryBuffer, 10); + } + + @Test + public void testHybridBufferWithoutFallback() throws Exception { + innerTestHybridBuffer(false); + } + + @Test + public void testHybridBufferWithFallback() throws Exception { + innerTestHybridBuffer(true); + } + + private void innerTestHybridBuffer(boolean fallbackToBinary) throws Exception { + Options options = new Options(); + if (fallbackToBinary) { + options.set(LOOKUP_MERGE_RECORDS_THRESHOLD, 100); + } + HybridBuffer buffer = + KeyValueBuffer.createHybridBuffer( + new CoreOptions(options), keyType, valueType, ioManager); + innerTestBuffer(buffer, 200); + assertThat(buffer.binaryBuffer() != null).isEqualTo(fallbackToBinary); + } + private void innerTestBuffer(KeyValueBuffer buffer, int recordNumber) throws Exception { // Create test data List testData = new ArrayList<>(); - for (int i = 0; i < 10; i++) { + for (int i = 0; i < recordNumber; i++) { // Create key as BinaryRow BinaryRow key = new BinaryRow(1); BinaryRowWriter keyWriter = new BinaryRowWriter(key); @@ -135,11 +148,11 @@ public void testBinaryBufferPutAndIterator() throws Exception { // Put data into buffer for (KeyValue kv : testData) { - binaryBuffer.put(kv); + buffer.put(kv); } // Verify data through iterator - try (CloseableIterator iterator = binaryBuffer.iterator()) { + try (CloseableIterator iterator = buffer.iterator()) { int count = 0; while (iterator.hasNext()) { KeyValue kv = iterator.next(); @@ -152,5 +165,25 @@ public void testBinaryBufferPutAndIterator() throws Exception { } assertThat(count).isEqualTo(testData.size()); } + + // Verify data through iterator without hasNext + try (CloseableIterator iterator = buffer.iterator()) { + int count = 0; + while (true) { + KeyValue kv; + try { + kv = iterator.next(); + } catch (NoSuchElementException e) { + break; + } + KeyValue expected = testData.get(count); + assertThat(kv.key().getInt(0)).isEqualTo(expected.key().getInt(0)); + assertThat(kv.value().getInt(0)).isEqualTo(expected.value().getInt(0)); + assertThat(kv.sequenceNumber()).isEqualTo(expected.sequenceNumber()); + assertThat(kv.valueKind()).isEqualTo(expected.valueKind()); + count++; + } + assertThat(count).isEqualTo(testData.size()); + } } } diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/aggregate/FieldAggregatorTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/aggregate/FieldAggregatorTest.java index 166b4479c83e..4e09b993d5ee 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/aggregate/FieldAggregatorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/aggregate/FieldAggregatorTest.java @@ -345,7 +345,8 @@ public void testFieldNestedUpdateAgg() { DataTypes.FIELD(0, "k0", DataTypes.INT()), DataTypes.FIELD(1, "k1", DataTypes.INT()), DataTypes.FIELD(2, "v", DataTypes.STRING()))), - Arrays.asList("k0", "k1")); + Arrays.asList("k0", "k1"), + Integer.MAX_VALUE); InternalArray accumulator; InternalArray.ElementGetter elementGetter = @@ -383,7 +384,8 @@ public void testFieldNestedAppendAgg() { new FieldNestedUpdateAgg( FieldNestedUpdateAggFactory.NAME, DataTypes.ARRAY(elementRowType), - Collections.emptyList()); + Collections.emptyList(), + Integer.MAX_VALUE); InternalArray accumulator = null; InternalArray.ElementGetter elementGetter = @@ -405,6 +407,41 @@ public void testFieldNestedAppendAgg() { .containsExactlyInAnyOrderElementsOf(Collections.singletonList(row(0, 1, "B"))); } + @Test + public void testFieldNestedAppendAggWithCountLimit() { + DataType elementRowType = + DataTypes.ROW( + DataTypes.FIELD(0, "k0", DataTypes.INT()), + DataTypes.FIELD(1, "k1", DataTypes.INT()), + DataTypes.FIELD(2, "v", DataTypes.STRING())); + FieldNestedUpdateAgg agg = + new FieldNestedUpdateAgg( + FieldNestedUpdateAggFactory.NAME, + DataTypes.ARRAY(elementRowType), + Collections.emptyList(), + 2); + + InternalArray accumulator = null; + InternalArray.ElementGetter elementGetter = + InternalArray.createElementGetter(elementRowType); + + InternalRow current = row(0, 1, "B"); + accumulator = (InternalArray) agg.agg(accumulator, singletonArray(current)); + assertThat(unnest(accumulator, elementGetter)) + .containsExactlyInAnyOrderElementsOf(Collections.singletonList(row(0, 1, "B"))); + + current = row(0, 1, "b"); + accumulator = (InternalArray) agg.agg(accumulator, singletonArray(current)); + assertThat(unnest(accumulator, elementGetter)) + .containsExactlyInAnyOrderElementsOf(Arrays.asList(row(0, 1, "B"), row(0, 1, "b"))); + + // count limit is 2, so the third element will be dropped + current = row(0, 1, "C"); + accumulator = (InternalArray) agg.agg(accumulator, singletonArray(current)); + assertThat(unnest(accumulator, elementGetter)) + .containsExactlyInAnyOrderElementsOf(Arrays.asList(row(0, 1, "B"), row(0, 1, "b"))); + } + private List unnest(InternalArray array, InternalArray.ElementGetter elementGetter) { return IntStream.range(0, array.size()) .mapToObj(i -> elementGetter.getElementOrNull(array, i)) diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java index 8e794fe7496b..9d17593c118f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java @@ -154,13 +154,13 @@ public void testMultiPartitions(boolean cleanEmptyDirs) throws Exception { FileStoreCommitImpl commit = store.newCommit(); Map partitionSpec = new HashMap<>(); partitionSpec.put("dt", "0401"); - commit.overwrite( + commit.overwritePartition( partitionSpec, new ManifestCommittable(commitIdentifier++), Collections.emptyMap()); // step 3: generate snapshot 3 by cleaning partition dt=0402/hr=10 partitionSpec.put("dt", "0402"); partitionSpec.put("hr", "8"); - commit.overwrite( + commit.overwritePartition( partitionSpec, new ManifestCommittable(commitIdentifier++), Collections.emptyMap()); commit.close(); @@ -751,7 +751,7 @@ public void testDataFileSkippingSetException() throws Exception { store.newStatsFileHandler(), store.options().changelogProducer() != CoreOptions.ChangelogProducer.NONE, store.options().cleanEmptyDirectories(), - store.options().deleteFileThreadNum()); + store.options().fileOperationThreadNum()); ExpireSnapshots expireSnapshots = new ExpireSnapshotsImpl( @@ -816,7 +816,7 @@ public void testManifestFileSkippingSetException() throws Exception { store.newStatsFileHandler(), store.options().changelogProducer() != CoreOptions.ChangelogProducer.NONE, store.options().cleanEmptyDirectories(), - store.options().deleteFileThreadNum()); + store.options().fileOperationThreadNum()); ExpireSnapshots expireSnapshots = new ExpireSnapshotsImpl( snapshotManager, changelogManager, snapshotDeletion, tagManager); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java index 47a876f34545..505417a4121f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java @@ -886,23 +886,25 @@ public void testDVIndexFiles(boolean bitmap64) throws Exception { Map options = new HashMap<>(); options.put(CoreOptions.DELETION_VECTOR_BITMAP64.key(), String.valueOf(bitmap64)); TestAppendFileStore store = TestAppendFileStore.createAppendStore(tempDir, options); + BinaryRow partition = gen.getPartition(gen.next()); + + // create files + CommitMessageImpl commitMessage0 = + store.writeDataFiles(partition, 0, Arrays.asList("f1", "f2")); + store.commit(commitMessage0); // commit 1 CommitMessageImpl commitMessage1 = store.writeDVIndexFiles( - BinaryRow.EMPTY_ROW, - 0, - Collections.singletonMap("f1", Arrays.asList(1, 3))); + partition, 0, Collections.singletonMap("f1", Arrays.asList(1, 3))); CommitMessageImpl commitMessage2 = store.writeDVIndexFiles( - BinaryRow.EMPTY_ROW, - 0, - Collections.singletonMap("f2", Arrays.asList(2, 4))); + partition, 0, Collections.singletonMap("f2", Arrays.asList(2, 4))); store.commit(commitMessage1, commitMessage2); // assert 1 - assertThat(store.scanDVIndexFiles(BinaryRow.EMPTY_ROW, 0).size()).isEqualTo(2); - BucketedDvMaintainer maintainer = store.createOrRestoreDVMaintainer(BinaryRow.EMPTY_ROW, 0); + assertThat(store.scanDVIndexFiles(partition, 0).size()).isEqualTo(2); + BucketedDvMaintainer maintainer = store.createOrRestoreDVMaintainer(partition, 0); Map dvs = maintainer.deletionVectors(); assertThat(dvs.size()).isEqualTo(2); assertThat(dvs.get("f2").isDeleted(2)).isTrue(); @@ -912,16 +914,16 @@ public void testDVIndexFiles(boolean bitmap64) throws Exception { // commit 2 CommitMessage commitMessage3 = store.writeDVIndexFiles( - BinaryRow.EMPTY_ROW, 0, Collections.singletonMap("f2", Arrays.asList(3))); + partition, 0, Collections.singletonMap("f2", Arrays.asList(3))); List deleted = - new ArrayList<>(commitMessage1.indexIncrement().newIndexFiles()); - deleted.addAll(commitMessage2.indexIncrement().newIndexFiles()); - CommitMessage commitMessage4 = store.removeIndexFiles(BinaryRow.EMPTY_ROW, 0, deleted); + new ArrayList<>(commitMessage1.newFilesIncrement().newIndexFiles()); + deleted.addAll(commitMessage2.newFilesIncrement().newIndexFiles()); + CommitMessage commitMessage4 = store.removeIndexFiles(partition, 0, deleted); store.commit(commitMessage3, commitMessage4); // assert 2 - assertThat(store.scanDVIndexFiles(BinaryRow.EMPTY_ROW, 0).size()).isEqualTo(1); - maintainer = store.createOrRestoreDVMaintainer(BinaryRow.EMPTY_ROW, 0); + assertThat(store.scanDVIndexFiles(partition, 0).size()).isEqualTo(1); + maintainer = store.createOrRestoreDVMaintainer(partition, 0); dvs = maintainer.deletionVectors(); assertThat(dvs.size()).isEqualTo(2); assertThat(dvs.get("f1").isDeleted(3)).isTrue(); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/TestCommitThread.java b/paimon-core/src/test/java/org/apache/paimon/operation/TestCommitThread.java index 2f409a91768c..cdd2d8fba11f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/TestCommitThread.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/TestCommitThread.java @@ -169,7 +169,7 @@ private void doOverwrite() throws Exception { runWithRetry( committable, () -> - commit.overwrite( + commit.overwritePartition( TestKeyValueGenerator.toPartitionMap(partition, MULTI_PARTITIONED), committable, Collections.emptyMap())); diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java index a6e1936f40f0..bf56514b4ff3 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java @@ -84,7 +84,9 @@ public void setUp() throws Exception { @AfterEach public void tearDown() throws Exception { - restCatalogServer.shutdown(); + if (restCatalogServer != null) { + restCatalogServer.shutdown(); + } } @Test @@ -177,10 +179,10 @@ private void testDlfAuth(RESTCatalog restCatalog) throws Exception { Identifier.create(databaseName, tableName), DEFAULT_TABLE_SCHEMA, false); } PagedList listTablesPaged = - restCatalog.listTablesPaged(databaseName, 1, "dt=20230101", null); + restCatalog.listTablesPaged(databaseName, 1, "dt=20230101", null, null); PagedList listTablesPaged2 = restCatalog.listTablesPaged( - databaseName, 1, listTablesPaged.getNextPageToken(), null); + databaseName, 1, listTablesPaged.getNextPageToken(), null, null); assertEquals(listTablesPaged.getElements().get(0), "dt=20230102"); assertEquals(listTablesPaged2.getElements().get(0), "dt=20230103"); } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogServer.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogServer.java index a2e1da2c4062..67d4ff7e91d1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogServer.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogServer.java @@ -143,6 +143,7 @@ import static org.apache.paimon.rest.RESTApi.PAGE_TOKEN; import static org.apache.paimon.rest.RESTApi.PARTITION_NAME_PATTERN; import static org.apache.paimon.rest.RESTApi.TABLE_NAME_PATTERN; +import static org.apache.paimon.rest.RESTApi.TABLE_TYPE; import static org.apache.paimon.rest.RESTApi.VIEW_NAME_PATTERN; import static org.apache.paimon.rest.ResourcePaths.FUNCTIONS; import static org.apache.paimon.rest.ResourcePaths.FUNCTION_DETAILS; @@ -1296,12 +1297,30 @@ private MockResponse tablesHandle( private List listTables(String databaseName, Map parameters) { String tableNamePattern = parameters.get(TABLE_NAME_PATTERN); + String tableType = parameters.get(TABLE_TYPE); List tables = new ArrayList<>(); for (Map.Entry entry : tableMetadataStore.entrySet()) { Identifier identifier = Identifier.fromString(entry.getKey()); if (databaseName.equals(identifier.getDatabaseName()) && (Objects.isNull(tableNamePattern) || matchNamePattern(identifier.getTableName(), tableNamePattern))) { + + // Check table type filter if specified + if (StringUtils.isNotEmpty(tableType)) { + String actualTableType = entry.getValue().schema().options().get(TYPE.key()); + if (StringUtils.equals(tableType, "table")) { + // When filtering by "table" type, return tables with null or "table" type + if (actualTableType != null && !"table".equals(actualTableType)) { + continue; + } + } else { + // For other table types, return exact matches + if (!StringUtils.equals(tableType, actualTableType)) { + continue; + } + } + } + tables.add(identifier.getTableName()); } } @@ -1361,12 +1380,30 @@ private MockResponse tableDetailsHandle(Map parameters, String d private List listTableDetails( String databaseName, Map parameters) { String tableNamePattern = parameters.get(TABLE_NAME_PATTERN); + String tableType = parameters.get(TABLE_TYPE); List tableDetails = new ArrayList<>(); for (Map.Entry entry : tableMetadataStore.entrySet()) { Identifier identifier = Identifier.fromString(entry.getKey()); if (databaseName.equals(identifier.getDatabaseName()) && (Objects.isNull(tableNamePattern) || matchNamePattern(identifier.getTableName(), tableNamePattern))) { + + // Check table type filter if specified + if (StringUtils.isNotEmpty(tableType)) { + String actualTableType = entry.getValue().schema().options().get(TYPE.key()); + if (StringUtils.equals(tableType, "table")) { + // When filtering by "table" type, return tables with null or "table" type + if (actualTableType != null && !"table".equals(actualTableType)) { + continue; + } + } else { + // For other table types, return exact matches + if (!StringUtils.equals(tableType, actualTableType)) { + continue; + } + } + } + GetTableResponse getTableResponse = new GetTableResponse( entry.getValue().uuid(), diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java index 51e2c986cfb6..8234cac4d312 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java @@ -251,10 +251,10 @@ void testApiWhenDatabaseNoExistAndNotIgnore() { false)); assertThrows( Catalog.DatabaseNotExistException.class, - () -> catalog.listTablesPaged(database, 100, null, null)); + () -> catalog.listTablesPaged(database, 100, null, null, null)); assertThrows( Catalog.DatabaseNotExistException.class, - () -> catalog.listTableDetailsPaged(database, 100, null, null)); + () -> catalog.listTableDetailsPaged(database, 100, null, null, null)); } @Test @@ -362,7 +362,8 @@ public void testListTablesPaged() throws Exception { // List tables paged returns an empty list when there are no tables in the database String databaseName = "tables_paged_db"; catalog.createDatabase(databaseName, false); - PagedList pagedTables = catalog.listTablesPaged(databaseName, null, null, null); + PagedList pagedTables = + catalog.listTablesPaged(databaseName, null, null, null, null); assertThat(pagedTables.getElements()).isEmpty(); assertNull(pagedTables.getNextPageToken()); @@ -374,7 +375,7 @@ public void testListTablesPaged() throws Exception { // when maxResults is null or 0, the page length is set to a server configured value String[] sortedTableNames = Arrays.stream(tableNames).sorted().toArray(String[]::new); - pagedTables = catalog.listTablesPaged(databaseName, null, null, null); + pagedTables = catalog.listTablesPaged(databaseName, null, null, null, null); List tables = pagedTables.getElements(); assertThat(tables).containsExactly(sortedTableNames); assertNull(pagedTables.getNextPageToken()); @@ -383,7 +384,7 @@ public void testListTablesPaged() throws Exception { // server configured value // when pageToken is null, will list tables from the beginning int maxResults = 2; - pagedTables = catalog.listTablesPaged(databaseName, maxResults, null, null); + pagedTables = catalog.listTablesPaged(databaseName, maxResults, null, null, null); tables = pagedTables.getElements(); assertEquals(maxResults, tables.size()); assertThat(tables).containsExactly("abd", "def"); @@ -392,7 +393,7 @@ public void testListTablesPaged() throws Exception { // when pageToken is not null, will list tables from the pageToken (exclusive) pagedTables = catalog.listTablesPaged( - databaseName, maxResults, pagedTables.getNextPageToken(), null); + databaseName, maxResults, pagedTables.getNextPageToken(), null, null); tables = pagedTables.getElements(); assertEquals(maxResults, tables.size()); assertThat(tables).containsExactly("opr", "table1"); @@ -400,7 +401,7 @@ public void testListTablesPaged() throws Exception { pagedTables = catalog.listTablesPaged( - databaseName, maxResults, pagedTables.getNextPageToken(), null); + databaseName, maxResults, pagedTables.getNextPageToken(), null, null); tables = pagedTables.getElements(); assertEquals(maxResults, tables.size()); assertThat(tables).containsExactly("table2", "table3"); @@ -408,18 +409,18 @@ public void testListTablesPaged() throws Exception { pagedTables = catalog.listTablesPaged( - databaseName, maxResults, pagedTables.getNextPageToken(), null); + databaseName, maxResults, pagedTables.getNextPageToken(), null, null); tables = pagedTables.getElements(); assertEquals(1, tables.size()); assertNull(pagedTables.getNextPageToken()); maxResults = 8; - pagedTables = catalog.listTablesPaged(databaseName, maxResults, null, null); + pagedTables = catalog.listTablesPaged(databaseName, maxResults, null, null, null); tables = pagedTables.getElements(); assertThat(tables).containsExactly(sortedTableNames); assertNull(pagedTables.getNextPageToken()); - pagedTables = catalog.listTablesPaged(databaseName, maxResults, "table1", null); + pagedTables = catalog.listTablesPaged(databaseName, maxResults, "table1", null, null); tables = pagedTables.getElements(); assertEquals(3, tables.size()); assertThat(tables).containsExactly("table2", "table3", "table_name"); @@ -429,24 +430,24 @@ public void testListTablesPaged() throws Exception { assertThatExceptionOfType(Catalog.DatabaseNotExistException.class) .isThrownBy(() -> catalog.listTables("non_existing_db")); - pagedTables = catalog.listTablesPaged(databaseName, null, null, "table%"); + pagedTables = catalog.listTablesPaged(databaseName, null, null, "table%", null); tables = pagedTables.getElements(); assertEquals(4, tables.size()); assertThat(tables).containsExactly("table1", "table2", "table3", "table_name"); assertNull(pagedTables.getNextPageToken()); - pagedTables = catalog.listTablesPaged(databaseName, null, null, "table_"); + pagedTables = catalog.listTablesPaged(databaseName, null, null, "table_", null); tables = pagedTables.getElements(); assertTrue(tables.isEmpty()); assertNull(pagedTables.getNextPageToken()); - pagedTables = catalog.listTablesPaged(databaseName, null, null, "table_%"); + pagedTables = catalog.listTablesPaged(databaseName, null, null, "table_%", null); tables = pagedTables.getElements(); assertEquals(1, tables.size()); assertThat(tables).containsExactly("table_name"); assertNull(pagedTables.getNextPageToken()); - pagedTables = catalog.listTablesPaged(databaseName, null, null, "table_name"); + pagedTables = catalog.listTablesPaged(databaseName, null, null, "table_name", null); tables = pagedTables.getElements(); assertEquals(1, tables.size()); assertThat(tables).containsExactly("table_name"); @@ -454,11 +455,11 @@ public void testListTablesPaged() throws Exception { Assertions.assertThrows( BadRequestException.class, - () -> catalog.listTablesPaged(databaseName, null, null, "%table")); + () -> catalog.listTablesPaged(databaseName, null, null, "%table", null)); Assertions.assertThrows( BadRequestException.class, - () -> catalog.listTablesPaged(databaseName, null, null, "ta%le")); + () -> catalog.listTablesPaged(databaseName, null, null, "ta%le", null)); } @Test @@ -467,7 +468,7 @@ public void testListTableDetailsPaged() throws Exception { String databaseName = "table_details_paged_db"; catalog.createDatabase(databaseName, false); PagedList
pagedTableDetails = - catalog.listTableDetailsPaged(databaseName, null, null, null); + catalog.listTableDetailsPaged(databaseName, null, null, null, null); assertThat(pagedTableDetails.getElements()).isEmpty(); assertNull(pagedTableDetails.getNextPageToken()); @@ -478,42 +479,44 @@ public void testListTableDetailsPaged() throws Exception { Identifier.create(databaseName, tableName), DEFAULT_TABLE_SCHEMA, false); } - pagedTableDetails = catalog.listTableDetailsPaged(databaseName, null, null, null); + pagedTableDetails = catalog.listTableDetailsPaged(databaseName, null, null, null, null); assertPagedTableDetails(pagedTableDetails, tableNames.length, expectedTableNames); assertNull(pagedTableDetails.getNextPageToken()); int maxResults = 2; - pagedTableDetails = catalog.listTableDetailsPaged(databaseName, maxResults, null, null); + pagedTableDetails = + catalog.listTableDetailsPaged(databaseName, maxResults, null, null, null); assertPagedTableDetails(pagedTableDetails, maxResults, "abd", "def"); assertEquals("def", pagedTableDetails.getNextPageToken()); pagedTableDetails = catalog.listTableDetailsPaged( - databaseName, maxResults, pagedTableDetails.getNextPageToken(), null); + databaseName, maxResults, pagedTableDetails.getNextPageToken(), null, null); assertPagedTableDetails(pagedTableDetails, maxResults, "opr", "table1"); assertEquals("table1", pagedTableDetails.getNextPageToken()); pagedTableDetails = catalog.listTableDetailsPaged( - databaseName, maxResults, pagedTableDetails.getNextPageToken(), null); + databaseName, maxResults, pagedTableDetails.getNextPageToken(), null, null); assertPagedTableDetails(pagedTableDetails, maxResults, "table2", "table3"); assertEquals("table3", pagedTableDetails.getNextPageToken()); pagedTableDetails = catalog.listTableDetailsPaged( - databaseName, maxResults, pagedTableDetails.getNextPageToken(), null); + databaseName, maxResults, pagedTableDetails.getNextPageToken(), null, null); assertEquals(1, pagedTableDetails.getElements().size()); assertNull(pagedTableDetails.getNextPageToken()); maxResults = 8; - pagedTableDetails = catalog.listTableDetailsPaged(databaseName, maxResults, null, null); + pagedTableDetails = + catalog.listTableDetailsPaged(databaseName, maxResults, null, null, null); assertPagedTableDetails( pagedTableDetails, Math.min(maxResults, tableNames.length), expectedTableNames); assertNull(pagedTableDetails.getNextPageToken()); String pageToken = "table1"; pagedTableDetails = - catalog.listTableDetailsPaged(databaseName, maxResults, pageToken, null); + catalog.listTableDetailsPaged(databaseName, maxResults, pageToken, null, null); assertPagedTableDetails(pagedTableDetails, 3, "table2", "table3", "table_name"); assertNull(pagedTableDetails.getNextPageToken()); @@ -523,31 +526,292 @@ public void testListTableDetailsPaged() throws Exception { .isThrownBy( () -> catalog.listTableDetailsPaged( - "non_existing_db", finalMaxResults, pageToken, null)); + "non_existing_db", finalMaxResults, pageToken, null, null)); // List tables throws DatabaseNotExistException when the database does not exist assertThatExceptionOfType(Catalog.DatabaseNotExistException.class) .isThrownBy(() -> catalog.listTables("non_existing_db")); - pagedTableDetails = catalog.listTableDetailsPaged(databaseName, null, null, "table%"); + pagedTableDetails = catalog.listTableDetailsPaged(databaseName, null, null, "table%", null); assertPagedTableDetails(pagedTableDetails, 4, "table1", "table2", "table3", "table_name"); assertNull(pagedTableDetails.getNextPageToken()); - pagedTableDetails = catalog.listTableDetailsPaged(databaseName, null, null, "table_"); + pagedTableDetails = catalog.listTableDetailsPaged(databaseName, null, null, "table_", null); Assertions.assertTrue(pagedTableDetails.getElements().isEmpty()); assertNull(pagedTableDetails.getNextPageToken()); - pagedTableDetails = catalog.listTableDetailsPaged(databaseName, null, null, "table_%"); + pagedTableDetails = + catalog.listTableDetailsPaged(databaseName, null, null, "table_%", null); assertPagedTableDetails(pagedTableDetails, 1, "table_name"); assertNull(pagedTableDetails.getNextPageToken()); Assertions.assertThrows( BadRequestException.class, - () -> catalog.listTableDetailsPaged(databaseName, null, null, "ta%le")); + () -> catalog.listTableDetailsPaged(databaseName, null, null, "ta%le", null)); Assertions.assertThrows( BadRequestException.class, - () -> catalog.listTableDetailsPaged(databaseName, null, null, "%tale")); + () -> catalog.listTableDetailsPaged(databaseName, null, null, "%tale", null)); + } + + @Test + public void testListTableDetailsPagedWithTableType() throws Exception { + String databaseName = "table_type_filter_db"; + catalog.createDatabase(databaseName, false); + + // Create tables with different types + Schema normalTableSchema = DEFAULT_TABLE_SCHEMA; + catalog.createTable( + Identifier.create(databaseName, "normal_table"), normalTableSchema, false); + + Schema formatTableSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .option("type", TableType.FORMAT_TABLE.toString()) + .build(); + catalog.createTable( + Identifier.create(databaseName, "format_table"), formatTableSchema, false); + + Schema objectTableSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .option("type", TableType.OBJECT_TABLE.toString()) + .build(); + catalog.createTable( + Identifier.create(databaseName, "object_table"), objectTableSchema, false); + + // Test filtering by table type + PagedList
allTables = + catalog.listTableDetailsPaged(databaseName, null, null, null, null); + assertThat(allTables.getElements()).hasSize(3); + + PagedList
normalTables = + catalog.listTableDetailsPaged( + databaseName, null, null, null, TableType.TABLE.toString()); + assertThat(normalTables.getElements()).hasSize(1); + assertThat(normalTables.getElements().get(0).name()).isEqualTo("normal_table"); + + PagedList
formatTables = + catalog.listTableDetailsPaged( + databaseName, null, null, null, TableType.FORMAT_TABLE.toString()); + assertThat(formatTables.getElements()).hasSize(1); + assertThat(formatTables.getElements().get(0).name()).isEqualTo("format_table"); + + PagedList
objectTables = + catalog.listTableDetailsPaged( + databaseName, null, null, null, TableType.OBJECT_TABLE.toString()); + assertThat(objectTables.getElements()).hasSize(1); + assertThat(objectTables.getElements().get(0).name()).isEqualTo("object_table"); + + // Test with non-existent table type + PagedList
nonExistentType = + catalog.listTableDetailsPaged(databaseName, null, null, null, "non-existent-type"); + assertThat(nonExistentType.getElements()).isEmpty(); + + // Test with table name pattern and table type filter combined + PagedList
filteredTables = + catalog.listTableDetailsPaged( + databaseName, null, null, "format_%", TableType.FORMAT_TABLE.toString()); + assertThat(filteredTables.getElements()).hasSize(1); + assertThat(filteredTables.getElements().get(0).name()).isEqualTo("format_table"); + + // Test with table name pattern and non-existent table type filter combined + PagedList
filteredNonExistentType = + catalog.listTableDetailsPaged( + databaseName, null, null, "format_%", "non-existent-type"); + assertThat(filteredNonExistentType.getElements()).isEmpty(); + + // Test maxResults parameter variations with table type filtering + // Test maxResults=1 with different table types + PagedList
singleNormalTable = + catalog.listTableDetailsPaged( + databaseName, 1, null, null, TableType.TABLE.toString()); + assertThat(singleNormalTable.getElements()).hasSize(1); + assertEquals("normal_table", singleNormalTable.getElements().get(0).name()); + assertEquals("normal_table", singleNormalTable.getNextPageToken()); + + PagedList
singleFormatTable = + catalog.listTableDetailsPaged( + databaseName, 1, null, null, TableType.FORMAT_TABLE.toString()); + assertThat(singleFormatTable.getElements()).hasSize(1); + assertEquals("format_table", singleFormatTable.getElements().get(0).name()); + assertEquals("format_table", singleFormatTable.getNextPageToken()); + + PagedList
singleObjectTable = + catalog.listTableDetailsPaged( + databaseName, 1, null, null, TableType.OBJECT_TABLE.toString()); + assertThat(singleObjectTable.getElements()).hasSize(1); + assertEquals("object_table", singleObjectTable.getElements().get(0).name()); + assertEquals("object_table", singleObjectTable.getNextPageToken()); + + // Test maxResults=2 with all table types + PagedList
allTablesWithMaxResults = + catalog.listTableDetailsPaged(databaseName, 2, null, null, null); + assertThat(allTablesWithMaxResults.getElements()).hasSize(2); + assertThat(allTablesWithMaxResults.getNextPageToken()).isNotNull(); + + // Test maxResults=2 with table name pattern and table type filter combined + PagedList
filteredTablesWithMaxResults = + catalog.listTableDetailsPaged( + databaseName, 2, null, "format_%", TableType.FORMAT_TABLE.toString()); + assertThat(filteredTablesWithMaxResults.getElements()).hasSize(1); + assertEquals("format_table", filteredTablesWithMaxResults.getElements().get(0).name()); + assertThat(filteredTablesWithMaxResults.getNextPageToken()).isNull(); + + // Test maxResults=0 (should return all tables) + PagedList
allTablesWithZeroMaxResults = + catalog.listTableDetailsPaged(databaseName, 0, null, null, null); + assertThat(allTablesWithZeroMaxResults.getElements()).hasSize(3); + assertThat(allTablesWithZeroMaxResults.getNextPageToken()).isNull(); + + // Test maxResults larger than total tables with table type filter + PagedList
largeMaxResultsWithType = + catalog.listTableDetailsPaged( + databaseName, 10, null, null, TableType.TABLE.toString()); + assertThat(largeMaxResultsWithType.getElements()).hasSize(1); + assertEquals("normal_table", largeMaxResultsWithType.getElements().get(0).name()); + assertThat(largeMaxResultsWithType.getNextPageToken()).isNull(); + + // Test maxResults with non-existent table type + PagedList
nonExistentTypeWithMaxResults = + catalog.listTableDetailsPaged(databaseName, 5, null, null, "non-existent-type"); + assertThat(nonExistentTypeWithMaxResults.getElements()).isEmpty(); + assertThat(nonExistentTypeWithMaxResults.getNextPageToken()).isNull(); + } + + @Test + public void testListTablesPagedWithTableType() throws Exception { + String databaseName = "tables_paged_table_type_db"; + catalog.createDatabase(databaseName, false); + + // Create tables with different types + Schema normalTableSchema = DEFAULT_TABLE_SCHEMA; + catalog.createTable( + Identifier.create(databaseName, "normal_table"), normalTableSchema, false); + + Schema formatTableSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .option("type", TableType.FORMAT_TABLE.toString()) + .build(); + catalog.createTable( + Identifier.create(databaseName, "format_table"), formatTableSchema, false); + + Schema objectTableSchema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .option("type", TableType.OBJECT_TABLE.toString()) + .build(); + catalog.createTable( + Identifier.create(databaseName, "object_table"), objectTableSchema, false); + + // Test filtering by table type + PagedList allTables = catalog.listTablesPaged(databaseName, null, null, null, null); + assertThat(allTables.getElements()).hasSize(3); + + PagedList normalTables = + catalog.listTablesPaged(databaseName, null, null, null, TableType.TABLE.toString()); + assertThat(normalTables.getElements()).hasSize(1); + assertThat(normalTables.getElements().get(0)).isEqualTo("normal_table"); + + PagedList formatTables = + catalog.listTablesPaged( + databaseName, null, null, null, TableType.FORMAT_TABLE.toString()); + assertThat(formatTables.getElements()).hasSize(1); + assertThat(formatTables.getElements().get(0)).isEqualTo("format_table"); + + PagedList objectTables = + catalog.listTablesPaged( + databaseName, null, null, null, TableType.OBJECT_TABLE.toString()); + assertThat(objectTables.getElements()).hasSize(1); + assertThat(objectTables.getElements().get(0)).isEqualTo("object_table"); + + // Test with non-existent table type + PagedList nonExistentType = + catalog.listTablesPaged(databaseName, null, null, null, "non-existent-type"); + assertThat(nonExistentType.getElements()).isEmpty(); + + // Test with table name pattern and table type filter combined + PagedList filteredTables = + catalog.listTablesPaged( + databaseName, null, null, "format_%", TableType.FORMAT_TABLE.toString()); + assertThat(filteredTables.getElements()).hasSize(1); + assertThat(filteredTables.getElements().get(0)).isEqualTo("format_table"); + + // Test with table name pattern and non-existent table type filter combined + PagedList filteredNonExistentType = + catalog.listTablesPaged(databaseName, null, null, "format_%", "non-existent-type"); + assertThat(filteredNonExistentType.getElements()).isEmpty(); + + // Test paging with table type filter + int maxResults = 10; + PagedList pagedNormalTables = + catalog.listTablesPaged( + databaseName, maxResults, null, null, TableType.TABLE.toString()); + assertThat(pagedNormalTables.getElements()).hasSize(1); + assertThat(pagedNormalTables.getElements().get(0)).isEqualTo("normal_table"); + assertNull(pagedNormalTables.getNextPageToken()); + + // Test maxResults parameter variations with table type filtering + // Test maxResults=0 (should return all tables) + PagedList allTablesWithZeroMaxResults = + catalog.listTablesPaged(databaseName, 0, null, null, null); + assertThat(allTablesWithZeroMaxResults.getElements()).hasSize(3); + assertNull(allTablesWithZeroMaxResults.getNextPageToken()); + + // Test maxResults=1 with different table types + PagedList singleNormalTable = + catalog.listTablesPaged(databaseName, 1, null, null, TableType.TABLE.toString()); + assertThat(singleNormalTable.getElements()).hasSize(1); + assertEquals("normal_table", singleNormalTable.getElements().get(0)); + assertEquals("normal_table", singleNormalTable.getNextPageToken()); + + PagedList singleFormatTable = + catalog.listTablesPaged( + databaseName, 1, null, null, TableType.FORMAT_TABLE.toString()); + assertThat(singleFormatTable.getElements()).hasSize(1); + assertEquals("format_table", singleFormatTable.getElements().get(0)); + assertEquals("format_table", singleFormatTable.getNextPageToken()); + + PagedList singleObjectTable = + catalog.listTablesPaged( + databaseName, 1, null, null, TableType.OBJECT_TABLE.toString()); + assertThat(singleObjectTable.getElements()).hasSize(1); + assertEquals("object_table", singleObjectTable.getElements().get(0)); + assertEquals("object_table", singleObjectTable.getNextPageToken()); + + // Test maxResults=2 with all table types + PagedList allTablesWithMaxResults = + catalog.listTablesPaged(databaseName, 2, null, null, null); + assertThat(allTablesWithMaxResults.getElements()).hasSize(2); + assertThat(allTablesWithMaxResults.getNextPageToken()).isNotNull(); + + // Test maxResults=2 with table name pattern and table type filter combined + PagedList filteredTablesWithMaxResults = + catalog.listTablesPaged( + databaseName, 2, null, "format_%", TableType.FORMAT_TABLE.toString()); + assertThat(filteredTablesWithMaxResults.getElements()).hasSize(1); + assertThat(filteredTablesWithMaxResults.getElements().get(0)).isEqualTo("format_table"); + assertNull(filteredTablesWithMaxResults.getNextPageToken()); + assertEquals("format_table", filteredTablesWithMaxResults.getElements().get(0)); + assertNull(filteredTablesWithMaxResults.getNextPageToken()); + + // Test maxResults larger than total tables with table type filter + PagedList largeMaxResultsWithType = + catalog.listTablesPaged(databaseName, 10, null, null, TableType.TABLE.toString()); + assertThat(largeMaxResultsWithType.getElements()).hasSize(1); + assertEquals("normal_table", largeMaxResultsWithType.getElements().get(0)); + assertNull(largeMaxResultsWithType.getNextPageToken()); + + // Test maxResults with non-existent table type + PagedList nonExistentTypeWithMaxResults = + catalog.listTablesPaged(databaseName, 5, null, null, "non-existent-type"); + assertThat(nonExistentTypeWithMaxResults.getElements()).isEmpty(); + assertNull(nonExistentTypeWithMaxResults.getNextPageToken()); } @Test diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java index 7f0cd6042567..275241455a80 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/TableSchemaTest.java @@ -77,13 +77,6 @@ public void testCrossPartition() { new TableSchema(1, fields, 10, partitionKeys, primaryKeys, options, ""); assertThat(schema.crossPartitionUpdate()).isTrue(); - options.put(BUCKET.key(), "1"); - assertThatThrownBy(() -> validateTableSchema(schema)) - .hasMessageContaining("You should use dynamic bucket"); - - options.put(BUCKET.key(), "-1"); - validateTableSchema(schema); - options.put(SEQUENCE_FIELD.key(), "f2"); assertThatThrownBy(() -> validateTableSchema(schema)) .hasMessageContaining("You can not use sequence.field"); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/DataEvolutionTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/DataEvolutionTableTest.java index 8cbc0413ff78..0cd2cf46086e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/DataEvolutionTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/DataEvolutionTableTest.java @@ -23,6 +23,8 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.predicate.PredicateBuilder; import org.apache.paimon.reader.DataEvolutionFileReader; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.Schema; @@ -31,6 +33,7 @@ import org.apache.paimon.table.sink.BatchWriteBuilder; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.ReadBuilder; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -385,6 +388,78 @@ public void testMoreData() throws Exception { }); } + @Test + public void testPredicate() throws Exception { + createTableDefault(); + Schema schema = schemaDefault(); + BatchWriteBuilder builder = getTableDefault().newBatchWriteBuilder(); + try (BatchTableWrite write = builder.newWrite().withWriteType(schema.rowType())) { + write.write( + GenericRow.of(1, BinaryString.fromString("a"), BinaryString.fromString("b"))); + BatchTableCommit commit = builder.newCommit(); + List commitables = write.prepareCommit(); + commit.commit(commitables); + } + + RowType writeType1 = schema.rowType().project(Collections.singletonList("f2")); + try (BatchTableWrite write1 = builder.newWrite().withWriteType(writeType1)) { + write1.write(GenericRow.of(BinaryString.fromString("c"))); + + BatchTableCommit commit = builder.newCommit(); + List commitables = write1.prepareCommit(); + setFirstRowId(commitables, 0L); + commit.commit(commitables); + } + + ReadBuilder readBuilder = getTableDefault().newReadBuilder(); + PredicateBuilder predicateBuilder = new PredicateBuilder(schema.rowType()); + Predicate predicate = predicateBuilder.notEqual(2, BinaryString.fromString("b")); + readBuilder.withFilter(predicate); + assertThat(((DataSplit) readBuilder.newScan().plan().splits().get(0)).dataFiles().size()) + .isEqualTo(2); + } + + @Test + public void testNonNullColumn() throws Exception { + Schema.Builder schemaBuilder = Schema.newBuilder(); + schemaBuilder.column("f0", DataTypes.INT()); + schemaBuilder.column("f1", DataTypes.STRING()); + schemaBuilder.column("f2", DataTypes.STRING().copy(false)); + schemaBuilder.option(CoreOptions.ROW_TRACKING_ENABLED.key(), "true"); + schemaBuilder.option(CoreOptions.DATA_EVOLUTION_ENABLED.key(), "true"); + + Schema schema = schemaBuilder.build(); + + catalog.createTable(identifier(), schema, true); + Table table = catalog.getTable(identifier()); + BatchWriteBuilder builder = table.newBatchWriteBuilder(); + BatchTableWrite write = builder.newWrite(); + write.write(GenericRow.of(1, BinaryString.fromString("a"), BinaryString.fromString("b"))); + BatchTableCommit commit = builder.newCommit(); + List commitables = write.prepareCommit(); + commit.commit(commitables); + + write = + builder.newWrite() + .withWriteType(schema.rowType().project(Collections.singletonList("f2"))); + write.write(GenericRow.of(BinaryString.fromString("c"))); + commit = builder.newCommit(); + commitables = write.prepareCommit(); + setFirstRowId(commitables, 0L); + commit.commit(commitables); + + ReadBuilder readBuilder = table.newReadBuilder(); + RecordReader reader = + readBuilder.newRead().createReader(readBuilder.newScan().plan()); + assertThat(reader).isInstanceOf(DataEvolutionFileReader.class); + reader.forEachRemaining( + r -> { + assertThat(r.getInt(0)).isEqualTo(1); + assertThat(r.getString(1).toString()).isEqualTo("a"); + assertThat(r.getString(2).toString()).isEqualTo("c"); + }); + } + protected Schema schemaDefault() { Schema.Builder schemaBuilder = Schema.newBuilder(); schemaBuilder.column("f0", DataTypes.INT()); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/DynamicBucketTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/DynamicBucketTableTest.java index 29e71e2b44da..0767200bb181 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/DynamicBucketTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/DynamicBucketTableTest.java @@ -61,7 +61,7 @@ public void testOverwriteDynamicBucketTable() throws Exception { batchTableWrite.write(rowWithBucket.getKey(), rowWithBucket.getValue()); assertThat( ((CommitMessageImpl) batchTableWrite.prepareCommit().get(0)) - .indexIncrement() + .newFilesIncrement() .newIndexFiles() .get(0) .rowCount()) diff --git a/paimon-core/src/test/java/org/apache/paimon/table/SchemaEvolutionTest.java b/paimon-core/src/test/java/org/apache/paimon/table/SchemaEvolutionTest.java index ad540d58b439..8b118e05621d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/SchemaEvolutionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/SchemaEvolutionTest.java @@ -227,6 +227,22 @@ public void testUpdateFieldType() throws Exception { assertThat(tableSchema.fields().get(0).type()).isEqualTo(DataTypes.STRING()); } + @Test + public void testUpdatePrimaryKeyType() throws Exception { + Schema schema = + Schema.newBuilder() + .column("k", DataTypes.INT()) + .column("v", DataTypes.BIGINT()) + .primaryKey("k") + .build(); + schemaManager.createTable(schema); + + List changes = + Collections.singletonList(SchemaChange.updateColumnType("k", DataTypes.STRING())); + assertThatThrownBy(() -> schemaManager.commitChanges(changes)) + .hasMessageContaining("Cannot update primary key"); + } + @Test public void testRenameField() throws Exception { Schema schema = diff --git a/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java b/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java index 351af6a3b72b..c4f519e84e52 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java @@ -20,7 +20,6 @@ import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataIncrement; -import org.apache.paimon.io.IndexIncrement; import org.junit.jupiter.api.Test; @@ -41,22 +40,31 @@ public void test() throws IOException { CommitMessageSerializer serializer = new CommitMessageSerializer(); DataIncrement dataIncrement = randomNewFilesIncrement(); + dataIncrement.newIndexFiles().addAll(Arrays.asList(randomIndexFile(), randomIndexFile())); + dataIncrement + .deletedIndexFiles() + .addAll(Arrays.asList(randomIndexFile(), randomIndexFile())); + CompactIncrement compactIncrement = randomCompactIncrement(); - IndexIncrement indexIncrement = - new IndexIncrement( - Arrays.asList(randomIndexFile(), randomIndexFile()), - Arrays.asList(randomIndexFile(), randomIndexFile())); + compactIncrement + .newIndexFiles() + .addAll(Arrays.asList(randomIndexFile(), randomIndexFile())); + compactIncrement + .deletedIndexFiles() + .addAll(Arrays.asList(randomIndexFile(), randomIndexFile())); + CommitMessageImpl committable = - new CommitMessageImpl( - row(0), 1, 2, dataIncrement, compactIncrement, indexIncrement); + new CommitMessageImpl(row(0), 1, 2, dataIncrement, compactIncrement); CommitMessageImpl newCommittable = - (CommitMessageImpl) serializer.deserialize(7, serializer.serialize(committable)); + (CommitMessageImpl) + serializer.deserialize( + CommitMessageSerializer.CURRENT_VERSION, + serializer.serialize(committable)); assertThat(newCommittable.partition()).isEqualTo(committable.partition()); assertThat(newCommittable.bucket()).isEqualTo(committable.bucket()); assertThat(newCommittable.totalBuckets()).isEqualTo(committable.totalBuckets()); assertThat(newCommittable.compactIncrement()).isEqualTo(committable.compactIncrement()); assertThat(newCommittable.newFilesIncrement()).isEqualTo(committable.newFilesIncrement()); - assertThat(newCommittable.indexIncrement()).isEqualTo(committable.indexIncrement()); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/CompatibilityUtils.java b/paimon-core/src/test/java/org/apache/paimon/utils/CompatibilityUtils.java new file mode 100644 index 000000000000..67b8a3d511ac --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/utils/CompatibilityUtils.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.utils; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; + +/** Utils for compatibility. */ +public class CompatibilityUtils { + + public static final String COMPATIBILITY_FILE_DIR = "/src/test/resources/compatibility/"; + + /** Write compatibility file. */ + public static void writeCompatibilityFile(String fileName, byte[] data) throws IOException { + File file = new File(System.getProperty("user.dir") + COMPATIBILITY_FILE_DIR + fileName); + try (FileOutputStream fos = new FileOutputStream(file)) { + fos.write(data); + } + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/ConflictDeletionUtilsTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/ConflictDeletionUtilsTest.java new file mode 100644 index 000000000000..34e6e59e7caa --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/utils/ConflictDeletionUtilsTest.java @@ -0,0 +1,342 @@ +/* + * 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. + */ + +package org.apache.paimon.utils; + +import org.apache.paimon.index.DeletionVectorMeta; +import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.manifest.FileEntry; +import org.apache.paimon.manifest.FileKind; +import org.apache.paimon.manifest.IndexManifestEntry; +import org.apache.paimon.manifest.SimpleFileEntry; +import org.apache.paimon.manifest.SimpleFileEntryWithDV; + +import org.junit.jupiter.api.Test; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; + +import static org.apache.paimon.data.BinaryRow.EMPTY_ROW; +import static org.apache.paimon.deletionvectors.DeletionVectorsIndexFile.DELETION_VECTORS_INDEX; +import static org.apache.paimon.manifest.FileKind.ADD; +import static org.apache.paimon.manifest.FileKind.DELETE; +import static org.apache.paimon.utils.ConflictDeletionUtils.buildBaseEntriesWithDV; +import static org.apache.paimon.utils.ConflictDeletionUtils.buildDeltaEntriesWithDV; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link ConflictDeletionUtils}. */ +public class ConflictDeletionUtilsTest { + + @Test + public void testBuildBaseEntriesWithDV() { + { + // Scene 1 + List baseEntries = new ArrayList<>(); + baseEntries.add(createFileEntry("f1", ADD)); + baseEntries.add(createFileEntry("f2", ADD)); + + List deltaIndexEntries = new ArrayList<>(); + deltaIndexEntries.add(createDvIndexEntry("dv1", ADD, Arrays.asList("f2"))); + + assertThat(buildBaseEntriesWithDV(baseEntries, deltaIndexEntries)) + .containsExactlyInAnyOrder( + createFileEntryWithDV("f1", ADD, null), + createFileEntryWithDV("f2", ADD, "dv1")); + } + + { + // Scene 2: skip delete dv + List baseEntries = new ArrayList<>(); + baseEntries.add(createFileEntry("f1", ADD)); + baseEntries.add(createFileEntry("f2", ADD)); + + List deltaIndexEntries = new ArrayList<>(); + deltaIndexEntries.add(createDvIndexEntry("dv1", DELETE, Arrays.asList("f2"))); + + assertThat(buildBaseEntriesWithDV(baseEntries, deltaIndexEntries)) + .containsExactlyInAnyOrder( + createFileEntryWithDV("f1", ADD, null), + createFileEntryWithDV("f2", ADD, null)); + } + } + + @Test + public void testBuildDeltaEntriesWithDV() { + { + // Scene 1: update f2's dv + List baseEntries = new ArrayList<>(); + baseEntries.add(createFileEntryWithDV("f1", ADD, "dv1")); + baseEntries.add(createFileEntryWithDV("f2", ADD, null)); + + List deltaEntries = new ArrayList<>(); + deltaEntries.add(createFileEntry("f2", DELETE)); + deltaEntries.add(createFileEntry("f2_new", ADD)); + + List deltaIndexEntries = new ArrayList<>(); + deltaIndexEntries.add(createDvIndexEntry("dv2", ADD, Arrays.asList("f2_new"))); + + assertThat(buildDeltaEntriesWithDV(baseEntries, deltaEntries, deltaIndexEntries)) + .containsExactlyInAnyOrder( + createFileEntryWithDV("f2", DELETE, null), + createFileEntryWithDV("f2_new", ADD, "dv2")); + } + + { + // Scene 2: update f2 and merge f1's dv + List baseEntries = new ArrayList<>(); + baseEntries.add(createFileEntryWithDV("f1", ADD, "dv1")); + baseEntries.add(createFileEntryWithDV("f2", ADD, null)); + + List deltaEntries = new ArrayList<>(); + deltaEntries.add(createFileEntry("f2", DELETE)); + deltaEntries.add(createFileEntry("f2_new", ADD)); + deltaEntries.add(createFileEntry("f3", ADD)); + + List deltaIndexEntries = new ArrayList<>(); + deltaIndexEntries.add(createDvIndexEntry("dv1", DELETE, Arrays.asList("f1"))); + deltaIndexEntries.add(createDvIndexEntry("dv2", ADD, Arrays.asList("f1", "f2_new"))); + + assertThat(buildDeltaEntriesWithDV(baseEntries, deltaEntries, deltaIndexEntries)) + .containsExactlyInAnyOrder( + createFileEntryWithDV("f1", DELETE, "dv1"), + createFileEntryWithDV("f1", ADD, "dv2"), + createFileEntryWithDV("f2", DELETE, null), + createFileEntryWithDV("f2_new", ADD, "dv2"), + createFileEntryWithDV("f3", ADD, null)); + } + + { + // Scene 3: update f2 (with dv) and merge f1's dv + List baseEntries = new ArrayList<>(); + baseEntries.add(createFileEntryWithDV("f1", ADD, "dv1")); + baseEntries.add(createFileEntryWithDV("f2", ADD, "dv2")); + + List deltaEntries = new ArrayList<>(); + deltaEntries.add(createFileEntry("f2", DELETE)); + deltaEntries.add(createFileEntry("f2_new", ADD)); + deltaEntries.add(createFileEntry("f3", ADD)); + + List deltaIndexEntries = new ArrayList<>(); + deltaIndexEntries.add(createDvIndexEntry("dv1", DELETE, Arrays.asList("f1"))); + deltaIndexEntries.add(createDvIndexEntry("dv2", DELETE, Arrays.asList("f2"))); + deltaIndexEntries.add(createDvIndexEntry("dv3", ADD, Arrays.asList("f1", "f2_new"))); + + assertThat(buildDeltaEntriesWithDV(baseEntries, deltaEntries, deltaIndexEntries)) + .containsExactlyInAnyOrder( + createFileEntryWithDV("f1", DELETE, "dv1"), + createFileEntryWithDV("f1", ADD, "dv3"), + createFileEntryWithDV("f2", DELETE, "dv2"), + createFileEntryWithDV("f2_new", ADD, "dv3"), + createFileEntryWithDV("f3", ADD, null)); + } + + { + // Scene 4: full compact + List baseEntries = new ArrayList<>(); + baseEntries.add(createFileEntryWithDV("f1", ADD, null)); + baseEntries.add(createFileEntryWithDV("f2", ADD, "dv1")); + baseEntries.add(createFileEntryWithDV("f3", ADD, "dv1")); + baseEntries.add(createFileEntryWithDV("f4", ADD, "dv2")); + + List deltaEntries = new ArrayList<>(); + deltaEntries.add(createFileEntry("f1", DELETE)); + deltaEntries.add(createFileEntry("f2", DELETE)); + deltaEntries.add(createFileEntry("f3", DELETE)); + deltaEntries.add(createFileEntry("f4", DELETE)); + deltaEntries.add(createFileEntry("f5_compact", ADD)); + + List deltaIndexEntries = new ArrayList<>(); + deltaIndexEntries.add(createDvIndexEntry("dv1", DELETE, Arrays.asList("f2", "f3"))); + deltaIndexEntries.add(createDvIndexEntry("dv2", DELETE, Arrays.asList("f4"))); + + assertThat(buildDeltaEntriesWithDV(baseEntries, deltaEntries, deltaIndexEntries)) + .containsExactlyInAnyOrder( + createFileEntryWithDV("f1", DELETE, null), + createFileEntryWithDV("f2", DELETE, "dv1"), + createFileEntryWithDV("f3", DELETE, "dv1"), + createFileEntryWithDV("f4", DELETE, "dv2"), + createFileEntryWithDV("f5_compact", ADD, null)); + } + + { + // Scene 5: merge into with update, delete and insert + List baseEntries = new ArrayList<>(); + baseEntries.add(createFileEntryWithDV("f1", ADD, null)); + baseEntries.add(createFileEntryWithDV("f2", ADD, null)); + baseEntries.add(createFileEntryWithDV("f3", ADD, "dv1")); + baseEntries.add(createFileEntryWithDV("f4", ADD, "dv1")); + baseEntries.add(createFileEntryWithDV("f5", ADD, "dv2")); + + List deltaEntries = new ArrayList<>(); + deltaEntries.add(createFileEntry("f2", DELETE)); + deltaEntries.add(createFileEntry("f3", DELETE)); + deltaEntries.add(createFileEntry("f3_new", ADD)); + deltaEntries.add(createFileEntry("f7", ADD)); + + List deltaIndexEntries = new ArrayList<>(); + deltaIndexEntries.add(createDvIndexEntry("dv1", DELETE, Arrays.asList("f3", "f4"))); + deltaIndexEntries.add(createDvIndexEntry("dv2", DELETE, Arrays.asList("f5"))); + deltaIndexEntries.add(createDvIndexEntry("dv3", ADD, Arrays.asList("f1", "f4", "f5"))); + + assertThat(buildDeltaEntriesWithDV(baseEntries, deltaEntries, deltaIndexEntries)) + .containsExactlyInAnyOrder( + createFileEntryWithDV("f1", DELETE, null), + createFileEntryWithDV("f1", ADD, "dv3"), + createFileEntryWithDV("f2", DELETE, null), + createFileEntryWithDV("f3", DELETE, "dv1"), + createFileEntryWithDV("f3_new", ADD, null), + createFileEntryWithDV("f4", DELETE, "dv1"), + createFileEntryWithDV("f4", ADD, "dv3"), + createFileEntryWithDV("f5", DELETE, "dv2"), + createFileEntryWithDV("f5", ADD, "dv3"), + createFileEntryWithDV("f7", ADD, null)); + } + } + + @Test + public void testConflictDeletionWithDV() { + { + // Scene 1: base -------------> update2 (conflict) + // f1 ^ + // | + // update1 (finished) + // + List update1Entries = new ArrayList<>(); + update1Entries.add(createFileEntryWithDV("f1", ADD, "dv1")); + + List update2DeltaEntries = new ArrayList<>(); + + List update2DeltaIndexEntries = new ArrayList<>(); + update2DeltaIndexEntries.add(createDvIndexEntry("dv2", ADD, Arrays.asList("f1"))); + + List update2DeltaEntriesWithDV = + buildDeltaEntriesWithDV( + update1Entries, update2DeltaEntries, update2DeltaIndexEntries); + assertThat(update2DeltaEntriesWithDV) + .containsExactlyInAnyOrder( + createFileEntryWithDV("f1", DELETE, null), + createFileEntryWithDV("f1", ADD, "dv2")); + assertConflict(update1Entries, update2DeltaEntriesWithDV); + } + + { + // Scene 2: base -------------> update2 (conflict) + // ^ + // | + // update1 (finished) + // + List update1Entries = new ArrayList<>(); + update1Entries.add(createFileEntryWithDV("f1", ADD, "dv1")); + + List update2DeltaEntries = new ArrayList<>(); + + List update2DeltaIndexEntries = new ArrayList<>(); + update2DeltaIndexEntries.add(createDvIndexEntry("dv0", DELETE, Arrays.asList("f1"))); + update2DeltaIndexEntries.add(createDvIndexEntry("dv2", ADD, Arrays.asList("f1"))); + + List update2DeltaEntriesWithDV = + buildDeltaEntriesWithDV( + update1Entries, update2DeltaEntries, update2DeltaIndexEntries); + assertThat(update2DeltaEntriesWithDV) + .containsExactlyInAnyOrder( + createFileEntryWithDV("f1", DELETE, "dv0"), + createFileEntryWithDV("f1", ADD, "dv2")); + assertConflict(update1Entries, update2DeltaEntriesWithDV); + } + + { + // Scene 3: base -------------> update2 (conflict) + // ^ <-f1, -dv0>, <+f3, null> + // | + // update1 (finished) + // <-f1, -dv0>, <+f2, dv1> + List update1Entries = new ArrayList<>(); + update1Entries.add(createFileEntryWithDV("f2", ADD, "dv1")); + + List update2DeltaEntries = new ArrayList<>(); + update2DeltaEntries.add(createFileEntry("f1", DELETE)); + update2DeltaEntries.add(createFileEntry("f3", ADD)); + + List update2DeltaIndexEntries = new ArrayList<>(); + update2DeltaIndexEntries.add(createDvIndexEntry("dv0", DELETE, Arrays.asList("f1"))); + + List update2DeltaEntriesWithDV = + buildDeltaEntriesWithDV( + update1Entries, update2DeltaEntries, update2DeltaIndexEntries); + assertThat(update2DeltaEntriesWithDV) + .containsExactlyInAnyOrder( + createFileEntryWithDV("f1", DELETE, "dv0"), + createFileEntryWithDV("f3", ADD, null)); + assertConflict(update1Entries, update2DeltaEntriesWithDV); + } + } + + private SimpleFileEntry createFileEntry(String fileName, FileKind kind) { + return new SimpleFileEntry( + kind, + EMPTY_ROW, + 0, + 1, + 0, + fileName, + Collections.emptyList(), + null, + EMPTY_ROW, + EMPTY_ROW, + null); + } + + private SimpleFileEntryWithDV createFileEntryWithDV( + String fileName, FileKind kind, @Nullable String dvFileName) { + return new SimpleFileEntryWithDV(createFileEntry(fileName, kind), dvFileName); + } + + private IndexManifestEntry createDvIndexEntry( + String fileName, FileKind kind, List fileNames) { + LinkedHashMap dvRanges = new LinkedHashMap<>(); + for (String name : fileNames) { + dvRanges.put(name, new DeletionVectorMeta(name, 1, 1, 1L)); + } + return new IndexManifestEntry( + kind, + EMPTY_ROW, + 0, + new IndexFileMeta( + DELETION_VECTORS_INDEX, fileName, 11, dvRanges.size(), dvRanges, null)); + } + + private void assertConflict( + List baseEntries, List deltaEntries) { + ArrayList simpleFileEntryWithDVS = new ArrayList<>(baseEntries); + simpleFileEntryWithDVS.addAll(deltaEntries); + Collection merged = FileEntry.mergeEntries(simpleFileEntryWithDVS); + int deleteCount = 0; + for (SimpleFileEntry simpleFileEntryWithDV : merged) { + if (simpleFileEntryWithDV.kind().equals(FileKind.DELETE)) { + deleteCount++; + } + } + assert (deleteCount > 0); + } +} diff --git a/paimon-core/src/test/resources/compatibility/manifest-committable-v10 b/paimon-core/src/test/resources/compatibility/manifest-committable-v10 new file mode 100644 index 000000000000..0ae6e7bfe366 Binary files /dev/null and b/paimon-core/src/test/resources/compatibility/manifest-committable-v10 differ diff --git a/paimon-filesystems/paimon-oss-impl/src/main/java/org/apache/paimon/oss/OSSFileIO.java b/paimon-filesystems/paimon-oss-impl/src/main/java/org/apache/paimon/oss/OSSFileIO.java index 4c457598e778..d18ddcc94b26 100644 --- a/paimon-filesystems/paimon-oss-impl/src/main/java/org/apache/paimon/oss/OSSFileIO.java +++ b/paimon-filesystems/paimon-oss-impl/src/main/java/org/apache/paimon/oss/OSSFileIO.java @@ -22,10 +22,14 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.options.Options; import org.apache.paimon.utils.IOUtils; +import org.apache.paimon.utils.ReflectionUtils; +import com.aliyun.oss.OSSClient; +import com.aliyun.oss.common.comm.ServiceClient; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem; +import org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystemStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +60,7 @@ public class OSSFileIO extends HadoopCompliantFileIO { private static final String OSS_ACCESS_KEY_ID = "fs.oss.accessKeyId"; private static final String OSS_ACCESS_KEY_SECRET = "fs.oss.accessKeySecret"; private static final String OSS_SECURITY_TOKEN = "fs.oss.securityToken"; + private static final String OSS_SECOND_LEVEL_DOMAIN_ENABLED = "fs.oss.sld.enabled"; private static final Map CASE_SENSITIVE_KEYS = new HashMap() { @@ -137,6 +142,11 @@ protected AliyunOSSFileSystem createFileSystem(org.apache.hadoop.fs.Path path) { } catch (IOException e) { throw new UncheckedIOException(e); } + + if (hadoopOptions.getBoolean(OSS_SECOND_LEVEL_DOMAIN_ENABLED, false)) { + enableSecondLevelDomain(fs); + } + return fs; }; @@ -156,6 +166,19 @@ public void close() { } } + public void enableSecondLevelDomain(AliyunOSSFileSystem fs) { + AliyunOSSFileSystemStore store = fs.getStore(); + try { + OSSClient ossClient = ReflectionUtils.getPrivateFieldValue(store, "ossClient"); + ServiceClient serviceClient = + ReflectionUtils.getPrivateFieldValue(ossClient, "serviceClient"); + serviceClient.getClientConfiguration().setSLDEnabled(true); + } catch (Exception e) { + LOG.error("Failed to enable second level domain.", e); + throw new RuntimeException("Failed to enable second level domain.", e); + } + } + private static class CacheKey { private final Options options; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java index ea6008d87277..da612bd337b7 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java @@ -31,8 +31,6 @@ import org.apache.paimon.flink.sink.StoreSinkWriteState; import org.apache.paimon.flink.sink.StoreSinkWriteStateImpl; import org.apache.paimon.flink.utils.RuntimeContextUtils; -import org.apache.paimon.memory.HeapMemorySegmentPool; -import org.apache.paimon.memory.MemoryPoolFactory; import org.apache.paimon.options.Options; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; @@ -70,11 +68,10 @@ public class CdcRecordStoreMultiWriteOperator private static final long serialVersionUID = 1L; - private final StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider; + private final StoreSinkWrite.Provider storeSinkWriteProvider; private final String initialCommitUser; private final CatalogLoader catalogLoader; - private MemoryPoolFactory memoryPoolFactory; private Catalog catalog; private Map tables; private StoreSinkWriteState state; @@ -85,7 +82,7 @@ public class CdcRecordStoreMultiWriteOperator private CdcRecordStoreMultiWriteOperator( StreamOperatorParameters parameters, CatalogLoader catalogLoader, - StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider, + StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser, Options options) { super(parameters, options); @@ -134,19 +131,6 @@ public void processElement(StreamRecord element) throws Exce int retryCnt = table.coreOptions().toConfiguration().get(MAX_RETRY_NUM_TIMES); boolean skipCorruptRecord = table.coreOptions().toConfiguration().get(SKIP_CORRUPT_RECORD); - // all table write should share one write buffer so that writers can preempt memory - // from those of other tables - if (memoryPoolFactory == null) { - memoryPoolFactory = - new MemoryPoolFactory( - memoryPool != null - ? memoryPool - // currently, the options of all tables are the same in CDC - : new HeapMemorySegmentPool( - table.coreOptions().writeBufferSize(), - table.coreOptions().pageSize())); - } - StoreSinkWrite write = writes.computeIfAbsent( tableId, @@ -294,13 +278,13 @@ public String commitUser() { /** {@link StreamOperatorFactory} of {@link CdcRecordStoreMultiWriteOperator}. */ public static class Factory extends PrepareCommitOperator.Factory { - private final StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider; + private final StoreSinkWrite.Provider storeSinkWriteProvider; private final String initialCommitUser; private final CatalogLoader catalogLoader; public Factory( CatalogLoader catalogLoader, - StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider, + StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser, Options options) { super(options); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java index a67e74ef55a4..c9ae95b69825 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java @@ -91,7 +91,7 @@ public FlinkCdcMultiTableSink( this.tableFilter = tableFilter; } - private StoreSinkWrite.WithWriteBufferProvider createWriteProvider() { + private StoreSinkWrite.Provider createWriteProvider() { // for now, no compaction for multiplexed sink return (table, commitUser, state, ioManager, memoryPoolFactory, metricGroup) -> new StoreSinkWriteImpl( @@ -118,7 +118,7 @@ public DataStreamSink sinkFrom(DataStream input) { public DataStreamSink sinkFrom( DataStream input, String commitUser, - StoreSinkWrite.WithWriteBufferProvider sinkProvider) { + StoreSinkWrite.Provider sinkProvider) { StreamExecutionEnvironment env = input.getExecutionEnvironment(); assertStreamingConfiguration(env); MultiTableCommittableTypeInfo typeInfo = new MultiTableCommittableTypeInfo(); @@ -151,8 +151,7 @@ public DataStreamSink sinkFrom( } protected OneInputStreamOperatorFactory - createWriteOperator( - StoreSinkWrite.WithWriteBufferProvider writeProvider, String commitUser) { + createWriteOperator(StoreSinkWrite.Provider writeProvider, String commitUser) { return new CdcRecordStoreMultiWriteOperator.Factory( catalogLoader, writeProvider, commitUser, new Options()); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java index d5c33da412ed..f5cd33f01929 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java @@ -270,7 +270,7 @@ private void buildDividedCdcSink() { case BUCKET_UNAWARE: buildForUnawareBucket(table, converted); break; - case CROSS_PARTITION: + case KEY_DYNAMIC: default: throw new UnsupportedOperationException( "Unsupported bucket mode: " + bucketMode); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java index 4436aa392d42..ee64162ad910 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java @@ -516,8 +516,9 @@ public void testMultiTableUpdateColumnType() throws Exception { // first table data = new HashMap<>(); data.put("pt", "1"); - data.put("k", "123456789876543211"); + data.put("k", "2"); data.put("v", "varchar"); + data.put("v2", "hello"); expected = CdcMultiplexRecord.fromCdcRecord( databaseName, @@ -528,7 +529,7 @@ public void testMultiTableUpdateColumnType() throws Exception { assertThat(actual).isNull(); schemaManager = new SchemaManager(table1.fileIO(), table1.location()); - schemaManager.commitChanges(SchemaChange.updateColumnType("k", DataTypes.BIGINT())); + schemaManager.commitChanges(SchemaChange.addColumn("v2", DataTypes.STRING())); actual = runner.take(); assertThat(actual).isEqualTo(expected); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java index f00229d99890..d27b919fa13d 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java @@ -256,7 +256,7 @@ private OneInputStreamOperatorTestHarness createTestHarn CdcRecordStoreWriteOperator.Factory operatorFactory = new CdcRecordStoreWriteOperator.Factory( table, - (t, commitUser, state, ioManager, memoryPool, metricGroup) -> + (t, commitUser, state, ioManager, memoryPoolFactory, metricGroup) -> new StoreSinkWriteImpl( t, commitUser, @@ -265,7 +265,7 @@ private OneInputStreamOperatorTestHarness createTestHarn false, false, true, - memoryPool, + memoryPoolFactory, metricGroup), commitUser); TypeSerializer inputSerializer = new JavaSerializer<>(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java index 3b1013835054..42e20f5aabcd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactAction.java @@ -19,9 +19,13 @@ package org.apache.paimon.flink.action; import org.apache.paimon.CoreOptions; +import org.apache.paimon.append.cluster.IncrementalClusterManager; +import org.apache.paimon.compact.CompactUnit; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.flink.FlinkConnectorOptions; +import org.apache.paimon.flink.cluster.IncrementalClusterSplitSource; +import org.apache.paimon.flink.cluster.RewriteIncrementalClusterCommittableOperator; import org.apache.paimon.flink.compact.AppendTableCompactBuilder; import org.apache.paimon.flink.postpone.PostponeBucketCompactSplitSource; import org.apache.paimon.flink.postpone.RewritePostponeBucketCommittableOperator; @@ -32,7 +36,10 @@ import org.apache.paimon.flink.sink.FixedBucketSink; import org.apache.paimon.flink.sink.FlinkSinkBuilder; import org.apache.paimon.flink.sink.FlinkStreamPartitioner; +import org.apache.paimon.flink.sink.RowAppendTableSink; import org.apache.paimon.flink.sink.RowDataChannelComputer; +import org.apache.paimon.flink.sorter.TableSortInfo; +import org.apache.paimon.flink.sorter.TableSorter; import org.apache.paimon.flink.source.CompactorSourceBuilder; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.options.Options; @@ -43,6 +50,8 @@ import org.apache.paimon.predicate.PredicateProjectionConverter; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.InternalRowPartitionComputer; import org.apache.paimon.utils.Pair; @@ -67,6 +76,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import static org.apache.paimon.partition.PartitionPredicate.createBinaryPartitions; import static org.apache.paimon.partition.PartitionPredicate.createPartitionPredicate; @@ -143,8 +153,12 @@ private boolean buildImpl() throws Exception { if (fileStoreTable.coreOptions().bucket() == BucketMode.POSTPONE_BUCKET) { return buildForPostponeBucketCompaction(env, fileStoreTable, isStreaming); } else if (fileStoreTable.bucketMode() == BucketMode.BUCKET_UNAWARE) { - buildForAppendTableCompact(env, fileStoreTable, isStreaming); - return true; + if (fileStoreTable.coreOptions().clusteringIncrementalEnabled()) { + return buildForIncrementalClustering(env, fileStoreTable, isStreaming); + } else { + buildForAppendTableCompact(env, fileStoreTable, isStreaming); + return true; + } } else { buildForBucketedTableCompact(env, fileStoreTable, isStreaming); return true; @@ -198,6 +212,124 @@ private void buildForAppendTableCompact( builder.build(); } + private boolean buildForIncrementalClustering( + StreamExecutionEnvironment env, FileStoreTable table, boolean isStreaming) + throws Exception { + checkArgument(!isStreaming, "Incremental clustering currently only supports batch mode"); + + IncrementalClusterManager incrementalClusterManager = + new IncrementalClusterManager(table, getPartitionPredicate()); + + // non-full strategy as default for incremental clustering + if (fullCompaction == null) { + fullCompaction = false; + } + Options options = new Options(table.options()); + int localSampleMagnification = table.coreOptions().getLocalSampleMagnification(); + if (localSampleMagnification < 20) { + throw new IllegalArgumentException( + String.format( + "the config '%s=%d' should not be set too small, greater than or equal to 20 is needed.", + CoreOptions.SORT_COMPACTION_SAMPLE_MAGNIFICATION.key(), + localSampleMagnification)); + } + String commitUser = CoreOptions.createCommitUser(options); + InternalRowPartitionComputer partitionComputer = + new InternalRowPartitionComputer( + table.coreOptions().partitionDefaultName(), + table.store().partitionType(), + table.partitionKeys().toArray(new String[0]), + table.coreOptions().legacyPartitionName()); + + // 1. pick cluster files for each partition + Map compactUnits = + incrementalClusterManager.prepareForCluster(fullCompaction); + if (compactUnits.isEmpty()) { + LOGGER.warn( + "No partition needs to be incrementally clustered. " + + "Please set '--compact_strategy full' if you need forcibly trigger the cluster." + + "Please set '--force_start_flink_job true' if you need forcibly start a flink job."); + return false; + } + + Map, CommitMessage>> partitionSplits = + incrementalClusterManager.toSplitsAndRewriteDvFiles(compactUnits); + + // 2. read,sort and write in partition + List> dataStreams = new ArrayList<>(); + + for (Map.Entry, CommitMessage>> entry : + partitionSplits.entrySet()) { + BinaryRow partition = entry.getKey(); + List splits = entry.getValue().getKey(); + CommitMessage dvCommitMessage = entry.getValue().getRight(); + LinkedHashMap partitionSpec = + partitionComputer.generatePartValues(partition); + + // 2.1 generate source for current partition + Pair, DataStream> sourcePair = + IncrementalClusterSplitSource.buildSource( + env, + table, + partitionSpec, + splits, + dvCommitMessage, + options.get(FlinkConnectorOptions.SCAN_PARALLELISM)); + + // 2.2 cluster in partition + Integer sinkParallelism = options.get(FlinkConnectorOptions.SINK_PARALLELISM); + if (sinkParallelism == null) { + sinkParallelism = sourcePair.getLeft().getParallelism(); + } + TableSortInfo sortInfo = + new TableSortInfo.Builder() + .setSortColumns(incrementalClusterManager.clusterKeys()) + .setSortStrategy(incrementalClusterManager.clusterCurve()) + .setSinkParallelism(sinkParallelism) + .setLocalSampleSize(sinkParallelism * localSampleMagnification) + .setGlobalSampleSize(sinkParallelism * 1000) + .setRangeNumber(sinkParallelism * 10) + .build(); + DataStream sorted = + TableSorter.getSorter(env, sourcePair.getLeft(), table, sortInfo).sort(); + + // 2.3 write and then reorganize the committable + // set parallelism to null, and it'll forward parallelism when doWrite() + RowAppendTableSink sink = new RowAppendTableSink(table, null, null, null); + DataStream written = + sink.doWrite( + FlinkSinkBuilder.mapToInternalRow(sorted, table.rowType()), + commitUser, + null); + DataStream clusterCommittable = + written.forward() + .transform( + "Rewrite cluster committable", + new CommittableTypeInfo(), + new RewriteIncrementalClusterCommittableOperator( + table, + compactUnits.entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, + unit -> + unit.getValue() + .outputLevel())))) + .setParallelism(written.getParallelism()); + dataStreams.add(clusterCommittable); + dataStreams.add(sourcePair.getRight()); + } + + // 3. commit + RowAppendTableSink sink = new RowAppendTableSink(table, null, null, null); + DataStream dataStream = dataStreams.get(0); + for (int i = 1; i < dataStreams.size(); i++) { + dataStream = dataStream.union(dataStreams.get(i)); + } + sink.doCommit(dataStream, commitUser); + return true; + } + protected PartitionPredicate getPartitionPredicate() throws Exception { checkArgument( partitions == null || whereSql == null, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/IncrementalClusterSplitSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/IncrementalClusterSplitSource.java new file mode 100644 index 000000000000..132dab660e83 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/IncrementalClusterSplitSource.java @@ -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. + */ + +package org.apache.paimon.flink.cluster; + +import org.apache.paimon.flink.LogicalTypeConversion; +import org.apache.paimon.flink.sink.Committable; +import org.apache.paimon.flink.sink.CommittableTypeInfo; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.operator.ReadOperator; +import org.apache.paimon.flink.utils.JavaTypeInfo; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.utils.Pair; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.transformations.PartitionTransformation; +import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Map; + +/** Source for Incremental Clustering. */ +public class IncrementalClusterSplitSource extends AbstractNonCoordinatedSource { + + private static final long serialVersionUID = 2L; + + private final List splits; + + public IncrementalClusterSplitSource(List splits) { + this.splits = splits; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) + throws Exception { + return new IncrementalClusterSplitSource.Reader(); + } + + private class Reader extends AbstractNonCoordinatedSourceReader { + + @Override + public InputStatus pollNext(ReaderOutput output) { + for (Split split : splits) { + DataSplit dataSplit = (DataSplit) split; + output.collect(dataSplit); + } + return InputStatus.END_OF_INPUT; + } + } + + public static Pair, DataStream> buildSource( + StreamExecutionEnvironment env, + FileStoreTable table, + Map partitionSpec, + List splits, + @Nullable CommitMessage dvCommitMessage, + @Nullable Integer parallelism) { + DataStream source = + env.fromSource( + new IncrementalClusterSplitSource((List) splits), + WatermarkStrategy.noWatermarks(), + String.format( + "Incremental-cluster split generator: %s - %s", + table.fullName(), partitionSpec), + new JavaTypeInfo<>(Split.class)) + .forceNonParallel(); + + PartitionTransformation partitioned = + new PartitionTransformation<>( + source.getTransformation(), new RebalancePartitioner<>()); + if (parallelism != null) { + partitioned.setParallelism(parallelism); + } + + return Pair.of( + new DataStream<>(source.getExecutionEnvironment(), partitioned) + .transform( + String.format( + "Incremental-cluster reader: %s - %s", + table.fullName(), partitionSpec), + InternalTypeInfo.of( + LogicalTypeConversion.toLogicalType(table.rowType())), + new ReadOperator(table::newRead, null, null)), + source.forward() + .transform( + "Remove files to be clustered", + new CommittableTypeInfo(), + new RemoveClusterBeforeFilesOperator(dvCommitMessage)) + .forceNonParallel()); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RemoveClusterBeforeFilesOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RemoveClusterBeforeFilesOperator.java new file mode 100644 index 000000000000..83b41940bc4e --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RemoveClusterBeforeFilesOperator.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.cluster; + +import org.apache.paimon.flink.sink.Committable; +import org.apache.paimon.flink.utils.BoundedOneInputOperator; +import org.apache.paimon.io.CompactIncrement; +import org.apache.paimon.io.DataIncrement; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.Split; + +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +import javax.annotation.Nullable; + +import java.util.Collections; + +/** Operator used with {@link IncrementalClusterSplitSource}, to remove files to be clustered. */ +public class RemoveClusterBeforeFilesOperator extends BoundedOneInputOperator { + + private static final long serialVersionUID = 2L; + + private final @Nullable CommitMessage dvCommitMessage; + + public RemoveClusterBeforeFilesOperator(@Nullable CommitMessage dvCommitMessage) { + this.dvCommitMessage = dvCommitMessage; + } + + @Override + public void processElement(StreamRecord element) throws Exception { + DataSplit dataSplit = (DataSplit) element.getValue(); + CommitMessageImpl message = + new CommitMessageImpl( + dataSplit.partition(), + dataSplit.bucket(), + dataSplit.totalBuckets(), + DataIncrement.emptyIncrement(), + new CompactIncrement( + dataSplit.dataFiles(), + Collections.emptyList(), + Collections.emptyList())); + output.collect( + new StreamRecord<>( + new Committable(Long.MAX_VALUE, Committable.Kind.FILE, message))); + } + + @Override + public void endInput() throws Exception { + emitDvIndexCommitMessages(Long.MAX_VALUE); + } + + private void emitDvIndexCommitMessages(long checkpointId) { + if (dvCommitMessage != null) { + output.collect( + new StreamRecord<>( + new Committable(checkpointId, Committable.Kind.FILE, dvCommitMessage))); + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RewriteIncrementalClusterCommittableOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RewriteIncrementalClusterCommittableOperator.java new file mode 100644 index 000000000000..01c2deba6466 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/cluster/RewriteIncrementalClusterCommittableOperator.java @@ -0,0 +1,117 @@ +/* + * 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. + */ + +package org.apache.paimon.flink.cluster; + +import org.apache.paimon.append.cluster.IncrementalClusterManager; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.flink.sink.Committable; +import org.apache.paimon.flink.utils.BoundedOneInputOperator; +import org.apache.paimon.io.CompactIncrement; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.io.DataIncrement; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.CommitMessageImpl; + +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.utils.Preconditions.checkArgument; + +/** Rewrite committable for new files written after clustered. */ +public class RewriteIncrementalClusterCommittableOperator + extends BoundedOneInputOperator { + + protected static final Logger LOG = + LoggerFactory.getLogger(RewriteIncrementalClusterCommittableOperator.class); + private static final long serialVersionUID = 1L; + + private final FileStoreTable table; + private final Map outputLevels; + + private transient Map> partitionFiles; + + public RewriteIncrementalClusterCommittableOperator( + FileStoreTable table, Map outputLevels) { + this.table = table; + this.outputLevels = outputLevels; + } + + @Override + public void open() throws Exception { + partitionFiles = new HashMap<>(); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + Committable committable = element.getValue(); + if (committable.kind() != Committable.Kind.FILE) { + output.collect(element); + } + + CommitMessageImpl message = (CommitMessageImpl) committable.wrappedCommittable(); + checkArgument(message.bucket() == 0); + BinaryRow partition = message.partition(); + partitionFiles + .computeIfAbsent(partition, file -> new ArrayList<>()) + .addAll(message.newFilesIncrement().newFiles()); + } + + @Override + public void endInput() throws Exception { + emitAll(Long.MAX_VALUE); + } + + protected void emitAll(long checkpointId) { + for (Map.Entry> partitionEntry : partitionFiles.entrySet()) { + BinaryRow partition = partitionEntry.getKey(); + // upgrade the clustered file to outputLevel + List clusterAfter = + IncrementalClusterManager.upgrade( + partitionEntry.getValue(), outputLevels.get(partition)); + LOG.info( + "Partition {}: upgrade file level to {}", + partition, + outputLevels.get(partition)); + CompactIncrement compactIncrement = + new CompactIncrement( + Collections.emptyList(), clusterAfter, Collections.emptyList()); + CommitMessageImpl clusterMessage = + new CommitMessageImpl( + partition, + // bucket 0 is bucket for unaware-bucket table + // for compatibility with the old design + 0, + table.coreOptions().bucket(), + DataIncrement.emptyIncrement(), + compactIncrement); + output.collect( + new StreamRecord<>( + new Committable(checkpointId, Committable.Kind.FILE, clusterMessage))); + } + + partitionFiles.clear(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperator.java index 674dd59e7ef3..827d79ca5aba 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperator.java @@ -128,9 +128,10 @@ public void processElement(StreamRecord record) throws Exception { new DataIncrement( skippedFiles, message.newFilesIncrement().deletedFiles(), - message.newFilesIncrement().changelogFiles()), - message.compactIncrement(), - message.indexIncrement()); + message.newFilesIncrement().changelogFiles(), + message.newFilesIncrement().newIndexFiles(), + message.newFilesIncrement().deletedIndexFiles()), + message.compactIncrement()); if (!newMessage.isEmpty()) { Committable newCommittable = new Committable(committable.checkpointId(), Committable.Kind.FILE, newMessage); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactWorkerOperator.java index 8d8a668f7930..0e71f9186546 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendPreCommitCompactWorkerOperator.java @@ -72,7 +72,7 @@ public void open() throws Exception { checkArgument( !coreOptions.dataEvolutionEnabled(), "Data evolution enabled table should not invoke compact yet."); - this.write.withWriteType(SpecialFields.rowTypeWithRowLineage(table.rowType())); + this.write.withWriteType(SpecialFields.rowTypeWithRowTracking(table.rowType())); } this.pathFactory = table.store().pathFactory(); this.fileIO = table.fileIO(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendTableCompactor.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendTableCompactor.java index 4b85eb51693e..1efbccbc199a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendTableCompactor.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/AppendTableCompactor.java @@ -78,7 +78,7 @@ public AppendTableCompactor( CoreOptions coreOptions = table.coreOptions(); this.write = (BaseAppendFileStoreWrite) table.store().newWrite(commitUser); if (coreOptions.rowTrackingEnabled()) { - write.withWriteType(SpecialFields.rowTypeWithRowLineage(table.rowType())); + write.withWriteType(SpecialFields.rowTypeWithRowTracking(table.rowType())); } this.result = new LinkedList<>(); this.compactExecutorsupplier = lazyCompactExecutor; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperator.java index 0190279a8b9f..d33671703164 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperator.java @@ -138,12 +138,15 @@ public void processElement(StreamRecord record) { new DataIncrement( message.newFilesIncrement().newFiles(), message.newFilesIncrement().deletedFiles(), - skippedNewChangelogs), + skippedNewChangelogs, + message.newFilesIncrement().newIndexFiles(), + message.newFilesIncrement().deletedIndexFiles()), new CompactIncrement( message.compactIncrement().compactBefore(), message.compactIncrement().compactAfter(), - skippedCompactChangelogs), - message.indexIncrement()); + skippedCompactChangelogs, + message.compactIncrement().newIndexFiles(), + message.compactIncrement().deletedIndexFiles())); Committable newCommittable = new Committable(committable.checkpointId(), Committable.Kind.FILE, newMessage); output.collect(new StreamRecord<>(Either.Left(newCommittable))); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperator.java index 92e0110f27f2..e6183ccb8a51 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactSortOperator.java @@ -23,7 +23,6 @@ import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataIncrement; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; @@ -95,12 +94,15 @@ public void processElement(StreamRecord record) throws Exception { new DataIncrement( message.newFilesIncrement().newFiles(), message.newFilesIncrement().deletedFiles(), - Collections.emptyList()), + Collections.emptyList(), + message.newFilesIncrement().newIndexFiles(), + message.newFilesIncrement().deletedIndexFiles()), new CompactIncrement( message.compactIncrement().compactBefore(), message.compactIncrement().compactAfter(), - Collections.emptyList()), - message.indexIncrement()); + Collections.emptyList(), + message.compactIncrement().newIndexFiles(), + message.compactIncrement().deletedIndexFiles())); if (!newMessage.isEmpty()) { Committable newCommittable = new Committable(committable.checkpointId(), Committable.Kind.FILE, newMessage); @@ -138,8 +140,8 @@ private void emitAll(long checkpointId) { new CompactIncrement( Collections.emptyList(), Collections.emptyList(), - sortedChangelogs(compactChangelogFiles, partition, bucket)), - new IndexIncrement(Collections.emptyList())); + sortedChangelogs( + compactChangelogFiles, partition, bucket))); Committable newCommittable = new Committable(checkpointId, Committable.Kind.FILE, newMessage); output.collect(new StreamRecord<>(newCommittable)); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCommittableRewriter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCommittableRewriter.java index 2991fb4ecca8..50eb5a079600 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCommittableRewriter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/postpone/PostponeBucketCommittableRewriter.java @@ -27,7 +27,6 @@ import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFilePathFactory; import org.apache.paimon.io.DataIncrement; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.utils.FileStorePathFactory; @@ -137,8 +136,8 @@ private void update(CommitMessageImpl message) { changelogFiles.addAll(message.newFilesIncrement().changelogFiles()); changelogFiles.addAll(message.compactIncrement().changelogFiles()); - newIndexFiles.addAll(message.indexIncrement().newIndexFiles()); - deletedIndexFiles.addAll(message.indexIncrement().deletedIndexFiles()); + newIndexFiles.addAll(message.compactIncrement().newIndexFiles()); + deletedIndexFiles.addAll(message.compactIncrement().deletedIndexFiles()); toDelete.forEach((fileName, path) -> fileIO.deleteQuietly(path)); } @@ -151,8 +150,12 @@ private CommitMessageImpl makeMessage(BinaryRow partition, int bucket) { bucket, totalBuckets, DataIncrement.emptyIncrement(), - new CompactIncrement(compactBefore, realCompactAfter, changelogFiles), - new IndexIncrement(newIndexFiles, deletedIndexFiles)); + new CompactIncrement( + compactBefore, + realCompactAfter, + changelogFiles, + newIndexFiles, + deletedIndexFiles)); } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendTableSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendTableSink.java index b95318df792e..ccad6b35ca1d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendTableSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendTableSink.java @@ -23,6 +23,7 @@ import org.apache.paimon.flink.compact.AppendPreCommitCompactWorkerOperator; import org.apache.paimon.flink.source.AppendBypassCoordinateOperatorFactory; import org.apache.paimon.options.Options; +import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; import org.apache.flink.api.common.RuntimeExecutionMode; @@ -98,7 +99,10 @@ public DataStream doWrite( } boolean enableCompaction = - !table.coreOptions().writeOnly() && !table.coreOptions().dataEvolutionEnabled(); + !table.coreOptions().writeOnly() + && !table.coreOptions().dataEvolutionEnabled() + && !(table.bucketMode() == BucketMode.BUCKET_UNAWARE + && table.coreOptions().clusteringIncrementalEnabled()); boolean isStreamingMode = input.getExecutionEnvironment() .getConfiguration() diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java index 847cf06990fa..4f40a5c5bcb7 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java @@ -224,7 +224,7 @@ public DataStreamSink build() { return buildForFixedBucket(input); case HASH_DYNAMIC: return buildDynamicBucketSink(input, false); - case CROSS_PARTITION: + case KEY_DYNAMIC: return buildDynamicBucketSink(input, true); case BUCKET_UNAWARE: return buildUnawareBucketSink(input); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSinkBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSinkBase.java index bf2bee15dc3a..051bc91a6704 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSinkBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSinkBase.java @@ -44,6 +44,7 @@ import static org.apache.paimon.CoreOptions.CHANGELOG_PRODUCER; import static org.apache.paimon.CoreOptions.CLUSTERING_COLUMNS; +import static org.apache.paimon.CoreOptions.CLUSTERING_INCREMENTAL; import static org.apache.paimon.CoreOptions.CLUSTERING_STRATEGY; import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; import static org.apache.paimon.CoreOptions.MERGE_ENGINE; @@ -134,12 +135,14 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { .forRowData( new DataStream<>( dataStream.getExecutionEnvironment(), - dataStream.getTransformation())) - .clusteringIfPossible( - conf.get(CLUSTERING_COLUMNS), - conf.get(CLUSTERING_STRATEGY), - conf.get(CLUSTERING_SORT_IN_CLUSTER), - conf.get(CLUSTERING_SAMPLE_FACTOR)); + dataStream.getTransformation())); + if (!conf.get(CLUSTERING_INCREMENTAL)) { + builder.clusteringIfPossible( + conf.get(CLUSTERING_COLUMNS), + conf.get(CLUSTERING_STRATEGY), + conf.get(CLUSTERING_SORT_IN_CLUSTER), + conf.get(CLUSTERING_SAMPLE_FACTOR)); + } if (overwrite) { builder.overwrite(staticPartitions); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/GlobalFullCompactionSinkWrite.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/GlobalFullCompactionSinkWrite.java index 62341a180dab..a5702d7c6d0c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/GlobalFullCompactionSinkWrite.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/GlobalFullCompactionSinkWrite.java @@ -21,7 +21,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.memory.MemorySegmentPool; +import org.apache.paimon.memory.MemoryPoolFactory; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.SinkRecord; import org.apache.paimon.utils.SnapshotManager; @@ -74,7 +74,7 @@ public GlobalFullCompactionSinkWrite( boolean waitCompaction, int deltaCommits, boolean isStreaming, - @Nullable MemorySegmentPool memoryPool, + MemoryPoolFactory memoryPoolFactory, MetricGroup metricGroup) { super( table, @@ -84,7 +84,7 @@ public GlobalFullCompactionSinkWrite( ignorePreviousFiles, waitCompaction, isStreaming, - memoryPool, + memoryPoolFactory, metricGroup); this.deltaCommits = deltaCommits; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LookupSinkWrite.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LookupSinkWrite.java index e6f683d4b15d..e8b27083b01c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LookupSinkWrite.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LookupSinkWrite.java @@ -19,15 +19,13 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.data.BinaryRow; -import org.apache.paimon.memory.MemorySegmentPool; +import org.apache.paimon.memory.MemoryPoolFactory; import org.apache.paimon.operation.AbstractFileStoreWrite; import org.apache.paimon.table.FileStoreTable; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import javax.annotation.Nullable; - import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -48,7 +46,7 @@ public LookupSinkWrite( boolean ignorePreviousFiles, boolean waitCompaction, boolean isStreaming, - @Nullable MemorySegmentPool memoryPool, + MemoryPoolFactory memoryPoolFactory, MetricGroup metricGroup) { super( table, @@ -58,7 +56,7 @@ public LookupSinkWrite( ignorePreviousFiles, waitCompaction, isStreaming, - memoryPool, + memoryPoolFactory, metricGroup); this.tableName = table.name(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java index 3687fea6f1c1..7d7ba29760b0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java @@ -165,7 +165,7 @@ public void processElement(StreamRecord element) throws Exception { commitUser, state, getContainingTask().getEnvironment().getIOManager(), - memoryPool, + memoryPoolFactory, getMetricGroup())); if (write.streamingMode()) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java index e66819398ea8..35f5ff15b9ae 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java @@ -18,8 +18,11 @@ package org.apache.paimon.flink.sink; +import org.apache.paimon.CoreOptions; import org.apache.paimon.flink.memory.FlinkMemorySegmentPool; import org.apache.paimon.flink.memory.MemorySegmentAllocator; +import org.apache.paimon.memory.HeapMemorySegmentPool; +import org.apache.paimon.memory.MemoryPoolFactory; import org.apache.paimon.memory.MemorySegmentPool; import org.apache.paimon.options.Options; @@ -51,7 +54,7 @@ public abstract class PrepareCommitOperator extends AbstractStreamOpera private static final long serialVersionUID = 1L; - @Nullable protected transient MemorySegmentPool memoryPool; + protected transient MemoryPoolFactory memoryPoolFactory; @Nullable private transient MemorySegmentAllocator memoryAllocator; protected final Options options; private boolean endOfInput = false; @@ -67,6 +70,8 @@ public void setup( StreamConfig config, Output> output) { super.setup(containingTask, config, output); + + MemorySegmentPool memoryPool; if (options.get(SINK_USE_MANAGED_MEMORY)) { MemoryManager memoryManager = containingTask.getEnvironment().getMemoryManager(); memoryAllocator = new MemorySegmentAllocator(containingTask, memoryManager); @@ -75,7 +80,13 @@ public void setup( computeManagedMemory(this), memoryManager.getPageSize(), memoryAllocator); + } else { + CoreOptions coreOptions = new CoreOptions(options); + memoryPool = + new HeapMemorySegmentPool( + coreOptions.writeBufferSize(), coreOptions.pageSize()); } + memoryPoolFactory = new MemoryPoolFactory(memoryPool); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java index 7676084b94c9..989c8e91dc83 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCompactOperator.java @@ -117,7 +117,7 @@ public void initializeState(StateInitializationContext context) throws Exception commitUser, state, getContainingTask().getEnvironment().getIOManager(), - memoryPool, + memoryPoolFactory, getMetricGroup()); this.writeRefresher = WriterRefresher.create(write.streamingMode(), table, write::replace); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWrite.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWrite.java index d88cbea6f0a3..a7b1434740de 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWrite.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWrite.java @@ -23,7 +23,6 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.memory.MemoryPoolFactory; -import org.apache.paimon.memory.MemorySegmentPool; import org.apache.paimon.operation.WriteRestore; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; @@ -96,7 +95,7 @@ StoreSinkWrite provide( String commitUser, StoreSinkWriteState state, IOManager ioManager, - @Nullable MemorySegmentPool memoryPool, + MemoryPoolFactory memoryPoolFactory, @Nullable MetricGroup metricGroup); } @@ -141,7 +140,7 @@ static StoreSinkWrite.Provider createWriteProvider( if (changelogProducer == CoreOptions.ChangelogProducer.FULL_COMPACTION || deltaCommits >= 0) { int finalDeltaCommits = Math.max(deltaCommits, 1); - return (table, commitUser, state, ioManager, memoryPool, metricGroup) -> { + return (table, commitUser, state, ioManager, memoryPoolFactory, metricGroup) -> { assertNoSinkMaterializer.run(); return new GlobalFullCompactionSinkWrite( table, @@ -152,13 +151,13 @@ static StoreSinkWrite.Provider createWriteProvider( waitCompaction, finalDeltaCommits, isStreaming, - memoryPool, + memoryPoolFactory, metricGroup); }; } if (coreOptions.needLookup()) { - return (table, commitUser, state, ioManager, memoryPool, metricGroup) -> { + return (table, commitUser, state, ioManager, memoryPoolFactory, metricGroup) -> { assertNoSinkMaterializer.run(); return new LookupSinkWrite( table, @@ -168,13 +167,13 @@ static StoreSinkWrite.Provider createWriteProvider( ignorePreviousFiles, waitCompaction, isStreaming, - memoryPool, + memoryPoolFactory, metricGroup); }; } } - return (table, commitUser, state, ioManager, memoryPool, metricGroup) -> { + return (table, commitUser, state, ioManager, memoryPoolFactory, metricGroup) -> { assertNoSinkMaterializer.run(); return new StoreSinkWriteImpl( table, @@ -184,25 +183,8 @@ static StoreSinkWrite.Provider createWriteProvider( ignorePreviousFiles, waitCompaction, isStreaming, - memoryPool, + memoryPoolFactory, metricGroup); }; } - - /** Provider of {@link StoreSinkWrite} that uses given write buffer. */ - @FunctionalInterface - interface WithWriteBufferProvider extends Serializable { - - /** - * TODO: The argument list has become too complicated. Build {@link TableWriteImpl} directly - * in caller and simplify the argument list. - */ - StoreSinkWrite provide( - FileStoreTable table, - String commitUser, - StoreSinkWriteState state, - IOManager ioManager, - @Nullable MemoryPoolFactory memoryPoolFactory, - MetricGroup metricGroup); - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java index 9c65f221aadb..2d3742a95ad2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java @@ -24,9 +24,7 @@ import org.apache.paimon.disk.IOManagerImpl; import org.apache.paimon.flink.metrics.FlinkMetricRegistry; import org.apache.paimon.io.DataFileMeta; -import org.apache.paimon.memory.HeapMemorySegmentPool; import org.apache.paimon.memory.MemoryPoolFactory; -import org.apache.paimon.memory.MemorySegmentPool; import org.apache.paimon.operation.FileStoreWrite; import org.apache.paimon.operation.WriteRestore; import org.apache.paimon.table.FileStoreTable; @@ -46,8 +44,6 @@ import java.util.List; import java.util.concurrent.ExecutorService; -import static org.apache.paimon.utils.Preconditions.checkArgument; - /** Default implementation of {@link StoreSinkWrite}. This writer does not have states. */ public class StoreSinkWriteImpl implements StoreSinkWrite { @@ -59,35 +55,10 @@ public class StoreSinkWriteImpl implements StoreSinkWrite { private final boolean ignorePreviousFiles; private final boolean waitCompaction; private final boolean isStreamingMode; - @Nullable private final MemorySegmentPool memoryPool; - @Nullable private final MemoryPoolFactory memoryPoolFactory; - - protected TableWriteImpl write; - + private final MemoryPoolFactory memoryPoolFactory; @Nullable private final MetricGroup metricGroup; - public StoreSinkWriteImpl( - FileStoreTable table, - String commitUser, - StoreSinkWriteState state, - IOManager ioManager, - boolean ignorePreviousFiles, - boolean waitCompaction, - boolean isStreamingMode, - @Nullable MemorySegmentPool memoryPool, - @Nullable MetricGroup metricGroup) { - this( - table, - commitUser, - state, - ioManager, - ignorePreviousFiles, - waitCompaction, - isStreamingMode, - memoryPool, - null, - metricGroup); - } + protected TableWriteImpl write; public StoreSinkWriteImpl( FileStoreTable table, @@ -99,67 +70,29 @@ public StoreSinkWriteImpl( boolean isStreamingMode, MemoryPoolFactory memoryPoolFactory, @Nullable MetricGroup metricGroup) { - this( - table, - commitUser, - state, - ioManager, - ignorePreviousFiles, - waitCompaction, - isStreamingMode, - null, - memoryPoolFactory, - metricGroup); - } - - private StoreSinkWriteImpl( - FileStoreTable table, - String commitUser, - StoreSinkWriteState state, - IOManager ioManager, - boolean ignorePreviousFiles, - boolean waitCompaction, - boolean isStreamingMode, - @Nullable MemorySegmentPool memoryPool, - @Nullable MemoryPoolFactory memoryPoolFactory, - @Nullable MetricGroup metricGroup) { this.commitUser = commitUser; this.state = state; this.paimonIOManager = new IOManagerImpl(ioManager.getSpillingDirectoriesPaths()); this.ignorePreviousFiles = ignorePreviousFiles; this.waitCompaction = waitCompaction; this.isStreamingMode = isStreamingMode; - this.memoryPool = memoryPool; this.memoryPoolFactory = memoryPoolFactory; this.metricGroup = metricGroup; this.write = newTableWrite(table); } private TableWriteImpl newTableWrite(FileStoreTable table) { - checkArgument( - !(memoryPool != null && memoryPoolFactory != null), - "memoryPool and memoryPoolFactory cannot be set at the same time."); - TableWriteImpl tableWrite = table.newWrite(commitUser, state.getSubtaskId()) .withIOManager(paimonIOManager) .withIgnorePreviousFiles(ignorePreviousFiles) - .withBucketMode(table.bucketMode()); + .withBucketMode(table.bucketMode()) + .withMemoryPoolFactory(memoryPoolFactory); if (metricGroup != null) { tableWrite.withMetricRegistry(new FlinkMetricRegistry(metricGroup)); } - - if (memoryPoolFactory != null) { - return tableWrite.withMemoryPoolFactory(memoryPoolFactory); - } else { - return tableWrite.withMemoryPool( - memoryPool != null - ? memoryPool - : new HeapMemorySegmentPool( - table.coreOptions().writeBufferSize(), - table.coreOptions().pageSize())); - } + return tableWrite; } public void withCompactExecutor(ExecutorService compactExecutor) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java index ca2705926583..caf92cb742c1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/TableWriteOperator.java @@ -94,7 +94,7 @@ public void initializeState(StateInitializationContext context) throws Exception getCommitUser(context), state, getContainingTask().getEnvironment().getIOManager(), - memoryPool, + memoryPoolFactory, getMetricGroup()); if (writeRestore != null) { write.setWriteRestore(writeRestore); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/listener/PartitionMarkDoneListener.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/listener/PartitionMarkDoneListener.java index 5dbd9f3a0961..0283b197d2fa 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/listener/PartitionMarkDoneListener.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/listener/PartitionMarkDoneListener.java @@ -147,9 +147,7 @@ private void markDoneByProcessTime(List committables) { for (ManifestCommittable committable : committables) { for (CommitMessage commitMessage : committable.fileCommittables()) { CommitMessageImpl message = (CommitMessageImpl) commitMessage; - if (waitCompaction - || !message.indexIncrement().isEmpty() - || !message.newFilesIncrement().isEmpty()) { + if (waitCompaction || !message.newFilesIncrement().isEmpty()) { partitions.add(message.partition()); } } @@ -199,9 +197,7 @@ private Tuple2, Boolean> extractPartitionWatermarks( if (watermark != null) { for (CommitMessage commitMessage : committable.fileCommittables()) { CommitMessageImpl message = (CommitMessageImpl) commitMessage; - if (waitCompaction - || !message.indexIncrement().isEmpty() - || !message.newFilesIncrement().isEmpty()) { + if (waitCompaction || !message.newFilesIncrement().isEmpty()) { partitionWatermarks.compute( message.partition(), (partition, old) -> diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplit.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplit.java index 769154b19683..b9ce94c413a2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplit.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplit.java @@ -79,4 +79,17 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(id, split, recordsToSkip); } + + @Override + public String toString() { + return "{" + + "id='" + + id + + '\'' + + ", split=" + + split + + ", recordsToSkip=" + + recordsToSkip + + '}'; + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitGenerator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitGenerator.java index f19ec828e0c8..2d89bbbe520a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitGenerator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceSplitGenerator.java @@ -18,10 +18,11 @@ package org.apache.paimon.flink.source; -import org.apache.paimon.table.source.Split; import org.apache.paimon.table.source.TableScan; import java.util.List; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; /** @@ -30,11 +31,8 @@ */ public class FileStoreSourceSplitGenerator { - /** - * The current Id as a mutable string representation. This covers more values than the integer - * value range, so we should never overflow. - */ - private final char[] currentId = "0000000000".toCharArray(); + private final String uuid = UUID.randomUUID().toString(); + private final AtomicInteger idCounter = new AtomicInteger(1); public List createSplits(TableScan.Plan plan) { return plan.splits().stream() @@ -42,32 +40,7 @@ public List createSplits(TableScan.Plan plan) { .collect(Collectors.toList()); } - public List createSplits(List splits) { - return splits.stream() - .map(s -> new FileStoreSourceSplit(getNextId(), s)) - .collect(Collectors.toList()); - } - protected final String getNextId() { - // because we just increment numbers, we increment the char representation directly, - // rather than incrementing an integer and converting it to a string representation - // every time again (requires quite some expensive conversion logic). - incrementCharArrayByOne(currentId, currentId.length - 1); - return new String(currentId); - } - - private static void incrementCharArrayByOne(char[] array, int pos) { - if (pos < 0) { - throw new RuntimeException("Produce too many splits."); - } - - char c = array[pos]; - c++; - - if (c > '9') { - c = '0'; - incrementCharArrayByOne(array, pos - 1); - } - array[pos] = c; + return uuid + "-" + idCounter.getAndIncrement(); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendTableITCase.java index 582ced3e109c..85ded57f2363 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendTableITCase.java @@ -95,69 +95,69 @@ public void testReadWrite() { } @Test - public void testReadWriteWithLineage() { - batchSql("INSERT INTO append_table_lineage VALUES (1, 'AAA'), (2, 'BBB')"); - List rows = batchSql("SELECT * FROM append_table_lineage$row_lineage"); + public void testReadWriteWithRowTracking() { + batchSql("INSERT INTO append_table_tracking VALUES (1, 'AAA'), (2, 'BBB')"); + List rows = batchSql("SELECT * FROM append_table_tracking$row_tracking"); assertThat(rows.size()).isEqualTo(2); assertThat(rows) .containsExactlyInAnyOrder(Row.of(1, "AAA", 0L, 1L), Row.of(2, "BBB", 1L, 1L)); - rows = batchSql("SELECT * FROM append_table_lineage"); + rows = batchSql("SELECT * FROM append_table_tracking"); assertThat(rows.size()).isEqualTo(2); assertThat(rows).containsExactlyInAnyOrder(Row.of(1, "AAA"), Row.of(2, "BBB")); } @Test - public void testCompactionWithRowLineage() throws Exception { - batchSql("ALTER TABLE append_table_lineage SET ('compaction.max.file-num' = '4')"); + public void testCompactionWithRowTracking() throws Exception { + batchSql("ALTER TABLE append_table_tracking SET ('compaction.max.file-num' = '4')"); assertExecuteExpected( - "INSERT INTO append_table_lineage VALUES (1, 'AAA'), (2, 'BBB')", + "INSERT INTO append_table_tracking VALUES (1, 'AAA'), (2, 'BBB')", 1L, Snapshot.CommitKind.APPEND, - "append_table_lineage"); + "append_table_tracking"); assertExecuteExpected( - "INSERT INTO append_table_lineage VALUES (3, 'CCC'), (4, 'DDD')", + "INSERT INTO append_table_tracking VALUES (3, 'CCC'), (4, 'DDD')", 2L, Snapshot.CommitKind.APPEND, - "append_table_lineage"); + "append_table_tracking"); assertExecuteExpected( - "INSERT INTO append_table_lineage VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC'), (4, 'DDD')", + "INSERT INTO append_table_tracking VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC'), (4, 'DDD')", 3L, Snapshot.CommitKind.APPEND, - "append_table_lineage"); + "append_table_tracking"); assertExecuteExpected( - "INSERT INTO append_table_lineage VALUES (5, 'EEE'), (6, 'FFF')", + "INSERT INTO append_table_tracking VALUES (5, 'EEE'), (6, 'FFF')", 4L, Snapshot.CommitKind.APPEND, - "append_table_lineage"); + "append_table_tracking"); assertExecuteExpected( - "INSERT INTO append_table_lineage VALUES (7, 'HHH'), (8, 'III')", + "INSERT INTO append_table_tracking VALUES (7, 'HHH'), (8, 'III')", 5L, Snapshot.CommitKind.APPEND, - "append_table_lineage"); + "append_table_tracking"); assertExecuteExpected( - "INSERT INTO append_table_lineage VALUES (9, 'JJJ'), (10, 'KKK')", + "INSERT INTO append_table_tracking VALUES (9, 'JJJ'), (10, 'KKK')", 6L, Snapshot.CommitKind.APPEND, - "append_table_lineage"); + "append_table_tracking"); assertExecuteExpected( - "INSERT INTO append_table_lineage VALUES (11, 'LLL'), (12, 'MMM')", + "INSERT INTO append_table_tracking VALUES (11, 'LLL'), (12, 'MMM')", 7L, Snapshot.CommitKind.APPEND, - "append_table_lineage"); + "append_table_tracking"); assertExecuteExpected( - "INSERT INTO append_table_lineage VALUES (13, 'NNN'), (14, 'OOO')", + "INSERT INTO append_table_tracking VALUES (13, 'NNN'), (14, 'OOO')", 8L, Snapshot.CommitKind.APPEND, - "append_table_lineage"); + "append_table_tracking"); - List originRowsWithId2 = batchSql("SELECT * FROM append_table_lineage$row_lineage"); - batchSql("call sys.compact('default.append_table_lineage')"); - waitCompactSnapshot(60000L, "append_table_lineage"); - List files = batchSql("SELECT * FROM append_table_lineage$files"); + List originRowsWithId2 = batchSql("SELECT * FROM append_table_tracking$row_tracking"); + batchSql("call sys.compact('default.append_table_tracking')"); + waitCompactSnapshot(60000L, "append_table_tracking"); + List files = batchSql("SELECT * FROM append_table_tracking$files"); assertThat(files.size()).isEqualTo(1); - List rowsAfter2 = batchSql("SELECT * FROM append_table_lineage$row_lineage"); + List rowsAfter2 = batchSql("SELECT * FROM append_table_tracking$row_tracking"); assertThat(originRowsWithId2).containsExactlyInAnyOrderElementsOf(rowsAfter2); assertThat(rowsAfter2) @@ -655,7 +655,7 @@ private int countNumRecords() throws Exception { protected List ddl() { return Arrays.asList( "CREATE TABLE IF NOT EXISTS append_table (id INT, data STRING) WITH ('bucket' = '-1')", - "CREATE TABLE IF NOT EXISTS append_table_lineage (id INT, data STRING) WITH ('bucket' = '-1', 'row-tracking.enabled' = 'true')", + "CREATE TABLE IF NOT EXISTS append_table_tracking (id INT, data STRING) WITH ('bucket' = '-1', 'row-tracking.enabled' = 'true')", "CREATE TABLE IF NOT EXISTS part_table (id INT, data STRING, dt STRING) PARTITIONED BY (dt) WITH ('bucket' = '-1')", "CREATE TABLE IF NOT EXISTS complex_table (id INT, data MAP) WITH ('bucket' = '-1')", "CREATE TABLE IF NOT EXISTS index_table (id INT, indexc STRING, data STRING) WITH ('bucket' = '-1', 'file-index.bloom-filter.columns'='indexc', 'file-index.bloom-filter.indexc.items' = '500')"); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java index 13425014f5e1..c5e394ca8bd7 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BatchFileStoreITCase.java @@ -1016,4 +1016,23 @@ public void testBatchReadSourceWithoutSnapshot() { "SELECT * FROM T /*+ OPTIONS('scan.dedicated-split-generation'='true') */")) .hasSize(0); } + + @Test + public void testLevel0FileCanBeReadForFilesTable() { + sql( + "CREATE TABLE test_table (a int PRIMARY KEY NOT ENFORCED, b string) " + + "WITH ('deletion-vectors.enabled' = 'true', 'write-only' = 'true');"); + sql("INSERT INTO test_table VALUES (1, 'A')"); + assertThat(sql("SELECT * FROM `test_table$files`")).isNotEmpty(); + } + + @Test + public void testLevel0FileCanBeReadForPartitionsTable() { + sql( + "CREATE TABLE test_table (a int PRIMARY KEY NOT ENFORCED, b string, dt string) " + + "PARTITIONED BY (dt)" + + "WITH ('deletion-vectors.enabled' = 'true', 'write-only' = 'true');"); + sql("INSERT INTO test_table VALUES (1, 'A', '2024-12-01')"); + assertThat(sql("SELECT * FROM `test_table$partitions`")).isNotEmpty(); + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java index ddf8b1e3fc77..b2bf1d7c90de 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogITCaseBase.java @@ -31,6 +31,7 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.internal.TableEnvironmentImpl; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogBaseTable; @@ -69,7 +70,11 @@ public abstract class CatalogITCaseBase extends AbstractTestBase { @BeforeEach public void before() throws IOException { - tEnv = tableEnvironmentBuilder().batchMode().build(); + TableEnvironmentBuilder tBuilder = tableEnvironmentBuilder().batchMode(); + if (sqlSyncMode() != null) { + tBuilder.setConf(TableConfigOptions.TABLE_DML_SYNC, sqlSyncMode()); + } + tEnv = tBuilder.build(); String catalog = "PAIMON"; path = getTempDirPath(); String inferScan = @@ -97,6 +102,11 @@ public void before() throws IOException { prepareEnv(); } + @Nullable + protected Boolean sqlSyncMode() { + return null; + } + protected Map catalogOptions() { return Collections.emptyMap(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java index bcdf1e2ef6dd..3d7429177a7a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CatalogTableITCase.java @@ -1172,6 +1172,26 @@ public void testBinlogTableStreamRead() throws Exception { iterator.close(); } + @Test + public void testBinlogTableStreamReadWithProjection() throws Exception { + sql( + "CREATE TABLE T (a INT, b INT, primary key (a) NOT ENFORCED) with ('changelog-producer' = 'lookup', " + + "'bucket' = '2')"); + BlockingIterator iterator = + streamSqlBlockIter( + "SELECT rowkind, a FROM T$binlog /*+ OPTIONS('scan.mode' = 'latest') */"); + sql("INSERT INTO T VALUES (1, 2)"); + sql("INSERT INTO T VALUES (1, 3)"); + sql("INSERT INTO T VALUES (2, 2)"); + List rows = iterator.collect(3); + assertThat(rows) + .containsExactly( + Row.of("+I", new Integer[] {1}), + Row.of("+U", new Integer[] {1, 1}), + Row.of("+I", new Integer[] {2})); + iterator.close(); + } + @Test public void testBinlogTableBatchRead() throws Exception { sql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/GlobalDynamicBucketTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CrossPartitionTableITCase.java similarity index 84% rename from paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/GlobalDynamicBucketTableITCase.java rename to paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CrossPartitionTableITCase.java index 210489592eaf..98747a31d6bf 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/GlobalDynamicBucketTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/CrossPartitionTableITCase.java @@ -22,13 +22,16 @@ import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +import javax.annotation.Nullable; + import java.util.Arrays; import java.util.List; +import java.util.concurrent.ExecutionException; import static org.assertj.core.api.Assertions.assertThat; /** ITCase for batch file store. */ -public class GlobalDynamicBucketTableITCase extends CatalogITCaseBase { +public class CrossPartitionTableITCase extends CatalogITCaseBase { @Override protected List ddl() { @@ -66,6 +69,12 @@ protected List ddl() { + ")"); } + @Nullable + @Override + protected Boolean sqlSyncMode() { + return true; + } + @Test public void testBulkLoad() { sql("INSERT INTO T VALUES (1, 1, 1), (2, 1, 2), (1, 3, 3), (2, 4, 4), (3, 3, 5)"); @@ -172,4 +181,28 @@ public void testPkContainsPartialPartitionFields() { sql("insert into partial_part values (1, 2, 1, 2)"); assertThat(sql("select * from partial_part")).containsExactlyInAnyOrder(Row.of(1, 2, 1, 2)); } + + @Test + public void testCrossPartitionWithFixedBucket() { + sql( + "create table cross_fixed (pt int, k int, v int, primary key (k) not enforced) " + + "partitioned by (pt) with ('bucket' = '2')"); + sql("insert into cross_fixed values (1, 1, 1)"); + sql("insert into cross_fixed values (2, 2, 2)"); + assertThat(sql("select * from cross_fixed")) + .containsExactlyInAnyOrder(Row.of(1, 1, 1), Row.of(2, 2, 2)); + } + + @Test + public void testCrossPartitionWithPostponeBucket() + throws ExecutionException, InterruptedException { + sql( + "create table cross_postpone (pt int, k int, v int, primary key (k) not enforced) " + + "partitioned by (pt) with ('bucket' = '-2')"); + sql("insert into cross_postpone values (1, 1, 1)"); + sql("insert into cross_postpone values (2, 2, 2)"); + tEnv.executeSql("CALL sys.compact(`table` => 'default.cross_postpone')").await(); + assertThat(sql("select * from cross_postpone")) + .containsExactlyInAnyOrder(Row.of(1, 1, 1), Row.of(2, 2, 2)); + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FilterPushdownWithSchemaChangeITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FilterPushdownWithSchemaChangeITCase.java index 73ea8f0ae119..317d5741946e 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FilterPushdownWithSchemaChangeITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FilterPushdownWithSchemaChangeITCase.java @@ -247,13 +247,12 @@ public void testPrimaryStringToNumericForStatsFilter() { sql("INSERT INTO T VALUES ('9', '9'), ('10', '10'), ('11', '11')"); // key filter - sql("ALTER TABLE T MODIFY (pk INT)"); assertThat(sql("SELECT * FROM T WHERE pk > 9")) - .containsExactlyInAnyOrder(Row.of(10, "10"), Row.of(11, "11")); + .containsExactlyInAnyOrder(Row.of("10", "10"), Row.of("11", "11")); // value filter sql("ALTER TABLE T MODIFY (v INT)"); assertThat(sql("SELECT * FROM T WHERE v > 9")) - .containsExactlyInAnyOrder(Row.of(10, 10), Row.of(11, 11)); + .containsExactlyInAnyOrder(Row.of("10", 10), Row.of("11", 11)); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java index bad5998e3994..fdd12ae2a70a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java @@ -254,7 +254,7 @@ private void createTargetTable(String tableName, BucketMode bucketMode) { "CREATE TABLE IF NOT EXISTS `%s` (k INT, f1 STRING, pt STRING, PRIMARY KEY(k, pt) NOT ENFORCED) WITH ('bucket'='-1', 'commit.force-create-snapshot'='true')", tableName)); return; - case CROSS_PARTITION: + case KEY_DYNAMIC: batchSql( String.format( "CREATE TABLE IF NOT EXISTS `%s` (k INT, f1 STRING, pt STRING, PRIMARY KEY(k) NOT ENFORCED) WITH ('bucket'='-1', 'commit.force-create-snapshot'='true')", diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/IncrementalClusterActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/IncrementalClusterActionITCase.java new file mode 100644 index 000000000000..bf7087e77e3b --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/IncrementalClusterActionITCase.java @@ -0,0 +1,807 @@ +/* + * 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. + */ + +package org.apache.paimon.flink.action; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.Snapshot; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.deletionvectors.BitmapDeletionVector; +import org.apache.paimon.deletionvectors.append.AppendDeleteFileMaintainer; +import org.apache.paimon.deletionvectors.append.BaseAppendDeleteFileMaintainer; +import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.io.CompactIncrement; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.io.DataIncrement; +import org.apache.paimon.manifest.FileKind; +import org.apache.paimon.manifest.IndexManifestEntry; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.BatchTableCommit; +import org.apache.paimon.table.sink.BatchTableWrite; +import org.apache.paimon.table.sink.BatchWriteBuilder; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.utils.StringUtils; + +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; + +/** IT cases for incremental clustering action. */ +public class IncrementalClusterActionITCase extends ActionITCaseBase { + + @Test + public void testClusterUnpartitionedTable() throws Exception { + FileStoreTable table = createTable(null, 1); + + BinaryString randomStr = BinaryString.fromString(randomString(150)); + List messages = new ArrayList<>(); + + // first write + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + messages.addAll(write(GenericRow.of(i, j, randomStr, 0))); + } + } + commit(messages); + ReadBuilder readBuilder = table.newReadBuilder().withProjection(new int[] {0, 1}); + List result1 = + getResult( + readBuilder.newRead(), + readBuilder.newScan().plan().splits(), + readBuilder.readType()); + List expected1 = + Lists.newArrayList( + "+I[0, 0]", + "+I[0, 1]", + "+I[0, 2]", + "+I[1, 0]", + "+I[1, 1]", + "+I[1, 2]", + "+I[2, 0]", + "+I[2, 1]", + "+I[2, 2]"); + assertThat(result1).containsExactlyElementsOf(expected1); + + // first cluster + runAction(Collections.emptyList()); + checkSnapshot(table); + List splits = readBuilder.newScan().plan().splits(); + assertThat(splits.size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(0).level()).isEqualTo(5); + List result2 = getResult(readBuilder.newRead(), splits, readBuilder.readType()); + List expected2 = + Lists.newArrayList( + "+I[0, 0]", + "+I[0, 1]", + "+I[1, 0]", + "+I[1, 1]", + "+I[0, 2]", + "+I[1, 2]", + "+I[2, 0]", + "+I[2, 1]", + "+I[2, 2]"); + assertThat(result2).containsExactlyElementsOf(expected2); + + // second write + messages.clear(); + messages.addAll( + write( + GenericRow.of(0, 3, null, 0), + GenericRow.of(1, 3, null, 0), + GenericRow.of(2, 3, null, 0))); + messages.addAll( + write( + GenericRow.of(3, 0, null, 0), + GenericRow.of(3, 1, null, 0), + GenericRow.of(3, 2, null, 0), + GenericRow.of(3, 3, null, 0))); + commit(messages); + + List result3 = + getResult( + readBuilder.newRead(), + readBuilder.newScan().plan().splits(), + readBuilder.readType()); + List expected3 = new ArrayList<>(expected2); + expected3.addAll( + Lists.newArrayList( + "+I[0, 3]", + "+I[1, 3]", + "+I[2, 3]", + "+I[3, 0]", + "+I[3, 1]", + "+I[3, 2]", + "+I[3, 3]")); + assertThat(result3).containsExactlyElementsOf(expected3); + + // second cluster + runAction(Collections.emptyList()); + checkSnapshot(table); + splits = readBuilder.newScan().plan().splits(); + List result4 = getResult(readBuilder.newRead(), splits, readBuilder.readType()); + List expected4 = new ArrayList<>(expected2); + expected4.addAll( + Lists.newArrayList( + "+I[0, 3]", + "+I[1, 3]", + "+I[3, 0]", + "+I[3, 1]", + "+I[2, 3]", + "+I[3, 2]", + "+I[3, 3]")); + assertThat(splits.size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(2); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(0).level()).isEqualTo(5); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(1).level()).isEqualTo(4); + assertThat(result4).containsExactlyElementsOf(expected4); + + // full cluster + runAction(Lists.newArrayList("--compact_strategy", "full")); + checkSnapshot(table); + splits = readBuilder.newScan().plan().splits(); + List result5 = getResult(readBuilder.newRead(), splits, readBuilder.readType()); + List expected5 = + new ArrayList<>( + Lists.newArrayList( + "+I[0, 0]", + "+I[0, 1]", + "+I[1, 0]", + "+I[1, 1]", + "+I[0, 2]", + "+I[0, 3]", + "+I[1, 2]", + "+I[1, 3]", + "+I[2, 0]", + "+I[2, 1]", + "+I[3, 0]", + "+I[3, 1]", + "+I[2, 2]", + "+I[2, 3]", + "+I[3, 2]", + "+I[3, 3]")); + assertThat(splits.size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(0).level()).isEqualTo(5); + assertThat(result5).containsExactlyElementsOf(expected5); + } + + @Test + public void testClusterPartitionedTable() throws Exception { + FileStoreTable table = createTable("pt", 1); + + BinaryString randomStr = BinaryString.fromString(randomString(150)); + List messages = new ArrayList<>(); + + // first write + List expected1 = new ArrayList<>(); + for (int pt = 0; pt < 2; pt++) { + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + messages.addAll(write(GenericRow.of(i, j, (pt == 0) ? randomStr : null, pt))); + expected1.add(String.format("+I[%s, %s, %s]", i, j, pt)); + } + } + } + commit(messages); + ReadBuilder readBuilder = table.newReadBuilder().withProjection(new int[] {0, 1, 3}); + List result1 = + getResult( + readBuilder.newRead(), + readBuilder.newScan().plan().splits(), + readBuilder.readType()); + assertThat(result1).containsExactlyElementsOf(expected1); + + // first cluster + runAction(Collections.emptyList()); + checkSnapshot(table); + List splits = readBuilder.newScan().plan().splits(); + assertThat(splits.size()).isEqualTo(2); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(0).level()).isEqualTo(5); + List result2 = getResult(readBuilder.newRead(), splits, readBuilder.readType()); + List expected2 = new ArrayList<>(); + for (int pt = 0; pt < 2; pt++) { + expected2.add(String.format("+I[0, 0, %s]", pt)); + expected2.add(String.format("+I[0, 1, %s]", pt)); + expected2.add(String.format("+I[1, 0, %s]", pt)); + expected2.add(String.format("+I[1, 1, %s]", pt)); + expected2.add(String.format("+I[0, 2, %s]", pt)); + expected2.add(String.format("+I[1, 2, %s]", pt)); + expected2.add(String.format("+I[2, 0, %s]", pt)); + expected2.add(String.format("+I[2, 1, %s]", pt)); + expected2.add(String.format("+I[2, 2, %s]", pt)); + } + assertThat(result2).containsExactlyElementsOf(expected2); + + // second write + messages.clear(); + for (int pt = 0; pt < 2; pt++) { + messages.addAll( + write( + GenericRow.of(0, 3, null, pt), + GenericRow.of(1, 3, null, pt), + GenericRow.of(2, 3, null, pt))); + messages.addAll( + write( + GenericRow.of(3, 0, null, pt), + GenericRow.of(3, 1, null, pt), + GenericRow.of(3, 2, null, pt), + GenericRow.of(3, 3, null, pt))); + } + commit(messages); + + List result3 = + getResult( + readBuilder.newRead(), + readBuilder.newScan().plan().splits(), + readBuilder.readType()); + List expected3 = new ArrayList<>(); + for (int pt = 0; pt < 2; pt++) { + expected3.addAll(expected2.subList(9 * pt, 9 * pt + 9)); + expected3.add(String.format("+I[0, 3, %s]", pt)); + expected3.add(String.format("+I[1, 3, %s]", pt)); + expected3.add(String.format("+I[2, 3, %s]", pt)); + expected3.add(String.format("+I[3, 0, %s]", pt)); + expected3.add(String.format("+I[3, 1, %s]", pt)); + expected3.add(String.format("+I[3, 2, %s]", pt)); + expected3.add(String.format("+I[3, 3, %s]", pt)); + } + assertThat(result3).containsExactlyElementsOf(expected3); + + // second cluster + runAction(Collections.emptyList()); + checkSnapshot(table); + splits = readBuilder.newScan().plan().splits(); + List result4 = getResult(readBuilder.newRead(), splits, readBuilder.readType()); + List expected4 = new ArrayList<>(); + // for partition-0: only file in level-0 will be picked for clustering, outputLevel is 4 + expected4.add("+I[0, 0, 0]"); + expected4.add("+I[0, 1, 0]"); + expected4.add("+I[1, 0, 0]"); + expected4.add("+I[1, 1, 0]"); + expected4.add("+I[0, 2, 0]"); + expected4.add("+I[1, 2, 0]"); + expected4.add("+I[2, 0, 0]"); + expected4.add("+I[2, 1, 0]"); + expected4.add("+I[2, 2, 0]"); + expected4.add("+I[0, 3, 0]"); + expected4.add("+I[1, 3, 0]"); + expected4.add("+I[3, 0, 0]"); + expected4.add("+I[3, 1, 0]"); + expected4.add("+I[2, 3, 0]"); + expected4.add("+I[3, 2, 0]"); + expected4.add("+I[3, 3, 0]"); + // for partition-1:all files will be picked for clustering, outputLevel is 5 + expected4.add("+I[0, 0, 1]"); + expected4.add("+I[0, 1, 1]"); + expected4.add("+I[1, 0, 1]"); + expected4.add("+I[1, 1, 1]"); + expected4.add("+I[0, 2, 1]"); + expected4.add("+I[0, 3, 1]"); + expected4.add("+I[1, 2, 1]"); + expected4.add("+I[1, 3, 1]"); + expected4.add("+I[2, 0, 1]"); + expected4.add("+I[2, 1, 1]"); + expected4.add("+I[3, 0, 1]"); + expected4.add("+I[3, 1, 1]"); + expected4.add("+I[2, 2, 1]"); + expected4.add("+I[2, 3, 1]"); + expected4.add("+I[3, 2, 1]"); + expected4.add("+I[3, 3, 1]"); + assertThat(splits.size()).isEqualTo(2); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(2); + assertThat(((DataSplit) splits.get(1)).dataFiles().size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(0).level()).isEqualTo(5); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(1).level()).isEqualTo(4); + assertThat(((DataSplit) splits.get(1)).dataFiles().get(0).level()).isEqualTo(5); + assertThat(result4).containsExactlyElementsOf(expected4); + } + + @Test + public void testClusterSpecifyPartition() throws Exception { + FileStoreTable table = createTable("pt", 1); + + BinaryString randomStr = BinaryString.fromString(randomString(150)); + List messages = new ArrayList<>(); + + // first write + List expected1 = new ArrayList<>(); + for (int pt = 0; pt < 2; pt++) { + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + messages.addAll(write(GenericRow.of(i, j, (pt == 0) ? randomStr : null, pt))); + expected1.add(String.format("+I[%s, %s, %s]", i, j, pt)); + } + } + } + commit(messages); + ReadBuilder readBuilder = table.newReadBuilder().withProjection(new int[] {0, 1, 3}); + List result1 = + getResult( + readBuilder.newRead(), + readBuilder.newScan().plan().splits(), + readBuilder.readType()); + assertThat(result1).containsExactlyElementsOf(expected1); + + runAction(Lists.newArrayList("--partition", "pt=0", "--compact_strategy", "full")); + checkSnapshot(table); + List splits = readBuilder.newScan().plan().splits(); + assertThat(splits.size()).isEqualTo(2); + for (Split split : splits) { + DataSplit dataSplit = (DataSplit) split; + if (dataSplit.partition().getInt(0) == 0) { + assertThat(dataSplit.dataFiles().size()).isEqualTo(1); + assertThat(dataSplit.dataFiles().get(0).level()).isEqualTo(5); + } else { + assertThat(dataSplit.dataFiles().size()).isGreaterThan(1); + assertThat(dataSplit.dataFiles().get(0).level()).isEqualTo(0); + } + } + } + + @Test + public void testClusterHistoryPartition() throws Exception { + Map options = new HashMap<>(); + options.put(CoreOptions.CLUSTERING_HISTORY_PARTITION_IDLE_TIME.key(), "3s"); + FileStoreTable table = createTable("pt", 1, options); + + BinaryString randomStr = BinaryString.fromString(randomString(150)); + List messages = new ArrayList<>(); + + // first write + List expected1 = new ArrayList<>(); + for (int pt = 0; pt < 4; pt++) { + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + messages.addAll(write(GenericRow.of(i, j, randomStr, pt))); + expected1.add(String.format("+I[%s, %s, %s]", i, j, pt)); + } + } + } + commit(messages); + ReadBuilder readBuilder = table.newReadBuilder().withProjection(new int[] {0, 1, 3}); + List result1 = + getResult( + readBuilder.newRead(), + readBuilder.newScan().plan().splits(), + readBuilder.readType()); + assertThat(result1).containsExactlyElementsOf(expected1); + + // first cluster, files in four partitions will be in top level + runAction(Collections.emptyList()); + checkSnapshot(table); + List splits = readBuilder.newScan().plan().splits(); + assertThat(splits.size()).isEqualTo(4); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(0).level()).isEqualTo(5); + List result2 = getResult(readBuilder.newRead(), splits, readBuilder.readType()); + List expected2 = new ArrayList<>(); + for (int pt = 0; pt < 4; pt++) { + expected2.add(String.format("+I[0, 0, %s]", pt)); + expected2.add(String.format("+I[0, 1, %s]", pt)); + expected2.add(String.format("+I[1, 0, %s]", pt)); + expected2.add(String.format("+I[1, 1, %s]", pt)); + expected2.add(String.format("+I[0, 2, %s]", pt)); + expected2.add(String.format("+I[1, 2, %s]", pt)); + expected2.add(String.format("+I[2, 0, %s]", pt)); + expected2.add(String.format("+I[2, 1, %s]", pt)); + expected2.add(String.format("+I[2, 2, %s]", pt)); + } + assertThat(result2).containsExactlyElementsOf(expected2); + + // second write + messages.clear(); + for (int pt = 0; pt < 4; pt++) { + messages.addAll( + write( + GenericRow.of(0, 3, null, pt), + GenericRow.of(1, 3, null, pt), + GenericRow.of(2, 3, null, pt))); + messages.addAll( + write( + GenericRow.of(3, 0, null, pt), + GenericRow.of(3, 1, null, pt), + GenericRow.of(3, 2, null, pt), + GenericRow.of(3, 3, null, pt))); + // pt-0, pt-1 will be history partition + if (pt == 1) { + Thread.sleep(3000); + } + } + commit(messages); + + List result3 = + getResult( + readBuilder.newRead(), + readBuilder.newScan().plan().splits(), + readBuilder.readType()); + List expected3 = new ArrayList<>(); + for (int pt = 0; pt < 4; pt++) { + expected3.addAll(expected2.subList(9 * pt, 9 * pt + 9)); + expected3.add(String.format("+I[0, 3, %s]", pt)); + expected3.add(String.format("+I[1, 3, %s]", pt)); + expected3.add(String.format("+I[2, 3, %s]", pt)); + expected3.add(String.format("+I[3, 0, %s]", pt)); + expected3.add(String.format("+I[3, 1, %s]", pt)); + expected3.add(String.format("+I[3, 2, %s]", pt)); + expected3.add(String.format("+I[3, 3, %s]", pt)); + } + assertThat(result3).containsExactlyElementsOf(expected3); + + // second cluster + runAction(Lists.newArrayList("--partition", "pt=3")); + checkSnapshot(table); + splits = readBuilder.newScan().plan().splits(); + List result4 = getResult(readBuilder.newRead(), splits, readBuilder.readType()); + List expected4 = new ArrayList<>(); + assertThat(splits.size()).isEqualTo(4); + // for pt-0 and pt-1: history partition, full clustering, all files will be + // picked for clustering, outputLevel is 5. + for (int pt = 0; pt <= 1; pt++) { + expected4.add(String.format("+I[0, 0, %s]", pt)); + expected4.add(String.format("+I[0, 1, %s]", pt)); + expected4.add(String.format("+I[1, 0, %s]", pt)); + expected4.add(String.format("+I[1, 1, %s]", pt)); + expected4.add(String.format("+I[0, 2, %s]", pt)); + expected4.add(String.format("+I[0, 3, %s]", pt)); + expected4.add(String.format("+I[1, 2, %s]", pt)); + expected4.add(String.format("+I[1, 3, %s]", pt)); + expected4.add(String.format("+I[2, 0, %s]", pt)); + expected4.add(String.format("+I[2, 1, %s]", pt)); + expected4.add(String.format("+I[3, 0, %s]", pt)); + expected4.add(String.format("+I[3, 1, %s]", pt)); + expected4.add(String.format("+I[2, 2, %s]", pt)); + expected4.add(String.format("+I[2, 3, %s]", pt)); + expected4.add(String.format("+I[3, 2, %s]", pt)); + expected4.add(String.format("+I[3, 3, %s]", pt)); + // the table has enabled 'scan.plan-sort-partition', so the splits has been sorted by + // partition + assertThat(((DataSplit) splits.get(pt)).dataFiles().size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(pt)).dataFiles().get(0).level()).isEqualTo(5); + } + // for pt-2, non history partition, nor specified partition, nothing happened + expected4.addAll(expected3.subList(32, 48)); + assertThat(((DataSplit) splits.get(2)).dataFiles().size()).isEqualTo(3); + // for pt-3: minor clustering, only file in level-0 will be picked for clustering, + // outputLevel is 4 + expected4.add("+I[0, 0, 3]"); + expected4.add("+I[0, 1, 3]"); + expected4.add("+I[1, 0, 3]"); + expected4.add("+I[1, 1, 3]"); + expected4.add("+I[0, 2, 3]"); + expected4.add("+I[1, 2, 3]"); + expected4.add("+I[2, 0, 3]"); + expected4.add("+I[2, 1, 3]"); + expected4.add("+I[2, 2, 3]"); + expected4.add("+I[0, 3, 3]"); + expected4.add("+I[1, 3, 3]"); + expected4.add("+I[3, 0, 3]"); + expected4.add("+I[3, 1, 3]"); + expected4.add("+I[2, 3, 3]"); + expected4.add("+I[3, 2, 3]"); + expected4.add("+I[3, 3, 3]"); + assertThat(((DataSplit) splits.get(3)).dataFiles().size()).isEqualTo(2); + assertThat( + ((DataSplit) splits.get(3)) + .dataFiles().stream() + .map(DataFileMeta::level) + .collect(Collectors.toList())) + .containsExactlyInAnyOrder(4, 5); + + assertThat(result4).containsExactlyElementsOf(expected4); + } + + @Test + public void testClusterOnEmptyData() throws Exception { + createTable("pt", 1); + assertThatCode(() -> runAction(Collections.emptyList())).doesNotThrowAnyException(); + } + + @Test + public void testMultiParallelism() throws Exception { + FileStoreTable table = createTable(null, 2); + + BinaryString randomStr = BinaryString.fromString(randomString(150)); + List messages = new ArrayList<>(); + + // first write + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + messages.addAll(write(GenericRow.of(i, j, randomStr, 0))); + } + } + commit(messages); + ReadBuilder readBuilder = table.newReadBuilder().withProjection(new int[] {0, 1}); + List result1 = + getResult( + readBuilder.newRead(), + readBuilder.newScan().plan().splits(), + readBuilder.readType()); + List expected1 = + Lists.newArrayList( + "+I[0, 0]", + "+I[0, 1]", + "+I[0, 2]", + "+I[1, 0]", + "+I[1, 1]", + "+I[1, 2]", + "+I[2, 0]", + "+I[2, 1]", + "+I[2, 2]"); + assertThat(result1).containsExactlyElementsOf(expected1); + + runAction(Lists.newArrayList("--table_conf", "scan.parallelism=2")); + checkSnapshot(table); + List splits = readBuilder.newScan().plan().splits(); + assertThat(splits.size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isGreaterThanOrEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(0).level()).isEqualTo(5); + } + + @Test + public void testClusterWithDeletionVector() throws Exception { + Map dynamicOptions = new HashMap<>(); + dynamicOptions.put(CoreOptions.DELETION_VECTORS_ENABLED.key(), "true"); + FileStoreTable table = createTable(null, 1, dynamicOptions); + + BinaryString randomStr = BinaryString.fromString(randomString(150)); + List messages = new ArrayList<>(); + // first write + for (int i = 0; i < 3; i++) { + for (int j = 0; j < 3; j++) { + messages.addAll(write(GenericRow.of(i, j, randomStr, 0))); + } + } + commit(messages); + ReadBuilder readBuilder = table.newReadBuilder().withProjection(new int[] {0, 1}); + + // first cluster + runAction(Collections.emptyList()); + + // second write + messages.clear(); + messages.addAll( + write( + GenericRow.of(0, 3, null, 0), + GenericRow.of(1, 3, null, 0), + GenericRow.of(2, 3, null, 0))); + messages.addAll( + write( + GenericRow.of(3, 0, null, 0), + GenericRow.of(3, 1, null, 0), + GenericRow.of(3, 2, null, 0), + GenericRow.of(3, 3, null, 0))); + commit(messages); + + // write deletion vector for the table + AppendDeleteFileMaintainer maintainer = + BaseAppendDeleteFileMaintainer.forUnawareAppend( + table.store().newIndexFileHandler(), + table.latestSnapshot().get(), + BinaryRow.EMPTY_ROW); + List files = + readBuilder.newScan().plan().splits().stream() + .map(s -> ((DataSplit) s).dataFiles()) + .flatMap(List::stream) + .collect(Collectors.toList()); + // delete (0,0) and (0,3) + for (DataFileMeta file : files) { + if (file.rowCount() == 9 || file.rowCount() == 3) { + BitmapDeletionVector dv = new BitmapDeletionVector(); + dv.delete(0); + maintainer.notifyNewDeletionVector(file.fileName(), dv); + } + } + commit(produceDvIndexMessages(table, maintainer)); + List result1 = + getResult( + readBuilder.newRead(), + readBuilder.newScan().plan().splits(), + readBuilder.readType()); + List expected1 = + Lists.newArrayList( + "+I[0, 1]", + "+I[1, 0]", + "+I[1, 1]", + "+I[0, 2]", + "+I[1, 2]", + "+I[2, 0]", + "+I[2, 1]", + "+I[2, 2]", + "+I[1, 3]", + "+I[2, 3]", + "+I[3, 0]", + "+I[3, 1]", + "+I[3, 2]", + "+I[3, 3]"); + assertThat(result1).containsExactlyElementsOf(expected1); + + // second cluster + runAction(Collections.emptyList()); + checkSnapshot(table); + List splits = readBuilder.newScan().plan().splits(); + List result2 = + getResult( + readBuilder.newRead(), + readBuilder.newScan().plan().splits(), + readBuilder.readType()); + assertThat(result2.size()).isEqualTo(expected1.size()); + assertThat(splits.size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(2); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(0).level()).isEqualTo(5); + // dv index for level-5 file should be retained + assertThat(splits.get(0).deletionFiles().get().get(0)).isNotNull(); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(1).level()).isEqualTo(4); + assertThat((splits.get(0).deletionFiles().get().get(1))).isNull(); + + // full cluster + runAction(Lists.newArrayList("--compact_strategy", "full")); + checkSnapshot(table); + splits = readBuilder.newScan().plan().splits(); + assertThat(splits.size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(1); + assertThat(((DataSplit) splits.get(0)).dataFiles().get(0).level()).isEqualTo(5); + assertThat(splits.get(0).deletionFiles().get().get(0)).isNull(); + } + + protected FileStoreTable createTable(String partitionKeys, int sinkParallelism) + throws Exception { + return createTable(partitionKeys, sinkParallelism, Collections.emptyMap()); + } + + protected FileStoreTable createTable( + String partitionKeys, int sinkParallelism, Map options) + throws Exception { + catalog.createDatabase(database, true); + catalog.createTable(identifier(), schema(partitionKeys, sinkParallelism, options), true); + return (FileStoreTable) catalog.getTable(identifier()); + } + + private FileStoreTable getTable() throws Exception { + return (FileStoreTable) catalog.getTable(identifier()); + } + + private Identifier identifier() { + return Identifier.create(database, tableName); + } + + private List write(GenericRow... data) throws Exception { + BatchWriteBuilder builder = getTable().newBatchWriteBuilder(); + try (BatchTableWrite batchTableWrite = builder.newWrite()) { + for (GenericRow row : data) { + batchTableWrite.write(row); + } + return batchTableWrite.prepareCommit(); + } + } + + private void commit(List messages) throws Exception { + BatchTableCommit commit = getTable().newBatchWriteBuilder().newCommit(); + commit.commit(messages); + commit.close(); + } + + private static Schema schema(String partitionKeys, int sinkParallelism) { + return schema(partitionKeys, sinkParallelism, Collections.emptyMap()); + } + + private static Schema schema( + String partitionKeys, int sinkParallelism, Map options) { + Schema.Builder schemaBuilder = Schema.newBuilder(); + schemaBuilder.column("a", DataTypes.INT()); + schemaBuilder.column("b", DataTypes.INT()); + schemaBuilder.column("c", DataTypes.STRING()); + schemaBuilder.column("pt", DataTypes.INT()); + schemaBuilder.option("bucket", "-1"); + schemaBuilder.option("num-levels", "6"); + schemaBuilder.option("num-sorted-run.compaction-trigger", "2"); + schemaBuilder.option("scan.plan-sort-partition", "true"); + schemaBuilder.option("clustering.columns", "a,b"); + schemaBuilder.option("clustering.strategy", "zorder"); + schemaBuilder.option("clustering.incremental", "true"); + schemaBuilder.option("scan.parallelism", "1"); + schemaBuilder.option("sink.parallelism", String.valueOf(sinkParallelism)); + for (String key : options.keySet()) { + schemaBuilder.option(key, options.get(key)); + } + if (!StringUtils.isNullOrWhitespaceOnly(partitionKeys)) { + schemaBuilder.partitionKeys(partitionKeys); + } + return schemaBuilder.build(); + } + + private static String randomString(int length) { + String chars = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789"; + StringBuilder sb = new StringBuilder(length); + ThreadLocalRandom random = ThreadLocalRandom.current(); + for (int i = 0; i < length; i++) { + sb.append(chars.charAt(random.nextInt(chars.length()))); + } + return sb.toString(); + } + + private void checkSnapshot(FileStoreTable table) { + assertThat(table.latestSnapshot().get().commitKind()) + .isEqualTo(Snapshot.CommitKind.COMPACT); + } + + private List produceDvIndexMessages( + FileStoreTable table, AppendDeleteFileMaintainer maintainer) { + List newIndexFiles = new ArrayList<>(); + List deletedIndexFiles = new ArrayList<>(); + List indexEntries = maintainer.persist(); + for (IndexManifestEntry entry : indexEntries) { + if (entry.kind() == FileKind.ADD) { + newIndexFiles.add(entry.indexFile()); + } else { + deletedIndexFiles.add(entry.indexFile()); + } + } + return Collections.singletonList( + new CommitMessageImpl( + maintainer.getPartition(), + 0, + table.coreOptions().bucket(), + DataIncrement.emptyIncrement(), + new CompactIncrement( + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList(), + newIndexFiles, + deletedIndexFiles))); + } + + private void runAction(List extra) throws Exception { + StreamExecutionEnvironment env = streamExecutionEnvironmentBuilder().batchMode().build(); + ArrayList baseArgs = + Lists.newArrayList("compact", "--database", database, "--table", tableName); + ThreadLocalRandom random = ThreadLocalRandom.current(); + if (random.nextBoolean()) { + baseArgs.addAll(Lists.newArrayList("--warehouse", warehouse)); + } else { + baseArgs.addAll(Lists.newArrayList("--catalog_conf", "warehouse=" + warehouse)); + } + baseArgs.addAll(extra); + + CompactAction action = createAction(CompactAction.class, baseArgs.toArray(new String[0])); + action.withStreamExecutionEnvironment(env); + action.run(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperatorTest.java index 333f315df6ed..ad9e9355e149 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/AppendPreCommitCompactCoordinatorOperatorTest.java @@ -190,7 +190,8 @@ private void assertCommittable(Object o, long checkpointId, BinaryRow partition, assertThat(message.newFilesIncrement().deletedFiles()).isEmpty(); assertThat(message.newFilesIncrement().changelogFiles()).isEmpty(); assertThat(message.compactIncrement().isEmpty()).isTrue(); - assertThat(message.indexIncrement().isEmpty()).isTrue(); + assertThat(message.newFilesIncrement().newIndexFiles().isEmpty()).isTrue(); + assertThat(message.newFilesIncrement().deletedIndexFiles().isEmpty()).isTrue(); assertThat(message.newFilesIncrement().newFiles().stream().map(DataFileMeta::fileSize)) .hasSameElementsAs( Arrays.stream(mbs) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java index 39664c3aa39e..7b058328cb75 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java @@ -19,11 +19,6 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.Snapshot; -import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.catalog.CatalogLoader; -import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.data.BinaryRow; -import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; import org.apache.paimon.flink.FlinkConnectorOptions; @@ -31,7 +26,6 @@ import org.apache.paimon.flink.util.AbstractTestBase; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; -import org.apache.paimon.options.Options; import org.apache.paimon.partition.PartitionPredicate; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; @@ -49,13 +43,8 @@ import org.apache.paimon.types.RowType; import org.apache.paimon.utils.SnapshotManager; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.data.RowData; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -70,7 +59,6 @@ import java.util.Random; import java.util.UUID; -import static org.apache.paimon.utils.SerializationUtils.serializeBinaryRow; import static org.assertj.core.api.Assertions.assertThat; /** IT cases for {@link CompactorSinkBuilder} and {@link CompactorSink}. */ @@ -216,98 +204,4 @@ private FileStoreTable createFileStoreTable() throws Exception { "")); return FileStoreTableFactory.create(LocalFileIO.create(), tablePath, tableSchema); } - - private FileStoreTable createCatalogTable(Catalog catalog, Identifier tableIdentifier) - throws Exception { - Schema tableSchema = - new Schema( - ROW_TYPE.getFields(), - Collections.emptyList(), - Collections.singletonList("k"), - Collections.singletonMap("bucket", "1"), - ""); - catalog.createTable(tableIdentifier, tableSchema, false); - return (FileStoreTable) catalog.getTable(tableIdentifier); - } - - private OneInputStreamOperatorTestHarness createTestHarness( - OneInputStreamOperator operator) throws Exception { - TypeSerializer serializer = - new CommittableTypeInfo().createSerializer(new ExecutionConfig()); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator); - harness.setup(serializer); - return harness; - } - - private OneInputStreamOperatorTestHarness - createMultiTablesTestHarness( - OneInputStreamOperator operator) - throws Exception { - TypeSerializer serializer = - new MultiTableCommittableTypeInfo().createSerializer(new ExecutionConfig()); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator); - harness.setup(serializer); - return harness; - } - - protected StoreCompactOperator.Factory createCompactOperator(FileStoreTable table) { - return new StoreCompactOperator.Factory( - table, - (t, commitUser, state, ioManager, memoryPool, metricGroup) -> - new StoreSinkWriteImpl( - t, - commitUser, - state, - ioManager, - false, - false, - false, - memoryPool, - metricGroup), - "test", - true); - } - - protected MultiTablesStoreCompactOperator.Factory createMultiTablesCompactOperator( - CatalogLoader catalogLoader) throws Exception { - return new MultiTablesStoreCompactOperator.Factory( - catalogLoader, - commitUser, - new CheckpointConfig(), - false, - false, - true, - new Options()); - } - - private static byte[] partition(String dt, int hh) { - BinaryRow row = new BinaryRow(2); - BinaryRowWriter writer = new BinaryRowWriter(row); - writer.writeString(0, BinaryString.fromString(dt)); - writer.writeInt(1, hh); - writer.complete(); - return serializeBinaryRow(row); - } - - private void prepareDataFile(FileStoreTable table) throws Exception { - StreamWriteBuilder streamWriteBuilder = - table.newStreamWriteBuilder().withCommitUser(commitUser); - StreamTableWrite write = streamWriteBuilder.newWrite(); - StreamTableCommit commit = streamWriteBuilder.newCommit(); - - write.write(rowData(1, 100, 15, BinaryString.fromString("20221208"))); - write.write(rowData(1, 100, 16, BinaryString.fromString("20221208"))); - write.write(rowData(1, 100, 15, BinaryString.fromString("20221209"))); - commit.commit(0, write.prepareCommit(true, 0)); - - write.write(rowData(2, 200, 15, BinaryString.fromString("20221208"))); - write.write(rowData(2, 200, 16, BinaryString.fromString("20221208"))); - write.write(rowData(2, 200, 15, BinaryString.fromString("20221209"))); - commit.commit(1, write.prepareCommit(true, 1)); - - write.close(); - commit.close(); - } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java index 0243270b6ebf..f90674c66865 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreCompactOperatorTest.java @@ -55,7 +55,7 @@ public void testCompactExactlyOnce(boolean streamingMode) throws Exception { StoreCompactOperator.Factory operatorFactory = new StoreCompactOperator.Factory( getTableDefault(), - (table, commitUser, state, ioManager, memoryPool, metricGroup) -> + (table, commitUser, state, ioManager, memoryPoolFactory, metricGroup) -> compactRememberStoreWrite, "10086", !streamingMode); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java index 757d25de1c55..276bf45193f3 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java @@ -27,7 +27,6 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.io.CompactIncrement; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.options.Options; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.Schema; @@ -550,7 +549,7 @@ private RowDataStoreWriteOperator.Factory getLookupWriteOperatorFactory( return new RowDataStoreWriteOperator.Factory( fileStoreTable, null, - (table, commitUser, state, ioManager, memoryPool, metricGroup) -> + (table, commitUser, state, ioManager, memoryPoolFactory, metricGroup) -> new LookupSinkWrite( table, commitUser, @@ -559,7 +558,7 @@ private RowDataStoreWriteOperator.Factory getLookupWriteOperatorFactory( false, waitCompaction, true, - memoryPool, + memoryPoolFactory, metricGroup), commitUser); } @@ -596,8 +595,7 @@ private void commitAppend( message.bucket(), message.totalBuckets(), message.newFilesIncrement(), - CompactIncrement.emptyIncrement(), - new IndexIncrement(Collections.emptyList())); + CompactIncrement.emptyIncrement()); commitMessages.add(newMessage); } commit.commit(commitIdentifier, commitMessages); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/listener/ListenerTestUtils.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/listener/ListenerTestUtils.java index 3439415c2f65..c873234105f0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/listener/ListenerTestUtils.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/listener/ListenerTestUtils.java @@ -24,7 +24,6 @@ import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileTestUtils; import org.apache.paimon.io.DataIncrement; -import org.apache.paimon.io.IndexIncrement; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.table.sink.CommitMessageImpl; @@ -67,8 +66,7 @@ static void notifyCommits( 0, 1, new DataIncrement(emptyList(), emptyList(), emptyList()), - new CompactIncrement(singletonList(file), emptyList(), emptyList()), - new IndexIncrement(emptyList())); + new CompactIncrement(singletonList(file), emptyList(), emptyList())); } else { compactMessage = new CommitMessageImpl( @@ -76,8 +74,7 @@ static void notifyCommits( 0, 1, new DataIncrement(singletonList(file), emptyList(), emptyList()), - new CompactIncrement(emptyList(), emptyList(), emptyList()), - new IndexIncrement(emptyList())); + new CompactIncrement(emptyList(), emptyList(), emptyList())); } committable.addFileCommittable(compactMessage); if (partitionMarkDoneRecoverFromState) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitGeneratorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitGeneratorTest.java index 932b7ee95118..cf63274169a4 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitGeneratorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitGeneratorTest.java @@ -70,23 +70,27 @@ public void test() { o -> ((DataSplit) ((FileStoreSourceSplit) o).split()).bucket())); // splitId should be the input order! - assertSplit(splits.get(0), "0000000001", 1, 0, Arrays.asList("f0", "f1")); - assertSplit(splits.get(1), "0000000002", 1, 1, Collections.singletonList("f2")); - assertSplit(splits.get(2), "0000000003", 2, 0, Arrays.asList("f3", "f4", "f5")); - assertSplit(splits.get(3), "0000000004", 2, 1, Collections.singletonList("f6")); - assertSplit(splits.get(4), "0000000005", 3, 0, Collections.singletonList("f7")); - assertSplit(splits.get(5), "0000000006", 3, 1, Collections.singletonList("f8")); - assertSplit(splits.get(6), "0000000007", 4, 0, Collections.singletonList("f9")); - assertSplit(splits.get(7), "0000000008", 4, 1, Collections.singletonList("f10")); - assertSplit(splits.get(8), "0000000009", 5, 0, Collections.singletonList("f11")); - assertSplit(splits.get(9), "0000000010", 5, 1, Collections.singletonList("f12")); - assertSplit(splits.get(10), "0000000011", 6, 0, Collections.singletonList("f13")); - assertSplit(splits.get(11), "0000000012", 6, 1, Collections.singletonList("f14")); + assertSplit(splits.get(0), "-1", 1, 0, Arrays.asList("f0", "f1")); + assertSplit(splits.get(1), "-2", 1, 1, Collections.singletonList("f2")); + assertSplit(splits.get(2), "-3", 2, 0, Arrays.asList("f3", "f4", "f5")); + assertSplit(splits.get(3), "-4", 2, 1, Collections.singletonList("f6")); + assertSplit(splits.get(4), "-5", 3, 0, Collections.singletonList("f7")); + assertSplit(splits.get(5), "-6", 3, 1, Collections.singletonList("f8")); + assertSplit(splits.get(6), "-7", 4, 0, Collections.singletonList("f9")); + assertSplit(splits.get(7), "-8", 4, 1, Collections.singletonList("f10")); + assertSplit(splits.get(8), "-9", 5, 0, Collections.singletonList("f11")); + assertSplit(splits.get(9), "-10", 5, 1, Collections.singletonList("f12")); + assertSplit(splits.get(10), "-11", 6, 0, Collections.singletonList("f13")); + assertSplit(splits.get(11), "-12", 6, 1, Collections.singletonList("f14")); } private void assertSplit( - FileStoreSourceSplit split, String splitId, int part, int bucket, List files) { - assertThat(split.splitId()).isEqualTo(splitId); + FileStoreSourceSplit split, + String splitIdSuffix, + int part, + int bucket, + List files) { + assertThat(split.splitId()).endsWith(splitIdSuffix); assertThat(((DataSplit) split.split()).partition().getInt(0)).isEqualTo(part); assertThat(((DataSplit) split.split()).bucket()).isEqualTo(bucket); assertThat( diff --git a/paimon-format/src/main/resources/META-INF/NOTICE b/paimon-format/src/main/resources/META-INF/NOTICE index 4fc2e2c9e5a6..c90ea4471730 100644 --- a/paimon-format/src/main/resources/META-INF/NOTICE +++ b/paimon-format/src/main/resources/META-INF/NOTICE @@ -19,12 +19,12 @@ This project bundles the following dependencies under the Apache Software Licens - com.fasterxml.jackson.core:jackson-annotations:2.14.2 - org.apache.commons:commons-compress:1.22 -- org.apache.parquet:parquet-hadoop:1.15.1 -- org.apache.parquet:parquet-column:1.15.1 -- org.apache.parquet:parquet-common:1.15.1 -- org.apache.parquet:parquet-encoding:1.15.1 -- org.apache.parquet:parquet-format-structures:1.15.1 -- org.apache.parquet:parquet-jackson:1.15.1 +- org.apache.parquet:parquet-hadoop:1.15.2 +- org.apache.parquet:parquet-column:1.15.2 +- org.apache.parquet:parquet-common:1.15.2 +- org.apache.parquet:parquet-encoding:1.15.2 +- org.apache.parquet:parquet-format-structures:1.15.2 +- org.apache.parquet:parquet-jackson:1.15.2 - commons-pool:commons-pool:1.6 This project bundles the following dependencies under the BSD license. diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/CloneActionITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/CloneActionITCase.java index 9c49ee92d53d..8f735b3aa3e8 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/CloneActionITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/procedure/CloneActionITCase.java @@ -40,6 +40,7 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import java.util.ArrayList; @@ -55,6 +56,7 @@ import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; /** Tests for {@link CloneAction}. */ +@Disabled // TODO fix unstable cases public class CloneActionITCase extends ActionITCaseBase { private static final TestHiveMetastore TEST_HIVE_METASTORE = new TestHiveMetastore(); diff --git a/paimon-lance/pom.xml b/paimon-lance/pom.xml index 0bd3c4f3d98e..401860a98c84 100644 --- a/paimon-lance/pom.xml +++ b/paimon-lance/pom.xml @@ -107,70 +107,25 @@ under the License. - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-paimon - package - - shade - - - - - - com.lancedb:lance-core - org.questdb:jar-jni - com.google.flatbuffers:flatbuffers-java - - - org.apache.paimon:paimon-arrow - org.apache.arrow:arrow-vector - org.apache.arrow:arrow-format - org.apache.arrow:arrow-memory-core - org.apache.arrow:arrow-memory-unsafe - org.apache.arrow:arrow-c-data - org.eclipse.collections:eclipse-collections - org.eclipse.collections:eclipse-collections-api - - - com.fasterxml.jackson.core:jackson-core - com.fasterxml.jackson.core:jackson-databind - com.fasterxml.jackson.core:jackson-annotations - com.fasterxml.jackson.datatype:jackson-datatype-jsr310 - - - - - - * - - META-INF/LICENSE.txt - LICENSE-EPL-1.0.txt - LICENSE-EDL-1.0.txt - about.html - - - - - - - com.fasterxml.jackson - org.apache.paimon.shade.lance.com.fasterxml.jackson - - - org.eclipse.collections - org.apache.paimon.shade.lance.org.eclipse.collections - - - - - - - - + + + + + + + + + + + + + + + + + + + + + diff --git a/paimon-lance/src/main/resources/META-INF/NOTICE b/paimon-lance/src/main/resources/META-INF/NOTICE deleted file mode 100644 index f294c7186d6a..000000000000 --- a/paimon-lance/src/main/resources/META-INF/NOTICE +++ /dev/null @@ -1,10 +0,0 @@ -paimon-format-lance -Copyright 2023-2025 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - -This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - -- com.lancedb:lancedb-core:0.30.0 - diff --git a/paimon-lance/src/test/java/org/apache/paimon/format/lance/LanceFileFormatTest.java b/paimon-lance/src/test/java/org/apache/paimon/format/lance/LanceFileFormatTest.java index c731ce259011..223a28246871 100644 --- a/paimon-lance/src/test/java/org/apache/paimon/format/lance/LanceFileFormatTest.java +++ b/paimon-lance/src/test/java/org/apache/paimon/format/lance/LanceFileFormatTest.java @@ -19,6 +19,8 @@ package org.apache.paimon.format.lance; import org.apache.paimon.format.FileFormatFactory; +import org.apache.paimon.format.FormatReaderFactory; +import org.apache.paimon.format.FormatWriterFactory; import org.apache.paimon.options.Options; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -48,7 +50,8 @@ public void testCreateWriterFactory() { } @Test - public void testValidateDataFields_UnsupportedType() { + public void testValidateDataFields_UnsupportedType_Map() { + // Test validation throws exception for unsupported MAP type LanceFileFormat format = new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); RowType rowType = RowType.of(DataTypes.MAP(DataTypes.STRING(), DataTypes.INT())); @@ -56,11 +59,21 @@ public void testValidateDataFields_UnsupportedType() { } @Test - public void testValidateDataFields_SupportedTypes() { + public void testValidateDataFields_UnsupportedType_LocalZonedTimestamp() { + // Test validation throws exception for unsupported TIMESTAMP_WITH_LOCAL_TIME_ZONE type + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + RowType rowType = RowType.of(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()); + assertThrows( + UnsupportedOperationException.class, () -> format.validateDataFields(rowType)); + } + + @Test + public void testValidateDataFields_SupportedTypes_Basic() { + // Test validation passes for basic supported types LanceFileFormat format = new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); - // Define a row type with various supported data types RowType rowType = RowType.builder() .field("id", DataTypes.INT()) @@ -74,7 +87,211 @@ public void testValidateDataFields_SupportedTypes() { .field("array", DataTypes.ARRAY(DataTypes.INT())) .build(); - // Validate that no exception is thrown for supported types assertDoesNotThrow(() -> format.validateDataFields(rowType)); } -} + + @Test + public void testValidateDataFields_AllNumericTypes() { + // Test validation passes for all numeric types + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + + RowType rowType = + RowType.builder() + .field("tinyInt", DataTypes.TINYINT()) + .field("smallInt", DataTypes.SMALLINT()) + .field("int", DataTypes.INT()) + .field("bigInt", DataTypes.BIGINT()) + .field("float", DataTypes.FLOAT()) + .field("double", DataTypes.DOUBLE()) + .field("decimal", DataTypes.DECIMAL(20, 6)) + .build(); + + assertDoesNotThrow(() -> format.validateDataFields(rowType)); + } + + @Test + public void testValidateDataFields_AllStringTypes() { + // Test validation passes for all string and binary types + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + + RowType rowType = + RowType.builder() + .field("char", DataTypes.CHAR(10)) + .field("varChar", DataTypes.VARCHAR(100)) + .field("binary", DataTypes.BINARY(50)) + .field("varBinary", DataTypes.VARBINARY(100)) + .build(); + + assertDoesNotThrow(() -> format.validateDataFields(rowType)); + } + + @Test + public void testValidateDataFields_TimeTypes() { + // Test validation passes for all time-related types with different precisions + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + + RowType rowType = + RowType.builder() + .field("date", DataTypes.DATE()) + .field("time", DataTypes.TIME()) + .field("timestamp", DataTypes.TIMESTAMP()) + .field("timestamp3", DataTypes.TIMESTAMP(3)) + .field("timestamp9", DataTypes.TIMESTAMP(9)) + .build(); + + assertDoesNotThrow(() -> format.validateDataFields(rowType)); + } + + @Test + public void testValidateDataFields_ComplexTypes() { + // Test validation passes for complex types: arrays, multisets, and variants + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + + RowType rowType = + RowType.builder() + .field("intArray", DataTypes.ARRAY(DataTypes.INT())) + .field("stringArray", DataTypes.ARRAY(DataTypes.STRING())) + .field("nestedArray", DataTypes.ARRAY(DataTypes.ARRAY(DataTypes.INT()))) + .field("multiset", DataTypes.MULTISET(DataTypes.STRING())) + .field("variant", DataTypes.VARIANT()) + .build(); + + assertDoesNotThrow(() -> format.validateDataFields(rowType)); + } + + @Test + public void testValidateDataFields_NestedRowType() { + // Test validation passes for nested RowType structures + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + + RowType nestedType = + RowType.builder() + .field("nestedId", DataTypes.INT()) + .field("nestedName", DataTypes.STRING()) + .build(); + + RowType rowType = + RowType.builder() + .field("id", DataTypes.INT()) + .field("name", DataTypes.STRING()) + .field("nested", nestedType) + .build(); + + assertDoesNotThrow(() -> format.validateDataFields(rowType)); + } + + @Test + public void testReaderFactory_WithProjectedTypes() { + // Test reader factory creation with projected types for column pruning + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 2048, 4096)); + + RowType fullType = + RowType.builder() + .field("id", DataTypes.INT()) + .field("name", DataTypes.STRING()) + .field("salary", DataTypes.DOUBLE()) + .field("department", DataTypes.STRING()) + .build(); + + RowType projectedType = + RowType.builder() + .field("id", DataTypes.INT()) + .field("name", DataTypes.STRING()) + .build(); + + FormatReaderFactory readerFactory = + format.createReaderFactory(fullType, projectedType, null); + + assertDoesNotThrow(() -> readerFactory); + assertDoesNotThrow(() -> ((LanceReaderFactory) readerFactory)); + } + + @Test + public void testReaderFactory_BatchSizeConfiguration() { + // Test reader factory with configured batch size + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + + RowType rowType = RowType.of(DataTypes.INT(), DataTypes.STRING()); + FormatReaderFactory readerFactory = format.createReaderFactory(rowType, rowType, null); + + assertDoesNotThrow(() -> ((LanceReaderFactory) readerFactory)); + } + + @Test + public void testWriterFactory_BatchSizeConfiguration() { + // Test writer factory with configured batch size and memory settings + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 2048, 8192)); + + RowType rowType = + RowType.builder() + .field("id", DataTypes.INT()) + .field("data", DataTypes.STRING()) + .field("value", DataTypes.DOUBLE()) + .build(); + + FormatWriterFactory writerFactory = format.createWriterFactory(rowType); + + assertDoesNotThrow(() -> writerFactory); + assertDoesNotThrow(() -> ((LanceWriterFactory) writerFactory)); + } + + @Test + public void testValidateDataFields_EmptyRowType() { + // Test validation passes for empty RowType + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + + RowType rowType = RowType.builder().build(); + + assertDoesNotThrow(() -> format.validateDataFields(rowType)); + } + + @Test + public void testValidateDataFields_SingleFieldTypes() { + // Test validation passes for single field types + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + + assertDoesNotThrow(() -> format.validateDataFields(RowType.of(DataTypes.INT()))); + assertDoesNotThrow(() -> format.validateDataFields(RowType.of(DataTypes.STRING()))); + assertDoesNotThrow(() -> format.validateDataFields(RowType.of(DataTypes.BOOLEAN()))); + assertDoesNotThrow(() -> format.validateDataFields(RowType.of(DataTypes.DATE()))); + assertDoesNotThrow(() -> format.validateDataFields(RowType.of(DataTypes.ARRAY(DataTypes.INT())))); + } + + @Test + public void testValidateDataFields_MixedArrayTypes() { + // Test validation passes for mixed array element types + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + + RowType rowType = + RowType.builder() + .field("intArray", DataTypes.ARRAY(DataTypes.INT())) + .field("stringArray", DataTypes.ARRAY(DataTypes.STRING())) + .field("doubleArray", DataTypes.ARRAY(DataTypes.DOUBLE())) + .field("booleanArray", DataTypes.ARRAY(DataTypes.BOOLEAN())) + .build(); + + assertDoesNotThrow(() -> format.validateDataFields(rowType)); + } + + @Test + public void testValidateDataFields_VariantType() { + // Test validation passes for VARIANT type + LanceFileFormat format = + new LanceFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + + RowType rowType = RowType.of(DataTypes.VARIANT()); + + assertDoesNotThrow(() -> format.validateDataFields(rowType)); + } +} \ No newline at end of file diff --git a/paimon-python/LICENSE b/paimon-python/LICENSE new file mode 100644 index 000000000000..d64569567334 --- /dev/null +++ b/paimon-python/LICENSE @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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/MANIFEST.in b/paimon-python/MANIFEST.in new file mode 100644 index 000000000000..fa6d79637e53 --- /dev/null +++ b/paimon-python/MANIFEST.in @@ -0,0 +1,20 @@ +################################################################################ +# 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. +################################################################################ +include dev/requirements.txt +include README.md +recursive-include pypaimon *.py \ No newline at end of file diff --git a/paimon-python/README.md b/paimon-python/README.md index 885125cb0024..0434536e8607 100644 --- a/paimon-python/README.md +++ b/paimon-python/README.md @@ -12,21 +12,8 @@ Pypaimon requires Python 3.10+ version. # Dependencies -Be building, you should install python setuptools: - -```commandline -brew install python-setuptools -pip3 install readerwriterlock -pip3 install fsspec -pip3 install cachetools -pip3 install ossfs -pip3 install pyarrow -pip3 install polars -pip3 install fastavro -pip3 install pandas -pip3 install ray -pip3 install duckdb -``` +The core dependencies are listed in `dev/requirements.txt`. +The development dependencies are listed in `dev/requirements-dev.txt`. # Build @@ -36,4 +23,11 @@ You can build the source package by executing the following command: python3 setup.py sdist ``` -The package is under `dist/`. +The package is under `dist/`. Then you can install the package by executing the following command: + +```commandline +pip3 install dist/*.tar.gz +``` + +The command will install the package and core dependencies to your local Python environment. + diff --git a/paimon-python/dev/lint-python.sh b/paimon-python/dev/lint-python.sh index 9cbb612137f8..fb57bfdd4191 100755 --- a/paimon-python/dev/lint-python.sh +++ b/paimon-python/dev/lint-python.sh @@ -119,13 +119,12 @@ function get_all_supported_checks() { # exec all selected check stages function check_stage() { print_function "STAGE" "checks starting" - for fun in ${SUPPORT_CHECKS[@]}; do + for fun in "${SUPPORT_CHECKS[@]}"; do $fun done echo "All the checks are finished, the detailed information can be found in: $LOG_FILE" } - ###############################################################All Checks Definitions############################################################### ######################### # This part defines all check functions such as tox_check and flake8_check @@ -176,7 +175,7 @@ function pytest_check() { TEST_DIR="pypaimon/tests/py36" echo "Running tests for Python 3.6: $TEST_DIR" else - TEST_DIR="pypaimon/tests --ignore=pypaimon/tests/py36" + TEST_DIR="pypaimon/tests --ignore=pypaimon/tests/py36 --ignore=pypaimon/tests/e2e" echo "Running tests for Python $PYTHON_VERSION (excluding py36): pypaimon/tests --ignore=pypaimon/tests/py36" fi @@ -194,6 +193,43 @@ function pytest_check() { print_function "STAGE" "pytest checks... [SUCCESS]" fi } + +# Mixed tests check - runs Java-Python interoperability tests +function mixed_check() { + # Get Python version + PYTHON_VERSION=$(python -c "import sys; print(f'{sys.version_info.major}.{sys.version_info.minor}')") + echo "Detected Python version: $PYTHON_VERSION" + if [ "$PYTHON_VERSION" = "3.6" ]; then + print_function "STAGE" "mixed tests checks... [SKIPPED]" + return + fi + print_function "STAGE" "mixed tests checks" + + # Path to the mixed tests script + MIXED_TESTS_SCRIPT="$CURRENT_DIR/dev/run_mixed_tests.sh" + + if [ ! -f "$MIXED_TESTS_SCRIPT" ]; then + echo "Mixed tests script not found at: $MIXED_TESTS_SCRIPT" + print_function "STAGE" "mixed tests checks... [FAILED]" + exit 1 + fi + + # Make sure the script is executable + chmod +x "$MIXED_TESTS_SCRIPT" + + # Run the mixed tests script + set -o pipefail + ($MIXED_TESTS_SCRIPT) 2>&1 | tee -a $LOG_FILE + + MIXED_TESTS_STATUS=$? + if [ $MIXED_TESTS_STATUS -ne 0 ]; then + print_function "STAGE" "mixed tests checks... [FAILED]" + # Stop the running script. + exit 1; + else + print_function "STAGE" "mixed tests checks... [SUCCESS]" + fi +} ###############################################################All Checks Definitions############################################################### # CURRENT_DIR is "paimon-python/" SCRIPT_PATH="$(readlink -f "$0")" @@ -236,15 +272,16 @@ INCLUDE_CHECKS="" USAGE=" usage: $0 [options] -h print this help message and exit --e [tox,flake8,sphinx,mypy] +-e [tox,flake8,sphinx,mypy,mixed] exclude checks which split by comma(,) --i [tox,flake8,sphinx,mypy] +-i [tox,flake8,sphinx,mypy,mixed] include checks which split by comma(,) -l list all checks supported. Examples: ./lint-python.sh => exec all checks. ./lint-python.sh -e tox,flake8 => exclude checks tox,flake8. ./lint-python.sh -i flake8 => include checks flake8. + ./lint-python.sh -i mixed => include checks mixed. ./lint-python.sh -l => list all checks supported. " while getopts "hfs:i:e:lr" arg; do @@ -261,7 +298,7 @@ while getopts "hfs:i:e:lr" arg; do ;; l) printf "current supported checks includes:\n" - for fun in ${SUPPORT_CHECKS[@]}; do + for fun in "${SUPPORT_CHECKS[@]}"; do echo ${fun%%_check*} done exit 2 diff --git a/paimon-python/dev/requirements-dev.txt b/paimon-python/dev/requirements-dev.txt new file mode 100644 index 000000000000..0f61cccb68ad --- /dev/null +++ b/paimon-python/dev/requirements-dev.txt @@ -0,0 +1,25 @@ +################################################################################ +# 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. +################################################################################ +# Core dependencies for pypaimon are in requirements.txt + +# Test dependencies for pypaimon are as follows +duckdb==1.3.2 +flake8==4.0.1 +pytest~=7.0 +ray==2.48.0 +requests \ No newline at end of file diff --git a/paimon-python/dev/requirements.txt b/paimon-python/dev/requirements.txt new file mode 100644 index 000000000000..9f4b80b9bdd9 --- /dev/null +++ b/paimon-python/dev/requirements.txt @@ -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. +################################################################################ +# Core dependencies for pypaimon +cachetools==4.2.4; python_version=="3.6" +cachetools==5.3.3; python_version>"3.6" +dataclasses==0.8.0; python_version < "3.7" +fastavro==1.4.7; python_version<"3.9" +fastavro==1.11.1; python_version>="3.9" +fsspec==2021.10.1; python_version<"3.8" +fsspec==2024.3.1; python_version>="3.8" +ossfs==2021.8.0; python_version<"3.8" +ossfs==2023.12.0; python_version>="3.8" +packaging==21.3.0; python_version<"3.8" +packaging==25.0; python_version>="3.8" +pandas==1.1.5; python_version < "3.7" +pandas==1.3.5; python_version >= "3.7" and python_version < "3.9" +pandas==2.3.2; python_version >= "3.9" +polars==0.9.12; python_version<"3.8" +polars==1.8.0; python_version=="3.8" +polars==1.32.0; python_version>"3.8" +pyarrow==6.0.1; python_version < "3.8" +pyarrow==16; python_version >= "3.8" +readerwriterlock==1.0.9 +zstandard==0.19.0; python_version<"3.9" +zstandard==0.24.0; python_version>="3.9" \ No newline at end of file diff --git a/paimon-python/dev/run_mixed_tests.sh b/paimon-python/dev/run_mixed_tests.sh new file mode 100755 index 000000000000..8387e79eb9c4 --- /dev/null +++ b/paimon-python/dev/run_mixed_tests.sh @@ -0,0 +1,217 @@ +#!/bin/bash + +################################################################################ +# 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. +################################################################################ + +# Mixed Java and Python test runner +# This script runs Java test first, then Python test to verify interoperability + +set -e # Exit on any error + +# Colors for output +RED='\033[0;31m' +GREEN='\033[0;32m' +YELLOW='\033[1;33m' +NC='\033[0m' # No Color + +# Get script directory +SCRIPT_DIR="$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)" +PROJECT_ROOT="$(cd "$SCRIPT_DIR/../.." && pwd)" +PAIMON_PYTHON_DIR="$PROJECT_ROOT/paimon-python/pypaimon/tests/e2e" +PAIMON_CORE_DIR="$PROJECT_ROOT/paimon-core" + +echo -e "${YELLOW}=== Mixed Java-Python Read Write Test Runner ===${NC}" +echo "Project root: $PROJECT_ROOT" +echo "Paimon Python dir: $PAIMON_PYTHON_DIR" +echo "Paimon Core dir: $PAIMON_CORE_DIR" +echo "" + +# Function to clean up warehouse directory +cleanup_warehouse() { + echo -e "${YELLOW}=== Cleaning up warehouse directory ===${NC}" + + local warehouse_dir="$PAIMON_PYTHON_DIR/warehouse" + + if [[ -d "$warehouse_dir" ]]; then + echo "Removing warehouse directory: $warehouse_dir" + rm -rf "$warehouse_dir" + echo -e "${GREEN}✓ Warehouse directory cleaned up successfully${NC}" + else + echo "Warehouse directory does not exist, no cleanup needed" + fi + + echo "" +} + +# Function to run Java test +run_java_write_test() { + echo -e "${YELLOW}=== Step 1: Running Java Test (JavaPyE2ETest.testJavaWriteRead) ===${NC}" + + cd "$PROJECT_ROOT" + + # Run the specific Java test method + echo "Running Maven test for JavaPyE2ETest.testJavaWriteRead..." + if mvn test -Dtest=org.apache.paimon.JavaPyE2ETest#testJavaWriteRead -pl paimon-core -q -Drun.e2e.tests=true; then + echo -e "${GREEN}✓ Java test completed successfully${NC}" + return 0 + else + echo -e "${RED}✗ Java test failed${NC}" + return 1 + fi +} + +# Function to run Python test +run_python_read_test() { + echo -e "${YELLOW}=== Step 2: Running Python Test (JavaPyReadWriteTest.testRead) ===${NC}" + + cd "$PAIMON_PYTHON_DIR" + + # Run the specific Python test method + echo "Running Python test for JavaPyReadWriteTest.testRead..." + if python -m pytest java_py_read_write_test.py::JavaPyReadWriteTest::test_read -v; then + echo -e "${GREEN}✓ Python test completed successfully${NC}" +# source deactivate + return 0 + else + echo -e "${RED}✗ Python test failed${NC}" +# source deactivate + return 1 + fi +} + +# Function to run Python Write test for Python-Write-Java-Read scenario +run_python_write_test() { + echo -e "${YELLOW}=== Step 3: Running Python Write Test (JavaPyReadWriteTest.test_py_write_read) ===${NC}" + + cd "$PAIMON_PYTHON_DIR" + + # Run the specific Python test method for writing data + echo "Running Python test for JavaPyReadWriteTest.test_py_write_read (Python Write)..." + if python -m pytest java_py_read_write_test.py::JavaPyReadWriteTest::test_py_write_read -v; then + echo -e "${GREEN}✓ Python write test completed successfully${NC}" + return 0 + else + echo -e "${RED}✗ Python write test failed${NC}" + return 1 + fi +} + +# Function to run Java Read test for Python-Write-Java-Read scenario +run_java_read_test() { + echo -e "${YELLOW}=== Step 4: Running Java Read Test (JavaPyE2ETest.testRead) ===${NC}" + + cd "$PROJECT_ROOT" + + # Run the specific Java test method for reading Python-written data + echo "Running Maven test for JavaPyE2ETest.testRead (Java Read)..." + if mvn test -Dtest=org.apache.paimon.JavaPyE2ETest#testRead -pl paimon-core -q -Drun.e2e.tests=true; then + echo -e "${GREEN}✓ Java read test completed successfully${NC}" + return 0 + else + echo -e "${RED}✗ Java read test failed${NC}" + return 1 + fi +} + +# Main execution +main() { + local java_write_result=0 + local python_read_result=0 + local python_write_result=0 + local java_read_result=0 + + echo -e "${YELLOW}Starting mixed language test execution...${NC}" + echo "" + + # Run Java write test + if ! run_java_write_test; then + java_write_result=1 + echo -e "${RED}Java test failed, but continuing with Python test...${NC}" + echo "" + else + echo "" + fi + + # Run Python read test + if ! run_python_read_test; then + python_read_result=1 + fi + + echo "" + + # Run Python Write - Java Read test sequence + echo -e "${YELLOW}Starting Python Write - Java Read test sequence...${NC}" + echo "" + + # Run Python write test + if ! run_python_write_test; then + python_write_result=1 + echo -e "${RED}Python write test failed, but continuing with Java read test...${NC}" + echo "" + else + echo "" + fi + + # Run Java read test + if ! run_java_read_test; then + java_read_result=1 + fi + + echo "" + echo -e "${YELLOW}=== Test Results Summary ===${NC}" + + if [[ $java_write_result -eq 0 ]]; then + echo -e "${GREEN}✓ Java Write Test (JavaPyE2ETest.testJavaWriteRead): PASSED${NC}" + else + echo -e "${RED}✗ Java Write Test (JavaPyE2ETest.testJavaWriteRead): FAILED${NC}" + fi + + if [[ $python_read_result -eq 0 ]]; then + echo -e "${GREEN}✓ Python Read Test (JavaPyReadWriteTest.testRead): PASSED${NC}" + else + echo -e "${RED}✗ Python Read Test (JavaPyReadWriteTest.testRead): FAILED${NC}" + fi + + if [[ $python_write_result -eq 0 ]]; then + echo -e "${GREEN}✓ Python Write Test (JavaPyReadWriteTest.test_py_write_read): PASSED${NC}" + else + echo -e "${RED}✗ Python Write Test (JavaPyReadWriteTest.test_py_write_read): FAILED${NC}" + fi + + if [[ $java_read_result -eq 0 ]]; then + echo -e "${GREEN}✓ Java Read Test (JavaPyE2ETest.testRead): PASSED${NC}" + else + echo -e "${RED}✗ Java Read Test (JavaPyE2ETest.testRead): FAILED${NC}" + fi + + echo "" + + # Clean up warehouse directory after all tests + cleanup_warehouse + + if [[ $java_write_result -eq 0 && $python_read_result -eq 0 && $python_write_result -eq 0 && $java_read_result -eq 0 ]]; then + echo -e "${GREEN}🎉 All tests passed! Java-Python interoperability verified.${NC}" + return 0 + else + echo -e "${RED}❌ Some tests failed. Please check the output above.${NC}" + return 1 + fi +} + +# Run main function +main "$@" \ No newline at end of file diff --git a/paimon-python/pypaimon/__init__.py b/paimon-python/pypaimon/__init__.py index 060529a185e2..5313e8e18aaf 100644 --- a/paimon-python/pypaimon/__init__.py +++ b/paimon-python/pypaimon/__init__.py @@ -15,8 +15,8 @@ # specific language governing permissions and limitations # under the License. -from pypaimon.filesystem.pvfs import PaimonVirtualFileSystem from pypaimon.catalog.catalog_factory import CatalogFactory +from pypaimon.filesystem.pvfs import PaimonVirtualFileSystem from pypaimon.schema.schema import Schema __all__ = [ diff --git a/paimon-python/pypaimon/api/rest_api.py b/paimon-python/pypaimon/api/rest_api.py old mode 100644 new mode 100755 index 037a531e58f0..d715c8a4aaf8 --- a/paimon-python/pypaimon/api/rest_api.py +++ b/paimon-python/pypaimon/api/rest_api.py @@ -219,15 +219,15 @@ def list_tables_paged( def create_table(self, identifier: Identifier, schema: Schema) -> None: request = CreateTableRequest(identifier, schema) return self.client.post( - self.resource_paths.tables(identifier.database_name), + self.resource_paths.tables(identifier.get_database_name()), request, self.rest_auth_function) def get_table(self, identifier: Identifier) -> GetTableResponse: return self.client.get( self.resource_paths.table( - identifier.database_name, - identifier.object_name), + identifier.get_database_name(), + identifier.get_object_name()), GetTableResponse, self.rest_auth_function, ) @@ -235,8 +235,8 @@ def get_table(self, identifier: Identifier) -> GetTableResponse: def drop_table(self, identifier: Identifier) -> GetTableResponse: return self.client.delete( self.resource_paths.table( - identifier.database_name, - identifier.object_name), + identifier.get_database_name(), + identifier.get_object_name()), self.rest_auth_function, ) @@ -250,8 +250,8 @@ def rename_table(self, source_identifier: Identifier, target_identifier: Identif def load_table_token(self, identifier: Identifier) -> GetTableTokenResponse: return self.client.get( self.resource_paths.table_token( - identifier.database_name, - identifier.object_name), + identifier.get_database_name(), + identifier.get_object_name()), GetTableTokenResponse, self.rest_auth_function, ) @@ -282,7 +282,7 @@ def commit_snapshot( request = CommitTableRequest(table_uuid, snapshot, statistics) response = self.client.post_with_response_type( self.resource_paths.commit_table( - identifier.database_name, identifier.object_name), + identifier.get_database_name(), identifier.get_object_name()), request, CommitTableResponse, self.rest_auth_function diff --git a/paimon-python/pypaimon/catalog/rest/rest_catalog.py b/paimon-python/pypaimon/catalog/rest/rest_catalog.py index f203169fbde0..53db2abbaa00 100644 --- a/paimon-python/pypaimon/catalog/rest/rest_catalog.py +++ b/paimon-python/pypaimon/catalog/rest/rest_catalog.py @@ -19,14 +19,18 @@ from typing import Any, Callable, Dict, List, Optional, Union from urllib.parse import urlparse +import pyarrow +from packaging.version import parse + from pypaimon.api.api_response import GetTableResponse, PagedList from pypaimon.api.options import Options from pypaimon.api.rest_api import RESTApi -from pypaimon.api.rest_exception import NoSuchResourceException +from pypaimon.api.rest_exception import NoSuchResourceException, AlreadyExistsException from pypaimon.catalog.catalog import Catalog from pypaimon.catalog.catalog_context import CatalogContext from pypaimon.catalog.catalog_environment import CatalogEnvironment -from pypaimon.catalog.catalog_exception import TableNotExistException +from pypaimon.catalog.catalog_exception import TableNotExistException, DatabaseAlreadyExistException, \ + TableAlreadyExistException, DatabaseNotExistException from pypaimon.catalog.database import Database from pypaimon.catalog.rest.property_change import PropertyChange from pypaimon.catalog.rest.rest_token_file_io import RESTTokenFileIO @@ -107,7 +111,12 @@ def list_databases_paged(self, max_results: Optional[int] = None, page_token: Op return self.rest_api.list_databases_paged(max_results, page_token, database_name_pattern) def create_database(self, name: str, ignore_if_exists: bool, properties: Dict[str, str] = None): - self.rest_api.create_database(name, properties) + try: + self.rest_api.create_database(name, properties) + except AlreadyExistsException as e: + if not ignore_if_exists: + # Convert REST API exception to catalog exception + raise DatabaseAlreadyExistException(name) from e def get_database(self, name: str) -> Database: response = self.rest_api.get_database(name) @@ -117,8 +126,13 @@ def get_database(self, name: str) -> Database: if response is not None: return Database(name, options) - def drop_database(self, name: str): - self.rest_api.drop_database(name) + def drop_database(self, name: str, ignore_if_exists: bool = False): + try: + self.rest_api.drop_database(name) + except NoSuchResourceException as e: + if not ignore_if_exists: + # Convert REST API exception to catalog exception + raise DatabaseNotExistException(name) from e def alter_database(self, name: str, changes: List[PropertyChange]): set_properties, remove_keys = PropertyChange.get_set_properties_to_remove_keys(changes) @@ -154,12 +168,20 @@ def get_table(self, identifier: Union[str, Identifier]) -> FileStoreTable: def create_table(self, identifier: Union[str, Identifier], schema: Schema, ignore_if_exists: bool): if not isinstance(identifier, Identifier): identifier = Identifier.from_string(identifier) - self.rest_api.create_table(identifier, schema) + try: + self.rest_api.create_table(identifier, schema) + except AlreadyExistsException as e: + if not ignore_if_exists: + raise TableAlreadyExistException(identifier) from e - def drop_table(self, identifier: Union[str, Identifier]): + def drop_table(self, identifier: Union[str, Identifier], ignore_if_exists: bool = False): if not isinstance(identifier, Identifier): identifier = Identifier.from_string(identifier) - self.rest_api.drop_table(identifier) + try: + self.rest_api.drop_table(identifier) + except NoSuchResourceException as e: + if not ignore_if_exists: + raise TableNotExistException(identifier) from e def load_table_metadata(self, identifier: Identifier) -> TableMetadata: response = self.rest_api.get_table(identifier) @@ -181,17 +203,17 @@ def to_table_metadata(self, db: str, response: GetTableResponse) -> TableMetadat uuid=response.get_id() ) - def file_io_from_options(self, table_path: Path) -> FileIO: - return FileIO(str(table_path), self.context.options.data) + def file_io_from_options(self, table_path: str) -> FileIO: + return FileIO(table_path, self.context.options.data) - def file_io_for_data(self, table_path: Path, identifier: Identifier): + def file_io_for_data(self, table_path: str, identifier: Identifier): return RESTTokenFileIO(identifier, table_path, self.context.options.data) \ if self.data_token_enabled else self.file_io_from_options(table_path) def load_table(self, identifier: Identifier, - internal_file_io: Callable[[Path], Any], - external_file_io: Callable[[Path], Any], + internal_file_io: Callable[[str], Any], + external_file_io: Callable[[str], Any], metadata_loader: Callable[[Identifier], TableMetadata], ) -> FileStoreTable: metadata = metadata_loader(identifier) @@ -204,9 +226,12 @@ def load_table(self, supports_version_management=True # REST catalogs support version management ) path_parsed = urlparse(schema.options.get(CoreOptions.PATH)) - path = Path(path_parsed.path) if path_parsed.scheme is None else Path(schema.options.get(CoreOptions.PATH)) - table_path = path_parsed.netloc + "/" + path_parsed.path \ - if path_parsed.scheme == "file" else path_parsed.path[1:] + path = path_parsed.path if path_parsed.scheme is None else schema.options.get(CoreOptions.PATH) + if path_parsed.scheme == "file": + table_path = path_parsed.path + else: + table_path = path_parsed.netloc + path_parsed.path \ + if parse(pyarrow.__version__) >= parse("7.0.0") else path_parsed.path[1:] table = self.create(data_file_io(path), Path(table_path), schema, diff --git a/paimon-python/pypaimon/catalog/rest/rest_token_file_io.py b/paimon-python/pypaimon/catalog/rest/rest_token_file_io.py index b9671c8ae951..a65e96695ce7 100644 --- a/paimon-python/pypaimon/catalog/rest/rest_token_file_io.py +++ b/paimon-python/pypaimon/catalog/rest/rest_token_file_io.py @@ -31,7 +31,7 @@ class RESTTokenFileIO(FileIO): - def __init__(self, identifier: Identifier, path: Path, + def __init__(self, identifier: Identifier, path: str, catalog_options: Optional[dict] = None): self.identifier = identifier self.path = path @@ -39,12 +39,12 @@ def __init__(self, identifier: Identifier, path: Path, self.api_instance: Optional[RESTApi] = None self.lock = threading.Lock() self.log = logging.getLogger(__name__) - super().__init__(str(path), catalog_options) + super().__init__(path, catalog_options) - def _initialize_oss_fs(self) -> FileSystem: + def _initialize_oss_fs(self, path) -> FileSystem: self.try_to_refresh_token() self.properties.update(self.token.token) - return super()._initialize_oss_fs() + return super()._initialize_oss_fs(path) def new_output_stream(self, path: Path): return self.filesystem.open_output_stream(str(path)) diff --git a/paimon-python/pypaimon/common/config.py b/paimon-python/pypaimon/common/config.py index b3c9a673b3b1..81b05b8f8458 100644 --- a/paimon-python/pypaimon/common/config.py +++ b/paimon-python/pypaimon/common/config.py @@ -21,7 +21,6 @@ class OssOptions: OSS_SECURITY_TOKEN = "fs.oss.securityToken" OSS_ENDPOINT = "fs.oss.endpoint" OSS_REGION = "fs.oss.region" - OSS_BUCKET = "fs.oss.bucket" class S3Options: @@ -48,6 +47,7 @@ class CatalogOptions: DLF_TOKEN_ECS_METADATA_URL = "dlf.token-ecs-metadata-url" PREFIX = 'prefix' HTTP_USER_AGENT_HEADER = 'header.HTTP_USER_AGENT' + BLOB_FILE_IO_DEFAULT_CACHE_SIZE = 2**31 - 1 class PVFSOptions: diff --git a/paimon-python/pypaimon/common/core_options.py b/paimon-python/pypaimon/common/core_options.py index 1a060bd20692..d6643399bbf9 100644 --- a/paimon-python/pypaimon/common/core_options.py +++ b/paimon-python/pypaimon/common/core_options.py @@ -18,6 +18,8 @@ from enum import Enum +from pypaimon.common.memory_size import MemorySize + class CoreOptions(str, Enum): """Core options for paimon.""" @@ -33,16 +35,44 @@ def __str__(self): BUCKET = "bucket" BUCKET_KEY = "bucket-key" WAREHOUSE = "warehouse" + SCAN_MANIFEST_PARALLELISM = "scan.manifest.parallelism" # File format options FILE_FORMAT = "file.format" FILE_FORMAT_ORC = "orc" FILE_FORMAT_AVRO = "avro" FILE_FORMAT_PARQUET = "parquet" + FILE_FORMAT_BLOB = "blob" FILE_COMPRESSION = "file.compression" FILE_COMPRESSION_PER_LEVEL = "file.compression.per.level" FILE_FORMAT_PER_LEVEL = "file.format.per.level" FILE_BLOCK_SIZE = "file.block-size" + FILE_BLOB_AS_DESCRIPTOR = "blob-as-descriptor" # Scan options SCAN_FALLBACK_BRANCH = "scan.fallback-branch" + INCREMENTAL_BETWEEN_TIMESTAMP = "incremental-between-timestamp" + SOURCE_SPLIT_TARGET_SIZE = "source.split.target-size" + SOURCE_SPLIT_OPEN_FILE_COST = "source.split.open-file-cost" # Commit options COMMIT_USER_PREFIX = "commit.user-prefix" + ROW_TRACKING_ENABLED = "row-tracking.enabled" + DATA_EVOLUTION_ENABLED = "data-evolution.enabled" + + @staticmethod + def get_blob_as_descriptor(options: dict) -> bool: + return options.get(CoreOptions.FILE_BLOB_AS_DESCRIPTOR, "false").lower() == 'true' + + @staticmethod + def get_split_target_size(options: dict) -> int: + """Get split target size from options, default to 128MB.""" + if CoreOptions.SOURCE_SPLIT_TARGET_SIZE in options: + size_str = options[CoreOptions.SOURCE_SPLIT_TARGET_SIZE] + return MemorySize.parse(size_str).get_bytes() + return MemorySize.of_mebi_bytes(128).get_bytes() + + @staticmethod + def get_split_open_file_cost(options: dict) -> int: + """Get split open file cost from options, default to 4MB.""" + if CoreOptions.SOURCE_SPLIT_OPEN_FILE_COST in options: + cost_str = options[CoreOptions.SOURCE_SPLIT_OPEN_FILE_COST] + return MemorySize.parse(cost_str).get_bytes() + return MemorySize.of_mebi_bytes(4).get_bytes() diff --git a/paimon-python/pypaimon/common/delta_varint_compressor.py b/paimon-python/pypaimon/common/delta_varint_compressor.py new file mode 100644 index 000000000000..dc849886bf74 --- /dev/null +++ b/paimon-python/pypaimon/common/delta_varint_compressor.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 io +from typing import List + + +class DeltaVarintCompressor: + + @staticmethod + def compress(data: List[int]) -> bytes: + if not data: + return b'' + + # Estimate output size (conservative: 5 bytes per varint max) + out = io.BytesIO() + out.seek(0) + + # Encode first value directly + DeltaVarintCompressor._encode_varint(data[0], out) + + # Encode deltas without intermediate list creation + prev = data[0] + for i in range(1, len(data)): + current = data[i] + delta = current - prev + DeltaVarintCompressor._encode_varint(delta, out) + prev = current + + # Return only the used portion of the buffer + position = out.tell() + result = out.getvalue() + out.close() + return result[:position] + + @staticmethod + def decompress(compressed: bytes) -> List[int]: + if not compressed: + return [] + + # Fast path: decode directly into result without intermediate deltas list + in_stream = io.BytesIO(compressed) + result = [] + + try: + # Decode first value + first_value = DeltaVarintCompressor._decode_varint(in_stream) + result.append(first_value) + + # Decode and reconstruct remaining values in one pass + current_value = first_value + while True: + try: + delta = DeltaVarintCompressor._decode_varint(in_stream) + current_value += delta + result.append(current_value) + except RuntimeError: + # End of stream reached + break + + except RuntimeError: + # Handle empty stream case + pass + finally: + in_stream.close() + + return result + + @staticmethod + def _encode_varint(value: int, out: io.BytesIO) -> None: + # ZigZag encoding: maps signed integers to unsigned integers + if value >= 0: + zigzag = value << 1 + else: + zigzag = ((-value) << 1) - 1 + + # Varint encoding + while zigzag >= 0x80: + out.write(bytes([(zigzag & 0x7F) | 0x80])) + zigzag >>= 7 + out.write(bytes([zigzag & 0x7F])) + + @staticmethod + def _decode_varint(in_stream: io.BytesIO) -> int: + result = 0 + shift = 0 + while True: + byte_data = in_stream.read(1) + if not byte_data: + if shift == 0: + # Natural end of stream + raise RuntimeError("End of stream") + else: + # Unexpected end in middle of varint + raise RuntimeError("Unexpected end of input") + + b = byte_data[0] + result |= (b & 0x7F) << shift + if (b & 0x80) == 0: + break + + shift += 7 + if shift > 63: + raise RuntimeError("Varint overflow") + + # ZigZag decoding: maps unsigned integers back to signed integers + if result & 1: + return -((result + 1) >> 1) + else: + return result >> 1 diff --git a/paimon-python/pypaimon/common/file_io.py b/paimon-python/pypaimon/common/file_io.py index 809d65338cc3..f881ba77bc4a 100644 --- a/paimon-python/pypaimon/common/file_io.py +++ b/paimon-python/pypaimon/common/file_io.py @@ -15,7 +15,6 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ - import logging import os import subprocess @@ -26,17 +25,23 @@ import pyarrow from packaging.version import parse from pyarrow._fs import FileSystem - from pypaimon.common.config import OssOptions, S3Options +from pypaimon.common.uri_reader import UriReaderFactory +from pypaimon.schema.data_types import DataField, AtomicType, PyarrowFieldParser +from pypaimon.table.row.blob import BlobData, BlobDescriptor, Blob +from pypaimon.table.row.generic_row import GenericRow +from pypaimon.table.row.row_kind import RowKind +from pypaimon.write.blob_format_writer import BlobFormatWriter class FileIO: - def __init__(self, warehouse: str, catalog_options: dict): + def __init__(self, path: str, catalog_options: dict): self.properties = catalog_options self.logger = logging.getLogger(__name__) - scheme, netloc, path = self.parse_location(warehouse) + scheme, netloc, _ = self.parse_location(path) + self.uri_reader_factory = UriReaderFactory(catalog_options) if scheme in {"oss"}: - self.filesystem = self._initialize_oss_fs() + self.filesystem = self._initialize_oss_fs(path) elif scheme in {"s3", "s3a", "s3n"}: self.filesystem = self._initialize_s3_fs() elif scheme in {"hdfs", "viewfs"}: @@ -56,7 +61,29 @@ def parse_location(location: str): else: return uri.scheme, uri.netloc, f"{uri.netloc}{uri.path}" - def _initialize_oss_fs(self) -> FileSystem: + def _extract_oss_bucket(self, location) -> str: + uri = urlparse(location) + if uri.scheme and uri.scheme != "oss": + raise ValueError("Not an OSS URI: {}".format(location)) + + netloc = uri.netloc or "" + # parse oss://access_id:secret_key@Endpoint/bucket/path/to/object + if (getattr(uri, "username", None) or getattr(uri, "password", None)) or ("@" in netloc): + first_segment = uri.path.lstrip("/").split("/", 1)[0] + if not first_segment: + raise ValueError("Invalid OSS URI without bucket: {}".format(location)) + return first_segment + + # parse oss://bucket/... or oss://bucket.endpoint/... + host = getattr(uri, "hostname", None) or netloc + if not host: + raise ValueError("Invalid OSS URI without host: {}".format(location)) + bucket = host.split(".", 1)[0] + if not bucket: + raise ValueError("Invalid OSS URI without bucket: {}".format(location)) + return bucket + + def _initialize_oss_fs(self, path) -> FileSystem: from pyarrow.fs import S3FileSystem client_kwargs = { @@ -71,7 +98,8 @@ def _initialize_oss_fs(self) -> FileSystem: client_kwargs['force_virtual_addressing'] = True client_kwargs['endpoint_override'] = self.properties.get(OssOptions.OSS_ENDPOINT) else: - client_kwargs['endpoint_override'] = (self.properties.get(OssOptions.OSS_BUCKET) + "." + + oss_bucket = self._extract_oss_bucket(path) + client_kwargs['endpoint_override'] = (oss_bucket + "." + self.properties.get(OssOptions.OSS_ENDPOINT)) return S3FileSystem(**client_kwargs) @@ -341,3 +369,57 @@ def record_generator(): with self.new_output_stream(path) as output_stream: fastavro.writer(output_stream, avro_schema, records, **kwargs) + + def write_blob(self, path: Path, data: pyarrow.Table, blob_as_descriptor: bool, **kwargs): + try: + # Validate input constraints + if data.num_columns != 1: + raise RuntimeError(f"Blob format only supports a single column, got {data.num_columns} columns") + # Check for null values + column = data.column(0) + if column.null_count > 0: + raise RuntimeError("Blob format does not support null values") + # Convert PyArrow schema to Paimon DataFields + # For blob files, we expect exactly one blob column + field = data.schema[0] + if pyarrow.types.is_large_binary(field.type): + fields = [DataField(0, field.name, AtomicType("BLOB"))] + else: + # Convert other types as needed + paimon_type = PyarrowFieldParser.to_paimon_type(field.type, field.nullable) + fields = [DataField(0, field.name, paimon_type)] + # Convert PyArrow Table to records + records_dict = data.to_pydict() + num_rows = data.num_rows + field_name = fields[0].name + with self.new_output_stream(path) as output_stream: + writer = BlobFormatWriter(output_stream) + # Write each row + for i in range(num_rows): + col_data = records_dict[field_name][i] + # Convert to appropriate type based on field type + if hasattr(fields[0].type, 'type') and fields[0].type.type == "BLOB": + if blob_as_descriptor: + blob_descriptor = BlobDescriptor.deserialize(col_data) + uri_reader = self.uri_reader_factory.create(blob_descriptor.uri) + blob_data = Blob.from_descriptor(uri_reader, blob_descriptor) + elif isinstance(col_data, bytes): + blob_data = BlobData(col_data) + else: + # Convert to bytes if needed + if hasattr(col_data, 'as_py'): + col_data = col_data.as_py() + if isinstance(col_data, str): + col_data = col_data.encode('utf-8') + blob_data = BlobData(col_data) + row_values = [blob_data] + else: + row_values = [col_data] + # Create GenericRow and write + row = GenericRow(row_values, fields, RowKind.INSERT) + writer.add_element(row) + writer.close() + + except Exception as e: + self.delete_quietly(path) + raise RuntimeError(f"Failed to write blob file {path}: {e}") from e diff --git a/paimon-python/pypaimon/common/identifier.py b/paimon-python/pypaimon/common/identifier.py old mode 100644 new mode 100755 index 0731db50db99..6851a180741a --- a/paimon-python/pypaimon/common/identifier.py +++ b/paimon-python/pypaimon/common/identifier.py @@ -27,13 +27,13 @@ @dataclass class Identifier: - database_name: str = json_field("database", default=None) - object_name: str = json_field("object", default=None) - branch_name: Optional[str] = json_field("branch", default=None) + database: str = json_field("database", default=None) + object: str = json_field("object", default=None) + branch: Optional[str] = json_field("branch", default=None) @classmethod - def create(cls, database_name: str, object_name: str) -> "Identifier": - return cls(database_name, object_name) + def create(cls, database: str, object: str) -> "Identifier": + return cls(database, object) @classmethod def from_string(cls, full_name: str) -> "Identifier": @@ -46,21 +46,21 @@ def from_string(cls, full_name: str) -> "Identifier": raise ValueError("Invalid identifier format: {}".format(full_name)) def get_full_name(self) -> str: - if self.branch_name: - return "{}.{}.{}".format(self.database_name, self.object_name, self.branch_name) - return "{}.{}".format(self.database_name, self.object_name) + if self.branch: + return "{}.{}.{}".format(self.database, self.object, self.branch) + return "{}.{}".format(self.database, self.object) def get_database_name(self) -> str: - return self.database_name + return self.database def get_table_name(self) -> str: - return self.object_name + return self.object def get_object_name(self) -> str: - return self.object_name + return self.object def get_branch_name(self) -> Optional[str]: - return self.branch_name + return self.branch def is_system_table(self) -> bool: - return self.object_name.startswith('$') + return self.object.startswith('$') diff --git a/paimon-python/pypaimon/common/memory_size.py b/paimon-python/pypaimon/common/memory_size.py new file mode 100644 index 000000000000..b68ccfc2103c --- /dev/null +++ b/paimon-python/pypaimon/common/memory_size.py @@ -0,0 +1,201 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import Optional + + +class MemorySize: + """MemorySize is a representation of a number of bytes, viewable in different units.""" + + ZERO = None + MAX_VALUE = None + + def __init__(self, bytes: int): + """Constructs a new MemorySize.""" + if bytes < 0: + raise ValueError("bytes must be >= 0") + self.bytes = bytes + + @staticmethod + def of_mebi_bytes(mebi_bytes: int) -> 'MemorySize': + return MemorySize(mebi_bytes << 20) + + @staticmethod + def of_kibi_bytes(kibi_bytes: int) -> 'MemorySize': + return MemorySize(kibi_bytes << 10) + + @staticmethod + def of_bytes(bytes: int) -> 'MemorySize': + return MemorySize(bytes) + + def get_bytes(self) -> int: + return self.bytes + + def get_kibi_bytes(self) -> int: + return self.bytes >> 10 + + def get_mebi_bytes(self) -> int: + return self.bytes >> 20 + + def get_gibi_bytes(self) -> int: + return self.bytes >> 30 + + def get_tebi_bytes(self) -> int: + return self.bytes >> 40 + + def __eq__(self, other) -> bool: + return isinstance(other, MemorySize) and self.bytes == other.bytes + + def __hash__(self) -> int: + return hash(self.bytes) + + def __str__(self) -> str: + return self.format_to_string() + + def format_to_string(self) -> str: + ORDERED_UNITS = [MemoryUnit.BYTES, MemoryUnit.KILO_BYTES, MemoryUnit.MEGA_BYTES, + MemoryUnit.GIGA_BYTES, MemoryUnit.TERA_BYTES] + + highest_integer_unit = MemoryUnit.BYTES + for idx, unit in enumerate(ORDERED_UNITS): + if self.bytes % unit.multiplier != 0: + if idx == 0: + highest_integer_unit = ORDERED_UNITS[0] + else: + highest_integer_unit = ORDERED_UNITS[idx - 1] + break + else: + highest_integer_unit = MemoryUnit.BYTES + + return f"{self.bytes // highest_integer_unit.multiplier} {highest_integer_unit.units[1]}" + + def __repr__(self) -> str: + return f"MemorySize({self.bytes})" + + def __lt__(self, other: 'MemorySize') -> bool: + return self.bytes < other.bytes + + def __le__(self, other: 'MemorySize') -> bool: + return self.bytes <= other.bytes + + def __gt__(self, other: 'MemorySize') -> bool: + return self.bytes > other.bytes + + def __ge__(self, other: 'MemorySize') -> bool: + return self.bytes >= other.bytes + + @staticmethod + def parse(text: str) -> 'MemorySize': + return MemorySize(MemorySize.parse_bytes(text)) + + @staticmethod + def parse_bytes(text: str) -> int: + if text is None: + raise ValueError("text cannot be None") + + trimmed = text.strip() + if not trimmed: + raise ValueError("argument is an empty- or whitespace-only string") + + pos = 0 + while pos < len(trimmed) and trimmed[pos].isdigit(): + pos += 1 + + number_str = trimmed[:pos] + unit_str = trimmed[pos:].strip().lower() + + if not number_str: + raise ValueError("text does not start with a number") + + try: + value = int(number_str) + except ValueError: + raise ValueError( + f"The value '{number_str}' cannot be represented as 64bit number (numeric overflow).") + + unit = MemorySize._parse_unit(unit_str) + multiplier = unit.multiplier if unit else 1 + result = value * multiplier + + if result // multiplier != value: + raise ValueError( + f"The value '{text}' cannot be represented as 64bit number of bytes (numeric overflow).") + + return result + + @staticmethod + def _parse_unit(unit_str: str) -> Optional['MemoryUnit']: + if not unit_str: + return None + + for unit in [MemoryUnit.BYTES, MemoryUnit.KILO_BYTES, MemoryUnit.MEGA_BYTES, + MemoryUnit.GIGA_BYTES, MemoryUnit.TERA_BYTES]: + if unit_str in unit.units: + return unit + + raise ValueError( + f"Memory size unit '{unit_str}' does not match any of the recognized units: " + f"{MemoryUnit.get_all_units()}") + + +class MemoryUnit: + """Enum which defines memory unit, mostly used to parse value from configuration file.""" + + def __init__(self, units: list, multiplier: int): + self.units = units + self.multiplier = multiplier + + BYTES = None + KILO_BYTES = None + MEGA_BYTES = None + GIGA_BYTES = None + TERA_BYTES = None + + @staticmethod + def get_all_units() -> str: + all_units = [] + for unit in [MemoryUnit.BYTES, MemoryUnit.KILO_BYTES, MemoryUnit.MEGA_BYTES, + MemoryUnit.GIGA_BYTES, MemoryUnit.TERA_BYTES]: + all_units.append("(" + " | ".join(unit.units) + ")") + return " / ".join(all_units) + + @staticmethod + def has_unit(text: str) -> bool: + if text is None: + raise ValueError("text cannot be None") + + trimmed = text.strip() + if not trimmed: + raise ValueError("argument is an empty- or whitespace-only string") + + pos = 0 + while pos < len(trimmed) and trimmed[pos].isdigit(): + pos += 1 + + unit = trimmed[pos:].strip().lower() + return len(unit) > 0 + + +MemoryUnit.BYTES = MemoryUnit(["b", "bytes"], 1) +MemoryUnit.KILO_BYTES = MemoryUnit(["k", "kb", "kibibytes"], 1024) +MemoryUnit.MEGA_BYTES = MemoryUnit(["m", "mb", "mebibytes"], 1024 * 1024) +MemoryUnit.GIGA_BYTES = MemoryUnit(["g", "gb", "gibibytes"], 1024 * 1024 * 1024) +MemoryUnit.TERA_BYTES = MemoryUnit(["t", "tb", "tebibytes"], 1024 * 1024 * 1024 * 1024) + +MemorySize.ZERO = MemorySize(0) +MemorySize.MAX_VALUE = MemorySize(2**63 - 1) diff --git a/paimon-python/pypaimon/common/predicate.py b/paimon-python/pypaimon/common/predicate.py index c8a4070c6a84..9ae2cdfce37c 100644 --- a/paimon-python/pypaimon/common/predicate.py +++ b/paimon-python/pypaimon/common/predicate.py @@ -16,14 +16,17 @@ # limitations under the License. ################################################################################ +from abc import ABC, ABCMeta, abstractmethod from dataclasses import dataclass from functools import reduce from typing import Any, Dict, List, Optional +from typing import ClassVar import pyarrow from pyarrow import compute as pyarrow_compute from pyarrow import dataset as pyarrow_dataset +from pypaimon.manifest.schema.simple_stats import SimpleStats from pypaimon.table.row.internal_row import InternalRow @@ -34,176 +37,70 @@ class Predicate: field: Optional[str] literals: Optional[List[Any]] = None - def test(self, record: InternalRow) -> bool: - if self.method == 'equal': - return record.get_field(self.index) == self.literals[0] - elif self.method == 'notEqual': - return record.get_field(self.index) != self.literals[0] - elif self.method == 'lessThan': - return record.get_field(self.index) < self.literals[0] - elif self.method == 'lessOrEqual': - return record.get_field(self.index) <= self.literals[0] - elif self.method == 'greaterThan': - return record.get_field(self.index) > self.literals[0] - elif self.method == 'greaterOrEqual': - return record.get_field(self.index) >= self.literals[0] - elif self.method == 'isNull': - return record.get_field(self.index) is None - elif self.method == 'isNotNull': - return record.get_field(self.index) is not None - elif self.method == 'startsWith': - field_value = record.get_field(self.index) - if not isinstance(field_value, str): - return False - return field_value.startswith(self.literals[0]) - elif self.method == 'endsWith': - field_value = record.get_field(self.index) - if not isinstance(field_value, str): - return False - return field_value.endswith(self.literals[0]) - elif self.method == 'contains': - field_value = record.get_field(self.index) - if not isinstance(field_value, str): - return False - return self.literals[0] in field_value - elif self.method == 'in': - return record.get_field(self.index) in self.literals - elif self.method == 'notIn': - return record.get_field(self.index) not in self.literals - elif self.method == 'between': - field_value = record.get_field(self.index) - return self.literals[0] <= field_value <= self.literals[1] - elif self.method == 'and': - return all(p.test(record) for p in self.literals) - elif self.method == 'or': - t = any(p.test(record) for p in self.literals) - return t - else: - raise ValueError("Unsupported predicate method: {}".format(self.method)) + testers: ClassVar[Dict[str, Any]] = {} - def test_by_value(self, value: Any) -> bool: + def new_index(self, index: int): + return Predicate( + method=self.method, + index=index, + field=self.field, + literals=self.literals) + + def new_literals(self, literals: List[Any]): + return Predicate( + method=self.method, + index=self.index, + field=self.field, + literals=literals) + + def test(self, record: InternalRow) -> bool: if self.method == 'and': - return all(p.test_by_value(value) for p in self.literals) + return all(p.test(record) for p in self.literals) if self.method == 'or': - t = any(p.test_by_value(value) for p in self.literals) + t = any(p.test(record) for p in self.literals) return t - if self.method == 'equal': - return value == self.literals[0] - if self.method == 'notEqual': - return value != self.literals[0] - if self.method == 'lessThan': - return value < self.literals[0] - if self.method == 'lessOrEqual': - return value <= self.literals[0] - if self.method == 'greaterThan': - return value > self.literals[0] - if self.method == 'greaterOrEqual': - return value >= self.literals[0] - if self.method == 'isNull': - return value is None - if self.method == 'isNotNull': - return value is not None - if self.method == 'startsWith': - if not isinstance(value, str): - return False - return value.startswith(self.literals[0]) - if self.method == 'endsWith': - if not isinstance(value, str): - return False - return value.endswith(self.literals[0]) - if self.method == 'contains': - if not isinstance(value, str): - return False - return self.literals[0] in value - if self.method == 'in': - return value in self.literals - if self.method == 'notIn': - return value not in self.literals - if self.method == 'between': - return self.literals[0] <= value <= self.literals[1] - - raise ValueError("Unsupported predicate method: {}".format(self.method)) + field_value = record.get_field(self.index) + tester = Predicate.testers.get(self.method) + if tester: + return tester.test_by_value(field_value, self.literals) + raise ValueError(f"Unsupported predicate method: {self.method}") - def test_by_stats(self, stat: Dict) -> bool: + def test_by_simple_stats(self, stat: SimpleStats, row_count: int) -> bool: + """Test predicate against BinaryRow stats with denseIndexMapping like Java implementation.""" if self.method == 'and': - return all(p.test_by_stats(stat) for p in self.literals) + return all(p.test_by_simple_stats(stat, row_count) for p in self.literals) if self.method == 'or': - t = any(p.test_by_stats(stat) for p in self.literals) - return t + return any(p.test_by_simple_stats(stat, row_count) for p in self.literals) - null_count = stat["null_counts"][self.field] - row_count = stat["row_count"] + null_count = stat.null_counts[self.index] if self.method == 'isNull': return null_count is not None and null_count > 0 if self.method == 'isNotNull': return null_count is None or row_count is None or null_count < row_count - min_value = stat["min_values"][self.field] - max_value = stat["max_values"][self.field] + min_value = stat.min_values.get_field(self.index) + max_value = stat.max_values.get_field(self.index) if min_value is None or max_value is None or (null_count is not None and null_count == row_count): - return False - - if self.method == 'equal': - return min_value <= self.literals[0] <= max_value - if self.method == 'notEqual': - return not (min_value == self.literals[0] == max_value) - if self.method == 'lessThan': - return self.literals[0] > min_value - if self.method == 'lessOrEqual': - return self.literals[0] >= min_value - if self.method == 'greaterThan': - return self.literals[0] < max_value - if self.method == 'greaterOrEqual': - return self.literals[0] <= max_value - if self.method == 'startsWith': - if not isinstance(min_value, str) or not isinstance(max_value, str): - raise RuntimeError("startsWith predicate on non-str field") - return ((min_value.startswith(self.literals[0]) or min_value < self.literals[0]) - and (max_value.startswith(self.literals[0]) or max_value > self.literals[0])) - if self.method == 'endsWith': + # invalid stats, skip validation return True - if self.method == 'contains': - return True - if self.method == 'in': - for literal in self.literals: - if min_value <= literal <= max_value: - return True - return False - if self.method == 'notIn': - for literal in self.literals: - if min_value == literal == max_value: - return False - return True - if self.method == 'between': - return self.literals[0] <= max_value and self.literals[1] >= min_value - else: - raise ValueError("Unsupported predicate method: {}".format(self.method)) + + tester = Predicate.testers.get(self.method) + if tester: + return tester.test_by_stats(min_value, max_value, self.literals) + raise ValueError(f"Unsupported predicate method: {self.method}") def to_arrow(self) -> Any: - if self.method == 'equal': - return pyarrow_dataset.field(self.field) == self.literals[0] - elif self.method == 'notEqual': - return pyarrow_dataset.field(self.field) != self.literals[0] - elif self.method == 'lessThan': - return pyarrow_dataset.field(self.field) < self.literals[0] - elif self.method == 'lessOrEqual': - return pyarrow_dataset.field(self.field) <= self.literals[0] - elif self.method == 'greaterThan': - return pyarrow_dataset.field(self.field) > self.literals[0] - elif self.method == 'greaterOrEqual': - return pyarrow_dataset.field(self.field) >= self.literals[0] - elif self.method == 'isNull': - return pyarrow_dataset.field(self.field).is_null() - elif self.method == 'isNotNull': - return pyarrow_dataset.field(self.field).is_valid() - elif self.method == 'in': - return pyarrow_dataset.field(self.field).isin(self.literals) - elif self.method == 'notIn': - return ~pyarrow_dataset.field(self.field).isin(self.literals) - elif self.method == 'startsWith': + if self.method == 'and': + return reduce(lambda x, y: x & y, + [p.to_arrow() for p in self.literals]) + if self.method == 'or': + return reduce(lambda x, y: x | y, + [p.to_arrow() for p in self.literals]) + + if self.method == 'startsWith': pattern = self.literals[0] # For PyArrow compatibility - improved approach try: @@ -215,7 +112,7 @@ def to_arrow(self) -> Any: except Exception: # Fallback to True return pyarrow_dataset.field(self.field).is_valid() | pyarrow_dataset.field(self.field).is_null() - elif self.method == 'endsWith': + if self.method == 'endsWith': pattern = self.literals[0] # For PyArrow compatibility try: @@ -227,7 +124,7 @@ def to_arrow(self) -> Any: except Exception: # Fallback to True return pyarrow_dataset.field(self.field).is_valid() | pyarrow_dataset.field(self.field).is_null() - elif self.method == 'contains': + if self.method == 'contains': pattern = self.literals[0] # For PyArrow compatibility try: @@ -239,14 +136,223 @@ def to_arrow(self) -> Any: except Exception: # Fallback to True return pyarrow_dataset.field(self.field).is_valid() | pyarrow_dataset.field(self.field).is_null() - elif self.method == 'between': - return (pyarrow_dataset.field(self.field) >= self.literals[0]) & \ - (pyarrow_dataset.field(self.field) <= self.literals[1]) - elif self.method == 'and': - return reduce(lambda x, y: x & y, - [p.to_arrow() for p in self.literals]) - elif self.method == 'or': - return reduce(lambda x, y: x | y, - [p.to_arrow() for p in self.literals]) - else: - raise ValueError("Unsupported predicate method: {}".format(self.method)) + + field = pyarrow_dataset.field(self.field) + tester = Predicate.testers.get(self.method) + if tester: + return tester.test_by_arrow(field, self.literals) + + raise ValueError("Unsupported predicate method: {}".format(self.method)) + + +class RegisterMeta(ABCMeta): + def __init__(cls, name, bases, dct): + super().__init__(name, bases, dct) + if not bool(cls.__abstractmethods__): + Predicate.testers[cls.name] = cls() + + +class Tester(ABC, metaclass=RegisterMeta): + name = None + + @abstractmethod + def test_by_value(self, val, literals) -> bool: + """ + Test based on the specific val and literals. + """ + + @abstractmethod + def test_by_stats(self, min_v, max_v, literals) -> bool: + """ + Test based on the specific min_value and max_value and literals. + """ + + @abstractmethod + def test_by_arrow(self, val, literals) -> bool: + """ + Test based on the specific arrow value and literals. + """ + + +class Equal(Tester): + name = 'equal' + + def test_by_value(self, val, literals) -> bool: + return val == literals[0] + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return min_v <= literals[0] <= max_v + + def test_by_arrow(self, val, literals) -> bool: + return val == literals[0] + + +class NotEqual(Tester): + name = "notEqual" + + def test_by_value(self, val, literals) -> bool: + return val != literals[0] + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return not (min_v == literals[0] == max_v) + + def test_by_arrow(self, val, literals) -> bool: + return val != literals[0] + + +class LessThan(Tester): + name = "lessThan" + + def test_by_value(self, val, literals) -> bool: + return val < literals[0] + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return literals[0] > min_v + + def test_by_arrow(self, val, literals) -> bool: + return val < literals[0] + + +class LessOrEqual(Tester): + name = "lessOrEqual" + + def test_by_value(self, val, literals) -> bool: + return val <= literals[0] + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return literals[0] >= min_v + + def test_by_arrow(self, val, literals) -> bool: + return val <= literals[0] + + +class GreaterThan(Tester): + name = "greaterThan" + + def test_by_value(self, val, literals) -> bool: + return val > literals[0] + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return literals[0] < max_v + + def test_by_arrow(self, val, literals) -> bool: + return val > literals[0] + + +class GreaterOrEqual(Tester): + name = "greaterOrEqual" + + def test_by_value(self, val, literals) -> bool: + return val >= literals[0] + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return literals[0] <= max_v + + def test_by_arrow(self, val, literals) -> bool: + return val >= literals[0] + + +class In(Tester): + name = "in" + + def test_by_value(self, val, literals) -> bool: + return val in literals + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return any(min_v <= l <= max_v for l in literals) + + def test_by_arrow(self, val, literals) -> bool: + return val.isin(literals) + + +class NotIn(Tester): + name = "notIn" + + def test_by_value(self, val, literals) -> bool: + return val not in literals + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return not any(min_v == l == max_v for l in literals) + + def test_by_arrow(self, val, literals) -> bool: + return ~val.isin(literals) + + +class Between(Tester): + name = "between" + + def test_by_value(self, val, literals) -> bool: + return literals[0] <= val <= literals[1] + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return literals[0] <= max_v and literals[1] >= min_v + + def test_by_arrow(self, val, literals) -> bool: + return (val >= literals[0]) & (val <= literals[1]) + + +class StartsWith(Tester): + name = "startsWith" + + def test_by_value(self, val, literals) -> bool: + return isinstance(val, str) and val.startswith(literals[0]) + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return ((isinstance(min_v, str) and isinstance(max_v, str)) and + ((min_v.startswith(literals[0]) or min_v < literals[0]) and + (max_v.startswith(literals[0]) or max_v > literals[0]))) + + def test_by_arrow(self, val, literals) -> bool: + return True + + +class EndsWith(Tester): + name = "endsWith" + + def test_by_value(self, val, literals) -> bool: + return isinstance(val, str) and val.endswith(literals[0]) + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return True + + def test_by_arrow(self, val, literals) -> bool: + return True + + +class Contains(Tester): + name = "contains" + + def test_by_value(self, val, literals) -> bool: + return isinstance(val, str) and literals[0] in val + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return True + + def test_by_arrow(self, val, literals) -> bool: + return True + + +class IsNull(Tester): + name = "isNull" + + def test_by_value(self, val, literals) -> bool: + return val is None + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return True + + def test_by_arrow(self, val, literals) -> bool: + return val.is_null() + + +class IsNotNull(Tester): + name = "isNotNull" + + def test_by_value(self, val, literals) -> bool: + return val is not None + + def test_by_stats(self, min_v, max_v, literals) -> bool: + return True + + def test_by_arrow(self, val, literals) -> bool: + return val.is_valid() diff --git a/paimon-python/pypaimon/common/predicate_builder.py b/paimon-python/pypaimon/common/predicate_builder.py index bf668eed5050..7b96bb613482 100644 --- a/paimon-python/pypaimon/common/predicate_builder.py +++ b/paimon-python/pypaimon/common/predicate_builder.py @@ -101,8 +101,13 @@ def between(self, field: str, included_lower_bound: Any, included_upper_bound: A """Create a between predicate.""" return self._build_predicate('between', field, [included_lower_bound, included_upper_bound]) - def and_predicates(self, predicates: List[Predicate]) -> Predicate: + @staticmethod + def and_predicates(predicates: List[Predicate]) -> Optional[Predicate]: """Create an AND predicate from multiple predicates.""" + if len(predicates) == 0: + return None + if len(predicates) == 1: + return predicates[0] return Predicate( method='and', index=None, @@ -110,8 +115,13 @@ def and_predicates(self, predicates: List[Predicate]) -> Predicate: literals=predicates ) - def or_predicates(self, predicates: List[Predicate]) -> Predicate: + @staticmethod + def or_predicates(predicates: List[Predicate]) -> Optional[Predicate]: """Create an OR predicate from multiple predicates.""" + if len(predicates) == 0: + return None + if len(predicates) == 1: + return predicates[0] return Predicate( method='or', index=None, diff --git a/paimon-python/pypaimon/common/uri_reader.py b/paimon-python/pypaimon/common/uri_reader.py new file mode 100644 index 000000000000..823020caafe7 --- /dev/null +++ b/paimon-python/pypaimon/common/uri_reader.py @@ -0,0 +1,171 @@ +################################################################################ +# 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 io +from abc import ABC, abstractmethod +from pathlib import Path +from typing import Any, Optional +from urllib.parse import urlparse, ParseResult + +import requests +from cachetools import LRUCache +from readerwriterlock import rwlock + +from pypaimon.common.config import CatalogOptions + + +class UriReader(ABC): + @classmethod + def from_http(cls) -> 'HttpUriReader': + return HttpUriReader() + + @classmethod + def from_file(cls, file_io: Any) -> 'FileUriReader': + return FileUriReader(file_io) + + @classmethod + def get_file_path(cls, uri: str): + parsed_uri = urlparse(uri) + if parsed_uri.scheme == 'file': + path = Path(parsed_uri.path) + elif parsed_uri.scheme and parsed_uri.scheme != '': + path = Path(parsed_uri.netloc + parsed_uri.path) + else: + path = Path(uri) + return path + + @abstractmethod + def new_input_stream(self, uri: str): + pass + + +class FileUriReader(UriReader): + + def __init__(self, file_io: Any): + self._file_io = file_io + + def new_input_stream(self, uri: str): + try: + path = self.get_file_path(uri) + return self._file_io.new_input_stream(path) + except Exception as e: + raise IOError(f"Failed to read file {uri}: {e}") + + +class HttpUriReader(UriReader): + + def new_input_stream(self, uri: str): + try: + response = requests.get(uri) + if response.status_code != 200: + raise RuntimeError(f"Failed to read HTTP URI {uri} status code {response.status_code}") + return io.BytesIO(response.content) + except Exception as e: + raise RuntimeError(f"Failed to read HTTP URI {uri}: {e}") + + +class UriKey: + + def __init__(self, scheme: Optional[str], authority: Optional[str]) -> None: + self._scheme = scheme + self._authority = authority + self._hash = hash((self._scheme, self._authority)) + + @property + def scheme(self) -> Optional[str]: + return self._scheme + + @property + def authority(self) -> Optional[str]: + return self._authority + + def __eq__(self, other: object) -> bool: + if not isinstance(other, UriKey): + return False + + return (self._scheme == other._scheme and + self._authority == other._authority) + + def __hash__(self) -> int: + return self._hash + + def __repr__(self) -> str: + return f"UriKey(scheme='{self._scheme}', authority='{self._authority}')" + + +class UriReaderFactory: + + def __init__(self, catalog_options: dict) -> None: + self.catalog_options = catalog_options + self._readers = LRUCache(CatalogOptions.BLOB_FILE_IO_DEFAULT_CACHE_SIZE) + self._readers_lock = rwlock.RWLockFair() + + def create(self, input_uri: str) -> UriReader: + try: + parsed_uri = urlparse(input_uri) + except Exception as e: + raise ValueError(f"Invalid URI: {input_uri}") from e + + key = UriKey(parsed_uri.scheme, parsed_uri.netloc or None) + rlock = self._readers_lock.gen_rlock() + rlock.acquire() + try: + reader = self._readers.get(key) + if reader is not None: + return reader + finally: + rlock.release() + wlock = self._readers_lock.gen_wlock() + wlock.acquire() + try: + reader = self._readers.get(key) + if reader is not None: + return reader + reader = self._new_reader(key, parsed_uri) + self._readers[key] = reader + return reader + finally: + wlock.release() + + def _new_reader(self, key: UriKey, parsed_uri: ParseResult) -> UriReader: + scheme = key.scheme + if scheme in ('http', 'https'): + return UriReader.from_http() + try: + # Import FileIO here to avoid circular imports + from pypaimon.common.file_io import FileIO + uri_string = parsed_uri.geturl() + file_io = FileIO(uri_string, self.catalog_options) + return UriReader.from_file(file_io) + except Exception as e: + raise RuntimeError(f"Failed to create reader for URI {parsed_uri.geturl()}") from e + + def clear_cache(self) -> None: + self._readers.clear() + + def get_cache_size(self) -> int: + return len(self._readers) + + def __getstate__(self): + state = self.__dict__.copy() + del state['_readers_lock'] + return state + + def __setstate__(self, state): + self.__dict__.update(state) + self._readers_lock = rwlock.RWLockFair() diff --git a/paimon-python/pypaimon/filesystem/pvfs.py b/paimon-python/pypaimon/filesystem/pvfs.py index d7dabbbcaaea..15f9f9f8dcd0 100644 --- a/paimon-python/pypaimon/filesystem/pvfs.py +++ b/paimon-python/pypaimon/filesystem/pvfs.py @@ -29,7 +29,7 @@ from fsspec.implementations.local import LocalFileSystem from readerwriterlock import rwlock -from pypaimon.api.api_response import GetTableTokenResponse, GetTableResponse +from pypaimon.api.api_response import GetTableResponse, GetTableTokenResponse from pypaimon.api.client import AlreadyExistsException, NoSuchResourceException from pypaimon.api.rest_api import RESTApi from pypaimon.common.config import CatalogOptions, OssOptions, PVFSOptions diff --git a/paimon-python/pypaimon/manifest/manifest_file_manager.py b/paimon-python/pypaimon/manifest/manifest_file_manager.py index aec8bc7ed0bd..567bba1cbd4f 100644 --- a/paimon-python/pypaimon/manifest/manifest_file_manager.py +++ b/paimon-python/pypaimon/manifest/manifest_file_manager.py @@ -15,16 +15,20 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import fastavro +from concurrent.futures import ThreadPoolExecutor from io import BytesIO from typing import List +import fastavro + from pypaimon.manifest.schema.data_file_meta import DataFileMeta from pypaimon.manifest.schema.manifest_entry import (MANIFEST_ENTRY_SCHEMA, ManifestEntry) +from pypaimon.manifest.schema.manifest_file_meta import ManifestFileMeta from pypaimon.manifest.schema.simple_stats import SimpleStats from pypaimon.table.row.generic_row import (GenericRowDeserializer, GenericRowSerializer) +from pypaimon.table.row.binary_row import BinaryRow class ManifestFileManager: @@ -36,11 +40,35 @@ def __init__(self, table): self.table: FileStoreTable = table self.manifest_path = table.table_path / "manifest" self.file_io = table.file_io - self.partition_key_fields = self.table.table_schema.get_partition_key_fields() - self.primary_key_fields = self.table.table_schema.get_primary_key_fields() - self.trimmed_primary_key_fields = self.table.table_schema.get_trimmed_primary_key_fields() + self.partition_keys_fields = self.table.partition_keys_fields + self.primary_keys_fields = self.table.primary_keys_fields + self.trimmed_primary_keys_fields = self.table.trimmed_primary_keys_fields + + def read_entries_parallel(self, manifest_files: List[ManifestFileMeta], manifest_entry_filter=None, + drop_stats=True, max_workers=8) -> List[ManifestEntry]: + + def _process_single_manifest(manifest_file: ManifestFileMeta) -> List[ManifestEntry]: + return self.read(manifest_file.file_name, manifest_entry_filter, drop_stats) + + deleted_entry_keys = set() + added_entries = [] + with ThreadPoolExecutor(max_workers=max_workers) as executor: + future_results = executor.map(_process_single_manifest, manifest_files) + for entries in future_results: + for entry in entries: + if entry.kind == 0: + added_entries.append(entry) + else: + key = (tuple(entry.partition.values), entry.bucket, entry.file.file_name) + deleted_entry_keys.add(key) - def read(self, manifest_file_name: str, shard_filter=None) -> List[ManifestEntry]: + final_entries = [ + entry for entry in added_entries + if (tuple(entry.partition.values), entry.bucket, entry.file.file_name) not in deleted_entry_keys + ] + return final_entries + + def read(self, manifest_file_name: str, manifest_entry_filter=None, drop_stats=True) -> List[ManifestEntry]: manifest_file_path = self.manifest_path / manifest_file_name entries = [] @@ -53,30 +81,25 @@ def read(self, manifest_file_name: str, shard_filter=None) -> List[ManifestEntry file_dict = dict(record['_FILE']) key_dict = dict(file_dict['_KEY_STATS']) key_stats = SimpleStats( - min_values=GenericRowDeserializer.from_bytes(key_dict['_MIN_VALUES'], - self.trimmed_primary_key_fields), - max_values=GenericRowDeserializer.from_bytes(key_dict['_MAX_VALUES'], - self.trimmed_primary_key_fields), + min_values=BinaryRow(key_dict['_MIN_VALUES'], self.trimmed_primary_keys_fields), + max_values=BinaryRow(key_dict['_MAX_VALUES'], self.trimmed_primary_keys_fields), null_counts=key_dict['_NULL_COUNTS'], ) + + schema_fields = self.table.schema_manager.get_schema(file_dict['_SCHEMA_ID']).fields + fields = self._get_value_stats_fields(file_dict, schema_fields) value_dict = dict(file_dict['_VALUE_STATS']) - if file_dict.get('_VALUE_STATS_COLS') is None: - fields = self.table.table_schema.fields - elif not file_dict.get('_VALUE_STATS_COLS'): - fields = [] - else: - fields = [self.table.field_dict[col] for col in file_dict['_VALUE_STATS_COLS']] value_stats = SimpleStats( - min_values=GenericRowDeserializer.from_bytes(value_dict['_MIN_VALUES'], fields), - max_values=GenericRowDeserializer.from_bytes(value_dict['_MAX_VALUES'], fields), + min_values=BinaryRow(value_dict['_MIN_VALUES'], fields), + max_values=BinaryRow(value_dict['_MAX_VALUES'], fields), null_counts=value_dict['_NULL_COUNTS'], ) file_meta = DataFileMeta( file_name=file_dict['_FILE_NAME'], file_size=file_dict['_FILE_SIZE'], row_count=file_dict['_ROW_COUNT'], - min_key=GenericRowDeserializer.from_bytes(file_dict['_MIN_KEY'], self.trimmed_primary_key_fields), - max_key=GenericRowDeserializer.from_bytes(file_dict['_MAX_KEY'], self.trimmed_primary_key_fields), + min_key=GenericRowDeserializer.from_bytes(file_dict['_MIN_KEY'], self.trimmed_primary_keys_fields), + max_key=GenericRowDeserializer.from_bytes(file_dict['_MAX_KEY'], self.trimmed_primary_keys_fields), key_stats=key_stats, value_stats=value_stats, min_sequence_number=file_dict['_MIN_SEQUENCE_NUMBER'], @@ -89,19 +112,40 @@ def read(self, manifest_file_name: str, shard_filter=None) -> List[ManifestEntry embedded_index=file_dict['_EMBEDDED_FILE_INDEX'], file_source=file_dict['_FILE_SOURCE'], value_stats_cols=file_dict.get('_VALUE_STATS_COLS'), + external_path=file_dict.get('_EXTERNAL_PATH'), + first_row_id=file_dict['_FIRST_ROW_ID'] if '_FIRST_ROW_ID' in file_dict else None, + write_cols=file_dict['_WRITE_COLS'] if '_WRITE_COLS' in file_dict else None, ) entry = ManifestEntry( kind=record['_KIND'], - partition=GenericRowDeserializer.from_bytes(record['_PARTITION'], self.partition_key_fields), + partition=GenericRowDeserializer.from_bytes(record['_PARTITION'], self.partition_keys_fields), bucket=record['_BUCKET'], total_buckets=record['_TOTAL_BUCKETS'], file=file_meta ) - if shard_filter is not None and not shard_filter(entry): + if manifest_entry_filter is not None and not manifest_entry_filter(entry): continue + if drop_stats: + entry = entry.copy_without_stats() entries.append(entry) return entries + def _get_value_stats_fields(self, file_dict: dict, schema_fields: list) -> List: + if file_dict['_VALUE_STATS_COLS'] is None: + if '_WRITE_COLS' in file_dict: + if file_dict['_WRITE_COLS'] is None: + fields = schema_fields + else: + read_fields = file_dict['_WRITE_COLS'] + fields = [self.table.field_dict[col] for col in read_fields] + else: + fields = schema_fields + elif not file_dict['_VALUE_STATS_COLS']: + fields = [] + else: + fields = [self.table.field_dict[col] for col in file_dict['_VALUE_STATS_COLS']] + return fields + def write(self, file_name, entries: List[ManifestEntry]): avro_records = [] for entry in entries: @@ -137,6 +181,9 @@ def write(self, file_name, entries: List[ManifestEntry]): "_EMBEDDED_FILE_INDEX": entry.file.embedded_index, "_FILE_SOURCE": entry.file.file_source, "_VALUE_STATS_COLS": entry.file.value_stats_cols, + "_EXTERNAL_PATH": entry.file.external_path, + "_FIRST_ROW_ID": entry.file.first_row_id, + "_WRITE_COLS": entry.file.write_cols, } } avro_records.append(avro_record) diff --git a/paimon-python/pypaimon/manifest/manifest_list_manager.py b/paimon-python/pypaimon/manifest/manifest_list_manager.py index 2fc1eea0115f..367f802de574 100644 --- a/paimon-python/pypaimon/manifest/manifest_list_manager.py +++ b/paimon-python/pypaimon/manifest/manifest_list_manager.py @@ -25,8 +25,8 @@ MANIFEST_FILE_META_SCHEMA, ManifestFileMeta) from pypaimon.manifest.schema.simple_stats import SimpleStats from pypaimon.snapshot.snapshot import Snapshot -from pypaimon.table.row.generic_row import (GenericRowDeserializer, - GenericRowSerializer) +from pypaimon.table.row.binary_row import BinaryRow +from pypaimon.table.row.generic_row import GenericRowSerializer class ManifestListManager: @@ -47,6 +47,9 @@ def read_all(self, snapshot: Snapshot) -> List[ManifestFileMeta]: manifest_files.extend(delta_manifests) return manifest_files + def read_delta(self, snapshot: Snapshot) -> List[ManifestFileMeta]: + return self.read(snapshot.delta_manifest_list) + def read(self, manifest_list_name: str) -> List[ManifestFileMeta]: manifest_files = [] @@ -58,13 +61,13 @@ def read(self, manifest_list_name: str) -> List[ManifestFileMeta]: for record in reader: stats_dict = dict(record['_PARTITION_STATS']) partition_stats = SimpleStats( - min_values=GenericRowDeserializer.from_bytes( + min_values=BinaryRow( stats_dict['_MIN_VALUES'], - self.table.table_schema.get_partition_key_fields() + self.table.partition_keys_fields ), - max_values=GenericRowDeserializer.from_bytes( + max_values=BinaryRow( stats_dict['_MAX_VALUES'], - self.table.table_schema.get_partition_key_fields() + self.table.partition_keys_fields ), null_counts=stats_dict['_NULL_COUNTS'], ) diff --git a/paimon-python/pypaimon/manifest/schema/data_file_meta.py b/paimon-python/pypaimon/manifest/schema/data_file_meta.py index 82dfb669186c..405c2e34831f 100644 --- a/paimon-python/pypaimon/manifest/schema/data_file_meta.py +++ b/paimon-python/pypaimon/manifest/schema/data_file_meta.py @@ -21,7 +21,7 @@ from pathlib import Path from typing import List, Optional -from pypaimon.manifest.schema.simple_stats import (SIMPLE_STATS_SCHEMA, +from pypaimon.manifest.schema.simple_stats import (KEY_STATS_SCHEMA, VALUE_STATS_SCHEMA, SimpleStats) from pypaimon.table.row.generic_row import GenericRow @@ -44,9 +44,11 @@ class DataFileMeta: creation_time: Optional[datetime] = None delete_row_count: Optional[int] = None embedded_index: Optional[bytes] = None - file_source: Optional[str] = None + file_source: Optional[int] = None value_stats_cols: Optional[List[str]] = None external_path: Optional[str] = None + first_row_id: Optional[int] = None + write_cols: Optional[List[str]] = None # not a schema field, just for internal usage file_path: str = None @@ -59,6 +61,84 @@ def set_file_path(self, table_path: Path, partition: GenericRow, bucket: int): path_builder = path_builder / ("bucket-" + str(bucket)) / self.file_name self.file_path = str(path_builder) + def copy_without_stats(self) -> 'DataFileMeta': + """Create a new DataFileMeta without value statistics.""" + return DataFileMeta( + file_name=self.file_name, + file_size=self.file_size, + row_count=self.row_count, + min_key=self.min_key, + max_key=self.max_key, + key_stats=self.key_stats, + value_stats=SimpleStats.empty_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=self.extra_files, + creation_time=self.creation_time, + delete_row_count=self.delete_row_count, + embedded_index=self.embedded_index, + file_source=self.file_source, + value_stats_cols=[], + external_path=self.external_path, + first_row_id=self.first_row_id, + write_cols=self.write_cols, + file_path=self.file_path + ) + + def assign_first_row_id(self, first_row_id: int) -> 'DataFileMeta': + """Create a new DataFileMeta with the assigned first_row_id.""" + return DataFileMeta( + file_name=self.file_name, + file_size=self.file_size, + row_count=self.row_count, + min_key=self.min_key, + max_key=self.max_key, + key_stats=self.key_stats, + value_stats=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=self.extra_files, + creation_time=self.creation_time, + delete_row_count=self.delete_row_count, + embedded_index=self.embedded_index, + file_source=self.file_source, + value_stats_cols=self.value_stats_cols, + external_path=self.external_path, + first_row_id=first_row_id, + write_cols=self.write_cols, + file_path=self.file_path + ) + + def assign_sequence_number(self, min_sequence_number: int, max_sequence_number: int) -> 'DataFileMeta': + """Create a new DataFileMeta with the assigned sequence numbers.""" + return DataFileMeta( + file_name=self.file_name, + file_size=self.file_size, + row_count=self.row_count, + min_key=self.min_key, + max_key=self.max_key, + key_stats=self.key_stats, + value_stats=self.value_stats, + min_sequence_number=min_sequence_number, + max_sequence_number=max_sequence_number, + schema_id=self.schema_id, + level=self.level, + extra_files=self.extra_files, + creation_time=self.creation_time, + delete_row_count=self.delete_row_count, + embedded_index=self.embedded_index, + file_source=self.file_source, + value_stats_cols=self.value_stats_cols, + external_path=self.external_path, + first_row_id=self.first_row_id, + write_cols=self.write_cols, + file_path=self.file_path + ) + DATA_FILE_META_SCHEMA = { "type": "record", @@ -69,8 +149,8 @@ def set_file_path(self, table_path: Path, partition: GenericRow, bucket: int): {"name": "_ROW_COUNT", "type": "long"}, {"name": "_MIN_KEY", "type": "bytes"}, {"name": "_MAX_KEY", "type": "bytes"}, - {"name": "_KEY_STATS", "type": SIMPLE_STATS_SCHEMA}, - {"name": "_VALUE_STATS", "type": "SimpleStats"}, + {"name": "_KEY_STATS", "type": KEY_STATS_SCHEMA}, + {"name": "_VALUE_STATS", "type": VALUE_STATS_SCHEMA}, {"name": "_MIN_SEQUENCE_NUMBER", "type": "long"}, {"name": "_MAX_SEQUENCE_NUMBER", "type": "long"}, {"name": "_SCHEMA_ID", "type": "long"}, @@ -87,5 +167,10 @@ def set_file_path(self, table_path: Path, partition: GenericRow, bucket: int): {"name": "_VALUE_STATS_COLS", "type": ["null", {"type": "array", "items": "string"}], "default": None}, + {"name": "_EXTERNAL_PATH", "type": ["null", "string"], "default": None}, + {"name": "_FIRST_ROW_ID", "type": ["null", "long"], "default": None}, + {"name": "_WRITE_COLS", + "type": ["null", {"type": "array", "items": "string"}], + "default": None}, ] } diff --git a/paimon-python/pypaimon/manifest/schema/manifest_entry.py b/paimon-python/pypaimon/manifest/schema/manifest_entry.py index 9a023411755a..b1fd244dafc0 100644 --- a/paimon-python/pypaimon/manifest/schema/manifest_entry.py +++ b/paimon-python/pypaimon/manifest/schema/manifest_entry.py @@ -31,6 +31,36 @@ class ManifestEntry: total_buckets: int file: DataFileMeta + def copy_without_stats(self) -> 'ManifestEntry': + """Create a new ManifestEntry without value statistics.""" + return ManifestEntry( + kind=self.kind, + partition=self.partition, + bucket=self.bucket, + total_buckets=self.total_buckets, + file=self.file.copy_without_stats() + ) + + def assign_first_row_id(self, first_row_id: int) -> 'ManifestEntry': + """Create a new ManifestEntry with the assigned first_row_id.""" + return ManifestEntry( + kind=self.kind, + partition=self.partition, + bucket=self.bucket, + total_buckets=self.total_buckets, + file=self.file.assign_first_row_id(first_row_id) + ) + + def assign_sequence_number(self, min_sequence_number: int, max_sequence_number: int) -> 'ManifestEntry': + """Create a new ManifestEntry with the assigned sequence numbers.""" + return ManifestEntry( + kind=self.kind, + partition=self.partition, + bucket=self.bucket, + total_buckets=self.total_buckets, + file=self.file.assign_sequence_number(min_sequence_number, max_sequence_number) + ) + MANIFEST_ENTRY_SCHEMA = { "type": "record", diff --git a/paimon-python/pypaimon/manifest/schema/manifest_file_meta.py b/paimon-python/pypaimon/manifest/schema/manifest_file_meta.py index 443c6a0944d9..a830be6cc2ea 100644 --- a/paimon-python/pypaimon/manifest/schema/manifest_file_meta.py +++ b/paimon-python/pypaimon/manifest/schema/manifest_file_meta.py @@ -18,7 +18,7 @@ from dataclasses import dataclass -from pypaimon.manifest.schema.simple_stats import (SIMPLE_STATS_SCHEMA, +from pypaimon.manifest.schema.simple_stats import (PARTITION_STATS_SCHEMA, SimpleStats) @@ -41,7 +41,7 @@ class ManifestFileMeta: {"name": "_FILE_SIZE", "type": "long"}, {"name": "_NUM_ADDED_FILES", "type": "long"}, {"name": "_NUM_DELETED_FILES", "type": "long"}, - {"name": "_PARTITION_STATS", "type": SIMPLE_STATS_SCHEMA}, + {"name": "_PARTITION_STATS", "type": PARTITION_STATS_SCHEMA}, {"name": "_SCHEMA_ID", "type": "long"}, ] } diff --git a/paimon-python/pypaimon/manifest/schema/simple_stats.py b/paimon-python/pypaimon/manifest/schema/simple_stats.py index 45982491b912..10d9e6242022 100644 --- a/paimon-python/pypaimon/manifest/schema/simple_stats.py +++ b/paimon-python/pypaimon/manifest/schema/simple_stats.py @@ -17,32 +17,58 @@ ################################################################################ from dataclasses import dataclass -from typing import List, Optional +from typing import List +from typing import ClassVar from pypaimon.table.row.generic_row import GenericRow +from pypaimon.table.row.internal_row import InternalRow @dataclass class SimpleStats: - min_values: GenericRow - max_values: GenericRow - null_counts: Optional[List[int]] + min_values: InternalRow + max_values: InternalRow + null_counts: List[int] + _empty_stats: ClassVar[object] = None -SIMPLE_STATS_SCHEMA = { + @classmethod + def empty_stats(cls): + if cls._empty_stats is None: + min_values = GenericRow([], []) + max_values = GenericRow([], []) + cls._empty_stats = cls(min_values, max_values, None) + return cls._empty_stats + + +SIMPLE_STATS_FIELDS = [ + {"name": "_MIN_VALUES", "type": "bytes"}, + {"name": "_MAX_VALUES", "type": "bytes"}, + {"name": "_NULL_COUNTS", + "type": [ + "null", + { + "type": "array", + "items": ["null", "long"] + } + ], + "default": None}, +] + +KEY_STATS_SCHEMA = { + "type": "record", + "name": "record_KEY_STATS", + "fields": SIMPLE_STATS_FIELDS +} + +VALUE_STATS_SCHEMA = { + "type": "record", + "name": "record_VALUE_STATS", + "fields": SIMPLE_STATS_FIELDS +} + +PARTITION_STATS_SCHEMA = { "type": "record", - "name": "SimpleStats", - "fields": [ - {"name": "_MIN_VALUES", "type": "bytes"}, - {"name": "_MAX_VALUES", "type": "bytes"}, - {"name": "_NULL_COUNTS", - "type": [ - "null", - { - "type": "array", - "items": ["null", "long"] - } - ], - "default": None}, - ] + "name": "record_PARTITION_STATS", + "fields": SIMPLE_STATS_FIELDS } diff --git a/paimon-python/pypaimon/manifest/simple_stats_evolution.py b/paimon-python/pypaimon/manifest/simple_stats_evolution.py new file mode 100644 index 000000000000..d2291c54de1f --- /dev/null +++ b/paimon-python/pypaimon/manifest/simple_stats_evolution.py @@ -0,0 +1,123 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import List, Optional, Dict, Any +import threading + +from pypaimon.schema.data_types import DataField +from pypaimon.manifest.schema.simple_stats import SimpleStats +from pypaimon.table.row.generic_row import GenericRow +from pypaimon.table.row.projected_row import ProjectedRow + + +class SimpleStatsEvolution: + """Converter for array of SimpleColStats.""" + + def __init__(self, data_fields: List[DataField], index_mapping: Optional[List[int]], + cast_field_getters: Optional[List[Any]]): + self.field_names = [field.name for field in data_fields] + self.index_mapping = index_mapping + self.cast_field_getters = cast_field_getters + self.index_mappings: Dict[tuple, List[int]] = {} + self._lock = threading.Lock() + + # Create empty values for optimization + self.empty_values = GenericRow([None] * len(self.field_names), data_fields) + self.empty_null_counts = [0] * len(self.field_names) + + def evolution(self, stats: SimpleStats, row_count: Optional[int], + stats_fields: Optional[List[str]]) -> 'SimpleStats': + min_values = stats.min_values + max_values = stats.max_values + null_counts = stats.null_counts + + if stats_fields is not None and not stats_fields: + # Optimize for empty dense fields + min_values = self.empty_values + max_values = self.empty_values + null_counts = self.empty_null_counts + elif stats_fields is not None: + # Apply dense field mapping + dense_index_mapping = self._get_dense_index_mapping(stats_fields) + min_values = self._project_row(min_values, dense_index_mapping) + max_values = self._project_row(max_values, dense_index_mapping) + null_counts = self._project_array(null_counts, dense_index_mapping) + + if self.index_mapping is not None: + min_values = self._project_row(min_values, self.index_mapping) + max_values = self._project_row(max_values, self.index_mapping) + + if row_count is None: + raise RuntimeError("Schema Evolution for stats needs row count.") + + null_counts = self._evolve_null_counts(null_counts, self.index_mapping, row_count) + + return SimpleStats(min_values, max_values, null_counts) + + def _get_dense_index_mapping(self, dense_fields: List[str]) -> List[int]: + """ + Get dense index mapping similar to Java: + fieldNames.stream().mapToInt(denseFields::indexOf).toArray() + """ + dense_fields_tuple = tuple(dense_fields) + + if dense_fields_tuple not in self.index_mappings: + with self._lock: + # Double-check locking + if dense_fields_tuple not in self.index_mappings: + mapping = [] + for field_name in self.field_names: + try: + index = dense_fields.index(field_name) + mapping.append(index) + except ValueError: + mapping.append(-1) # Field not found + self.index_mappings[dense_fields_tuple] = mapping + + return self.index_mappings[dense_fields_tuple] + + def _project_row(self, row: Any, index_mapping: List[int]) -> Any: + """Project row based on index mapping using ProjectedRow.""" + projected_row = ProjectedRow.from_index_mapping(index_mapping) + return projected_row.replace_row(row) + + def _project_array(self, array: List[Any], index_mapping: List[int]) -> List[Any]: + """Project array based on index mapping.""" + if not array: + return [0] * len(index_mapping) + + projected = [] + for mapped_index in index_mapping: + if mapped_index >= 0 and mapped_index < len(array): + projected.append(array[mapped_index]) + else: + projected.append(0) # Default value for missing fields + + return projected + + def _evolve_null_counts(self, null_counts: List[Any], index_mapping: List[int], + not_found_value: int) -> List[Any]: + """Evolve null counts with schema evolution mapping.""" + evolved = [] + for mapped_index in index_mapping: + if mapped_index >= 0 and mapped_index < len(null_counts): + evolved.append(null_counts[mapped_index]) + else: + evolved.append(not_found_value) # Use row count for missing fields + + return evolved diff --git a/paimon-python/pypaimon/manifest/simple_stats_evolutions.py b/paimon-python/pypaimon/manifest/simple_stats_evolutions.py new file mode 100644 index 000000000000..f7f5946162f0 --- /dev/null +++ b/paimon-python/pypaimon/manifest/simple_stats_evolutions.py @@ -0,0 +1,71 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import Callable, Dict, List, Optional + +from pypaimon.manifest.simple_stats_evolution import SimpleStatsEvolution +from pypaimon.schema.data_types import DataField + + +class SimpleStatsEvolutions: + """Converters to create col stats array serializer.""" + + def __init__(self, schema_fields: Callable[[int], List[DataField]], table_schema_id: int): + self.schema_fields = schema_fields + self.table_schema_id = table_schema_id + self.table_fields = schema_fields(table_schema_id) + self.evolutions: Dict[int, SimpleStatsEvolution] = {} + + def get_or_create(self, data_schema_id: int) -> SimpleStatsEvolution: + """Get or create SimpleStatsEvolution for given schema id.""" + if data_schema_id in self.evolutions: + return self.evolutions[data_schema_id] + + if self.table_schema_id == data_schema_id: + evolution = SimpleStatsEvolution(self.schema_fields(data_schema_id), None, None) + else: + data_fields = self.schema_fields(data_schema_id) + index_cast_mapping = self._create_index_cast_mapping(self.table_fields, data_fields) + index_mapping = index_cast_mapping.get('index_mapping') + cast_mapping = index_cast_mapping.get('cast_mapping') + evolution = SimpleStatsEvolution(data_fields, index_mapping, cast_mapping) + + self.evolutions[data_schema_id] = evolution + return evolution + + @staticmethod + def _create_index_cast_mapping(table_fields: List[DataField], + data_fields: List[DataField]) -> Dict[str, Optional[List[int]]]: + """ + Create index and cast mapping between table fields and data fields. + This is a simplified implementation. + """ + # Create a mapping from field names to indices in data_fields + data_field_map = {field.name: i for i, field in enumerate(data_fields)} + + index_mapping = [] + for table_field in table_fields: + if table_field.name in data_field_map: + index_mapping.append(data_field_map[table_field.name]) + else: + index_mapping.append(-1) # Field not found in data schema + + return { + 'index_mapping': index_mapping, + 'cast_mapping': None # Simplified - no casting for now + } diff --git a/paimon-python/pypaimon/read/plan.py b/paimon-python/pypaimon/read/plan.py index 9a65fd6f12b7..8c69a41a9b0e 100644 --- a/paimon-python/pypaimon/read/plan.py +++ b/paimon-python/pypaimon/read/plan.py @@ -19,18 +19,14 @@ from dataclasses import dataclass from typing import List -from pypaimon.manifest.schema.manifest_entry import ManifestEntry + from pypaimon.read.split import Split @dataclass class Plan: """Implementation of Plan for native Python reading.""" - _files: List[ManifestEntry] _splits: List[Split] - def files(self) -> List[ManifestEntry]: - return self._files - def splits(self) -> List[Split]: return self._splits diff --git a/paimon-python/pypaimon/read/push_down_utils.py b/paimon-python/pypaimon/read/push_down_utils.py index 95a99d900584..f8123411490c 100644 --- a/paimon-python/pypaimon/read/push_down_utils.py +++ b/paimon-python/pypaimon/read/push_down_utils.py @@ -19,28 +19,51 @@ from typing import Dict, List, Set from pypaimon.common.predicate import Predicate +from pypaimon.common.predicate_builder import PredicateBuilder -def extract_predicate_to_list(result: list, input_predicate: 'Predicate', keys: List[str]): - if not input_predicate or not keys: - return +def trim_and_transform_predicate(input_predicate: Predicate, all_fields: List[str], trimmed_keys: List[str]): + new_predicate = trim_predicate_by_fields(input_predicate, trimmed_keys) + part_to_index = {element: idx for idx, element in enumerate(trimmed_keys)} + mapping: Dict[int, int] = { + i: part_to_index.get(all_fields[i], -1) + for i in range(len(all_fields)) + } + return _change_index(new_predicate, mapping) + + +def trim_predicate_by_fields(input_predicate: Predicate, trimmed_keys: List[str]): + if not input_predicate or not trimmed_keys: + return None + + predicates: list[Predicate] = _split_and(input_predicate) + predicates = [element for element in predicates if _get_all_fields(element).issubset(trimmed_keys)] + return PredicateBuilder.and_predicates(predicates) + + +def _split_and(input_predicate: Predicate): + if not input_predicate: + return list() if input_predicate.method == 'and': - for sub_predicate in input_predicate.literals: - extract_predicate_to_list(result, sub_predicate, keys) - return - elif input_predicate.method == 'or': - # condition: involved keys all belong to primary keys - involved_fields = _get_all_fields(input_predicate) - if involved_fields and involved_fields.issubset(keys): - result.append(input_predicate) - return + return [p for element in (input_predicate.literals or []) for p in _split_and(element)] + + return [input_predicate] + - if input_predicate.field in keys: - result.append(input_predicate) +def _change_index(input_predicate: Predicate, mapping: Dict[int, int]): + if not input_predicate: + return None + if input_predicate.method == 'and' or input_predicate.method == 'or': + predicates: list[Predicate] = input_predicate.literals + new_predicates = [_change_index(element, mapping) for element in predicates] + return input_predicate.new_literals(new_predicates) -def _get_all_fields(predicate: 'Predicate') -> Set[str]: + return input_predicate.new_index(mapping[input_predicate.index]) + + +def _get_all_fields(predicate: Predicate) -> Set[str]: if predicate.field is not None: return {predicate.field} involved_fields = set() @@ -48,26 +71,3 @@ def _get_all_fields(predicate: 'Predicate') -> Set[str]: for sub_predicate in predicate.literals: involved_fields.update(_get_all_fields(sub_predicate)) return involved_fields - - -def extract_predicate_to_dict(result: Dict, input_predicate: 'Predicate', keys: List[str]): - if not input_predicate or not keys: - return - - if input_predicate.method == 'and': - for sub_predicate in input_predicate.literals: - extract_predicate_to_dict(result, sub_predicate, keys) - return - elif input_predicate.method == 'or': - # ensure no recursive and/or - if not input_predicate.literals or any(p.field is None for p in input_predicate.literals): - return - # condition: only one key for 'or', and the key belongs to keys - involved_fields = {p.field for p in input_predicate.literals} - field = involved_fields.pop() if len(involved_fields) == 1 else None - if field is not None and field in keys: - result[field].append(input_predicate) - return - - if input_predicate.field in keys: - result[input_predicate.field].append(input_predicate) diff --git a/paimon-python/pypaimon/read/read_builder.py b/paimon-python/pypaimon/read/read_builder.py index 30f824a698d2..6fc8026c4311 100644 --- a/paimon-python/pypaimon/read/read_builder.py +++ b/paimon-python/pypaimon/read/read_builder.py @@ -52,8 +52,7 @@ def new_scan(self) -> TableScan: return TableScan( table=self.table, predicate=self._predicate, - limit=self._limit, - read_type=self.read_type() + limit=self._limit ) def new_read(self) -> TableRead: diff --git a/paimon-python/pypaimon/read/reader/concat_batch_reader.py b/paimon-python/pypaimon/read/reader/concat_batch_reader.py index 76b9f10c71c4..de4f10c15b75 100644 --- a/paimon-python/pypaimon/read/reader/concat_batch_reader.py +++ b/paimon-python/pypaimon/read/reader/concat_batch_reader.py @@ -19,6 +19,7 @@ import collections from typing import Callable, List, Optional +import pyarrow as pa from pyarrow import RecordBatch from pypaimon.read.reader.iface.record_batch_reader import RecordBatchReader @@ -49,3 +50,97 @@ def close(self) -> None: self.current_reader.close() self.current_reader = None self.queue.clear() + + +class ShardBatchReader(ConcatBatchReader): + + def __init__(self, readers, split_start_row, split_end_row): + super().__init__(readers) + self.split_start_row = split_start_row + self.split_end_row = split_end_row + self.cur_end = 0 + + def read_arrow_batch(self) -> Optional[RecordBatch]: + batch = super().read_arrow_batch() + if batch is None: + return None + if self.split_start_row is not None or self.split_end_row is not None: + cur_begin = self.cur_end # begin idx of current batch based on the split + self.cur_end += batch.num_rows + # shard the first batch and the last batch + if self.split_start_row <= cur_begin < self.cur_end <= self.split_end_row: + return batch + elif cur_begin <= self.split_start_row < self.cur_end: + return batch.slice(self.split_start_row - cur_begin, + min(self.split_end_row, self.cur_end) - self.split_start_row) + elif cur_begin < self.split_end_row <= self.cur_end: + return batch.slice(0, self.split_end_row - cur_begin) + else: + return batch + + +class MergeAllBatchReader(RecordBatchReader): + """ + A reader that accepts multiple reader suppliers and concatenates all their arrow batches + into one big batch. This is useful when you want to merge all data from multiple sources + into a single batch for processing. + """ + + def __init__(self, reader_suppliers: List[Callable]): + self.reader_suppliers = reader_suppliers + self.merged_batch: Optional[RecordBatch] = None + self.batch_created = False + + def read_arrow_batch(self) -> Optional[RecordBatch]: + if self.batch_created: + return None + + all_batches = [] + + # Read all batches from all reader suppliers + for supplier in self.reader_suppliers: + reader = supplier() + try: + while True: + batch = reader.read_arrow_batch() + if batch is None: + break + all_batches.append(batch) + finally: + reader.close() + + # Concatenate all batches into one big batch + if all_batches: + # For PyArrow < 17.0.0, use Table.concat_tables approach + # Convert batches to tables and concatenate + tables = [pa.Table.from_batches([batch]) for batch in all_batches] + if len(tables) == 1: + # Single table, just get the first batch + self.merged_batch = tables[0].to_batches()[0] + else: + # Multiple tables, concatenate them + concatenated_table = pa.concat_tables(tables) + # Convert back to a single batch by taking all batches and combining + all_concatenated_batches = concatenated_table.to_batches() + if len(all_concatenated_batches) == 1: + self.merged_batch = all_concatenated_batches[0] + else: + # If still multiple batches, we need to manually combine them + # This shouldn't happen with concat_tables, but just in case + combined_arrays = [] + for i in range(len(all_concatenated_batches[0].columns)): + column_arrays = [batch.column(i) for batch in all_concatenated_batches] + combined_arrays.append(pa.concat_arrays(column_arrays)) + self.merged_batch = pa.RecordBatch.from_arrays( + combined_arrays, + names=all_concatenated_batches[0].schema.names + ) + else: + self.merged_batch = None + + self.batch_created = True + return self.merged_batch + + def close(self) -> None: + self.merged_batch = None + self.batch_created = False diff --git a/paimon-python/pypaimon/read/reader/data_evolution_merge_reader.py b/paimon-python/pypaimon/read/reader/data_evolution_merge_reader.py new file mode 100644 index 000000000000..43bf92686221 --- /dev/null +++ b/paimon-python/pypaimon/read/reader/data_evolution_merge_reader.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. +################################################################################ +from typing import List, Optional + +import pyarrow as pa +from pyarrow import RecordBatch + +from pypaimon.read.reader.iface.record_batch_reader import RecordBatchReader + + +class DataEvolutionMergeReader(RecordBatchReader): + """ + This is a union reader which contains multiple inner readers, Each reader is responsible for reading one file. + + This reader, assembling multiple reader into one big and great reader, will merge the batches from all readers. + + For example, if rowOffsets is {0, 2, 0, 1, 2, 1} and fieldOffsets is {0, 0, 1, 1, 1, 0}, it means: + - The first field comes from batch0, and it is at offset 0 in batch0. + - The second field comes from batch2, and it is at offset 0 in batch2. + - The third field comes from batch0, and it is at offset 1 in batch0. + - The fourth field comes from batch1, and it is at offset 1 in batch1. + - The fifth field comes from batch2, and it is at offset 1 in batch2. + - The sixth field comes from batch1, and it is at offset 0 in batch1. + """ + + def __init__(self, row_offsets: List[int], field_offsets: List[int], readers: List[Optional[RecordBatchReader]]): + if row_offsets is None: + raise ValueError("Row offsets must not be null") + if field_offsets is None: + raise ValueError("Field offsets must not be null") + if len(row_offsets) != len(field_offsets): + raise ValueError("Row offsets and field offsets must have the same length") + if not row_offsets: + raise ValueError("Row offsets must not be empty") + if not readers or len(readers) < 1: + raise ValueError("Readers should be more than 0") + self.row_offsets = row_offsets + self.field_offsets = field_offsets + self.readers = readers + + def read_arrow_batch(self) -> Optional[RecordBatch]: + batches: List[Optional[RecordBatch]] = [None] * len(self.readers) + for i, reader in enumerate(self.readers): + if reader is not None: + batch = reader.read_arrow_batch() + if batch is None: + # all readers are aligned, as long as one returns null, the others will also have no data + return None + batches[i] = batch + # Assemble record batches from batches based on row_offsets and field_offsets + columns = [] + names = [] + for i in range(len(self.row_offsets)): + batch_index = self.row_offsets[i] + field_index = self.field_offsets[i] + if batches[batch_index] is not None: + column = batches[batch_index].column(field_index) + columns.append(column) + names.append(batches[batch_index].schema.names[field_index]) + if columns: + return pa.RecordBatch.from_arrays(columns, names) + return None + + def close(self) -> None: + try: + for reader in self.readers: + if reader is not None: + reader.close() + except Exception as e: + raise IOError("Failed to close inner readers") from e diff --git a/paimon-python/pypaimon/read/reader/data_file_record_reader.py b/paimon-python/pypaimon/read/reader/data_file_batch_reader.py similarity index 100% rename from paimon-python/pypaimon/read/reader/data_file_record_reader.py rename to paimon-python/pypaimon/read/reader/data_file_batch_reader.py diff --git a/paimon-python/pypaimon/read/reader/field_bunch.py b/paimon-python/pypaimon/read/reader/field_bunch.py new file mode 100644 index 000000000000..4ba82bd80e39 --- /dev/null +++ b/paimon-python/pypaimon/read/reader/field_bunch.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. +""" +""" +FieldBunch classes for organizing files by field in data evolution. + +These classes help organize DataFileMeta objects into groups based on their field content, +supporting both regular data files and blob files. +""" +from abc import ABC +from typing import List +from pypaimon.manifest.schema.data_file_meta import DataFileMeta + + +class FieldBunch(ABC): + """Interface for files organized by field.""" + + def row_count(self) -> int: + """Return the total row count for this bunch.""" + ... + + def files(self) -> List[DataFileMeta]: + """Return the list of files in this bunch.""" + ... + + +class DataBunch(FieldBunch): + """Files for a single data file.""" + + def __init__(self, data_file: DataFileMeta): + self.data_file = data_file + + def row_count(self) -> int: + return self.data_file.row_count + + def files(self) -> List[DataFileMeta]: + return [self.data_file] + + +class BlobBunch(FieldBunch): + """Files for partial field (blob files).""" + + def __init__(self, expected_row_count: int): + self._files: List[DataFileMeta] = [] + self.expected_row_count = expected_row_count + self.latest_first_row_id = -1 + self.expected_next_first_row_id = -1 + self.latest_max_sequence_number = -1 + self._row_count = 0 + + def add(self, file: DataFileMeta) -> None: + """Add a blob file to this bunch.""" + if not self._is_blob_file(file.file_name): + raise ValueError("Only blob file can be added to a blob bunch.") + + if file.first_row_id == self.latest_first_row_id: + if file.max_sequence_number >= self.latest_max_sequence_number: + raise ValueError( + "Blob file with same first row id should have decreasing sequence number." + ) + return + + if self._files: + first_row_id = file.first_row_id + if first_row_id < self.expected_next_first_row_id: + if file.max_sequence_number >= self.latest_max_sequence_number: + raise ValueError( + "Blob file with overlapping row id should have decreasing sequence number." + ) + return + elif first_row_id > self.expected_next_first_row_id: + raise ValueError( + f"Blob file first row id should be continuous, expect " + f"{self.expected_next_first_row_id} but got {first_row_id}" + ) + + if file.schema_id != self._files[0].schema_id: + raise ValueError( + "All files in a blob bunch should have the same schema id." + ) + if file.write_cols != self._files[0].write_cols: + raise ValueError( + "All files in a blob bunch should have the same write columns." + ) + + self._files.append(file) + self._row_count += file.row_count + if self._row_count > self.expected_row_count: + raise ValueError( + f"Blob files row count exceed the expect {self.expected_row_count}" + ) + + self.latest_max_sequence_number = file.max_sequence_number + self.latest_first_row_id = file.first_row_id + self.expected_next_first_row_id = self.latest_first_row_id + file.row_count + + def row_count(self) -> int: + return self._row_count + + def files(self) -> List[DataFileMeta]: + return self._files + + @staticmethod + def _is_blob_file(file_name: str) -> bool: + """Check if a file is a blob file based on its extension.""" + return file_name.endswith('.blob') diff --git a/paimon-python/pypaimon/read/reader/format_blob_reader.py b/paimon-python/pypaimon/read/reader/format_blob_reader.py new file mode 100644 index 000000000000..e6846dc5680e --- /dev/null +++ b/paimon-python/pypaimon/read/reader/format_blob_reader.py @@ -0,0 +1,189 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import struct +from pathlib import Path +from typing import List, Optional, Any, Iterator + +import pyarrow as pa +import pyarrow.dataset as ds +from pyarrow import RecordBatch + +from pypaimon.common.delta_varint_compressor import DeltaVarintCompressor +from pypaimon.common.file_io import FileIO +from pypaimon.read.reader.iface.record_batch_reader import RecordBatchReader +from pypaimon.schema.data_types import DataField, PyarrowFieldParser, AtomicType +from pypaimon.table.row.blob import Blob +from pypaimon.table.row.generic_row import GenericRow +from pypaimon.table.row.row_kind import RowKind + + +class FormatBlobReader(RecordBatchReader): + + def __init__(self, file_io: FileIO, file_path: str, read_fields: List[str], + full_fields: List[DataField], push_down_predicate: Any, blob_as_descriptor: bool): + self._file_io = file_io + self._file_path = file_path + self._push_down_predicate = push_down_predicate + self._blob_as_descriptor = blob_as_descriptor + + # Get file size + self._file_size = file_io.get_file_size(Path(file_path)) + + # Initialize the low-level blob format reader + self.file_path = file_path + self.blob_lengths: List[int] = [] + self.blob_offsets: List[int] = [] + self.returned = False + self._read_index() + + # Set up fields and schema + if len(read_fields) > 1: + raise RuntimeError("Blob reader only supports one field.") + self._fields = read_fields + full_fields_map = {field.name: field for field in full_fields} + projected_data_fields = [full_fields_map[name] for name in read_fields] + self._schema = PyarrowFieldParser.from_paimon_schema(projected_data_fields) + + # Initialize iterator + self._blob_iterator = None + self._current_batch = None + + def read_arrow_batch(self) -> Optional[RecordBatch]: + if self._blob_iterator is None: + if self.returned: + return None + self.returned = True + batch_iterator = BlobRecordIterator( + self._file_io, self.file_path, self.blob_lengths, + self.blob_offsets, self._fields[0] + ) + self._blob_iterator = iter(batch_iterator) + + # Collect records for this batch + pydict_data = {name: [] for name in self._fields} + records_in_batch = 0 + + try: + while True: + blob_row = next(self._blob_iterator) + if blob_row is None: + break + blob = blob_row.values[0] + for field_name in self._fields: + blob_descriptor = blob.to_descriptor() + if self._blob_as_descriptor: + blob_data = blob_descriptor.serialize() + else: + blob_data = blob.to_data() + pydict_data[field_name].append(blob_data) + + records_in_batch += 1 + + except StopIteration: + # Stop immediately when StopIteration occurs + pass + + if records_in_batch == 0: + return None + + # Create RecordBatch + if self._push_down_predicate is None: + # Convert to Table first, then to RecordBatch + table = pa.Table.from_pydict(pydict_data, self._schema) + if table.num_rows > 0: + return table.to_batches()[0] + else: + return None + else: + # Apply predicate filtering + pa_batch = pa.Table.from_pydict(pydict_data, self._schema) + dataset = ds.InMemoryDataset(pa_batch) + scanner = dataset.scanner(filter=self._push_down_predicate) + combine_chunks = scanner.to_table().combine_chunks() + if combine_chunks.num_rows > 0: + return combine_chunks.to_batches()[0] + else: + return None + + def close(self): + self._blob_iterator = None + + def _read_index(self) -> None: + with self._file_io.new_input_stream(Path(self.file_path)) as f: + # Seek to header: last 5 bytes + f.seek(self._file_size - 5) + header = f.read(5) + + if len(header) != 5: + raise IOError("Invalid blob file: cannot read header") + + # Parse header + index_length = struct.unpack(' Iterator[GenericRow]: + return self + + def __next__(self) -> GenericRow: + if self.current_position >= len(self.blob_lengths): + raise StopIteration + # Create blob reference for the current blob + # Skip magic number (4 bytes) and exclude length (8 bytes) + CRC (4 bytes) = 12 bytes + blob_offset = self.blob_offsets[self.current_position] + self.MAGIC_NUMBER_SIZE # Skip magic number + blob_length = self.blob_lengths[self.current_position] - self.METADATA_OVERHEAD + blob = Blob.from_file(self.file_io, self.file_path, blob_offset, blob_length) + self.current_position += 1 + fields = [DataField(0, self.field_name, AtomicType("BLOB"))] + return GenericRow([blob], fields, RowKind.INSERT) + + def returned_position(self) -> int: + return self.current_position diff --git a/paimon-python/pypaimon/read/scanner/__init__.py b/paimon-python/pypaimon/read/scanner/__init__.py new file mode 100644 index 000000000000..53ed4d36c2ce --- /dev/null +++ b/paimon-python/pypaimon/read/scanner/__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/read/scanner/empty_starting_scanner.py b/paimon-python/pypaimon/read/scanner/empty_starting_scanner.py new file mode 100644 index 000000000000..9c870aa588f3 --- /dev/null +++ b/paimon-python/pypaimon/read/scanner/empty_starting_scanner.py @@ -0,0 +1,25 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +from pypaimon.read.plan import Plan +from pypaimon.read.scanner.starting_scanner import StartingScanner + + +class EmptyStartingScanner(StartingScanner): + + def scan(self) -> Plan: + return Plan([]) diff --git a/paimon-python/pypaimon/read/scanner/full_starting_scanner.py b/paimon-python/pypaimon/read/scanner/full_starting_scanner.py new file mode 100644 index 000000000000..dfbe329f438b --- /dev/null +++ b/paimon-python/pypaimon/read/scanner/full_starting_scanner.py @@ -0,0 +1,523 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +import os +from collections import defaultdict +from typing import Callable, List, Optional + +from pypaimon.common.core_options import CoreOptions +from pypaimon.common.predicate import Predicate +from pypaimon.manifest.manifest_file_manager import ManifestFileManager +from pypaimon.manifest.manifest_list_manager import ManifestListManager +from pypaimon.manifest.schema.data_file_meta import DataFileMeta +from pypaimon.manifest.schema.manifest_entry import ManifestEntry +from pypaimon.manifest.schema.manifest_file_meta import ManifestFileMeta +from pypaimon.read.interval_partition import IntervalPartition, SortedRun +from pypaimon.read.plan import Plan +from pypaimon.read.push_down_utils import (trim_and_transform_predicate) +from pypaimon.read.scanner.starting_scanner import StartingScanner +from pypaimon.read.split import Split +from pypaimon.snapshot.snapshot_manager import SnapshotManager +from pypaimon.table.bucket_mode import BucketMode +from pypaimon.manifest.simple_stats_evolutions import SimpleStatsEvolutions + + +class FullStartingScanner(StartingScanner): + def __init__(self, table, predicate: Optional[Predicate], limit: Optional[int]): + from pypaimon.table.file_store_table import FileStoreTable + + self.table: FileStoreTable = table + self.predicate = predicate + self.limit = limit + + self.snapshot_manager = SnapshotManager(table) + self.manifest_list_manager = ManifestListManager(table) + self.manifest_file_manager = ManifestFileManager(table) + + self.primary_key_predicate = trim_and_transform_predicate( + self.predicate, self.table.field_names, self.table.trimmed_primary_keys) + + self.partition_key_predicate = trim_and_transform_predicate( + self.predicate, self.table.field_names, self.table.partition_keys) + + # Get split target size and open file cost from table options + self.target_split_size = CoreOptions.get_split_target_size(self.table.options) + self.open_file_cost = CoreOptions.get_split_open_file_cost(self.table.options) + + self.idx_of_this_subtask = None + self.number_of_para_subtasks = None + + self.only_read_real_buckets = True if int( + self.table.options.get('bucket', -1)) == BucketMode.POSTPONE_BUCKET.value else False + self.data_evolution = self.table.options.get(CoreOptions.DATA_EVOLUTION_ENABLED, 'false').lower() == 'true' + + def schema_fields_func(schema_id: int): + return self.table.schema_manager.get_schema(schema_id).fields + + self.simple_stats_evolutions = SimpleStatsEvolutions( + schema_fields_func, + self.table.table_schema.id + ) + + def scan(self) -> Plan: + file_entries = self.plan_files() + if not file_entries: + return Plan([]) + if self.table.is_primary_key_table: + splits = self._create_primary_key_splits(file_entries) + elif self.data_evolution: + splits = self._create_data_evolution_splits(file_entries) + else: + splits = self._create_append_only_splits(file_entries) + + splits = self._apply_push_down_limit(splits) + return Plan(splits) + + def plan_files(self) -> List[ManifestEntry]: + latest_snapshot = self.snapshot_manager.get_latest_snapshot() + if not latest_snapshot: + return [] + manifest_files = self.manifest_list_manager.read_all(latest_snapshot) + return self.read_manifest_entries(manifest_files) + + def read_manifest_entries(self, manifest_files: List[ManifestFileMeta]) -> List[ManifestEntry]: + max_workers = int(self.table.options.get(CoreOptions.SCAN_MANIFEST_PARALLELISM, os.cpu_count() or 8)) + if max_workers < 8: + max_workers = 8 + manifest_files = [entry for entry in manifest_files if self._filter_manifest_file(entry)] + return self.manifest_file_manager.read_entries_parallel(manifest_files, + self._filter_manifest_entry, + max_workers=max_workers) + + def with_shard(self, idx_of_this_subtask, number_of_para_subtasks) -> 'FullStartingScanner': + if idx_of_this_subtask >= number_of_para_subtasks: + raise Exception("idx_of_this_subtask must be less than number_of_para_subtasks") + self.idx_of_this_subtask = idx_of_this_subtask + self.number_of_para_subtasks = number_of_para_subtasks + return self + + def _append_only_filter_by_shard(self, partitioned_files: defaultdict) -> (defaultdict, int, int): + total_row = 0 + # Sort by file creation time to ensure consistent sharding + for key, file_entries in partitioned_files.items(): + for entry in file_entries: + total_row += entry.file.row_count + + # Calculate number of rows this shard should process + # Last shard handles all remaining rows (handles non-divisible cases) + if self.idx_of_this_subtask == self.number_of_para_subtasks - 1: + num_row = total_row - total_row // self.number_of_para_subtasks * self.idx_of_this_subtask + else: + num_row = total_row // self.number_of_para_subtasks + # Calculate start row and end row position for current shard in all data + start_row = self.idx_of_this_subtask * (total_row // self.number_of_para_subtasks) + end_row = start_row + num_row + + plan_start_row = 0 + plan_end_row = 0 + entry_end_row = 0 # end row position of current file in all data + splits_start_row = 0 + filtered_partitioned_files = defaultdict(list) + # Iterate through all file entries to find files that overlap with current shard range + for key, file_entries in partitioned_files.items(): + filtered_entries = [] + for entry in file_entries: + entry_begin_row = entry_end_row # Starting row position of current file in all data + entry_end_row += entry.file.row_count # Update to row position after current file + + # If current file is completely after shard range, stop iteration + if entry_begin_row >= end_row: + break + # If current file is completely before shard range, skip it + if entry_end_row <= start_row: + continue + if entry_begin_row <= start_row < entry_end_row: + splits_start_row = entry_begin_row + plan_start_row = start_row - entry_begin_row + # If shard end position is within current file, record relative end position + if entry_begin_row < end_row <= entry_end_row: + plan_end_row = end_row - splits_start_row + # Add files that overlap with shard range to result + filtered_entries.append(entry) + if filtered_entries: + filtered_partitioned_files[key] = filtered_entries + + return filtered_partitioned_files, plan_start_row, plan_end_row + + def _data_evolution_filter_by_shard(self, partitioned_files: defaultdict) -> (defaultdict, int, int): + total_row = 0 + first_row_id_set = set() + # Sort by file creation time to ensure consistent sharding + for key, file_entries in partitioned_files.items(): + for entry in file_entries: + if entry.file.first_row_id is None: + total_row += entry.file.row_count + elif entry.file.first_row_id not in first_row_id_set: + first_row_id_set.add(entry.file.first_row_id) + total_row += entry.file.row_count + + # Calculate number of rows this shard should process + # Last shard handles all remaining rows (handles non-divisible cases) + if self.idx_of_this_subtask == self.number_of_para_subtasks - 1: + num_row = total_row - total_row // self.number_of_para_subtasks * self.idx_of_this_subtask + else: + num_row = total_row // self.number_of_para_subtasks + # Calculate start row and end row position for current shard in all data + start_row = self.idx_of_this_subtask * (total_row // self.number_of_para_subtasks) + end_row = start_row + num_row + + plan_start_row = 0 + plan_end_row = 0 + entry_end_row = 0 # end row position of current file in all data + splits_start_row = 0 + filtered_partitioned_files = defaultdict(list) + # Iterate through all file entries to find files that overlap with current shard range + for key, file_entries in partitioned_files.items(): + filtered_entries = [] + first_row_id_set = set() + for entry in file_entries: + if entry.file.first_row_id is not None: + if entry.file.first_row_id in first_row_id_set: + filtered_entries.append(entry) + continue + else: + first_row_id_set.add(entry.file.first_row_id) + entry_begin_row = entry_end_row # Starting row position of current file in all data + entry_end_row += entry.file.row_count # Update to row position after current file + + # If current file is completely after shard range, stop iteration + if entry_begin_row >= end_row: + break + # If current file is completely before shard range, skip it + if entry_end_row <= start_row: + continue + if entry_begin_row <= start_row < entry_end_row: + splits_start_row = entry_begin_row + plan_start_row = start_row - entry_begin_row + # If shard end position is within current file, record relative end position + if entry_begin_row < end_row <= entry_end_row: + plan_end_row = end_row - splits_start_row + # Add files that overlap with shard range to result + filtered_entries.append(entry) + if filtered_entries: + filtered_partitioned_files[key] = filtered_entries + + return filtered_partitioned_files, plan_start_row, plan_end_row + + def _compute_split_start_end_row(self, splits: List[Split], plan_start_row, plan_end_row): + file_end_row = 0 # end row position of current file in all data + for split in splits: + files = split.files + split_start_row = file_end_row + # Iterate through all file entries to find files that overlap with current shard range + for file in files: + file_begin_row = file_end_row # Starting row position of current file in all data + file_end_row += file.row_count # Update to row position after current file + + # If shard start position is within current file, record actual start position and relative offset + if file_begin_row <= plan_start_row < file_end_row: + split.split_start_row = plan_start_row - file_begin_row + + # If shard end position is within current file, record relative end position + if file_begin_row < plan_end_row <= file_end_row: + split.split_end_row = plan_end_row - split_start_row + if split.split_start_row is None: + split.split_start_row = 0 + if split.split_end_row is None: + split.split_end_row = split.row_count + + def _primary_key_filter_by_shard(self, file_entries: List[ManifestEntry]) -> List[ManifestEntry]: + filtered_entries = [] + for entry in file_entries: + if entry.bucket % self.number_of_para_subtasks == self.idx_of_this_subtask: + filtered_entries.append(entry) + return filtered_entries + + def _apply_push_down_limit(self, splits: List[Split]) -> List[Split]: + if self.limit is None: + return splits + scanned_row_count = 0 + limited_splits = [] + + for split in splits: + if split.raw_convertible: + limited_splits.append(split) + scanned_row_count += split.row_count + if scanned_row_count >= self.limit: + return limited_splits + + return limited_splits + + def _filter_manifest_file(self, file: ManifestFileMeta) -> bool: + if not self.partition_key_predicate: + return True + return self.partition_key_predicate.test_by_simple_stats( + file.partition_stats, + file.num_added_files + file.num_deleted_files) + + def _filter_manifest_entry(self, entry: ManifestEntry) -> bool: + if self.only_read_real_buckets and entry.bucket < 0: + return False + if self.partition_key_predicate and not self.partition_key_predicate.test(entry.partition): + return False + + # Get SimpleStatsEvolution for this schema + evolution = self.simple_stats_evolutions.get_or_create(entry.file.schema_id) + + # Apply evolution to stats + if self.table.is_primary_key_table: + if not self.primary_key_predicate: + return True + return self.primary_key_predicate.test_by_simple_stats( + entry.file.key_stats, + entry.file.row_count + ) + else: + if not self.predicate: + return True + if entry.file.value_stats_cols is None and entry.file.write_cols is not None: + stats_fields = entry.file.write_cols + else: + stats_fields = entry.file.value_stats_cols + evolved_stats = evolution.evolution( + entry.file.value_stats, + entry.file.row_count, + stats_fields + ) + return self.predicate.test_by_simple_stats( + evolved_stats, + entry.file.row_count + ) + + def _create_append_only_splits(self, file_entries: List[ManifestEntry]) -> List['Split']: + partitioned_files = defaultdict(list) + for entry in file_entries: + partitioned_files[(tuple(entry.partition.values), entry.bucket)].append(entry) + + if self.idx_of_this_subtask is not None: + partitioned_files, plan_start_row, plan_end_row = self._append_only_filter_by_shard(partitioned_files) + + def weight_func(f: DataFileMeta) -> int: + return max(f.file_size, self.open_file_cost) + + splits = [] + for key, file_entries in partitioned_files.items(): + if not file_entries: + return [] + + data_files: List[DataFileMeta] = [e.file for e in file_entries] + + packed_files: List[List[DataFileMeta]] = self._pack_for_ordered(data_files, weight_func, + self.target_split_size) + splits += self._build_split_from_pack(packed_files, file_entries, False) + if self.idx_of_this_subtask is not None: + self._compute_split_start_end_row(splits, plan_start_row, plan_end_row) + return splits + + def _create_primary_key_splits(self, file_entries: List[ManifestEntry]) -> List['Split']: + if self.idx_of_this_subtask is not None: + file_entries = self._primary_key_filter_by_shard(file_entries) + partitioned_files = defaultdict(list) + for entry in file_entries: + partitioned_files[(tuple(entry.partition.values), entry.bucket)].append(entry) + + def weight_func(fl: List[DataFileMeta]) -> int: + return max(sum(f.file_size for f in fl), self.open_file_cost) + + splits = [] + for key, file_entries in partitioned_files.items(): + if not file_entries: + return [] + + data_files: List[DataFileMeta] = [e.file for e in file_entries] + partition_sort_runs: List[List[SortedRun]] = IntervalPartition(data_files).partition() + sections: List[List[DataFileMeta]] = [ + [file for s in sl for file in s.files] + for sl in partition_sort_runs + ] + + packed_files: List[List[List[DataFileMeta]]] = self._pack_for_ordered(sections, weight_func, + self.target_split_size) + flatten_packed_files: List[List[DataFileMeta]] = [ + [file for sub_pack in pack for file in sub_pack] + for pack in packed_files + ] + splits += self._build_split_from_pack(flatten_packed_files, file_entries, True) + return splits + + def _build_split_from_pack(self, packed_files, file_entries, for_primary_key_split: bool) -> List['Split']: + splits = [] + for file_group in packed_files: + raw_convertible = True + if for_primary_key_split: + raw_convertible = len(file_group) == 1 + + file_paths = [] + total_file_size = 0 + total_record_count = 0 + + for data_file in file_group: + data_file.set_file_path(self.table.table_path, file_entries[0].partition, + file_entries[0].bucket) + file_paths.append(data_file.file_path) + total_file_size += data_file.file_size + total_record_count += data_file.row_count + + if file_paths: + split = Split( + files=file_group, + partition=file_entries[0].partition, + bucket=file_entries[0].bucket, + _file_paths=file_paths, + _row_count=total_record_count, + _file_size=total_file_size, + raw_convertible=raw_convertible + ) + splits.append(split) + return splits + + @staticmethod + def _pack_for_ordered(items: List, weight_func: Callable, target_weight: int) -> List[List]: + packed = [] + bin_items = [] + bin_weight = 0 + + for item in items: + weight = weight_func(item) + if bin_weight + weight > target_weight and len(bin_items) > 0: + packed.append(list(bin_items)) + bin_items.clear() + bin_weight = 0 + + bin_weight += weight + bin_items.append(item) + + if len(bin_items) > 0: + packed.append(bin_items) + + return packed + + def _create_data_evolution_splits(self, file_entries: List[ManifestEntry]) -> List['Split']: + partitioned_files = defaultdict(list) + for entry in file_entries: + partitioned_files[(tuple(entry.partition.values), entry.bucket)].append(entry) + + if self.idx_of_this_subtask is not None: + partitioned_files, plan_start_row, plan_end_row = self._data_evolution_filter_by_shard(partitioned_files) + + def weight_func(file_list: List[DataFileMeta]) -> int: + return max(sum(f.file_size for f in file_list), self.open_file_cost) + + splits = [] + for key, file_entries in partitioned_files.items(): + if not file_entries: + continue + + data_files: List[DataFileMeta] = [e.file for e in file_entries] + + # Split files by firstRowId for data evolution + split_by_row_id = self._split_by_row_id(data_files) + + # Pack the split groups for optimal split sizes + packed_files: List[List[List[DataFileMeta]]] = self._pack_for_ordered(split_by_row_id, weight_func, + self.target_split_size) + + # Flatten the packed files and build splits + flatten_packed_files: List[List[DataFileMeta]] = [ + [file for sub_pack in pack for file in sub_pack] + for pack in packed_files + ] + + splits += self._build_split_from_pack(flatten_packed_files, file_entries, False) + + if self.idx_of_this_subtask is not None: + self._compute_split_start_end_row(splits, plan_start_row, plan_end_row) + return splits + + def _split_by_row_id(self, files: List[DataFileMeta]) -> List[List[DataFileMeta]]: + split_by_row_id = [] + + def sort_key(file: DataFileMeta) -> tuple: + first_row_id = file.first_row_id if file.first_row_id is not None else float('-inf') + is_blob = 1 if self._is_blob_file(file.file_name) else 0 + # For files with same firstRowId, sort by maxSequenceNumber in descending order + # (larger sequence number means more recent data) + max_seq = file.max_sequence_number + return (first_row_id, is_blob, -max_seq) + + sorted_files = sorted(files, key=sort_key) + + # Filter blob files to only include those within the row ID range of non-blob files + sorted_files = self._filter_blob(sorted_files) + + # Split files by firstRowId + last_row_id = -1 + check_row_id_start = 0 + current_split = [] + + for file in sorted_files: + first_row_id = file.first_row_id + if first_row_id is None: + # Files without firstRowId are treated as individual splits + split_by_row_id.append([file]) + continue + + if not self._is_blob_file(file.file_name) and first_row_id != last_row_id: + if current_split: + split_by_row_id.append(current_split) + + # Validate that files don't overlap + if first_row_id < check_row_id_start: + file_names = [f.file_name for f in sorted_files] + raise ValueError( + f"There are overlapping files in the split: {file_names}, " + f"the wrong file is: {file.file_name}" + ) + + current_split = [] + last_row_id = first_row_id + check_row_id_start = first_row_id + file.row_count + + current_split.append(file) + + if current_split: + split_by_row_id.append(current_split) + + return split_by_row_id + + @staticmethod + def _is_blob_file(file_name: str) -> bool: + return file_name.endswith('.blob') + + @staticmethod + def _filter_blob(files: List[DataFileMeta]) -> List[DataFileMeta]: + result = [] + row_id_start = -1 + row_id_end = -1 + + for file in files: + if not FullStartingScanner._is_blob_file(file.file_name): + if file.first_row_id is not None: + row_id_start = file.first_row_id + row_id_end = file.first_row_id + file.row_count + result.append(file) + else: + if file.first_row_id is not None and row_id_start != -1: + if row_id_start <= file.first_row_id < row_id_end: + result.append(file) + + return result diff --git a/paimon-python/pypaimon/read/scanner/incremental_starting_scanner.py b/paimon-python/pypaimon/read/scanner/incremental_starting_scanner.py new file mode 100644 index 000000000000..56c186188aa1 --- /dev/null +++ b/paimon-python/pypaimon/read/scanner/incremental_starting_scanner.py @@ -0,0 +1,69 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +from typing import List, Optional + +from pypaimon.common.predicate import Predicate +from pypaimon.manifest.schema.manifest_entry import ManifestEntry +from pypaimon.read.scanner.full_starting_scanner import FullStartingScanner +from pypaimon.snapshot.snapshot_manager import SnapshotManager + + +class IncrementalStartingScanner(FullStartingScanner): + def __init__(self, table, predicate: Optional[Predicate], limit: Optional[int], + start: int, end: int): + super().__init__(table, predicate, limit) + self.startingSnapshotId = start + self.endingSnapshotId = end + + def plan_files(self) -> List[ManifestEntry]: + snapshots_in_range = [] + for snapshot_id in range(self.startingSnapshotId + 1, self.endingSnapshotId + 1): + snapshot = self.snapshot_manager.get_snapshot_by_id(snapshot_id) + if snapshot.commit_kind == "APPEND": + snapshots_in_range.append(snapshot) + + # Collect all file entries from all snapshots in range + file_entries = [] + + for snapshot in snapshots_in_range: + manifest_files = self.manifest_list_manager.read_delta(snapshot) + entries = self.read_manifest_entries(manifest_files) + file_entries.extend(entries) + return file_entries + + @staticmethod + def between_timestamps(table, predicate: Optional[Predicate], limit: Optional[int], + start_timestamp: int, end_timestamp: int) -> 'IncrementalStartingScanner': + """ + Create an IncrementalStartingScanner for snapshots between two timestamps. + """ + snapshot_manager = SnapshotManager(table) + starting_snapshot = snapshot_manager.earlier_or_equal_time_mills(start_timestamp) + earliest_snapshot = snapshot_manager.try_get_earliest_snapshot() + + # If earliest_snapshot.time_millis > start_timestamp we should include the earliest_snapshot + if starting_snapshot is None or (earliest_snapshot and earliest_snapshot.time_millis > start_timestamp): + start_id = earliest_snapshot.id - 1 if earliest_snapshot else -1 + else: + start_id = starting_snapshot.id + + end_snapshot = snapshot_manager.earlier_or_equal_time_mills(end_timestamp) + latest_snapshot = snapshot_manager.get_latest_snapshot() + end_id = end_snapshot.id if end_snapshot else (latest_snapshot.id if latest_snapshot else -1) + + return IncrementalStartingScanner(table, predicate, limit, start_id, end_id) diff --git a/paimon-python/pypaimon/read/scanner/starting_scanner.py b/paimon-python/pypaimon/read/scanner/starting_scanner.py new file mode 100644 index 000000000000..7e6cdfd81aea --- /dev/null +++ b/paimon-python/pypaimon/read/scanner/starting_scanner.py @@ -0,0 +1,28 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +from abc import ABC, abstractmethod + +from pypaimon.read.plan import Plan + + +class StartingScanner(ABC): + """Helper class for the first planning of TableScan.""" + + @abstractmethod + def scan(self) -> Plan: + """Plan the files to read.""" diff --git a/paimon-python/pypaimon/read/split.py b/paimon-python/pypaimon/read/split.py index 9b802d98807d..f1ab5f3a5bf1 100644 --- a/paimon-python/pypaimon/read/split.py +++ b/paimon-python/pypaimon/read/split.py @@ -32,6 +32,8 @@ class Split: _file_paths: List[str] _row_count: int _file_size: int + split_start_row: int = None + split_end_row: int = None raw_convertible: bool = False @property diff --git a/paimon-python/pypaimon/read/split_read.py b/paimon-python/pypaimon/read/split_read.py index 1fe0a89d0ee3..a744fbc4f0ec 100644 --- a/paimon-python/pypaimon/read/split_read.py +++ b/paimon-python/pypaimon/read/split_read.py @@ -21,16 +21,22 @@ from functools import partial from typing import List, Optional, Tuple +from pypaimon.common.core_options import CoreOptions from pypaimon.common.predicate import Predicate +from pypaimon.manifest.schema.data_file_meta import DataFileMeta from pypaimon.read.interval_partition import IntervalPartition, SortedRun from pypaimon.read.partition_info import PartitionInfo -from pypaimon.read.reader.concat_batch_reader import ConcatBatchReader +from pypaimon.read.push_down_utils import trim_predicate_by_fields +from pypaimon.read.reader.concat_batch_reader import ConcatBatchReader, ShardBatchReader, MergeAllBatchReader from pypaimon.read.reader.concat_record_reader import ConcatRecordReader -from pypaimon.read.reader.data_file_record_reader import DataFileBatchReader +from pypaimon.read.reader.data_file_batch_reader import DataFileBatchReader +from pypaimon.read.reader.data_evolution_merge_reader import DataEvolutionMergeReader +from pypaimon.read.reader.field_bunch import FieldBunch, DataBunch, BlobBunch from pypaimon.read.reader.drop_delete_reader import DropDeleteRecordReader from pypaimon.read.reader.empty_record_reader import EmptyFileRecordReader from pypaimon.read.reader.filter_record_reader import FilterRecordReader from pypaimon.read.reader.format_avro_reader import FormatAvroReader +from pypaimon.read.reader.format_blob_reader import FormatBlobReader from pypaimon.read.reader.format_pyarrow_reader import FormatPyArrowReader from pypaimon.read.reader.iface.record_batch_reader import RecordBatchReader from pypaimon.read.reader.iface.record_reader import RecordReader @@ -49,36 +55,54 @@ class SplitRead(ABC): """Abstract base class for split reading operations.""" - def __init__(self, table, predicate: Optional[Predicate], push_down_predicate, - read_type: List[DataField], split: Split): + def __init__(self, table, predicate: Optional[Predicate], read_type: List[DataField], split: Split): from pypaimon.table.file_store_table import FileStoreTable self.table: FileStoreTable = table self.predicate = predicate - self.push_down_predicate = push_down_predicate + self.push_down_predicate = self._push_down_predicate() self.split = split self.value_arity = len(read_type) - self.trimmed_primary_key = [field.name for field in self.table.table_schema.get_trimmed_primary_key_fields()] + self.trimmed_primary_key = self.table.trimmed_primary_keys self.read_fields = read_type if isinstance(self, MergeFileSplitRead): self.read_fields = self._create_key_value_fields(read_type) + self.schema_id_2_fields = {} + + def _push_down_predicate(self) -> Optional[Predicate]: + if self.predicate is None: + return None + elif self.table.is_primary_key_table: + pk_predicate = trim_predicate_by_fields(self.predicate, self.table.primary_keys) + if not pk_predicate: + return None + return pk_predicate + else: + return self.predicate @abstractmethod def create_reader(self) -> RecordReader: """Create a record reader for the given split.""" - def file_reader_supplier(self, file_path: str, for_merge_read: bool): + def file_reader_supplier(self, file: DataFileMeta, for_merge_read: bool, read_fields: List[str]): + (read_file_fields, read_arrow_predicate) = self._get_fields_and_predicate(file.schema_id, read_fields) + + file_path = file.file_path _, extension = os.path.splitext(file_path) file_format = extension[1:] format_reader: RecordBatchReader - if file_format == "avro": - format_reader = FormatAvroReader(self.table.file_io, file_path, self._get_final_read_data_fields(), - self.read_fields, self.push_down_predicate) - elif file_format == "parquet" or file_format == "orc": + if file_format == CoreOptions.FILE_FORMAT_AVRO: + format_reader = FormatAvroReader(self.table.file_io, file_path, read_file_fields, + self.read_fields, read_arrow_predicate) + elif file_format == CoreOptions.FILE_FORMAT_BLOB: + blob_as_descriptor = CoreOptions.get_blob_as_descriptor(self.table.options) + format_reader = FormatBlobReader(self.table.file_io, file_path, read_file_fields, + self.read_fields, read_arrow_predicate, blob_as_descriptor) + elif file_format == CoreOptions.FILE_FORMAT_PARQUET or file_format == CoreOptions.FILE_FORMAT_ORC: format_reader = FormatPyArrowReader(self.table.file_io, file_format, file_path, - self._get_final_read_data_fields(), self.push_down_predicate) + read_file_fields, read_arrow_predicate) else: raise ValueError(f"Unexpected file format: {file_format}") @@ -91,6 +115,20 @@ def file_reader_supplier(self, file_path: str, for_merge_read: bool): return DataFileBatchReader(format_reader, index_mapping, partition_info, None, self.table.table_schema.fields) + def _get_fields_and_predicate(self, schema_id: int, read_fields): + key = (schema_id, tuple(read_fields)) + if key not in self.schema_id_2_fields: + schema = self.table.schema_manager.get_schema(schema_id) + schema_field_names = set(field.name for field in schema.fields) + if self.table.is_primary_key_table: + schema_field_names.add('_SEQUENCE_NUMBER') + schema_field_names.add('_VALUE_KIND') + read_file_fields = [read_field for read_field in read_fields if read_field in schema_field_names] + read_predicate = trim_predicate_by_fields(self.push_down_predicate, read_file_fields) + read_arrow_predicate = read_predicate.to_arrow() if read_predicate else None + self.schema_id_2_fields[key] = (read_file_fields, read_arrow_predicate) + return self.schema_id_2_fields[key] + @abstractmethod def _get_all_data_fields(self): """Get all data fields""" @@ -243,13 +281,21 @@ class RawFileSplitRead(SplitRead): def create_reader(self) -> RecordReader: data_readers = [] - for file_path in self.split.file_paths: - supplier = partial(self.file_reader_supplier, file_path=file_path, for_merge_read=False) + for file in self.split.files: + supplier = partial( + self.file_reader_supplier, + file=file, + for_merge_read=False, + read_fields=self._get_final_read_data_fields(), + ) data_readers.append(supplier) if not data_readers: return EmptyFileRecordReader() - concat_reader = ConcatBatchReader(data_readers) + if self.split.split_start_row is not None: + concat_reader = ShardBatchReader(data_readers, self.split.split_start_row, self.split.split_end_row) + else: + concat_reader = ConcatBatchReader(data_readers) # if the table is appendonly table, we don't need extra filter, all predicates has pushed down if self.table.is_primary_key_table and self.predicate: return FilterRecordReader(concat_reader, self.predicate) @@ -261,8 +307,13 @@ def _get_all_data_fields(self): class MergeFileSplitRead(SplitRead): - def kv_reader_supplier(self, file_path): - reader_supplier = partial(self.file_reader_supplier, file_path=file_path, for_merge_read=True) + def kv_reader_supplier(self, file): + reader_supplier = partial( + self.file_reader_supplier, + file=file, + for_merge_read=True, + read_fields=self._get_final_read_data_fields() + ) return KeyValueWrapReader(reader_supplier(), len(self.trimmed_primary_key), self.value_arity) def section_reader_supplier(self, section: List[SortedRun]): @@ -270,7 +321,7 @@ def section_reader_supplier(self, section: List[SortedRun]): for sorter_run in section: data_readers = [] for file in sorter_run.files: - supplier = partial(self.kv_reader_supplier, file.file_path) + supplier = partial(self.kv_reader_supplier, file) data_readers.append(supplier) readers.append(ConcatRecordReader(data_readers)) return SortMergeReaderWithMinHeap(readers, self.table.table_schema) @@ -290,3 +341,200 @@ def create_reader(self) -> RecordReader: def _get_all_data_fields(self): return self._create_key_value_fields(self.table.fields) + + +class DataEvolutionSplitRead(SplitRead): + + def create_reader(self) -> RecordReader: + files = self.split.files + suppliers = [] + + # Split files by row ID using the same logic as Java DataEvolutionSplitGenerator.split + split_by_row_id = self._split_by_row_id(files) + + for need_merge_files in split_by_row_id: + if len(need_merge_files) == 1 or not self.read_fields: + # No need to merge fields, just create a single file reader + suppliers.append( + lambda f=need_merge_files[0]: self._create_file_reader(f, self._get_final_read_data_fields()) + ) + else: + suppliers.append( + lambda files=need_merge_files: self._create_union_reader(files) + ) + if self.split.split_start_row is not None: + return ShardBatchReader(suppliers, self.split.split_start_row, self.split.split_end_row) + else: + return ConcatBatchReader(suppliers) + + def _split_by_row_id(self, files: List[DataFileMeta]) -> List[List[DataFileMeta]]: + """Split files by firstRowId for data evolution.""" + + # Sort files by firstRowId and then by maxSequenceNumber + def sort_key(file: DataFileMeta) -> tuple: + first_row_id = file.first_row_id if file.first_row_id is not None else float('-inf') + is_blob = 1 if self._is_blob_file(file.file_name) else 0 + max_seq = file.max_sequence_number + return (first_row_id, is_blob, -max_seq) + + sorted_files = sorted(files, key=sort_key) + + # Split files by firstRowId + split_by_row_id = [] + last_row_id = -1 + check_row_id_start = 0 + current_split = [] + + for file in sorted_files: + first_row_id = file.first_row_id + if first_row_id is None: + split_by_row_id.append([file]) + continue + + if not self._is_blob_file(file.file_name) and first_row_id != last_row_id: + if current_split: + split_by_row_id.append(current_split) + if first_row_id < check_row_id_start: + raise ValueError( + f"There are overlapping files in the split: {files}, " + f"the wrong file is: {file}" + ) + current_split = [] + last_row_id = first_row_id + check_row_id_start = first_row_id + file.row_count + current_split.append(file) + + if current_split: + split_by_row_id.append(current_split) + + return split_by_row_id + + def _create_union_reader(self, need_merge_files: List[DataFileMeta]) -> RecordReader: + """Create a DataEvolutionFileReader for merging multiple files.""" + # Split field bunches + fields_files = self._split_field_bunches(need_merge_files) + + # Validate row counts and first row IDs + row_count = fields_files[0].row_count() + first_row_id = fields_files[0].files()[0].first_row_id + + for bunch in fields_files: + if bunch.row_count() != row_count: + raise ValueError("All files in a field merge split should have the same row count.") + if bunch.files()[0].first_row_id != first_row_id: + raise ValueError( + "All files in a field merge split should have the same first row id and could not be null." + ) + + # Create the union reader + all_read_fields = self.read_fields + file_record_readers = [None] * len(fields_files) + read_field_index = [field.id for field in all_read_fields] + + # Initialize offsets + row_offsets = [-1] * len(all_read_fields) + field_offsets = [-1] * len(all_read_fields) + + for i, bunch in enumerate(fields_files): + first_file = bunch.files()[0] + + # Get field IDs for this bunch + if self._is_blob_file(first_file.file_name): + # For blob files, we need to get the field ID from the write columns + field_ids = [self._get_field_id_from_write_cols(first_file)] + elif first_file.write_cols: + field_ids = self._get_field_ids_from_write_cols(first_file.write_cols) + else: + # For regular files, get all field IDs from the schema + field_ids = [field.id for field in self.table.fields] + + read_fields = [] + for j, read_field_id in enumerate(read_field_index): + for field_id in field_ids: + if read_field_id == field_id: + if row_offsets[j] == -1: + row_offsets[j] = i + field_offsets[j] = len(read_fields) + read_fields.append(all_read_fields[j]) + break + + if not read_fields: + file_record_readers[i] = None + else: + table_fields = self.read_fields + self.read_fields = read_fields # create reader based on read_fields + # Create reader for this bunch + if len(bunch.files()) == 1: + file_record_readers[i] = self._create_file_reader( + bunch.files()[0], [field.name for field in read_fields] + ) + else: + # Create concatenated reader for multiple files + suppliers = [ + lambda f=file: self._create_file_reader( + f, [field.name for field in read_fields] + ) for file in bunch.files() + ] + file_record_readers[i] = MergeAllBatchReader(suppliers) + self.read_fields = table_fields + + # Validate that all required fields are found + for i, field in enumerate(all_read_fields): + if row_offsets[i] == -1: + if not field.type.nullable: + raise ValueError(f"Field {field} is not null but can't find any file contains it.") + + return DataEvolutionMergeReader(row_offsets, field_offsets, file_record_readers) + + def _create_file_reader(self, file: DataFileMeta, read_fields: [str]) -> RecordReader: + """Create a file reader for a single file.""" + return self.file_reader_supplier(file=file, for_merge_read=False, read_fields=read_fields) + + def _split_field_bunches(self, need_merge_files: List[DataFileMeta]) -> List[FieldBunch]: + """Split files into field bunches.""" + + fields_files = [] + blob_bunch_map = {} + row_count = -1 + + for file in need_merge_files: + if self._is_blob_file(file.file_name): + field_id = self._get_field_id_from_write_cols(file) + if field_id not in blob_bunch_map: + blob_bunch_map[field_id] = BlobBunch(row_count) + blob_bunch_map[field_id].add(file) + else: + # Normal file, just add it to the current merge split + fields_files.append(DataBunch(file)) + row_count = file.row_count + + fields_files.extend(blob_bunch_map.values()) + return fields_files + + def _get_field_id_from_write_cols(self, file: DataFileMeta) -> int: + """Get field ID from write columns for blob files.""" + if not file.write_cols or len(file.write_cols) == 0: + raise ValueError("Blob file must have write columns") + + # Find the field by name in the table schema + field_name = file.write_cols[0] + for field in self.table.fields: + if field.name == field_name: + return field.id + raise ValueError(f"Field {field_name} not found in table schema") + + def _get_field_ids_from_write_cols(self, write_cols: List[str]) -> List[int]: + field_ids = [] + for field_name in write_cols: + for field in self.table.fields: + if field.name == field_name: + field_ids.append(field.id) + return field_ids + + @staticmethod + def _is_blob_file(file_name: str) -> bool: + """Check if a file is a blob file based on its extension.""" + return file_name.endswith('.blob') + + def _get_all_data_fields(self): + return self.table.fields diff --git a/paimon-python/pypaimon/read/table_read.py b/paimon-python/pypaimon/read/table_read.py index b5f7a7b7653a..15aaf35462f2 100644 --- a/paimon-python/pypaimon/read/table_read.py +++ b/paimon-python/pypaimon/read/table_read.py @@ -15,18 +15,17 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -from typing import Iterator, List, Optional, Any +from typing import Iterator, List, Optional import pandas import pyarrow +from pypaimon.common.core_options import CoreOptions from pypaimon.common.predicate import Predicate -from pypaimon.common.predicate_builder import PredicateBuilder -from pypaimon.read.push_down_utils import extract_predicate_to_list from pypaimon.read.reader.iface.record_batch_reader import RecordBatchReader from pypaimon.read.split import Split from pypaimon.read.split_read import (MergeFileSplitRead, RawFileSplitRead, - SplitRead) + SplitRead, DataEvolutionSplitRead) from pypaimon.schema.data_types import DataField, PyarrowFieldParser from pypaimon.table.row.offset_row import OffsetRow @@ -39,7 +38,6 @@ def __init__(self, table, predicate: Optional[Predicate], read_type: List[DataFi self.table: FileStoreTable = table self.predicate = predicate - self.push_down_predicate = self._push_down_predicate() self.read_type = read_type def to_iterator(self, splits: List[Split]) -> Iterator: @@ -59,10 +57,35 @@ def to_arrow_batch_reader(self, splits: List[Split]) -> pyarrow.ipc.RecordBatchR batch_iterator = self._arrow_batch_generator(splits, schema) return pyarrow.ipc.RecordBatchReader.from_batches(schema, batch_iterator) + @staticmethod + def _try_to_pad_batch_by_schema(batch: pyarrow.RecordBatch, target_schema): + if batch.schema.names == target_schema.names: + return batch + + columns = [] + num_rows = batch.num_rows + + for field in target_schema: + if field.name in batch.column_names: + col = batch.column(field.name) + else: + col = pyarrow.nulls(num_rows, type=field.type) + columns.append(col) + + return pyarrow.RecordBatch.from_arrays(columns, schema=target_schema) + def to_arrow(self, splits: List[Split]) -> Optional[pyarrow.Table]: batch_reader = self.to_arrow_batch_reader(splits) - arrow_table = batch_reader.read_all() - return arrow_table + + schema = PyarrowFieldParser.from_paimon_schema(self.read_type) + table_list = [] + for batch in iter(batch_reader.read_next_batch, None): + table_list.append(self._try_to_pad_batch_by_schema(batch, schema)) + + if not table_list: + return pyarrow.Table.from_arrays([pyarrow.array([], type=field.type) for field in schema], schema=schema) + else: + return pyarrow.Table.from_batches(table_list) def _arrow_batch_generator(self, splits: List[Split], schema: pyarrow.Schema) -> Iterator[pyarrow.RecordBatch]: chunk_size = 65536 @@ -108,27 +131,18 @@ def to_ray(self, splits: List[Split]) -> "ray.data.dataset.Dataset": return ray.data.from_arrow(self.to_arrow(splits)) - def _push_down_predicate(self) -> Any: - if self.predicate is None: - return None - elif self.table.is_primary_key_table: - result = [] - extract_predicate_to_list(result, self.predicate, self.table.primary_keys) - if result: - # the field index is unused for arrow field - pk_predicates = (PredicateBuilder(self.table.fields).and_predicates(result)).to_arrow() - return pk_predicates - else: - return None - else: - return self.predicate.to_arrow() - def _create_split_read(self, split: Split) -> SplitRead: if self.table.is_primary_key_table and not split.raw_convertible: return MergeFileSplitRead( table=self.table, predicate=self.predicate, - push_down_predicate=self.push_down_predicate, + read_type=self.read_type, + split=split + ) + elif self.table.options.get(CoreOptions.DATA_EVOLUTION_ENABLED, 'false').lower() == 'true': + return DataEvolutionSplitRead( + table=self.table, + predicate=self.predicate, read_type=self.read_type, split=split ) @@ -136,7 +150,6 @@ def _create_split_read(self, split: Split) -> SplitRead: return RawFileSplitRead( table=self.table, predicate=self.predicate, - push_down_predicate=self.push_down_predicate, read_type=self.read_type, split=split ) diff --git a/paimon-python/pypaimon/read/table_scan.py b/paimon-python/pypaimon/read/table_scan.py index 0b9f97db4f9f..22994d4bb28a 100644 --- a/paimon-python/pypaimon/read/table_scan.py +++ b/paimon-python/pypaimon/read/table_scan.py @@ -16,256 +16,58 @@ # limitations under the License. ################################################################################ -from collections import defaultdict -from typing import Callable, List, Optional +from typing import Optional +from pypaimon.common.core_options import CoreOptions from pypaimon.common.predicate import Predicate -from pypaimon.common.predicate_builder import PredicateBuilder -from pypaimon.manifest.manifest_file_manager import ManifestFileManager -from pypaimon.manifest.manifest_list_manager import ManifestListManager -from pypaimon.manifest.schema.data_file_meta import DataFileMeta -from pypaimon.manifest.schema.manifest_entry import ManifestEntry -from pypaimon.read.interval_partition import IntervalPartition, SortedRun + from pypaimon.read.plan import Plan -from pypaimon.read.push_down_utils import (extract_predicate_to_dict, - extract_predicate_to_list) -from pypaimon.read.split import Split -from pypaimon.schema.data_types import DataField +from pypaimon.read.scanner.empty_starting_scanner import EmptyStartingScanner +from pypaimon.read.scanner.full_starting_scanner import FullStartingScanner +from pypaimon.read.scanner.incremental_starting_scanner import \ + IncrementalStartingScanner +from pypaimon.read.scanner.starting_scanner import StartingScanner from pypaimon.snapshot.snapshot_manager import SnapshotManager -from pypaimon.table.bucket_mode import BucketMode -from pypaimon.write.row_key_extractor import FixedBucketRowKeyExtractor class TableScan: """Implementation of TableScan for native Python reading.""" - def __init__(self, table, predicate: Optional[Predicate], limit: Optional[int], - read_type: List[DataField]): + def __init__(self, table, predicate: Optional[Predicate], limit: Optional[int]): from pypaimon.table.file_store_table import FileStoreTable self.table: FileStoreTable = table self.predicate = predicate self.limit = limit - self.read_type = read_type - - self.snapshot_manager = SnapshotManager(table) - self.manifest_list_manager = ManifestListManager(table) - self.manifest_file_manager = ManifestFileManager(table) - - pk_conditions = [] - trimmed_pk = [field.name for field in self.table.table_schema.get_trimmed_primary_key_fields()] - extract_predicate_to_list(pk_conditions, self.predicate, trimmed_pk) - self.primary_key_predicate = PredicateBuilder(self.table.fields).and_predicates(pk_conditions) - - partition_conditions = defaultdict(list) - extract_predicate_to_dict(partition_conditions, self.predicate, self.table.partition_keys) - self.partition_key_predicate = partition_conditions - - self.target_split_size = 128 * 1024 * 1024 - self.open_file_cost = 4 * 1024 * 1024 - - self.idx_of_this_subtask = None - self.number_of_para_subtasks = None - - self.only_read_real_buckets = True if int( - self.table.options.get('bucket', -1)) == BucketMode.POSTPONE_BUCKET.value else False + self.starting_scanner = self._create_starting_scanner() def plan(self) -> Plan: - latest_snapshot = self.snapshot_manager.get_latest_snapshot() - if not latest_snapshot: - return Plan([], []) - manifest_files = self.manifest_list_manager.read_all(latest_snapshot) - - deleted_entries = set() - added_entries = [] - # TODO: filter manifest files by predicate - for manifest_file in manifest_files: - manifest_entries = self.manifest_file_manager.read(manifest_file.file_name, - lambda row: self._bucket_filter(row)) - for entry in manifest_entries: - if entry.kind == 0: - added_entries.append(entry) - else: - deleted_entries.add((tuple(entry.partition.values), entry.bucket, entry.file.file_name)) - - file_entries = [ - entry for entry in added_entries - if (tuple(entry.partition.values), entry.bucket, entry.file.file_name) not in deleted_entries - ] - - if self.predicate: - file_entries = self._filter_by_predicate(file_entries) - - partitioned_split = defaultdict(list) - for entry in file_entries: - partitioned_split[(tuple(entry.partition.values), entry.bucket)].append(entry) - - splits = [] - for key, values in partitioned_split.items(): - if self.table.is_primary_key_table: - splits += self._create_primary_key_splits(values) - else: - splits += self._create_append_only_splits(values) - - splits = self._apply_push_down_limit(splits) - - return Plan(file_entries, splits) + return self.starting_scanner.scan() + + def _create_starting_scanner(self) -> Optional[StartingScanner]: + options = self.table.options + if CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP in options: + ts = options[CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP].split(",") + if len(ts) != 2: + raise ValueError( + "The incremental-between-timestamp must specific start(exclusive) and end timestamp. But is: " + + options[CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP]) + earliest_snapshot = SnapshotManager(self.table).try_get_earliest_snapshot() + latest_snapshot = SnapshotManager(self.table).get_latest_snapshot() + if earliest_snapshot is None or latest_snapshot is None: + return EmptyStartingScanner() + start_timestamp = int(ts[0]) + end_timestamp = int(ts[1]) + if start_timestamp >= end_timestamp: + raise ValueError( + "Ending timestamp %s should be >= starting timestamp %s." % (end_timestamp, start_timestamp)) + if (start_timestamp == end_timestamp or start_timestamp > latest_snapshot.time_millis + or end_timestamp < earliest_snapshot.time_millis): + return EmptyStartingScanner() + return IncrementalStartingScanner.between_timestamps(self.table, self.predicate, self.limit, + start_timestamp, end_timestamp) + return FullStartingScanner(self.table, self.predicate, self.limit) def with_shard(self, idx_of_this_subtask, number_of_para_subtasks) -> 'TableScan': - self.idx_of_this_subtask = idx_of_this_subtask - self.number_of_para_subtasks = number_of_para_subtasks + self.starting_scanner.with_shard(idx_of_this_subtask, number_of_para_subtasks) return self - - def _bucket_filter(self, entry: Optional[ManifestEntry]) -> bool: - bucket = entry.bucket - if self.only_read_real_buckets and bucket < 0: - return False - if self.idx_of_this_subtask is not None: - if self.table.is_primary_key_table: - return bucket % self.number_of_para_subtasks == self.idx_of_this_subtask - else: - file = entry.file.file_name - return FixedBucketRowKeyExtractor.hash(file) % self.number_of_para_subtasks == self.idx_of_this_subtask - return True - - def _apply_push_down_limit(self, splits: List[Split]) -> List[Split]: - if self.limit is None: - return splits - scanned_row_count = 0 - limited_splits = [] - - for split in splits: - if split.raw_convertible: - limited_splits.append(split) - scanned_row_count += split.row_count - if scanned_row_count >= self.limit: - return limited_splits - - return limited_splits - - def _filter_by_predicate(self, file_entries: List[ManifestEntry]) -> List[ManifestEntry]: - if not self.predicate: - return file_entries - - filtered_files = [] - for file_entry in file_entries: - if self.partition_key_predicate and not self._filter_by_partition(file_entry): - continue - if not self._filter_by_stats(file_entry): - continue - filtered_files.append(file_entry) - - return filtered_files - - def _filter_by_partition(self, file_entry: ManifestEntry) -> bool: - partition_dict = file_entry.partition.to_dict() - for field_name, conditions in self.partition_key_predicate.items(): - partition_value = partition_dict[field_name] - for predicate in conditions: - if not predicate.test_by_value(partition_value): - return False - return True - - def _filter_by_stats(self, file_entry: ManifestEntry) -> bool: - if file_entry.kind != 0: - return False - if self.table.is_primary_key_table: - predicate = self.primary_key_predicate - stats = file_entry.file.key_stats - else: - predicate = self.predicate - stats = file_entry.file.value_stats - return predicate.test_by_stats({ - "min_values": stats.min_values.to_dict(), - "max_values": stats.max_values.to_dict(), - "null_counts": { - stats.min_values.fields[i].name: stats.null_counts[i] for i in range(len(stats.min_values.fields)) - }, - "row_count": file_entry.file.row_count, - }) - - def _create_append_only_splits(self, file_entries: List[ManifestEntry]) -> List['Split']: - if not file_entries: - return [] - - data_files: List[DataFileMeta] = [e.file for e in file_entries] - - def weight_func(f: DataFileMeta) -> int: - return max(f.file_size, self.open_file_cost) - - packed_files: List[List[DataFileMeta]] = self._pack_for_ordered(data_files, weight_func, self.target_split_size) - return self._build_split_from_pack(packed_files, file_entries, False) - - def _create_primary_key_splits(self, file_entries: List[ManifestEntry]) -> List['Split']: - if not file_entries: - return [] - - data_files: List[DataFileMeta] = [e.file for e in file_entries] - partition_sort_runs: List[List[SortedRun]] = IntervalPartition(data_files).partition() - sections: List[List[DataFileMeta]] = [ - [file for s in sl for file in s.files] - for sl in partition_sort_runs - ] - - def weight_func(fl: List[DataFileMeta]) -> int: - return max(sum(f.file_size for f in fl), self.open_file_cost) - - packed_files: List[List[List[DataFileMeta]]] = self._pack_for_ordered(sections, weight_func, - self.target_split_size) - flatten_packed_files: List[List[DataFileMeta]] = [ - [file for sub_pack in pack for file in sub_pack] - for pack in packed_files - ] - return self._build_split_from_pack(flatten_packed_files, file_entries, True) - - def _build_split_from_pack(self, packed_files, file_entries, for_primary_key_split: bool) -> List['Split']: - splits = [] - for file_group in packed_files: - raw_convertible = True - if for_primary_key_split: - raw_convertible = len(file_group) == 1 - - file_paths = [] - total_file_size = 0 - total_record_count = 0 - - for data_file in file_group: - data_file.set_file_path(self.table.table_path, file_entries[0].partition, - file_entries[0].bucket) - file_paths.append(data_file.file_path) - total_file_size += data_file.file_size - total_record_count += data_file.row_count - - if file_paths: - split = Split( - files=file_group, - partition=file_entries[0].partition, - bucket=file_entries[0].bucket, - _file_paths=file_paths, - _row_count=total_record_count, - _file_size=total_file_size, - raw_convertible=raw_convertible - ) - splits.append(split) - return splits - - @staticmethod - def _pack_for_ordered(items: List, weight_func: Callable, target_weight: int) -> List[List]: - packed = [] - bin_items = [] - bin_weight = 0 - - for item in items: - weight = weight_func(item) - if bin_weight + weight > target_weight and len(bin_items) > 0: - packed.append(list(bin_items)) - bin_items.clear() - bin_weight = 0 - - bin_weight += weight - bin_items.append(item) - - if len(bin_items) > 0: - packed.append(bin_items) - - return packed diff --git a/paimon-python/pypaimon/schema/data_types.py b/paimon-python/pypaimon/schema/data_types.py old mode 100644 new mode 100755 index 5255ac6b1e04..91404cb1931c --- a/paimon-python/pypaimon/schema/data_types.py +++ b/paimon-python/pypaimon/schema/data_types.py @@ -121,7 +121,8 @@ def __init__(self, nullable: bool, element_type: DataType): def to_dict(self) -> Dict[str, Any]: return { - "type": "MULTISET{}".format('<' + str(self.element) + '>' if self.element else ''), + "type": "MULTISET{}{}".format('<' + str(self.element) + '>' if self.element else '', + " NOT NULL" if not self.nullable else ""), "element": self.element.to_dict() if self.element else None, "nullable": self.nullable, } @@ -234,7 +235,10 @@ def from_dict(cls, data: Dict[str, Any]) -> "RowType": return DataTypeParser.parse_data_type(data) def __str__(self) -> str: - field_strs = ["{}: {}".format(field.name, field.type) for field in self.fields] + field_strs = [] + for field in self.fields: + description = " COMMENT {}".format(field.description) if field.description else "" + field_strs.append("{}: {}{}".format(field.name, field.type, description)) null_suffix = "" if self.nullable else " NOT NULL" return "ROW<{}>{}".format(', '.join(field_strs), null_suffix) @@ -247,6 +251,7 @@ class Keyword(Enum): BINARY = "BINARY" VARBINARY = "VARBINARY" BYTES = "BYTES" + BLOB = "BLOB" DECIMAL = "DECIMAL" NUMERIC = "NUMERIC" DEC = "DEC" @@ -407,6 +412,8 @@ def from_paimon_type(data_type: DataType) -> pyarrow.DataType: return pyarrow.string() elif type_name == 'BYTES' or type_name.startswith('VARBINARY'): return pyarrow.binary() + elif type_name == 'BLOB': + return pyarrow.large_binary() elif type_name.startswith('BINARY'): if type_name == 'BINARY': return pyarrow.binary(1) @@ -506,6 +513,8 @@ def to_paimon_type(pa_type: pyarrow.DataType, nullable: bool) -> DataType: type_name = f'BINARY({pa_type.byte_width})' elif types.is_binary(pa_type): type_name = 'BYTES' + elif types.is_large_binary(pa_type): + type_name = 'BLOB' elif types.is_decimal(pa_type): type_name = f'DECIMAL({pa_type.precision}, {pa_type.scale})' elif types.is_timestamp(pa_type) and pa_type.tz is None: @@ -543,6 +552,7 @@ def to_paimon_field(field_idx: int, pa_field: pyarrow.Field) -> DataField: @staticmethod def to_paimon_schema(pa_schema: pyarrow.Schema) -> List[DataField]: + # Convert PyArrow schema to Paimon fields fields = [] for i, pa_field in enumerate(pa_schema): pa_field: pyarrow.Field diff --git a/paimon-python/pypaimon/schema/schema.py b/paimon-python/pypaimon/schema/schema.py index 965fe2255bb6..0ad53f99d341 100644 --- a/paimon-python/pypaimon/schema/schema.py +++ b/paimon-python/pypaimon/schema/schema.py @@ -20,6 +20,7 @@ import pyarrow as pa +from pypaimon.common.core_options import CoreOptions from pypaimon.common.json_util import json_field from pypaimon.schema.data_types import DataField, PyarrowFieldParser @@ -51,4 +52,37 @@ def __init__(self, fields: Optional[List[DataField]] = None, partition_keys: Opt def from_pyarrow_schema(pa_schema: pa.Schema, partition_keys: Optional[List[str]] = None, primary_keys: Optional[List[str]] = None, options: Optional[Dict] = None, comment: Optional[str] = None): - return Schema(PyarrowFieldParser.to_paimon_schema(pa_schema), partition_keys, primary_keys, options, comment) + # Convert PyArrow schema to Paimon fields + fields = PyarrowFieldParser.to_paimon_schema(pa_schema) + + # Check if Blob type exists in the schema + has_blob_type = any( + 'blob' in str(field.type).lower() + for field in fields + ) + + # If Blob type exists, validate required options + if has_blob_type: + if options is None: + options = {} + + required_options = { + CoreOptions.ROW_TRACKING_ENABLED: 'true', + CoreOptions.DATA_EVOLUTION_ENABLED: 'true' + } + + missing_options = [] + for key, expected_value in required_options.items(): + if key not in options or options[key] != expected_value: + missing_options.append(f"{key}='{expected_value}'") + + if missing_options: + raise ValueError( + f"Schema contains Blob type but is missing required options: {', '.join(missing_options)}. " + f"Please add these options to the schema." + ) + + if primary_keys is not None: + raise ValueError("Blob type is not supported with primary key.") + + return Schema(fields, partition_keys, primary_keys, options, comment) diff --git a/paimon-python/pypaimon/schema/schema_manager.py b/paimon-python/pypaimon/schema/schema_manager.py index 31297cc2b3de..33c6d4af8ba1 100644 --- a/paimon-python/pypaimon/schema/schema_manager.py +++ b/paimon-python/pypaimon/schema/schema_manager.py @@ -31,6 +31,7 @@ def __init__(self, file_io: FileIO, table_path: Path): self.file_io = file_io self.table_path = table_path self.schema_path = table_path / "schema" + self.schema_cache = {} def latest(self) -> Optional['TableSchema']: try: @@ -39,7 +40,7 @@ def latest(self) -> Optional['TableSchema']: return None max_version = max(versions) - return self._read_schema(max_version) + return self.get_schema(max_version) except Exception as e: raise RuntimeError(f"Failed to load schema from path: {self.schema_path}") from e @@ -57,19 +58,24 @@ def create_table(self, schema: Schema) -> TableSchema: def commit(self, new_schema: TableSchema) -> bool: schema_path = self._to_schema_path(new_schema.id) try: - return self.file_io.try_to_write_atomic(schema_path, JSON.to_json(new_schema, indent=2)) + result = self.file_io.try_to_write_atomic(schema_path, JSON.to_json(new_schema, indent=2)) + if result: + self.schema_cache[new_schema.id] = new_schema + return result except Exception as e: raise RuntimeError(f"Failed to commit schema: {e}") from e def _to_schema_path(self, schema_id: int) -> Path: return self.schema_path / f"{self.schema_prefix}{schema_id}" - def _read_schema(self, schema_id: int) -> Optional['TableSchema']: - schema_path = self._to_schema_path(schema_id) - if not self.file_io.exists(schema_path): - return None - - return TableSchema.from_path(self.file_io, schema_path) + def get_schema(self, schema_id: int) -> Optional[TableSchema]: + if schema_id not in self.schema_cache: + schema_path = self._to_schema_path(schema_id) + if not self.file_io.exists(schema_path): + return None + schema = TableSchema.from_path(self.file_io, schema_path) + self.schema_cache[schema_id] = schema + return self.schema_cache[schema_id] def _list_versioned_files(self) -> List[int]: if not self.file_io.exists(self.schema_path): diff --git a/paimon-python/pypaimon/schema/table_schema.py b/paimon-python/pypaimon/schema/table_schema.py index f74d713e9fd5..25dd19398b8e 100644 --- a/paimon-python/pypaimon/schema/table_schema.py +++ b/paimon-python/pypaimon/schema/table_schema.py @@ -145,28 +145,3 @@ def copy(self, new_options: Optional[Dict[str, str]] = None) -> "TableSchema": comment=self.comment, time_millis=self.time_millis ) - - def get_primary_key_fields(self) -> List[DataField]: - if not self.primary_keys: - return [] - field_map = {field.name: field for field in self.fields} - return [field_map[name] for name in self.primary_keys if name in field_map] - - def get_partition_key_fields(self) -> List[DataField]: - if not self.partition_keys: - return [] - field_map = {field.name: field for field in self.fields} - return [field_map[name] for name in self.partition_keys if name in field_map] - - def get_trimmed_primary_key_fields(self) -> List[DataField]: - if not self.primary_keys or not self.partition_keys: - return self.get_primary_key_fields() - adjusted = [pk for pk in self.primary_keys if pk not in self.partition_keys] - # Validate that filtered list is not empty - if not adjusted: - raise ValueError( - f"Primary key constraint {self.primary_keys} " - f"should not be same with partition fields {self.partition_keys}, " - "this will result in only one record in a partition") - field_map = {field.name: field for field in self.fields} - return [field_map[name] for name in adjusted if name in field_map] diff --git a/paimon-python/pypaimon/snapshot/catalog_snapshot_commit.py b/paimon-python/pypaimon/snapshot/catalog_snapshot_commit.py old mode 100644 new mode 100755 index 26796f77668a..1ffc80af57b5 --- a/paimon-python/pypaimon/snapshot/catalog_snapshot_commit.py +++ b/paimon-python/pypaimon/snapshot/catalog_snapshot_commit.py @@ -57,9 +57,9 @@ def commit(self, snapshot: Snapshot, branch: str, statistics: List[PartitionStat Exception: If commit fails """ new_identifier = Identifier( - database_name=self.identifier.get_database_name(), - object_name=self.identifier.get_table_name(), - branch_name=branch + database=self.identifier.get_database_name(), + object=self.identifier.get_table_name(), + branch=branch ) # Call catalog's commit_snapshot method diff --git a/paimon-python/pypaimon/snapshot/snapshot.py b/paimon-python/pypaimon/snapshot/snapshot.py index 5bc92dcad4c3..d5f72dedcf46 100644 --- a/paimon-python/pypaimon/snapshot/snapshot.py +++ b/paimon-python/pypaimon/snapshot/snapshot.py @@ -21,6 +21,8 @@ from pypaimon.common.json_util import json_field +BATCH_COMMIT_IDENTIFIER = 0x7fffffffffffffff + @dataclass class Snapshot: @@ -43,3 +45,4 @@ class Snapshot: changelog_record_count: Optional[int] = json_field("changelogRecordCount", default=None) watermark: Optional[int] = json_field("watermark", default=None) statistics: Optional[str] = json_field("statistics", default=None) + next_row_id: Optional[int] = json_field("nextRowId", default=None) diff --git a/paimon-python/pypaimon/snapshot/snapshot_manager.py b/paimon-python/pypaimon/snapshot/snapshot_manager.py index 2ded3578028e..87b42c6f3500 100644 --- a/paimon-python/pypaimon/snapshot/snapshot_manager.py +++ b/paimon-python/pypaimon/snapshot/snapshot_manager.py @@ -59,3 +59,58 @@ def get_snapshot_path(self, snapshot_id: int) -> Path: Path to the snapshot file """ return self.snapshot_dir / f"snapshot-{snapshot_id}" + + def try_get_earliest_snapshot(self) -> Optional[Snapshot]: + if self.file_io.exists(self.snapshot_dir / "EARLIEST"): + earliest_content = self.file_io.read_file_utf8(self.snapshot_dir / "EARLIEST") + earliest_snapshot_id = int(earliest_content.strip()) + return self.get_snapshot_by_id(earliest_snapshot_id) + else: + return self.get_snapshot_by_id(1) + + def earlier_or_equal_time_mills(self, timestamp: int) -> Optional[Snapshot]: + """ + Find the latest snapshot with time_millis <= the given timestamp. + + Args: + timestamp: The timestamp to compare against + + Returns: + The latest snapshot with time_millis <= timestamp, or None if no such snapshot exists + """ + earliest = 1 + latest = self.get_latest_snapshot().id + final_snapshot = None + + while earliest <= latest: + mid = earliest + (latest - earliest) // 2 + snapshot = self.get_snapshot_by_id(mid) + commit_time = snapshot.time_millis + + if commit_time > timestamp: + latest = mid - 1 + elif commit_time < timestamp: + earliest = mid + 1 + final_snapshot = snapshot + else: + final_snapshot = snapshot + break + + return final_snapshot + + def get_snapshot_by_id(self, snapshot_id: int) -> Optional[Snapshot]: + """ + Get a snapshot by its ID. + + Args: + snapshot_id: The snapshot ID + + Returns: + The snapshot with the specified ID, or None if not found + """ + snapshot_file = self.get_snapshot_path(snapshot_id) + if not self.file_io.exists(snapshot_file): + return None + + snapshot_content = self.file_io.read_file_utf8(snapshot_file) + return JSON.from_json(snapshot_content, Snapshot) diff --git a/paimon-python/pypaimon/table/file_store_table.py b/paimon-python/pypaimon/table/file_store_table.py index e0d7c7e6033a..6132c9bd6926 100644 --- a/paimon-python/pypaimon/table/file_store_table.py +++ b/paimon-python/pypaimon/table/file_store_table.py @@ -28,7 +28,7 @@ from pypaimon.schema.table_schema import TableSchema from pypaimon.table.bucket_mode import BucketMode from pypaimon.table.table import Table -from pypaimon.write.batch_write_builder import BatchWriteBuilder +from pypaimon.write.write_builder import BatchWriteBuilder, StreamWriteBuilder from pypaimon.write.row_key_extractor import (DynamicBucketRowKeyExtractor, FixedBucketRowKeyExtractor, PostponeBucketRowKeyExtractor, @@ -46,9 +46,15 @@ def __init__(self, file_io: FileIO, identifier: Identifier, table_path: Path, self.table_schema = table_schema self.fields = table_schema.fields + self.field_names = [field.name for field in table_schema.fields] self.field_dict = {field.name: field for field in self.fields} self.primary_keys = table_schema.primary_keys + self.primary_keys_fields = [self.field_dict[name] for name in self.primary_keys] self.partition_keys = table_schema.partition_keys + self.partition_keys_fields = [self.field_dict[name] for name in self.partition_keys] + self.trimmed_primary_keys = [pk for pk in self.primary_keys if pk not in self.partition_keys] + self.trimmed_primary_keys_fields = [self.field_dict[name] for name in self.trimmed_primary_keys] + self.options = table_schema.options self.cross_partition_update = self.table_schema.cross_partition_update() self.is_primary_key_table = bool(self.primary_keys) @@ -92,6 +98,9 @@ def new_read_builder(self) -> 'ReadBuilder': def new_batch_write_builder(self) -> BatchWriteBuilder: return BatchWriteBuilder(self) + def new_stream_write_builder(self) -> StreamWriteBuilder: + return StreamWriteBuilder(self) + def create_row_key_extractor(self) -> RowKeyExtractor: bucket_mode = self.bucket_mode() if bucket_mode == BucketMode.HASH_FIXED: @@ -104,3 +113,20 @@ def create_row_key_extractor(self) -> RowKeyExtractor: return DynamicBucketRowKeyExtractor(self.table_schema) else: raise ValueError(f"Unsupported bucket mode: {bucket_mode}") + + def copy(self, options: dict) -> 'FileStoreTable': + if CoreOptions.BUCKET in options and options.get(CoreOptions.BUCKET) != self.options.get(CoreOptions.BUCKET): + raise ValueError("Cannot change bucket number") + new_options = self.options.copy() + for k, v in options.items(): + if v is None: + new_options.pop(k) + else: + new_options[k] = v + new_table_schema = self.table_schema.copy(new_options=new_options) + return FileStoreTable(self.file_io, self.identifier, self.table_path, new_table_schema, + self.catalog_environment) + + def add_options(self, options: dict): + for key, value in options.items(): + self.options[key] = value diff --git a/paimon-python/pypaimon/table/row/binary_row.py b/paimon-python/pypaimon/table/row/binary_row.py new file mode 100644 index 000000000000..41773b57e3c6 --- /dev/null +++ b/paimon-python/pypaimon/table/row/binary_row.py @@ -0,0 +1,58 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +from typing import Any, List +from pypaimon.schema.data_types import DataField +from pypaimon.table.row.internal_row import InternalRow +from pypaimon.table.row.row_kind import RowKind + + +class BinaryRow(InternalRow): + """ + BinaryRow is a compact binary format for storing a row of data. + """ + + def __init__(self, data: bytes, fields: List[DataField]): + """ + Initialize BinaryRow with raw binary data and field definitions. + """ + self.data = data + self.arity = int.from_bytes(data[:4], 'big') + # Skip the arity prefix (4 bytes) if present + self.actual_data = data[4:] if len(data) >= 4 else data + self.fields = fields + self.row_kind = RowKind(self.actual_data[0]) + + def get_field(self, index: int) -> Any: + from pypaimon.table.row.generic_row import GenericRowDeserializer + """Get field value by index.""" + if index >= self.arity or index < 0: + raise IndexError(f"Field index {index} out of range [0, {self.arity})") + + if GenericRowDeserializer.is_null_at(self.actual_data, 0, index): + return None + + return GenericRowDeserializer.parse_field_value(self.actual_data, 0, + GenericRowDeserializer.calculate_bit_set_width_in_bytes( + self.arity), + index, self.fields[index].type) + + def get_row_kind(self) -> RowKind: + return self.row_kind + + def __len__(self): + return self.arity diff --git a/paimon-python/pypaimon/table/row/blob.py b/paimon-python/pypaimon/table/row/blob.py new file mode 100644 index 000000000000..b92ddb3e8281 --- /dev/null +++ b/paimon-python/pypaimon/table/row/blob.py @@ -0,0 +1,250 @@ +################################################################################ +# 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 io +from abc import ABC, abstractmethod +from typing import Optional, Union +from urllib.parse import urlparse + +from pypaimon.common.uri_reader import UriReader, FileUriReader + + +class BlobDescriptor: + CURRENT_VERSION = 1 + + def __init__(self, uri: str, offset: int, length: int, version: int = CURRENT_VERSION): + self._version = version + self._uri = uri + self._offset = offset + self._length = length + + @property + def uri(self) -> str: + return self._uri + + @property + def offset(self) -> int: + return self._offset + + @property + def length(self) -> int: + return self._length + + @property + def version(self) -> int: + return self._version + + def serialize(self) -> bytes: + import struct + + uri_bytes = self._uri.encode('utf-8') + uri_length = len(uri_bytes) + + # Pack using little endian format + data = struct.pack(' 'BlobDescriptor': + import struct + + if len(data) < 5: # minimum size: version(1) + uri_length(4) + raise ValueError("Invalid BlobDescriptor data: too short") + + offset = 0 + + # Read version + version = struct.unpack(' len(data): + raise ValueError("Invalid BlobDescriptor data: URI length exceeds data size") + + uri_bytes = data[offset:offset + uri_length] + uri = uri_bytes.decode('utf-8') + offset += uri_length + + # Read offset and length + if offset + 16 > len(data): + raise ValueError("Invalid BlobDescriptor data: missing offset/length") + + blob_offset = struct.unpack(' bool: + """Check equality with another BlobDescriptor.""" + if not isinstance(other, BlobDescriptor): + return False + return (self._version == other._version and + self._uri == other._uri and + self._offset == other._offset and + self._length == other._length) + + def __hash__(self) -> int: + """Calculate hash for the BlobDescriptor.""" + return hash((self._version, self._uri, self._offset, self._length)) + + def __str__(self) -> str: + """String representation of BlobDescriptor.""" + return (f"BlobDescriptor(version={self._version}, uri='{self._uri}', " + f"offset={self._offset}, length={self._length})") + + def __repr__(self) -> str: + """Detailed representation of BlobDescriptor.""" + return self.__str__() + + +class Blob(ABC): + + @abstractmethod + def to_data(self) -> bytes: + pass + + @abstractmethod + def to_descriptor(self) -> BlobDescriptor: + pass + + @abstractmethod + def new_input_stream(self) -> io.BytesIO: + pass + + @staticmethod + def from_data(data: bytes) -> 'Blob': + return BlobData(data) + + @staticmethod + def from_local(file: str) -> 'Blob': + # Import FileIO locally to avoid circular imports + from pypaimon.common.file_io import FileIO + + parsed = urlparse(file) + if parsed.scheme == "file": + file_uri = file + else: + file_uri = f"file://{file}" + file_io = FileIO(file_uri, {}) + uri_reader = FileUriReader(file_io) + descriptor = BlobDescriptor(file, 0, -1) + return Blob.from_descriptor(uri_reader, descriptor) + + @staticmethod + def from_http(uri: str) -> 'Blob': + descriptor = BlobDescriptor(uri, 0, -1) + return BlobRef(UriReader.from_http(), descriptor) + + @staticmethod + def from_file(file_io, file_path: str, offset: int, length: int) -> 'Blob': + uri_reader = FileUriReader(file_io) + descriptor = BlobDescriptor(file_path, offset, length) + return Blob.from_descriptor(uri_reader, descriptor) + + @staticmethod + def from_descriptor(uri_reader: UriReader, descriptor: BlobDescriptor) -> 'Blob': + return BlobRef(uri_reader, descriptor) + + +class BlobData(Blob): + + def __init__(self, data: Optional[Union[bytes, bytearray]] = None): + if data is None: + self._data = b'' + elif isinstance(data, (bytes, bytearray)): + self._data = bytes(data) + else: + raise TypeError(f"BlobData expects bytes, bytearray, or None, got {type(data)}") + + @classmethod + def from_bytes(cls, data: bytes) -> 'BlobData': + return cls(data) + + @property + def data(self) -> bytes: + return self._data + + def to_data(self) -> bytes: + return self._data + + def to_descriptor(self) -> 'BlobDescriptor': + raise RuntimeError("Blob data can not convert to descriptor.") + + def new_input_stream(self) -> io.BytesIO: + return io.BytesIO(self._data) + + def __eq__(self, other) -> bool: + if other is None or not isinstance(other, BlobData): + return False + return self._data == other._data + + def __hash__(self) -> int: + return hash(self._data) + + +class BlobRef(Blob): + + def __init__(self, uri_reader: UriReader, descriptor: BlobDescriptor): + self._uri_reader = uri_reader + self._descriptor = descriptor + + def to_data(self) -> bytes: + try: + with self.new_input_stream() as stream: + return stream.read() + except Exception as e: + raise IOError(f"Failed to read blob data: {e}") + + def to_descriptor(self) -> BlobDescriptor: + return self._descriptor + + def new_input_stream(self) -> io.BytesIO: + uri = self._descriptor.uri + offset = self._descriptor.offset + length = self._descriptor.length + with self._uri_reader.new_input_stream(uri) as input_stream: + if offset > 0: + input_stream.seek(offset) + if length == -1: + data = input_stream.read() + else: + data = input_stream.read(length) + return io.BytesIO(data) + + def __eq__(self, other) -> bool: + if not isinstance(other, BlobRef): + return False + return self._descriptor == other._descriptor + + def __hash__(self) -> int: + return hash(self._descriptor) diff --git a/paimon-python/pypaimon/table/row/generic_row.py b/paimon-python/pypaimon/table/row/generic_row.py index a7612168d926..b05e475951e8 100644 --- a/paimon-python/pypaimon/table/row/generic_row.py +++ b/paimon-python/pypaimon/table/row/generic_row.py @@ -17,24 +17,40 @@ ################################################################################ import struct -from dataclasses import dataclass from datetime import date, datetime, time, timedelta from decimal import Decimal -from typing import Any, List +from typing import Any, List, Union + +from dataclasses import dataclass from pypaimon.schema.data_types import AtomicType, DataField, DataType -from pypaimon.table.row.row_kind import RowKind +from pypaimon.table.row.binary_row import BinaryRow +from pypaimon.table.row.internal_row import InternalRow, RowKind +from pypaimon.table.row.blob import BlobData @dataclass -class GenericRow: - values: List[Any] - fields: List[DataField] - row_kind: RowKind = RowKind.INSERT +class GenericRow(InternalRow): + + def __init__(self, values: List[Any], fields: List[DataField], row_kind: RowKind = RowKind.INSERT): + self.values = values + self.fields = fields + self.row_kind = row_kind def to_dict(self): return {self.fields[i].name: self.values[i] for i in range(len(self.fields))} + def get_field(self, pos: int) -> Any: + if pos >= len(self.values): + raise IndexError(f"Position {pos} is out of bounds for row arity {len(self.values)}") + return self.values[pos] + + def get_row_kind(self) -> RowKind: + return self.row_kind + + def __len__(self) -> int: + return len(self.values) + class GenericRowDeserializer: HEADER_SIZE_IN_BITS = 8 @@ -57,28 +73,28 @@ def from_bytes( actual_data = bytes_data[4:] fields = [] - null_bits_size_in_bytes = cls._calculate_bit_set_width_in_bytes(arity) + null_bits_size_in_bytes = cls.calculate_bit_set_width_in_bytes(arity) for i, data_field in enumerate(data_fields): value = None - if not cls._is_null_at(actual_data, 0, i): - value = cls._parse_field_value(actual_data, 0, null_bits_size_in_bytes, i, data_field.type) + if not cls.is_null_at(actual_data, 0, i): + value = cls.parse_field_value(actual_data, 0, null_bits_size_in_bytes, i, data_field.type) fields.append(value) return GenericRow(fields, data_fields, RowKind(actual_data[0])) @classmethod - def _calculate_bit_set_width_in_bytes(cls, arity: int) -> int: + def calculate_bit_set_width_in_bytes(cls, arity: int) -> int: return ((arity + 63 + cls.HEADER_SIZE_IN_BITS) // 64) * 8 @classmethod - def _is_null_at(cls, bytes_data: bytes, offset: int, pos: int) -> bool: + def is_null_at(cls, bytes_data: bytes, offset: int, pos: int) -> bool: index = pos + cls.HEADER_SIZE_IN_BITS byte_index = offset + (index // 8) bit_index = index % 8 return (bytes_data[byte_index] & (1 << bit_index)) != 0 @classmethod - def _parse_field_value( + def parse_field_value( cls, bytes_data: bytes, base_offset: int, @@ -111,6 +127,8 @@ def _parse_field_value( return cls._parse_string(bytes_data, base_offset, field_offset) elif type_name.startswith('BINARY') or type_name.startswith('VARBINARY') or type_name == 'BYTES': return cls._parse_binary(bytes_data, base_offset, field_offset) + elif type_name == 'BLOB': + return cls._parse_blob(bytes_data, base_offset, field_offset) elif type_name.startswith('DECIMAL') or type_name.startswith('NUMERIC'): return cls._parse_decimal(bytes_data, base_offset, field_offset, data_type) elif type_name.startswith('TIMESTAMP'): @@ -193,6 +211,13 @@ def _parse_binary(cls, bytes_data: bytes, base_offset: int, field_offset: int) - length = (offset_and_len & cls.HIGHEST_SECOND_TO_EIGHTH_BIT) >> 56 return bytes_data[field_offset:field_offset + length] + @classmethod + def _parse_blob(cls, bytes_data: bytes, base_offset: int, field_offset: int) -> BlobData: + """Parse BLOB data from binary format and return a BlobData instance.""" + # BLOB uses the same binary format as regular binary data + binary_data = cls._parse_binary(bytes_data, base_offset, field_offset) + return BlobData.from_bytes(binary_data) + @classmethod def _parse_decimal(cls, bytes_data: bytes, base_offset: int, field_offset: int, data_type: DataType) -> Decimal: unscaled_long = struct.unpack(' bytes: - arity = len(binary_row.fields) + def to_bytes(cls, row: Union[GenericRow, BinaryRow]) -> bytes: + if isinstance(row, BinaryRow): + return row.data + arity = len(row.fields) null_bits_size_in_bytes = cls._calculate_bit_set_width_in_bytes(arity) fixed_part_size = null_bits_size_in_bytes + arity * 8 fixed_part = bytearray(fixed_part_size) - fixed_part[0] = binary_row.row_kind.value + fixed_part[0] = row.row_kind.value variable_part_data = [] current_variable_offset = 0 - for i, (value, field) in enumerate(zip(binary_row.values, binary_row.fields)): + for i, (value, field) in enumerate(zip(row.values, row.fields)): field_fixed_offset = null_bits_size_in_bytes + i * 8 if value is None: @@ -260,9 +287,12 @@ def to_bytes(cls, binary_row: GenericRow) -> bytes: raise ValueError(f"BinaryRow only support AtomicType yet, meet {field.type.__class__}") type_name = field.type.type.upper() - if any(type_name.startswith(p) for p in ['CHAR', 'VARCHAR', 'STRING', 'BINARY', 'VARBINARY', 'BYTES']): + if any(type_name.startswith(p) for p in ['CHAR', 'VARCHAR', 'STRING', + 'BINARY', 'VARBINARY', 'BYTES', 'BLOB']): if any(type_name.startswith(p) for p in ['CHAR', 'VARCHAR', 'STRING']): value_bytes = str(value).encode('utf-8') + elif type_name == 'BLOB': + value_bytes = value.to_data() else: value_bytes = bytes(value) diff --git a/paimon-python/pypaimon/table/row/internal_row.py b/paimon-python/pypaimon/table/row/internal_row.py index ca19ebcddf53..e70468348cbb 100644 --- a/paimon-python/pypaimon/table/row/internal_row.py +++ b/paimon-python/pypaimon/table/row/internal_row.py @@ -33,12 +33,6 @@ def get_field(self, pos: int) -> Any: Returns the value at the given position. """ - @abstractmethod - def is_null_at(self, pos: int) -> bool: - """ - Returns true if the element is null at the given position. - """ - @abstractmethod def get_row_kind(self) -> RowKind: """ diff --git a/paimon-python/pypaimon/table/row/offset_row.py b/paimon-python/pypaimon/table/row/offset_row.py index 90f23d212676..b6e6f8f432e9 100644 --- a/paimon-python/pypaimon/table/row/offset_row.py +++ b/paimon-python/pypaimon/table/row/offset_row.py @@ -36,22 +36,19 @@ def replace(self, row_tuple: tuple) -> 'OffsetRow': raise ValueError(f"Offset {self.offset} plus arity {self.arity} is out of row length {len(row_tuple)}") return self + def set_row_kind_byte(self, row_kind_byte: int) -> None: + """ + Store RowKind as a byte and instantiate it lazily to avoid performance overhead. + """ + self.row_kind_byte = row_kind_byte + def get_field(self, pos: int): if pos >= self.arity: raise IndexError(f"Position {pos} is out of bounds for row arity {self.arity}") return self.row_tuple[self.offset + pos] - def is_null_at(self, pos: int) -> bool: - return self.get_field(pos) is None - def get_row_kind(self) -> RowKind: return RowKind(self.row_kind_byte) - def set_row_kind_byte(self, row_kind_byte: int) -> None: - """ - Store RowKind as a byte and instantiate it lazily to avoid performance overhead. - """ - self.row_kind_byte = row_kind_byte - def __len__(self) -> int: return self.arity diff --git a/paimon-python/pypaimon/table/row/projected_row.py b/paimon-python/pypaimon/table/row/projected_row.py new file mode 100644 index 000000000000..dff63c372c6f --- /dev/null +++ b/paimon-python/pypaimon/table/row/projected_row.py @@ -0,0 +1,76 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +from typing import Any, List +from pypaimon.table.row.internal_row import InternalRow +from pypaimon.table.row.row_kind import RowKind + + +class ProjectedRow(InternalRow): + """ + An implementation of InternalRow which provides a projected view of the underlying InternalRow. + Projection includes both reducing the accessible fields and reordering them. + Note: This class supports only top-level projections, not nested projections. + """ + + def __init__(self, index_mapping: List[int]): + """ + Initialize ProjectedRow with index mapping. + Args: + index_mapping: Array representing the mapping of fields. For example, + [0, 2, 1] specifies to include in the following order the 1st field, the 3rd field + and the 2nd field of the row. + """ + self.index_mapping = index_mapping + self.row = None + + def replace_row(self, row: InternalRow) -> 'ProjectedRow': + self.row = row + return self + + def get_field(self, pos: int) -> Any: + """Returns the value at the given position.""" + if self.index_mapping[pos] < 0: + return None + return self.row.get_field(self.index_mapping[pos]) + + def get_row_kind(self) -> RowKind: + """Returns the kind of change that this row describes in a changelog.""" + return self.row.get_row_kind() + + def __len__(self) -> int: + """Returns the number of fields in this row.""" + return len(self.row) + + def __str__(self) -> str: + """String representation of the projected row.""" + return (f"{self.row.get_row_kind().name if self.row else 'None'}" + f"{{index_mapping={self.index_mapping}, row={self.row}}}") + + @staticmethod + def from_index_mapping(projection: List[int]) -> 'ProjectedRow': + """ + Create an empty ProjectedRow starting from a projection array. + Args: + projection: Array representing the mapping of fields. For example, + [0, 2, 1] specifies to include in the following order + the 1st field, the 3rd field and the 2nd field of the row. + Returns: + ProjectedRow instance + """ + return ProjectedRow(projection) diff --git a/paimon-python/pypaimon/table/table.py b/paimon-python/pypaimon/table/table.py index 3a1fe2e6228c..e20784f1fc9d 100644 --- a/paimon-python/pypaimon/table/table.py +++ b/paimon-python/pypaimon/table/table.py @@ -19,7 +19,7 @@ from abc import ABC, abstractmethod from pypaimon.read.read_builder import ReadBuilder -from pypaimon.write.batch_write_builder import BatchWriteBuilder +from pypaimon.write.write_builder import BatchWriteBuilder, StreamWriteBuilder class Table(ABC): @@ -32,3 +32,7 @@ def new_read_builder(self) -> ReadBuilder: @abstractmethod def new_batch_write_builder(self) -> BatchWriteBuilder: """Returns a builder for building batch table write and table commit.""" + + @abstractmethod + def new_stream_write_builder(self) -> StreamWriteBuilder: + """Returns a builder for building stream table write and table commit.""" diff --git a/paimon-python/pypaimon/tests/binary_row_test.py b/paimon-python/pypaimon/tests/binary_row_test.py new file mode 100644 index 000000000000..5bfabfb121ce --- /dev/null +++ b/paimon-python/pypaimon/tests/binary_row_test.py @@ -0,0 +1,334 @@ +################################################################################ +# 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 random +import tempfile +import unittest +from typing import List + +import pyarrow as pa + +from pypaimon import CatalogFactory, Schema +from pypaimon.manifest.schema.manifest_entry import ManifestEntry +from pypaimon.manifest.schema.simple_stats import SimpleStats +from pypaimon.read.scanner.full_starting_scanner import FullStartingScanner +from pypaimon.table.row.generic_row import GenericRow, GenericRowDeserializer + + +def _random_format(): + return random.choice(['parquet', 'avro', 'orc']) + + +class BinaryRowTest(unittest.TestCase): + + @classmethod + def setUpClass(cls): + cls.tempdir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.tempdir, 'warehouse') + cls.catalog = CatalogFactory.create({'warehouse': cls.warehouse}) + cls.catalog.create_database('default', False) + pa_schema = pa.schema([ + ('f0', pa.int64()), + ('f1', pa.string()), + ('f2', pa.int64()), + ]) + cls.catalog.create_table('default.test_append', Schema.from_pyarrow_schema( + pa_schema, partition_keys=['f0'], options={'file.format': _random_format()}), False) + cls.catalog.create_table('default.test_pk', Schema.from_pyarrow_schema( + pa_schema, partition_keys=['f2'], primary_keys=['f0'], + options={'bucket': '1', 'file.format': _random_format()}), False) + cls.data = pa.Table.from_pydict({ + 'f0': [1, 2, 3, 4, 5], + 'f1': ['abc', 'abbc', 'bc', 'd', None], + 'f2': [6, 7, 8, 9, 10], + }, schema=pa_schema) + + append_table = cls.catalog.get_table('default.test_append') + write_builder = append_table.new_batch_write_builder() + write = write_builder.new_write() + commit = write_builder.new_commit() + write.write_arrow(cls.data) + commit.commit(write.prepare_commit()) + write.close() + commit.close() + + pk_table = cls.catalog.get_table('default.test_pk') + write_builder = pk_table.new_batch_write_builder() + write = write_builder.new_write() + commit = write_builder.new_commit() + write.write_arrow(cls.data) + commit.commit(write.prepare_commit()) + write.close() + commit.close() + + def test_not_equal_append(self): + table = self.catalog.get_table('default.test_append') + self._overwrite_manifest_entry(table) + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.not_equal('f2', 6) # test stats filter when filtering ManifestEntry + splits, actual = self._read_result(read_builder.with_filter(predicate)) + expected = self.data.slice(1, 4) + self.assertEqual(expected, actual) + self.assertEqual(len(expected), len(splits)) + + def test_less_than_append(self): + table = self.catalog.get_table('default.test_append') + + self._overwrite_manifest_entry(table) + + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.less_than('f2', 8) + splits, actual = self._read_result(read_builder.with_filter(predicate)) + expected = self.data.slice(0, 2) + self.assertEqual(actual, expected) + self.assertEqual(len(expected), len(splits)) # test stats filter when filtering ManifestEntry + + def test_is_null_append(self): + table = self.catalog.get_table('default.test_append') + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.is_null('f1') # value_stats_cols=None + splits, actual = self._read_result(read_builder.with_filter(predicate)) + expected = self.data.slice(4, 1) + self.assertEqual(expected, actual) + self.assertEqual(len(expected), len(splits)) + + def test_is_not_null_append(self): + table = self.catalog.get_table('default.test_append') + starting_scanner = FullStartingScanner(table, None, None) + latest_snapshot = starting_scanner.snapshot_manager.get_latest_snapshot() + manifest_files = starting_scanner.manifest_list_manager.read_all(latest_snapshot) + manifest_entries = starting_scanner.manifest_file_manager.read(manifest_files[0].file_name) + self._transform_manifest_entries(manifest_entries, []) + l = ['abc', 'abbc', 'bc', 'd', None] + for i, entry in enumerate(manifest_entries): + entry.file.value_stats_cols = ['f1'] + entry.file.value_stats = SimpleStats( + GenericRow([l[i]], [table.fields[1]]), + GenericRow([l[i]], [table.fields[1]]), + [1 if l[i] is None else 0], + ) + starting_scanner.manifest_file_manager.write(manifest_files[0].file_name, manifest_entries) + + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.is_not_null('f1') + splits, actual = self._read_result(read_builder.with_filter(predicate)) + expected = self.data.slice(0, 4) + self.assertEqual(expected, actual) + self.assertEqual(len(expected), len(splits)) + + def test_is_in_append(self): + table = self.catalog.get_table('default.test_append') + self._overwrite_manifest_entry(table) + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.is_in('f2', [6, 8]) + splits, actual = self._read_result(read_builder.with_filter(predicate)) + expected = self.data.take([0, 2]) + self.assertEqual(expected, actual) + self.assertEqual(len(expected), len(splits)) + + def test_equal_pk(self): + table = self.catalog.get_table('default.test_pk') + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.equal('f2', 6) + splits, actual = self._read_result(read_builder.with_filter(predicate)) + expected = self.data.slice(0, 1) + self.assertEqual(expected, actual) + self.assertEqual(len(splits), len(expected)) # test partition filter when filtering ManifestEntry + + def test_not_equal_pk(self): + table = self.catalog.get_table('default.test_pk') + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.not_equal('f2', 6) + splits, actual = self._read_result(read_builder.with_filter(predicate)) + expected = self.data.slice(1, 4) + self.assertEqual(actual, expected) + self.assertEqual(len(splits), len(expected)) # test partition filter when filtering ManifestEntry + + def test_less_than_pk(self): + table = self.catalog.get_table('default.test_pk') + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.less_than('f0', 3) + splits, actual = self._read_result(read_builder.with_filter(predicate)) + expected = self.data.slice(0, 2) + self.assertEqual(expected, actual) + self.assertEqual(len(expected), len(splits)) # test key stats filter when filtering ManifestEntry + + def test_is_null_pk(self): + table = self.catalog.get_table('default.test_pk') + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.is_null('f1') + splits, actual = self._read_result(read_builder.with_filter(predicate)) + expected = self.data.slice(4, 1) + self.assertEqual(actual, expected) + + def test_is_not_null_pk(self): + table = self.catalog.get_table('default.test_pk') + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.is_not_null('f1') + splits, actual = self._read_result(read_builder.with_filter(predicate)) + expected = self.data.slice(0, 4) + self.assertEqual(actual, expected) + + def test_is_in_pk(self): + table = self.catalog.get_table('default.test_pk') + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.is_in('f0', [1, 5]) + splits, actual = self._read_result(read_builder.with_filter(predicate)) + # expected rows: indices [0, 3] + expected = self.data.take([0, 4]) + self.assertEqual(actual, expected) + self.assertEqual(len(splits), len(expected)) # test key stats filter when filtering ManifestEntry + + def test_append_multi_cols(self): + # Create a 10-column append table and write 10 rows + pa_schema = pa.schema([ + ('f0', pa.int64()), + ('f1', pa.string()), + ('f2', pa.int64()), + ('f3', pa.string()), + ('f4', pa.int64()), + ('f5', pa.string()), + ('f6', pa.int64()), + ('f7', pa.string()), + ('f8', pa.int64()), + ('f9', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + pa_schema, + partition_keys=['f0'], + options={'file.format': _random_format()} + ) + self.catalog.create_table('default.test_append_10cols', schema, False) + table = self.catalog.get_table('default.test_append_10cols') + + write_builder = table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + + data = { + 'f0': list(range(1, 11)), # 0..9 + 'f1': ['a0', 'bb', 'a2', 'a3', 'a4', 'a5', 'a6', 'a7', 'a8', 'a9'], # contains 'bb' at index 1 + 'f2': [10, 20, 30, 40, 50, 60, 70, 80, 90, 100], + 'f3': ['x0', 'x1', 'x2', 'x3', 'x4', 'x5', 'x6', 'x7', 'x8', 'x9'], + 'f4': [0, 1, 0, 1, 0, 1, 0, 1, 0, 1], + 'f5': ['y0', 'y1', 'y2', 'y3', 'y4', 'y5', 'y6', 'y7', 'y8', 'y9'], + 'f6': [100, 200, 300, 400, 500, 600, 700, 800, 900, 1000], + 'f7': ['z0', 'z1', 'z2', 'z3', 'z4', 'z5', 'z6', 'z7', 'z8', 'z9'], + 'f8': [5, 4, 3, 2, 1, 0, -1, -2, -3, -4], + 'f9': ['w0', 'w1', 'w2', 'w3', 'w4', 'w5', 'w6', 'w7', 'w8', 'w9'], + } + pa_table = pa.Table.from_pydict(data, schema=pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + starting_scanner = FullStartingScanner(table, None, None) + latest_snapshot = starting_scanner.snapshot_manager.get_latest_snapshot() + manifest_files = starting_scanner.manifest_list_manager.read_all(latest_snapshot) + manifest_entries = starting_scanner.manifest_file_manager.read(manifest_files[0].file_name) + self._transform_manifest_entries(manifest_entries, []) + for i, entry in enumerate(manifest_entries): + entry.file.value_stats_cols = ['f2', 'f6', 'f8'] + entry.file.value_stats = SimpleStats( + GenericRow([10 * (i + 1), 100 * (i + 1), 5 - i], [table.fields[2], table.fields[6], table.fields[8]]), + GenericRow([10 * (i + 1), 100 * (i + 1), 5 - i], [table.fields[2], table.fields[6], table.fields[8]]), + [0, 0, 0], + ) + starting_scanner.manifest_file_manager.write(manifest_files[0].file_name, manifest_entries) + # Build multiple predicates and combine them + read_builder = table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + p_in = predicate_builder.is_in('f6', [100, 600, 1000]) + p_contains = predicate_builder.less_or_equal('f8', -3) + p_not_null = predicate_builder.is_not_null('f2') + p_ge = predicate_builder.greater_or_equal('f2', 50) + p_or = predicate_builder.or_predicates([p_in, p_contains]) + combined = predicate_builder.and_predicates([p_or, p_not_null, p_ge]) + + splits, actual = self._read_result(read_builder.with_filter(combined)) + + # Expected rows after filter: indices 5 and 9 + expected_data = {'f0': [6, 9, 10], + 'f1': ['a5', 'a8', 'a9'], + 'f2': [60, 90, 100], + 'f3': ['x5', 'x8', 'x9'], + 'f4': [1, 0, 1], + 'f5': ['y5', 'y8', 'y9'], + 'f6': [600, 900, 1000], + 'f7': ['z5', 'z8', 'z9'], + 'f8': [0, -3, -4], + 'f9': ['w5', 'w8', 'w9'] + } + self.assertEqual(expected_data, actual.to_pydict()) + + starting_scanner = FullStartingScanner(table, None, None) + latest_snapshot = starting_scanner.snapshot_manager.get_latest_snapshot() + manifest_files = starting_scanner.manifest_list_manager.read_all(latest_snapshot) + manifest_entries = starting_scanner.manifest_file_manager.read(manifest_files[0].file_name) + self._transform_manifest_entries(manifest_entries, []) + for i, entry in enumerate(manifest_entries): + entry.file.value_stats_cols = ['f2', 'f6', 'f8'] + entry.file.value_stats = SimpleStats( + GenericRow([0, 100 * (i + 1), 5 - i], [table.fields[2], table.fields[6], table.fields[8]]), + GenericRow([0, 100 * (i + 1), 5 - i], [table.fields[2], table.fields[6], table.fields[8]]), + [0, 0, 0], + ) + starting_scanner.manifest_file_manager.write(manifest_files[0].file_name, manifest_entries) + splits, actual = self._read_result(read_builder.with_filter(combined)) + self.assertFalse(actual) + + def _read_result(self, read_builder): + scan = read_builder.new_scan() + read = read_builder.new_read() + splits = scan.plan().splits() + actual = read.to_arrow(splits) + return splits, actual + + def _transform_manifest_entries(self, manifest_entries: List[ManifestEntry], trimmed_pk_fields): + for entry in manifest_entries: + entry.file.key_stats.min_values = GenericRowDeserializer.from_bytes(entry.file.key_stats.min_values.data, + trimmed_pk_fields) + entry.file.key_stats.max_values = GenericRowDeserializer.from_bytes(entry.file.key_stats.max_values.data, + trimmed_pk_fields) + + def _overwrite_manifest_entry(self, table): + starting_scanner = FullStartingScanner(table, None, None) + latest_snapshot = starting_scanner.snapshot_manager.get_latest_snapshot() + manifest_files = starting_scanner.manifest_list_manager.read_all(latest_snapshot) + manifest_entries = starting_scanner.manifest_file_manager.read(manifest_files[0].file_name) + self._transform_manifest_entries(manifest_entries, []) + for i, entry in enumerate(manifest_entries): + entry.file.value_stats_cols = ['f2'] + entry.file.value_stats = SimpleStats( + GenericRow([6 + i], [table.fields[2]]), + GenericRow([6 + i], [table.fields[2]]), + [0], + ) + starting_scanner.manifest_file_manager.write(manifest_files[0].file_name, manifest_entries) diff --git a/paimon-python/pypaimon/tests/blob_table_test.py b/paimon-python/pypaimon/tests/blob_table_test.py new file mode 100644 index 000000000000..b9e2760e3eed --- /dev/null +++ b/paimon-python/pypaimon/tests/blob_table_test.py @@ -0,0 +1,1419 @@ +""" +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 +from pypaimon.table.file_store_table import FileStoreTable +from pypaimon.write.commit_message import CommitMessage + + +class DataBlobWriterTest(unittest.TestCase): + """Tests for DataBlobWriter functionality with paimon table operations.""" + + @classmethod + def setUpClass(cls): + """Set up test environment.""" + cls.temp_dir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.temp_dir, 'warehouse') + # Create catalog for table operations + cls.catalog = CatalogFactory.create({ + 'warehouse': cls.warehouse + }) + cls.catalog.create_database('test_db', False) + + @classmethod + def tearDownClass(cls): + """Clean up test environment.""" + try: + shutil.rmtree(cls.temp_dir) + except OSError: + pass + + def test_data_blob_writer_basic_functionality(self): + """Test basic DataBlobWriter functionality with paimon table.""" + from pypaimon import Schema + + # Create schema with normal and blob columns + pa_schema = pa.schema([ + ('id', pa.int32()), + ('name', pa.string()), + ('blob_data', pa.large_binary()), # This will be detected as blob + ]) + + # Create Paimon schema + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + + # Create table + self.catalog.create_table('test_db.blob_writer_test', schema, False) + table = self.catalog.get_table('test_db.blob_writer_test') + + # Test data + test_data = pa.Table.from_pydict({ + 'id': [1, 2, 3], + 'name': ['Alice', 'Bob', 'Charlie'], + 'blob_data': [b'blob_data_1', b'blob_data_2', b'blob_data_3'] + }, schema=pa_schema) + + # Test DataBlobWriter initialization using proper table API + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Write test data using BatchTableWrite API + blob_writer.write_arrow(test_data) + + # Test prepare commit + commit_messages = blob_writer.prepare_commit() + self.assertIsInstance(commit_messages, list) + self.assertGreater(len(commit_messages), 0) + + # Verify commit message structure + for commit_msg in commit_messages: + self.assertIsInstance(commit_msg.new_files, list) + self.assertGreater(len(commit_msg.new_files), 0) + + # Verify file metadata structure + for file_meta in commit_msg.new_files: + self.assertIsNotNone(file_meta.file_name) + self.assertGreater(file_meta.file_size, 0) + self.assertGreater(file_meta.row_count, 0) + + blob_writer.close() + + def test_data_blob_writer_schema_detection(self): + """Test that DataBlobWriter correctly detects blob columns from schema.""" + from pypaimon import Schema + + # Test schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('blob_field', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.blob_detection_test', schema, False) + table = self.catalog.get_table('test_db.blob_detection_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Test that DataBlobWriter was created internally + # We can verify this by checking the internal data writers + test_data = pa.Table.from_pydict({ + 'id': [1, 2, 3], + 'blob_field': [b'blob1', b'blob2', b'blob3'] + }, schema=pa_schema) + + # Write data to trigger writer creation + blob_writer.write_arrow(test_data) + + # Verify that a DataBlobWriter was created internally + data_writers = blob_writer.file_store_write.data_writers + self.assertGreater(len(data_writers), 0) + + # Check that the writer is a DataBlobWriter + for writer in data_writers.values(): + from pypaimon.write.writer.data_blob_writer import DataBlobWriter + self.assertIsInstance(writer, DataBlobWriter) + + blob_writer.close() + + def test_data_blob_writer_no_blob_column(self): + """Test that DataBlobWriter raises error when no blob column is found.""" + from pypaimon import Schema + + # Test schema without blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('name', pa.string()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.no_blob_test', schema, False) + table = self.catalog.get_table('test_db.no_blob_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + writer = write_builder.new_write() + + # Test that a regular writer (not DataBlobWriter) was created + test_data = pa.Table.from_pydict({ + 'id': [1, 2, 3], + 'name': ['Alice', 'Bob', 'Charlie'] + }, schema=pa_schema) + + # Write data to trigger writer creation + writer.write_arrow(test_data) + + # Verify that a regular writer was created (not DataBlobWriter) + data_writers = writer.file_store_write.data_writers + self.assertGreater(len(data_writers), 0) + + # Check that the writer is NOT a DataBlobWriter + for writer_instance in data_writers.values(): + from pypaimon.write.writer.data_blob_writer import DataBlobWriter + self.assertNotIsInstance(writer_instance, DataBlobWriter) + + writer.close() + + def test_data_blob_writer_multiple_blob_columns(self): + """Test that DataBlobWriter raises error when multiple blob columns are found.""" + from pypaimon import Schema + + # Test schema with multiple blob columns + pa_schema = pa.schema([ + ('id', pa.int32()), + ('blob1', pa.large_binary()), + ('blob2', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.multiple_blob_test', schema, False) + table = self.catalog.get_table('test_db.multiple_blob_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + writer = write_builder.new_write() + + # Test data with multiple blob columns + test_data = pa.Table.from_pydict({ + 'id': [1, 2, 3], + 'blob1': [b'blob1_1', b'blob1_2', b'blob1_3'], + 'blob2': [b'blob2_1', b'blob2_2', b'blob2_3'] + }, schema=pa_schema) + + # This should raise an error when DataBlobWriter is created internally + with self.assertRaises(ValueError) as context: + writer.write_arrow(test_data) + self.assertIn("Limit exactly one blob field in one paimon table yet", str(context.exception)) + + def test_data_blob_writer_write_operations(self): + """Test DataBlobWriter write operations with real data.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('name', pa.string()), + ('document', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.write_test', schema, False) + table = self.catalog.get_table('test_db.write_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Test data + test_data = pa.Table.from_pydict({ + 'id': [1, 2], + 'name': ['Alice', 'Bob'], + 'document': [b'document_content_1', b'document_content_2'] + }, schema=pa_schema) + + # Test writing data + for batch in test_data.to_batches(): + blob_writer.write_arrow_batch(batch) + + # Test prepare commit + commit_messages = blob_writer.prepare_commit() + self.assertIsInstance(commit_messages, list) + + blob_writer.close() + + def test_data_blob_writer_write_large_blob(self): + """Test DataBlobWriter with very large blob data (50MB per item) in 10 batches.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('description', pa.string()), + ('large_blob', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.large_blob_test', schema, False) + table = self.catalog.get_table('test_db.large_blob_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Create 50MB blob data per item + # Using a pattern to make the data more realistic and compressible + target_size = 50 * 1024 * 1024 # 50MB in bytes + blob_pattern = b'LARGE_BLOB_DATA_PATTERN_' + b'X' * 1024 # ~1KB pattern + pattern_size = len(blob_pattern) + repetitions = target_size // pattern_size + large_blob_data = blob_pattern * repetitions + + # Verify the blob size is approximately 50MB + blob_size_mb = len(large_blob_data) / (1024 * 1024) + self.assertGreater(blob_size_mb, 49) # Should be at least 49MB + self.assertLess(blob_size_mb, 51) # Should be less than 51MB + + total_rows = 0 + + # Write 10 batches, each with 5 rows (50 rows total) + # Total data volume: 50 rows * 50MB = 2.5GB of blob data + for batch_num in range(10): + batch_data = pa.Table.from_pydict({ + 'id': [batch_num * 5 + i for i in range(5)], + 'description': [f'Large blob batch {batch_num}, row {i}' for i in range(5)], + 'large_blob': [large_blob_data] * 5 # 5 rows per batch, each with 50MB blob + }, schema=pa_schema) + + # Write each batch + for batch in batch_data.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + # Log progress for large data processing + print(f"Completed batch {batch_num + 1}/10 with {batch.num_rows} rows") + + # Record count is tracked internally by DataBlobWriter + + # Test prepare commit + commit_messages: CommitMessage = blob_writer.prepare_commit() + self.assertIsInstance(commit_messages, list) + # Verify we have commit messages + self.assertEqual(len(commit_messages), 1) + commit_message = commit_messages[0] + normal_file_meta = commit_message.new_files[0] + blob_file_metas = commit_message.new_files[1:] + # Validate row count consistency + parquet_row_count = normal_file_meta.row_count + blob_row_count_sum = sum(meta.row_count for meta in blob_file_metas) + self.assertEqual(parquet_row_count, blob_row_count_sum, + f"Parquet row count ({parquet_row_count}) should equal " + f"sum of blob row counts ({blob_row_count_sum})") + + # Verify commit message structure and file metadata + total_file_size = 0 + total_row_count = parquet_row_count + for commit_msg in commit_messages: + self.assertIsInstance(commit_msg.new_files, list) + self.assertGreater(len(commit_msg.new_files), 0) + + # Verify file metadata structure + for file_meta in commit_msg.new_files: + self.assertIsNotNone(file_meta.file_name) + self.assertGreater(file_meta.file_size, 0) + self.assertGreater(file_meta.row_count, 0) + total_file_size += file_meta.file_size + + # Verify total data written (50 rows of normal data + 50 rows of blob data = 100 total) + self.assertEqual(total_row_count, 50) + + # Verify total file size is substantial (should be much larger than 2.5GB due to overhead) + total_size_mb = total_file_size / (1024 * 1024) + self.assertGreater(total_size_mb, 2000) # Should be at least 2GB due to overhead + + total_files = sum(len(commit_msg.new_files) for commit_msg in commit_messages) + print(f"Total data written: {total_size_mb:.2f}MB across {total_files} files") + print(f"Total rows processed: {total_row_count}") + + blob_writer.close() + + def test_data_blob_writer_abort_functionality(self): + """Test DataBlobWriter abort functionality.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('blob_data', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.abort_test', schema, False) + table = self.catalog.get_table('test_db.abort_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Test data + test_data = pa.Table.from_pydict({ + 'id': [1, 2], + 'blob_data': [b'blob_1', b'blob_2'] + }, schema=pa_schema) + + # Write some data + for batch in test_data.to_batches(): + blob_writer.write_arrow_batch(batch) + + # Test abort - BatchTableWrite doesn't have abort method + # The abort functionality is handled internally by DataBlobWriter + + blob_writer.close() + + def test_data_blob_writer_multiple_batches(self): + """Test DataBlobWriter with multiple batches and verify results.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('name', pa.string()), + ('document', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.multiple_batches_test', schema, False) + table = self.catalog.get_table('test_db.multiple_batches_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Test data - multiple batches + batch1_data = pa.Table.from_pydict({ + 'id': [1, 2], + 'name': ['Alice', 'Bob'], + 'document': [b'document_1_content', b'document_2_content'] + }, schema=pa_schema) + + batch2_data = pa.Table.from_pydict({ + 'id': [3, 4, 5], + 'name': ['Charlie', 'David', 'Eve'], + 'document': [b'document_3_content', b'document_4_content', b'document_5_content'] + }, schema=pa_schema) + + batch3_data = pa.Table.from_pydict({ + 'id': [6], + 'name': ['Frank'], + 'document': [b'document_6_content'] + }, schema=pa_schema) + + # Write multiple batches + total_rows = 0 + for batch in batch1_data.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + for batch in batch2_data.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + for batch in batch3_data.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + # Record count is tracked internally by DataBlobWriter + + # Test prepare commit + commit_messages = blob_writer.prepare_commit() + self.assertIsInstance(commit_messages, list) + + # Verify we have committed files + self.assertGreater(len(commit_messages), 0) + + blob_writer.close() + + def test_data_blob_writer_large_batches(self): + """Test DataBlobWriter with large batches to test rolling behavior.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('description', pa.string()), + ('large_blob', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.large_batches_test', schema, False) + table = self.catalog.get_table('test_db.large_batches_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Create large batches with substantial blob data + large_blob_data = b'L' * 10000 # 10KB blob data + + # Batch 1: 100 rows + batch1_data = pa.Table.from_pydict({ + 'id': list(range(1, 101)), + 'description': [f'Description for row {i}' for i in range(1, 101)], + 'large_blob': [large_blob_data] * 100 + }, schema=pa_schema) + + # Batch 2: 50 rows + batch2_data = pa.Table.from_pydict({ + 'id': list(range(101, 151)), + 'description': [f'Description for row {i}' for i in range(101, 151)], + 'large_blob': [large_blob_data] * 50 + }, schema=pa_schema) + + # Write large batches + total_rows = 0 + for batch in batch1_data.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + for batch in batch2_data.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + # Record count is tracked internally by DataBlobWriter + + # Test prepare commit + commit_messages = blob_writer.prepare_commit() + self.assertIsInstance(commit_messages, list) + + # Verify we have committed files + self.assertGreater(len(commit_messages), 0) + + blob_writer.close() + + def test_data_blob_writer_mixed_data_types(self): + """Test DataBlobWriter with mixed data types in blob column.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('type', pa.string()), + ('data', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.mixed_data_test', schema, False) + table = self.catalog.get_table('test_db.mixed_data_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Test data with different types of blob content + test_data = pa.Table.from_pydict({ + 'id': [1, 2, 3, 4, 5], + 'type': ['text', 'json', 'binary', 'image', 'pdf'], + 'data': [ + b'This is text content', + b'{"key": "value", "number": 42}', + b'\x00\x01\x02\x03\xff\xfe\xfd', + b'PNG_IMAGE_DATA_PLACEHOLDER', + b'%PDF-1.4\nPDF_CONTENT_PLACEHOLDER' + ] + }, schema=pa_schema) + + # Write mixed data + total_rows = 0 + for batch in test_data.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + # Record count is tracked internally by DataBlobWriter + + # Test prepare commit + commit_messages = blob_writer.prepare_commit() + self.assertIsInstance(commit_messages, list) + + # Verify we have committed files + self.assertGreater(len(commit_messages), 0) + + # Verify commit message structure + for commit_msg in commit_messages: + self.assertIsInstance(commit_msg.new_files, list) + self.assertGreater(len(commit_msg.new_files), 0) + + # Verify file metadata structure + for file_meta in commit_msg.new_files: + self.assertIsNotNone(file_meta.file_name) + self.assertGreater(file_meta.file_size, 0) + self.assertGreater(file_meta.row_count, 0) + + # Should have both normal and blob files + file_names = [f.file_name for f in commit_msg.new_files] + parquet_files = [f for f in file_names if f.endswith('.parquet')] + blob_files = [f for f in file_names if f.endswith('.blob')] + + self.assertGreater(len(parquet_files), 0, "Should have at least one parquet file") + self.assertGreater(len(blob_files), 0, "Should have at least one blob file") + + # Create commit and commit the data + commit = write_builder.new_commit() + commit.commit(commit_messages) + blob_writer.close() + + # Read data back using table API + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + result = table_read.to_arrow(table_scan.plan().splits()) + + # Verify the data was read back correctly + self.assertEqual(result.num_rows, 5, "Should have 5 rows") + self.assertEqual(result.num_columns, 3, "Should have 3 columns") + + # Convert result to pandas for easier comparison + result_df = result.to_pandas() + + # Verify each row matches the original data + for i in range(5): + original_id = test_data.column('id')[i].as_py() + original_type = test_data.column('type')[i].as_py() + original_data = test_data.column('data')[i].as_py() + + result_id = result_df.iloc[i]['id'] + result_type = result_df.iloc[i]['type'] + result_data = result_df.iloc[i]['data'] + + self.assertEqual(result_id, original_id, f"Row {i+1}: ID should match") + self.assertEqual(result_type, original_type, f"Row {i+1}: Type should match") + self.assertEqual(result_data, original_data, f"Row {i+1}: Blob data should match") + + def test_data_blob_writer_empty_batches(self): + """Test DataBlobWriter with empty batches.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('data', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.empty_batches_test', schema, False) + table = self.catalog.get_table('test_db.empty_batches_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Test data with some empty batches + batch1_data = pa.Table.from_pydict({ + 'id': [1, 2], + 'data': [b'data1', b'data2'] + }, schema=pa_schema) + + # Empty batch + empty_batch = pa.Table.from_pydict({ + 'id': [], + 'data': [] + }, schema=pa_schema) + + batch2_data = pa.Table.from_pydict({ + 'id': [3], + 'data': [b'data3'] + }, schema=pa_schema) + + # Write batches including empty ones + total_rows = 0 + for batch in batch1_data.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + for batch in empty_batch.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + for batch in batch2_data.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + # Verify record count (empty batch should not affect count) + # Record count is tracked internally by DataBlobWriter + # Record count is tracked internally by DataBlobWriter + + # Test prepare commit + commit_messages = blob_writer.prepare_commit() + self.assertIsInstance(commit_messages, list) + + blob_writer.close() + + def test_data_blob_writer_rolling_behavior(self): + """Test DataBlobWriter rolling behavior with multiple commits.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('content', pa.string()), + ('blob_data', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.rolling_test', schema, False) + table = self.catalog.get_table('test_db.rolling_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Create data that should trigger rolling + large_content = 'X' * 1000 # Large string content + large_blob = b'B' * 5000 # Large blob data + + # Write multiple batches to test rolling + for i in range(10): # 10 batches + batch_data = pa.Table.from_pydict({ + 'id': [i * 10 + j for j in range(10)], + 'content': [f'{large_content}_{i}_{j}' for j in range(10)], + 'blob_data': [large_blob] * 10 + }, schema=pa_schema) + + for batch in batch_data.to_batches(): + blob_writer.write_arrow_batch(batch) + + # Verify total record count + # Record count is tracked internally by DataBlobWriter + + # Test prepare commit + commit_messages = blob_writer.prepare_commit() + self.assertIsInstance(commit_messages, list) + + # Verify we have committed files + self.assertGreater(len(commit_messages), 0) + + # Verify file metadata structure + for commit_msg in commit_messages: + for file_meta in commit_msg.new_files: + self.assertIsNotNone(file_meta.file_name) + self.assertGreater(file_meta.file_size, 0) + self.assertGreater(file_meta.row_count, 0) + + blob_writer.close() + + def test_blob_write_read_end_to_end(self): + """Test complete end-to-end blob functionality: write blob data and read it back to verify correctness.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('name', pa.string()), + ('description', pa.string()), + ('blob_data', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.blob_write_read_e2e', schema, False) + table = self.catalog.get_table('test_db.blob_write_read_e2e') + + # Test data with various blob sizes and types + test_data = pa.Table.from_pydict({ + 'id': [1, 2, 3, 4, 5], + 'name': ['Alice', 'Bob', 'Charlie', 'David', 'Eve'], + 'description': ['User 1', 'User 2', 'User 3', 'User 4', 'User 5'], + 'blob_data': [ + b'small_blob_1', + b'medium_blob_data_2_with_more_content', + b'large_blob_data_3_with_even_more_content_and_details', + b'very_large_blob_data_4_with_extensive_content_and_multiple_details_here', # noqa: E501 + b'extremely_large_blob_data_5_with_comprehensive_content_and_extensive_details_covering_multiple_aspects' # noqa: E501 + ] + }, schema=pa_schema) + + # Write data using table API + write_builder = table.new_batch_write_builder() + writer = write_builder.new_write() + writer.write_arrow(test_data) + + # Commit the data + commit_messages = writer.prepare_commit() + self.assertGreater(len(commit_messages), 0) + + # Verify commit message structure + for commit_msg in commit_messages: + self.assertIsInstance(commit_msg.new_files, list) + self.assertGreater(len(commit_msg.new_files), 0) + + # Should have both normal and blob files + file_names = [f.file_name for f in commit_msg.new_files] + parquet_files = [f for f in file_names if f.endswith('.parquet')] + blob_files = [f for f in file_names if f.endswith('.blob')] + + self.assertGreater(len(parquet_files), 0, "Should have at least one parquet file") + self.assertGreater(len(blob_files), 0, "Should have at least one blob file") + + # Create commit and commit the data + commit = write_builder.new_commit() + commit.commit(commit_messages) + writer.close() + + # Read data back using table API + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + result = table_read.to_arrow(table_scan.plan().splits()) + + # Verify the data was read back correctly + self.assertEqual(result.num_rows, 5, "Should have 5 rows") + self.assertEqual(result.num_columns, 4, "Should have 4 columns") + + # Verify normal columns + self.assertEqual(result.column('id').to_pylist(), [1, 2, 3, 4, 5], "ID column should match") + self.assertEqual(result.column('name').to_pylist(), ['Alice', 'Bob', 'Charlie', 'David', 'Eve'], "Name column should match") # noqa: E501 + self.assertEqual(result.column('description').to_pylist(), ['User 1', 'User 2', 'User 3', 'User 4', 'User 5'], "Description column should match") # noqa: E501 + + # Verify blob data correctness + blob_data = result.column('blob_data').to_pylist() + expected_blobs = [ + b'small_blob_1', + b'medium_blob_data_2_with_more_content', + b'large_blob_data_3_with_even_more_content_and_details', + b'very_large_blob_data_4_with_extensive_content_and_multiple_details_here', # noqa: E501 + b'extremely_large_blob_data_5_with_comprehensive_content_and_extensive_details_covering_multiple_aspects' # noqa: E501 + ] + + self.assertEqual(len(blob_data), 5, "Should have 5 blob records") + self.assertEqual(blob_data, expected_blobs, "Blob data should match exactly") + + # Verify individual blob sizes + for i, (actual_blob, expected_blob) in enumerate(zip(blob_data, expected_blobs)): + self.assertEqual(len(actual_blob), len(expected_blob), f"Blob {i+1} size should match") + self.assertEqual(actual_blob, expected_blob, f"Blob {i+1} content should match exactly") + + print(f"✅ End-to-end blob write/read test passed: wrote and read back {len(blob_data)} blob records correctly") # noqa: E501 + + def test_blob_write_read_end_to_end_with_descriptor(self): + """Test end-to-end blob functionality using blob descriptors.""" + import random + from pypaimon import Schema + from pypaimon.table.row.blob import BlobDescriptor, Blob + from pypaimon.common.uri_reader import UriReaderFactory + from pypaimon.common.config import CatalogOptions + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('name', pa.string()), + ('picture', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true', + 'blob-as-descriptor': 'true' + } + ) + + # Create table + self.catalog.create_table('test_db.blob_descriptor_test', schema, False) + table: FileStoreTable = self.catalog.get_table('test_db.blob_descriptor_test') + + # Create test blob data (1MB) + blob_data = bytearray(1024 * 1024) + random.seed(42) # For reproducible tests + for i in range(len(blob_data)): + blob_data[i] = random.randint(0, 255) + blob_data = bytes(blob_data) + + # Create external blob file + external_blob_path = os.path.join(self.temp_dir, 'external_blob') + with open(external_blob_path, 'wb') as f: + f.write(blob_data) + + # Create blob descriptor pointing to external file + blob_descriptor = BlobDescriptor(external_blob_path, 0, len(blob_data)) + + # Create test data with blob descriptor + test_data = pa.Table.from_pydict({ + 'id': [1], + 'name': ['paimon'], + 'picture': [blob_descriptor.serialize()] + }, schema=pa_schema) + + # Write data using table API + write_builder = table.new_batch_write_builder() + writer = write_builder.new_write() + writer.write_arrow(test_data) + + # Commit the data + commit_messages = writer.prepare_commit() + commit = write_builder.new_commit() + commit.commit(commit_messages) + + # Read data back + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + result = table_read.to_arrow(table_scan.plan().splits()) + + # Verify the data was written and read correctly + self.assertEqual(result.num_rows, 1, "Should have 1 row") + self.assertEqual(result.column('id').to_pylist(), [1], "ID should match") + self.assertEqual(result.column('name').to_pylist(), ['paimon'], "Name should match") + + # Get the blob descriptor bytes from the result + picture_bytes = result.column('picture').to_pylist()[0] + self.assertIsInstance(picture_bytes, bytes, "Picture should be bytes") + + # Deserialize the blob descriptor + new_blob_descriptor = BlobDescriptor.deserialize(picture_bytes) + + # The URI might be different if the blob was stored in the table's data directory + # Let's verify the descriptor properties and try to read the data + # Note: offset might be non-zero due to blob file format overhead + self.assertGreaterEqual(new_blob_descriptor.offset, 0, "Offset should be non-negative") + self.assertEqual(new_blob_descriptor.length, len(blob_data), "Length should match") + + # Create URI reader factory and read the blob data + catalog_options = {CatalogOptions.WAREHOUSE: self.warehouse} + uri_reader_factory = UriReaderFactory(catalog_options) + uri_reader = uri_reader_factory.create(new_blob_descriptor.uri) + blob = Blob.from_descriptor(uri_reader, new_blob_descriptor) + + # Verify the blob data matches the original + self.assertEqual(blob.to_data(), blob_data, "Blob data should match original") + + print("✅ Blob descriptor end-to-end test passed:") + print(" - Created external blob file and descriptor") + print(" - Wrote and read blob descriptor successfully") + print(" - Verified blob data can be read from descriptor") + print(" - Tested blob-as-descriptor=true mode") + + def test_blob_write_read_large_data_end_to_end(self): + """Test end-to-end blob functionality with large blob data (1MB per blob).""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('metadata', pa.string()), + ('large_blob', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.blob_large_write_read_e2e', schema, False) + table = self.catalog.get_table('test_db.blob_large_write_read_e2e') + + # Create large blob data (1MB per blob) + large_blob_size = 1024 * 1024 # 1MB + blob_pattern = b'LARGE_BLOB_PATTERN_' + b'X' * 1024 # ~1KB pattern + pattern_size = len(blob_pattern) + repetitions = large_blob_size // pattern_size + large_blob_data = blob_pattern * repetitions + + # Test data with large blobs + test_data = pa.Table.from_pydict({ + 'id': [1, 2, 3], + 'metadata': ['Large blob 1', 'Large blob 2', 'Large blob 3'], + 'large_blob': [large_blob_data, large_blob_data, large_blob_data] + }, schema=pa_schema) + + # Write data + write_builder = table.new_batch_write_builder() + writer = write_builder.new_write() + writer.write_arrow(test_data) + + # Commit the data + commit_messages = writer.prepare_commit() + self.assertGreater(len(commit_messages), 0) + + # Verify commit message structure + for commit_msg in commit_messages: + self.assertIsInstance(commit_msg.new_files, list) + self.assertGreater(len(commit_msg.new_files), 0) + + # Should have both normal and blob files + file_names = [f.file_name for f in commit_msg.new_files] + parquet_files = [f for f in file_names if f.endswith('.parquet')] + blob_files = [f for f in file_names if f.endswith('.blob')] + + self.assertGreater(len(parquet_files), 0, "Should have at least one parquet file") + self.assertGreater(len(blob_files), 0, "Should have at least one blob file") + + commit = write_builder.new_commit() + commit.commit(commit_messages) + writer.close() + + # Read data back + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + result = table_read.to_arrow(table_scan.plan().splits()) + + # Verify the data + self.assertEqual(result.num_rows, 3, "Should have 3 rows") + self.assertEqual(result.num_columns, 3, "Should have 3 columns") + + # Verify normal columns + self.assertEqual(result.column('id').to_pylist(), [1, 2, 3], "ID column should match") + self.assertEqual(result.column('metadata').to_pylist(), ['Large blob 1', 'Large blob 2', 'Large blob 3'], "Metadata column should match") # noqa: E501 + + # Verify blob data integrity + blob_data = result.column('large_blob').to_pylist() + self.assertEqual(len(blob_data), 3, "Should have 3 blob records") + + for i, blob in enumerate(blob_data): + self.assertEqual(len(blob), len(large_blob_data), f"Blob {i+1} should be {large_blob_size} bytes") + self.assertEqual(blob, large_blob_data, f"Blob {i+1} content should match exactly") + print(f"✅ Verified large blob {i+1}: {len(blob)} bytes") + + print(f"✅ Large blob end-to-end test passed: wrote and read back {len(blob_data)} large blob records correctly") # noqa: E501 + + def test_blob_write_read_mixed_sizes_end_to_end(self): + """Test end-to-end blob functionality with mixed blob sizes.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('size_category', pa.string()), + ('blob_data', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.blob_mixed_sizes_write_read_e2e', schema, False) + table = self.catalog.get_table('test_db.blob_mixed_sizes_write_read_e2e') + + # Create blobs of different sizes + tiny_blob = b'tiny' + small_blob = b'small_blob_data' * 10 # ~140 bytes + medium_blob = b'medium_blob_data' * 100 # ~1.4KB + large_blob = b'large_blob_data' * 1000 # ~14KB + huge_blob = b'huge_blob_data' * 10000 # ~140KB + + test_data = pa.Table.from_pydict({ + 'id': [1, 2, 3, 4, 5], + 'size_category': ['tiny', 'small', 'medium', 'large', 'huge'], + 'blob_data': [tiny_blob, small_blob, medium_blob, large_blob, huge_blob] + }, schema=pa_schema) + + # Write data + write_builder = table.new_batch_write_builder() + writer = write_builder.new_write() + writer.write_arrow(test_data) + + # Commit + commit_messages = writer.prepare_commit() + self.assertGreater(len(commit_messages), 0) + + # Verify commit message structure + for commit_msg in commit_messages: + self.assertIsInstance(commit_msg.new_files, list) + self.assertGreater(len(commit_msg.new_files), 0) + + # Should have both normal and blob files + file_names = [f.file_name for f in commit_msg.new_files] + parquet_files = [f for f in file_names if f.endswith('.parquet')] + blob_files = [f for f in file_names if f.endswith('.blob')] + + self.assertGreater(len(parquet_files), 0, "Should have at least one parquet file") + self.assertGreater(len(blob_files), 0, "Should have at least one blob file") + + commit = write_builder.new_commit() + commit.commit(commit_messages) + writer.close() + + # Read back + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + result = table_read.to_arrow(table_scan.plan().splits()) + + # Verify + self.assertEqual(result.num_rows, 5, "Should have 5 rows") + self.assertEqual(result.num_columns, 3, "Should have 3 columns") + + # Verify normal columns + self.assertEqual(result.column('id').to_pylist(), [1, 2, 3, 4, 5], "ID column should match") + self.assertEqual(result.column('size_category').to_pylist(), ['tiny', 'small', 'medium', 'large', 'huge'], "Size category column should match") # noqa: E501 + + # Verify blob data + blob_data = result.column('blob_data').to_pylist() + expected_blobs = [tiny_blob, small_blob, medium_blob, large_blob, huge_blob] + + self.assertEqual(len(blob_data), 5, "Should have 5 blob records") + self.assertEqual(blob_data, expected_blobs, "Blob data should match exactly") + + # Verify sizes + sizes = [len(blob) for blob in blob_data] + expected_sizes = [len(blob) for blob in expected_blobs] + self.assertEqual(sizes, expected_sizes, "Blob sizes should match") + + # Verify individual blob content + for i, (actual_blob, expected_blob) in enumerate(zip(blob_data, expected_blobs)): + self.assertEqual(actual_blob, expected_blob, f"Blob {i+1} content should match exactly") + + print(f"✅ Mixed sizes end-to-end test passed: wrote and read back blobs ranging from {min(sizes)} to {max(sizes)} bytes") # noqa: E501 + + def test_blob_write_read_large_data_end_to_end_with_rolling(self): + """Test end-to-end blob functionality with large blob data (50MB per blob) and rolling behavior (40 blobs).""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('batch_id', pa.int32()), + ('metadata', pa.string()), + ('large_blob', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.blob_large_rolling_e2e', schema, False) + table = self.catalog.get_table('test_db.blob_large_rolling_e2e') + + # Create large blob data (50MB per blob) + large_blob_size = 50 * 1024 * 1024 # 50MB + blob_pattern = b'LARGE_BLOB_PATTERN_' + b'X' * 1024 # ~1KB pattern + pattern_size = len(blob_pattern) + repetitions = large_blob_size // pattern_size + large_blob_data = blob_pattern * repetitions + + # Verify the blob size is exactly 50MB + actual_size = len(large_blob_data) + print(f"Created blob data: {actual_size:,} bytes ({actual_size / (1024*1024):.2f} MB)") + + # Write 40 batches of data (each with 1 blob of 50MB) + write_builder = table.new_batch_write_builder() + writer = write_builder.new_write() + + # Write all 40 batches first + for batch_id in range(40): + # Create test data for this batch + test_data = pa.Table.from_pydict({ + 'id': [batch_id + 1], + 'batch_id': [batch_id], + 'metadata': [f'Large blob batch {batch_id + 1}'], + 'large_blob': [large_blob_data] + }, schema=pa_schema) + + # Write data + writer.write_arrow(test_data) + + # Print progress every 10 batches + if (batch_id + 1) % 10 == 0: + print(f"✅ Written batch {batch_id + 1}/40: {len(large_blob_data):,} bytes") + + print("✅ Successfully wrote all 40 batches of 50MB blobs") + + # Commit all data at once + commit_messages = writer.prepare_commit() + self.assertGreater(len(commit_messages), 0) + + # Verify commit message structure + for commit_msg in commit_messages: + self.assertIsInstance(commit_msg.new_files, list) + self.assertGreater(len(commit_msg.new_files), 0) + + # Should have both normal and blob files + file_names = [f.file_name for f in commit_msg.new_files] + parquet_files = [f for f in file_names if f.endswith('.parquet')] + blob_files = [f for f in file_names if f.endswith('.blob')] + + self.assertGreater(len(parquet_files), 0, "Should have at least one parquet file") + self.assertGreater(len(blob_files), 0, "Should have at least one blob file") + + # Commit the data + commit = write_builder.new_commit() + commit.commit(commit_messages) + writer.close() + + print(f"✅ Successfully committed {len(commit_messages)} commit messages with 40 batches of 50MB blobs") + + # Read data back + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + result = table_read.to_arrow(table_scan.plan().splits()) + + # Verify the data + self.assertEqual(result.num_rows, 40, "Should have 40 rows") + self.assertEqual(result.num_columns, 4, "Should have 4 columns") + + # Verify normal columns + expected_ids = list(range(1, 41)) + expected_batch_ids = list(range(40)) + expected_metadata = [f'Large blob batch {i}' for i in range(1, 41)] + + self.assertEqual(result.column('id').to_pylist(), expected_ids, "ID column should match") + self.assertEqual(result.column('batch_id').to_pylist(), expected_batch_ids, "Batch ID column should match") # noqa: E501 + self.assertEqual(result.column('metadata').to_pylist(), expected_metadata, "Metadata column should match") # noqa: E501 + + # Verify blob data integrity + blob_data = result.column('large_blob').to_pylist() + self.assertEqual(len(blob_data), 40, "Should have 40 blob records") + + # Verify each blob + for i, blob in enumerate(blob_data): + self.assertEqual(len(blob), len(large_blob_data), f"Blob {i+1} should be {large_blob_size:,} bytes") + self.assertEqual(blob, large_blob_data, f"Blob {i+1} content should match exactly") + + # Print progress every 10 blobs + if (i + 1) % 10 == 0: + print(f"✅ Verified blob {i+1}/40: {len(blob):,} bytes") + + # Verify total data size + total_blob_size = sum(len(blob) for blob in blob_data) + expected_total_size = 40 * len(large_blob_data) + self.assertEqual(total_blob_size, expected_total_size, + f"Total blob size should be {expected_total_size:,} bytes") + + print("✅ Large blob rolling end-to-end test passed:") + print(" - Wrote and read back 40 blobs of 50MB each") + print(f" - Total data size: {total_blob_size:,} bytes ({total_blob_size / (1024*1024*1024):.2f} GB)") # noqa: E501 + print(" - All blob content verified as correct") + + def test_data_blob_writer_with_shard(self): + """Test DataBlobWriter with mixed data types in blob column.""" + from pypaimon import Schema + + # Create schema with blob column + pa_schema = pa.schema([ + ('id', pa.int32()), + ('type', pa.string()), + ('data', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.with_shard_test', schema, False) + table = self.catalog.get_table('test_db.with_shard_test') + + # Use proper table API to create writer + write_builder = table.new_batch_write_builder() + blob_writer = write_builder.new_write() + + # Test data with different types of blob content + test_data = pa.Table.from_pydict({ + 'id': [1, 2, 3, 4, 5], + 'type': ['text', 'json', 'binary', 'image', 'pdf'], + 'data': [ + b'This is text content', + b'{"key": "value", "number": 42}', + b'\x00\x01\x02\x03\xff\xfe\xfd', + b'PNG_IMAGE_DATA_PLACEHOLDER', + b'%PDF-1.4\nPDF_CONTENT_PLACEHOLDER' + ] + }, schema=pa_schema) + + # Write mixed data + total_rows = 0 + for batch in test_data.to_batches(): + blob_writer.write_arrow_batch(batch) + total_rows += batch.num_rows + + # Test prepare commit + commit_messages = blob_writer.prepare_commit() + # Create commit and commit the data + commit = write_builder.new_commit() + commit.commit(commit_messages) + blob_writer.close() + + # Read data back using table API + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan().with_shard(1, 2) + table_read = read_builder.new_read() + splits = table_scan.plan().splits() + result = table_read.to_arrow(splits) + + # Verify the data was read back correctly + self.assertEqual(result.num_rows, 3, "Should have 5 rows") + self.assertEqual(result.num_columns, 3, "Should have 3 columns") + + def test_blob_read_row_by_row_iterator(self): + """Test reading blob data row by row using to_iterator().""" + from pypaimon import Schema + from pypaimon.table.row.blob import Blob + from pypaimon.table.row.internal_row import RowKind + + pa_schema = pa.schema([ + ('id', pa.int32()), + ('name', pa.string()), + ('blob_data', pa.large_binary()), + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + options={ + 'row-tracking.enabled': 'true', + 'data-evolution.enabled': 'true' + } + ) + self.catalog.create_table('test_db.blob_iterator_test', schema, False) + table = self.catalog.get_table('test_db.blob_iterator_test') + + expected_data = { + 1: {'name': 'Alice', 'blob': b'blob_1'}, + 2: {'name': 'Bob', 'blob': b'blob_2_data'}, + 3: {'name': 'Charlie', 'blob': b'blob_3_content'}, + 4: {'name': 'David', 'blob': b'blob_4_large_content'}, + 5: {'name': 'Eve', 'blob': b'blob_5_very_large_content_data'} + } + + test_data = pa.Table.from_pydict({ + 'id': list(expected_data.keys()), + 'name': [expected_data[i]['name'] for i in expected_data.keys()], + 'blob_data': [expected_data[i]['blob'] for i in expected_data.keys()] + }, schema=pa_schema) + + write_builder = table.new_batch_write_builder() + writer = write_builder.new_write() + writer.write_arrow(test_data) + commit_messages = writer.prepare_commit() + commit = write_builder.new_commit() + commit.commit(commit_messages) + writer.close() + + # Verify blob files were created + file_names = [f.file_name for f in commit_messages[0].new_files] + self.assertGreater( + len([f for f in file_names if f.endswith('.blob')]), 0, + "Should have at least one blob file") + + # Read using to_iterator + iterator = table.new_read_builder().new_read().to_iterator( + table.new_read_builder().new_scan().plan().splits()) + + rows = [] + value = next(iterator, None) + while value is not None: + rows.append(value) + value = next(iterator, None) + + self.assertEqual(len(rows), 5, "Should have 5 rows") + + for row in rows: + row_id = row.get_field(0) + self.assertIn(row_id, expected_data, f"ID {row_id} should be in expected data") + + expected = expected_data[row_id] + self.assertEqual(row.get_field(1), expected['name'], f"Row {row_id}: name should match") + + row_blob = row.get_field(2) + blob_bytes = row_blob.to_data() if isinstance(row_blob, Blob) else row_blob + self.assertIsInstance(blob_bytes, bytes, f"Row {row_id}: blob should be bytes") + self.assertEqual(blob_bytes, expected['blob'], f"Row {row_id}: blob data should match") + self.assertEqual(len(blob_bytes), len(expected['blob']), f"Row {row_id}: blob size should match") + + self.assertIn( + row.get_row_kind(), + [RowKind.INSERT, RowKind.UPDATE_BEFORE, RowKind.UPDATE_AFTER, RowKind.DELETE], + f"Row {row_id}: RowKind should be valid") + + +if __name__ == '__main__': + unittest.main() diff --git a/paimon-python/pypaimon/tests/blob_test.py b/paimon-python/pypaimon/tests/blob_test.py new file mode 100644 index 000000000000..b66c1dec9a9c --- /dev/null +++ b/paimon-python/pypaimon/tests/blob_test.py @@ -0,0 +1,1066 @@ +""" +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 pathlib import Path + +import pyarrow as pa + +from pypaimon import CatalogFactory +from pypaimon.common.file_io import FileIO +from pypaimon.read.reader.format_blob_reader import FormatBlobReader +from pypaimon.schema.data_types import AtomicType, DataField +from pypaimon.table.row.blob import Blob, BlobData, BlobRef, BlobDescriptor +from pypaimon.table.row.generic_row import GenericRowDeserializer, GenericRowSerializer, GenericRow +from pypaimon.table.row.row_kind import RowKind + + +class MockFileIO: + """Mock FileIO for testing.""" + + def __init__(self, file_io: FileIO): + self._file_io = file_io + + def get_file_size(self, path: str) -> int: + """Get file size.""" + return self._file_io.get_file_size(Path(path)) + + def new_input_stream(self, path: Path): + """Create new input stream for reading.""" + return self._file_io.new_input_stream(path) + + +class BlobTest(unittest.TestCase): + """Tests for Blob interface following org.apache.paimon.data.BlobTest.""" + + def setUp(self): + """Set up test environment with temporary file.""" + # Create a temporary directory and file + self.temp_dir = tempfile.mkdtemp() + self.file = os.path.join(self.temp_dir, "test.txt") + + # Write test data to the file + with open(self.file, 'wb') as f: + f.write(b"test data") + + def tearDown(self): + """Clean up temporary files.""" + try: + if os.path.exists(self.file): + os.remove(self.file) + os.rmdir(self.temp_dir) + except OSError: + pass # Ignore cleanup errors + + def test_from_data(self): + """Test Blob.from_data() method.""" + test_data = b"test data" + blob = Blob.from_data(test_data) + + # Verify it returns a BlobData instance + self.assertIsInstance(blob, BlobData) + + # Verify the data matches + self.assertEqual(blob.to_data(), test_data) + + def test_from_local(self): + """Test Blob.from_local() method.""" + blob = Blob.from_local(self.file) + + # Verify it returns a BlobRef instance + self.assertIsInstance(blob, BlobRef) + + # Verify the data matches + self.assertEqual(blob.to_data(), b"test data") + + def test_from_file_with_offset_and_length(self): + """Test Blob.from_file() method with offset and length.""" + file_io = FileIO(self.file if self.file.startswith('file://') else f"file://{self.file}", {}) + blob = Blob.from_file(file_io, self.file, 0, 4) + + # Verify it returns a BlobRef instance + self.assertIsInstance(blob, BlobRef) + + # Verify the data matches (first 4 bytes: "test") + self.assertEqual(blob.to_data(), b"test") + + def test_from_http(self): + """Test Blob.from_http() method.""" + uri = "http://example.com/file.txt" + blob = Blob.from_http(uri) + + # Verify it returns a BlobRef instance + self.assertIsInstance(blob, BlobRef) + + # Verify the descriptor has the correct URI + descriptor = blob.to_descriptor() + self.assertEqual(descriptor.uri, uri) + self.assertEqual(descriptor.offset, 0) + self.assertEqual(descriptor.length, -1) + + def test_blob_data_interface_compliance(self): + """Test that BlobData properly implements Blob interface.""" + test_data = b"interface test data" + blob_data = BlobData(test_data) + + # Test that it's a Blob + self.assertIsInstance(blob_data, Blob) + + # Test interface methods + self.assertEqual(blob_data.to_data(), test_data) + + # Test to_descriptor raises RuntimeError + with self.assertRaises(RuntimeError) as context: + blob_data.to_descriptor() + self.assertIn("Blob data can not convert to descriptor", str(context.exception)) + + # Test new_input_stream + stream = blob_data.new_input_stream() + self.assertEqual(stream.read(), test_data) + stream.close() + + def test_blob_ref_interface_compliance(self): + """Test that BlobRef properly implements Blob interface.""" + blob_ref = Blob.from_local(self.file) + + # Test that it's a Blob + self.assertIsInstance(blob_ref, Blob) + + # Test interface methods + self.assertEqual(blob_ref.to_data(), b"test data") + + # Test to_descriptor returns valid descriptor + descriptor = blob_ref.to_descriptor() + self.assertEqual(descriptor.uri, self.file) + self.assertEqual(descriptor.offset, 0) + self.assertEqual(descriptor.length, -1) + + # Test new_input_stream + stream = blob_ref.new_input_stream() + self.assertEqual(stream.read(), b"test data") + stream.close() + + def test_blob_equality_and_hashing(self): + """Test blob equality and hashing behavior.""" + # Test BlobData equality + data1 = BlobData(b"same data") + data2 = BlobData(b"same data") + data3 = BlobData(b"different data") + + self.assertEqual(data1, data2) + self.assertNotEqual(data1, data3) + self.assertEqual(hash(data1), hash(data2)) + + # Test BlobRef equality + ref1 = Blob.from_local(self.file) + ref2 = Blob.from_local(self.file) + + self.assertEqual(ref1, ref2) + self.assertEqual(hash(ref1), hash(ref2)) + + def test_blob_factory_methods_return_correct_types(self): + """Test that all factory methods return the expected types.""" + # from_data should return BlobData + blob_data = Blob.from_data(b"test") + self.assertIsInstance(blob_data, BlobData) + self.assertIsInstance(blob_data, Blob) + + # from_local should return BlobRef + blob_ref = Blob.from_local(self.file) + self.assertIsInstance(blob_ref, BlobRef) + self.assertIsInstance(blob_ref, Blob) + + # from_file should return BlobRef + file_io = FileIO(self.file if self.file.startswith('file://') else f"file://{self.file}", {}) + blob_file = Blob.from_file(file_io, self.file, 0, os.path.getsize(self.file)) + self.assertIsInstance(blob_file, BlobRef) + self.assertIsInstance(blob_file, Blob) + + # from_http should return BlobRef + blob_http = Blob.from_http("http://example.com/test.bin") + self.assertIsInstance(blob_http, BlobRef) + self.assertIsInstance(blob_http, Blob) + + def test_blob_data_convenience_methods(self): + # Test from_bytes class method + blob2 = BlobData.from_bytes(b"from bytes") + self.assertEqual(blob2.to_data(), b"from bytes") + + def test_generic_row_deserializer_parse_blob(self): + """Test GenericRowDeserializer._parse_blob method.""" + # Create test data with BLOB field + test_blob_data = b"Test BLOB data for parsing" + blob_data = BlobData(test_blob_data) + + # Create fields with BLOB type + fields = [ + DataField(0, "id", AtomicType("INT")), + DataField(1, "blob_field", AtomicType("BLOB")), + ] + + # Create and serialize a row with blob data + original_row = GenericRow([42, blob_data], fields, RowKind.INSERT) + serialized_bytes = GenericRowSerializer.to_bytes(original_row) + + # Test the full deserialization process (which uses _parse_blob internally) + deserialized_row = GenericRowDeserializer.from_bytes(serialized_bytes, fields) + + # Verify the deserialized blob + deserialized_blob = deserialized_row.values[1] + self.assertIsInstance(deserialized_blob, BlobData) + self.assertEqual(deserialized_blob.to_data(), test_blob_data) + + # Test with empty blob data + empty_blob = BlobData(b"") + empty_row = GenericRow([1, empty_blob], fields, RowKind.INSERT) + empty_serialized = GenericRowSerializer.to_bytes(empty_row) + empty_deserialized = GenericRowDeserializer.from_bytes(empty_serialized, fields) + + empty_deserialized_blob = empty_deserialized.values[1] + self.assertIsInstance(empty_deserialized_blob, BlobData) + self.assertEqual(empty_deserialized_blob.to_data(), b"") + + # Test with binary data containing null bytes + binary_blob_data = b"\x00\x01\x02\x03\xff\xfe\xfd" + binary_blob = BlobData(binary_blob_data) + binary_row = GenericRow([99, binary_blob], fields, RowKind.INSERT) + binary_serialized = GenericRowSerializer.to_bytes(binary_row) + binary_deserialized = GenericRowDeserializer.from_bytes(binary_serialized, fields) + + binary_deserialized_blob = binary_deserialized.values[1] + self.assertIsInstance(binary_deserialized_blob, BlobData) + self.assertEqual(binary_deserialized_blob.to_data(), binary_blob_data) + + def test_generic_row_deserializer_parse_blob_with_multiple_fields(self): + """Test _parse_blob with multiple BLOB fields in a row.""" + # Create test data with multiple BLOB fields + blob1_data = b"First BLOB data" + blob2_data = b"Second BLOB with different content" + blob3_data = b"" # Empty blob + + blob1 = BlobData(blob1_data) + blob2 = BlobData(blob2_data) + blob3 = BlobData(blob3_data) + + # Create fields with multiple BLOB types + fields = [ + DataField(0, "id", AtomicType("INT")), + DataField(1, "name", AtomicType("STRING")), + DataField(2, "blob1", AtomicType("BLOB")), + DataField(3, "blob2", AtomicType("BLOB")), + DataField(4, "blob3", AtomicType("BLOB")), + ] + + # Create and serialize a row with multiple blobs + original_row = GenericRow([123, "test_row", blob1, blob2, blob3], fields, RowKind.INSERT) + serialized_bytes = GenericRowSerializer.to_bytes(original_row) + + # Deserialize and verify all blobs + deserialized_row = GenericRowDeserializer.from_bytes(serialized_bytes, fields) + + # Verify each blob field + self.assertEqual(deserialized_row.values[0], 123) + self.assertEqual(deserialized_row.values[1], "test_row") + + deserialized_blob1 = deserialized_row.values[2] + self.assertIsInstance(deserialized_blob1, BlobData) + self.assertEqual(deserialized_blob1.to_data(), blob1_data) + + deserialized_blob2 = deserialized_row.values[3] + self.assertIsInstance(deserialized_blob2, BlobData) + self.assertEqual(deserialized_blob2.to_data(), blob2_data) + + deserialized_blob3 = deserialized_row.values[4] + self.assertIsInstance(deserialized_blob3, BlobData) + self.assertEqual(deserialized_blob3.to_data(), blob3_data) + + def test_generic_row_deserializer_parse_blob_with_null_values(self): + """Test _parse_blob with null BLOB values.""" + # Create fields with BLOB type + fields = [ + DataField(0, "id", AtomicType("INT")), + DataField(1, "blob_field", AtomicType("BLOB")), + DataField(2, "name", AtomicType("STRING")), + ] + + # Create row with null blob (None value) + original_row = GenericRow([456, None, "test_with_null"], fields, RowKind.INSERT) + serialized_bytes = GenericRowSerializer.to_bytes(original_row) + + # Deserialize and verify null blob is handled correctly + deserialized_row = GenericRowDeserializer.from_bytes(serialized_bytes, fields) + + self.assertEqual(deserialized_row.values[0], 456) + self.assertIsNone(deserialized_row.values[1]) # Null blob should remain None + self.assertEqual(deserialized_row.values[2], "test_with_null") + + def test_generic_row_deserializer_parse_blob_large_data(self): + """Test _parse_blob with large BLOB data.""" + # Create large blob data (1MB) + large_blob_data = b"X" * (1024 * 1024) # 1MB of 'X' characters + large_blob = BlobData(large_blob_data) + + fields = [ + DataField(0, "id", AtomicType("INT")), + DataField(1, "large_blob", AtomicType("BLOB")), + ] + + # Create and serialize row with large blob + original_row = GenericRow([789, large_blob], fields, RowKind.INSERT) + serialized_bytes = GenericRowSerializer.to_bytes(original_row) + + # Deserialize and verify large blob + deserialized_row = GenericRowDeserializer.from_bytes(serialized_bytes, fields) + + deserialized_large_blob = deserialized_row.values[1] + self.assertIsInstance(deserialized_large_blob, BlobData) + self.assertEqual(len(deserialized_large_blob.to_data()), 1024 * 1024) + self.assertEqual(deserialized_large_blob.to_data(), large_blob_data) + + def test_blob_descriptor_creation(self): + """Test BlobDescriptor creation and properties.""" + # Test basic creation + descriptor = BlobDescriptor("test://example.uri", 100, 200) + + self.assertEqual(descriptor.uri, "test://example.uri") + self.assertEqual(descriptor.offset, 100) + self.assertEqual(descriptor.length, 200) + self.assertEqual(descriptor.version, BlobDescriptor.CURRENT_VERSION) + + def test_blob_descriptor_creation_with_version(self): + """Test BlobDescriptor creation with explicit version.""" + descriptor = BlobDescriptor("test://example.uri", 50, 150, version=2) + + self.assertEqual(descriptor.uri, "test://example.uri") + self.assertEqual(descriptor.offset, 50) + self.assertEqual(descriptor.length, 150) + self.assertEqual(descriptor.version, 2) + + def test_blob_descriptor_serialization_deserialization(self): + """Test BlobDescriptor serialization and deserialization.""" + # Test with various URIs and parameters + test_cases = [ + ("file:///path/to/file.bin", 0, -1), + ("https://example.com/data.blob", 1024, 2048), + ("s3://bucket/key", 0, 1000000), + ("test://simple", 42, 84), + ] + + for uri, offset, length in test_cases: + with self.subTest(uri=uri, offset=offset, length=length): + # Create original descriptor + original = BlobDescriptor(uri, offset, length) + + # Serialize + serialized = original.serialize() + self.assertIsInstance(serialized, bytes) + self.assertGreater(len(serialized), 0) + + # Deserialize + deserialized = BlobDescriptor.deserialize(serialized) + + # Verify equality + self.assertEqual(deserialized, original) + self.assertEqual(deserialized.uri, uri) + self.assertEqual(deserialized.offset, offset) + self.assertEqual(deserialized.length, length) + self.assertEqual(deserialized.version, BlobDescriptor.CURRENT_VERSION) + + def test_blob_descriptor_serialization_with_unicode(self): + """Test BlobDescriptor serialization with Unicode characters.""" + # Test with Unicode characters in URI + unicode_uri = "file:///测试/文件.bin" + descriptor = BlobDescriptor(unicode_uri, 0, 100) + + # Serialize and deserialize + serialized = descriptor.serialize() + deserialized = BlobDescriptor.deserialize(serialized) + + # Verify Unicode is preserved + self.assertEqual(deserialized.uri, unicode_uri) + self.assertEqual(deserialized, descriptor) + + def test_blob_descriptor_deserialization_invalid_data(self): + """Test BlobDescriptor deserialization with invalid data.""" + # Test with too short data + with self.assertRaises(ValueError) as context: + BlobDescriptor.deserialize(b"sho") # Only 3 bytes, need at least 5 + self.assertIn("too short", str(context.exception)) + + # Test with invalid version (version 0) + # Create valid data but with wrong version + valid_descriptor = BlobDescriptor("test://uri", 0, 100) + valid_data = bytearray(valid_descriptor.serialize()) + valid_data[0] = 0 # Set invalid version (0) + + with self.assertRaises(ValueError) as context: + BlobDescriptor.deserialize(bytes(valid_data)) + self.assertIn("Unsupported BlobDescriptor version", str(context.exception)) + + # Test with incomplete data (missing URI bytes) + incomplete_data = b'\x01\x00\x00\x00\x10' # Version 1, URI length 16, but no URI bytes + with self.assertRaises(ValueError) as context: + BlobDescriptor.deserialize(incomplete_data) + self.assertIn("URI length exceeds data size", str(context.exception)) + + def test_blob_descriptor_equality_and_hashing(self): + """Test BlobDescriptor equality and hashing.""" + # Create identical descriptors + desc1 = BlobDescriptor("test://uri", 100, 200) + desc2 = BlobDescriptor("test://uri", 100, 200) + desc3 = BlobDescriptor("test://uri", 100, 201) # Different length + desc4 = BlobDescriptor("test://other", 100, 200) # Different URI + + # Test equality + self.assertEqual(desc1, desc2) + self.assertNotEqual(desc1, desc3) + self.assertNotEqual(desc1, desc4) + self.assertNotEqual(desc1, None) + self.assertNotEqual(desc1, "not a descriptor") + + # Test hashing + self.assertEqual(hash(desc1), hash(desc2)) + # Hash should be different for different descriptors (though not guaranteed) + self.assertNotEqual(hash(desc1), hash(desc3)) + self.assertNotEqual(hash(desc1), hash(desc4)) + + def test_blob_descriptor_string_representation(self): + """Test BlobDescriptor string representation.""" + descriptor = BlobDescriptor("test://example.uri", 42, 84) + + str_repr = str(descriptor) + self.assertIn("test://example.uri", str_repr) + self.assertIn("42", str_repr) + self.assertIn("84", str_repr) + self.assertIn("BlobDescriptor", str_repr) + + # __repr__ should be the same as __str__ + self.assertEqual(str_repr, repr(descriptor)) + + def test_blob_descriptor_version_handling(self): + """Test BlobDescriptor version handling.""" + # Test current version + descriptor = BlobDescriptor("test://uri", 0, 100) + self.assertEqual(descriptor.version, BlobDescriptor.CURRENT_VERSION) + + # Test explicit version + descriptor_v2 = BlobDescriptor("test://uri", 0, 100, version=2) + self.assertEqual(descriptor_v2.version, 2) + + # Serialize and deserialize should preserve version + serialized = descriptor_v2.serialize() + deserialized = BlobDescriptor.deserialize(serialized) + self.assertEqual(deserialized.version, 2) + + def test_blob_descriptor_edge_cases(self): + """Test BlobDescriptor with edge cases.""" + # Test with empty URI + empty_uri_desc = BlobDescriptor("", 0, 0) + serialized = empty_uri_desc.serialize() + deserialized = BlobDescriptor.deserialize(serialized) + self.assertEqual(deserialized.uri, "") + + # Test with very long URI + long_uri = "file://" + "a" * 1000 + "/file.bin" + long_uri_desc = BlobDescriptor(long_uri, 0, 1000000) + serialized = long_uri_desc.serialize() + deserialized = BlobDescriptor.deserialize(serialized) + self.assertEqual(deserialized.uri, long_uri) + + # Test with negative values + negative_desc = BlobDescriptor("test://uri", -1, -1) + serialized = negative_desc.serialize() + deserialized = BlobDescriptor.deserialize(serialized) + self.assertEqual(deserialized.offset, -1) + self.assertEqual(deserialized.length, -1) + + def test_blob_descriptor_with_blob_ref(self): + """Test BlobDescriptor integration with BlobRef.""" + # Create a descriptor + descriptor = BlobDescriptor(self.file, 0, -1) + + # Create BlobRef from descriptor + blob_ref = Blob.from_local(self.file) + + # Verify descriptor is preserved + returned_descriptor = blob_ref.to_descriptor() + self.assertEqual(returned_descriptor, descriptor) + + # Verify data can be read through BlobRef + data = blob_ref.to_data() + self.assertEqual(data, b"test data") + + def test_blob_descriptor_serialization_format(self): + """Test BlobDescriptor serialization format details.""" + descriptor = BlobDescriptor("test", 12345, 67890) + serialized = descriptor.serialize() + + # Check that serialized data starts with version byte + self.assertEqual(serialized[0], BlobDescriptor.CURRENT_VERSION) + + # Check minimum length (version + uri_length + uri + offset + length) + # 1 + 4 + len("test") + 8 + 8 = 25 bytes + self.assertEqual(len(serialized), 25) + + # Verify round-trip consistency + deserialized = BlobDescriptor.deserialize(serialized) + re_serialized = deserialized.serialize() + self.assertEqual(serialized, re_serialized) + + +class BlobEndToEndTest(unittest.TestCase): + """End-to-end tests for blob functionality with schema definition, file writing, and reading.""" + + def setUp(self): + """Set up test environment.""" + self.temp_dir = tempfile.mkdtemp() + self.warehouse = os.path.join(self.temp_dir, 'warehouse') + # Create catalog for table operations + self.catalog = CatalogFactory.create({ + 'warehouse': self.warehouse + }) + self.catalog.create_database('test_db', False) + + def tearDown(self): + """Clean up test environment.""" + try: + shutil.rmtree(self.temp_dir) + except OSError: + pass + + def test_blob_end_to_end(self): + # Set up file I/O + file_io = FileIO(self.temp_dir, {}) + + blob_field_name = "blob_field" + # ========== Step 1: Check Type Validation ========== + blob_fields = [DataField(0, blob_field_name, AtomicType("BLOB"))] + for blob_field in blob_fields: + self.assertIsInstance(blob_field.type, AtomicType) + self.assertEqual(blob_field.type.type, "BLOB") + + # ========== Step 2: Write Data ========== + test_data = {blob_field_name: BlobData(b'End-to-end test: PDF header %PDF-1.4\n...')} + blob_files = {} + blob_data = [test_data[blob_field_name].to_data()] + schema = pa.schema([pa.field(blob_field_name, pa.large_binary())]) + table = pa.table([blob_data], schema=schema) + blob_files[blob_field_name] = Path(self.temp_dir) / (blob_field_name + ".blob") + file_io.write_blob(blob_files[blob_field_name], table, False) + self.assertTrue(file_io.exists(blob_files[blob_field_name])) + + # ========== Step 3: Read Data and Check Data ========== + for field_name, file_path in blob_files.items(): + read_fields = blob_fields + reader = FormatBlobReader( + file_io=file_io, + file_path=str(file_path), + read_fields=[field_name], + full_fields=read_fields, + push_down_predicate=None, + blob_as_descriptor=False + ) + + # Read data + batch = reader.read_arrow_batch() + self.assertIsNotNone(batch, f"{field_name} batch should not be None") + self.assertEqual(batch.num_rows, 1, f"{field_name} should have 1 row") + + # Verify data integrity + read_blob_data = batch.column(0)[0].as_py() + expected_blob_data = test_data[field_name].to_data() + self.assertEqual(read_blob_data, expected_blob_data, f"{field_name} data should match") + + reader.close() + + def test_blob_complex_types_throw_exception(self): + """Test that complex types containing BLOB elements throw exceptions during read/write operations.""" + from pypaimon.schema.data_types import DataField, AtomicType, ArrayType, MultisetType, MapType + from pypaimon.table.row.blob import BlobData + from pypaimon.common.file_io import FileIO + from pypaimon.table.row.generic_row import GenericRow, GenericRowSerializer + from pypaimon.table.row.row_kind import RowKind + from pathlib import Path + + # Set up file I/O + file_io = FileIO(self.temp_dir, {}) + + # ========== Test ArrayType(nullable=True, element_type=AtomicType("BLOB")) ========== + array_fields = [ + DataField(0, "id", AtomicType("INT")), + DataField(1, "blob_array", ArrayType(nullable=True, element_type=AtomicType("BLOB"))), + ] + + # Test serialization throws exception for ArrayType + array_blob_data = [ + BlobData(b"Array blob 1"), + BlobData(b"Array blob 2"), + BlobData(b"Array blob 3") + ] + + array_row = GenericRow([1, array_blob_data], array_fields, RowKind.INSERT) + + # GenericRowSerializer should throw exception for complex types + with self.assertRaises(ValueError) as context: + GenericRowSerializer.to_bytes(array_row) + self.assertIn("AtomicType", str(context.exception)) + + # Note: FileIO.write_blob validation for complex types is tested separately below + + # ========== Test MultisetType(nullable=True, element_type=AtomicType("BLOB")) ========== + multiset_fields = [ + DataField(0, "id", AtomicType("INT")), + DataField(1, "blob_multiset", MultisetType(nullable=True, element_type=AtomicType("BLOB"))), + ] + + # Test serialization throws exception for MultisetType + multiset_blob_data = [ + BlobData(b"Multiset blob 1"), + BlobData(b"Multiset blob 2"), + BlobData(b"Multiset blob 1"), # Duplicate allowed in multiset + ] + + multiset_row = GenericRow([2, multiset_blob_data], multiset_fields, RowKind.INSERT) + + # GenericRowSerializer should throw exception for complex types + with self.assertRaises(ValueError) as context: + GenericRowSerializer.to_bytes(multiset_row) + self.assertIn("AtomicType", str(context.exception)) + map_fields = [ + DataField(0, "id", AtomicType("INT")), + DataField(1, "blob_map", MapType( + nullable=True, key_type=AtomicType("STRING"), value_type=AtomicType("BLOB") + )), + ] + + # Test serialization throws exception for MapType + map_blob_data = { + "document": BlobData(b"Document content"), + "image": BlobData(b"Image data"), + "metadata": BlobData(b"Metadata content") + } + + map_row = GenericRow([3, map_blob_data], map_fields, RowKind.INSERT) + + # GenericRowSerializer should throw exception for complex types + with self.assertRaises(ValueError) as context: + GenericRowSerializer.to_bytes(map_row) + self.assertIn("AtomicType", str(context.exception)) + + # ========== Test FileIO.write_blob validation for complex types ========== + # Test that FileIO.write_blob properly validates and rejects complex types + + # Create a table with multiple columns (should fail - blob format requires single column) + multi_column_schema = pa.schema([ + pa.field("blob1", pa.large_binary()), + pa.field("blob2", pa.large_binary()) + ]) + multi_column_table = pa.table([ + [b"blob1_data"], + [b"blob2_data"] + ], schema=multi_column_schema) + + multi_column_file = Path(self.temp_dir) / "multi_column.blob" + + # Should throw RuntimeError for multiple columns + with self.assertRaises(RuntimeError) as context: + file_io.write_blob(multi_column_file, multi_column_table, False) + self.assertIn("single column", str(context.exception)) + + # Test that FileIO.write_blob rejects null values + null_schema = pa.schema([pa.field("blob_with_nulls", pa.large_binary())]) + null_table = pa.table([[b"data", None]], schema=null_schema) + + null_file = Path(self.temp_dir) / "null_data.blob" + + # Should throw RuntimeError for null values + with self.assertRaises(RuntimeError) as context: + file_io.write_blob(null_file, null_table, False) + self.assertIn("null values", str(context.exception)) + + # ========== Test FormatBlobReader with complex type schema ========== + # Create a valid blob file first + valid_blob_data = [b"Valid blob content"] + valid_schema = pa.schema([pa.field("valid_blob", pa.large_binary())]) + valid_table = pa.table([valid_blob_data], schema=valid_schema) + + valid_blob_file = Path(self.temp_dir) / "valid_blob.blob" + file_io.write_blob(valid_blob_file, valid_table, False) + + # Try to read with complex type field definition - this should fail + # because FormatBlobReader tries to create PyArrow schema with complex types + complex_read_fields = [ + DataField(0, "valid_blob", ArrayType(nullable=True, element_type=AtomicType("BLOB"))) + ] + + # FormatBlobReader creation should work, but reading should fail due to schema mismatch + reader = FormatBlobReader( + file_io=file_io, + file_path=str(valid_blob_file), + read_fields=["valid_blob"], + full_fields=complex_read_fields, + push_down_predicate=None, + blob_as_descriptor=False + ) + + # Reading should fail because the schema expects complex type but data is atomic + with self.assertRaises(Exception) as context: + reader.read_arrow_batch() + # The error could be ArrowTypeError or other PyArrow-related errors + self.assertTrue( + "ArrowTypeError" in str(type(context.exception)) or + "TypeError" in str(type(context.exception)) or + "ValueError" in str(type(context.exception)) + ) + + reader.close() + + def test_blob_advanced_scenarios(self): + """Test advanced blob scenarios: corruption, truncation, zero-length, large blobs, compression, cross-format.""" + from pypaimon.schema.data_types import DataField, AtomicType + from pypaimon.common.file_io import FileIO + from pypaimon.common.delta_varint_compressor import DeltaVarintCompressor + from pathlib import Path + + # Set up file I/O + file_io = FileIO(self.temp_dir, {}) + + # ========== Test 1: Corrupted file header test ========== + + # Create a valid blob file first + valid_blob_data = [b"Test blob content for corruption test"] + valid_schema = pa.schema([pa.field("test_blob", pa.large_binary())]) + valid_table = pa.table([valid_blob_data], schema=valid_schema) + + header_test_file = Path(self.temp_dir) / "header_test.blob" + file_io.write_blob(header_test_file, valid_table, False) + + # Read the file and corrupt the header (last 5 bytes: index_length + version) + with open(header_test_file, 'rb') as f: + original_data = f.read() + + # Corrupt the version byte (last byte) + corrupted_data = bytearray(original_data) + corrupted_data[-1] = 99 # Invalid version (should be 1) + + corrupted_header_file = Path(self.temp_dir) / "corrupted_header.blob" + with open(corrupted_header_file, 'wb') as f: + f.write(corrupted_data) + + # Try to read corrupted file - should detect invalid version + fields = [DataField(0, "test_blob", AtomicType("BLOB"))] + + # Reading should fail due to invalid version + with self.assertRaises(IOError) as context: + FormatBlobReader( + file_io=file_io, + file_path=str(corrupted_header_file), + read_fields=["test_blob"], + full_fields=fields, + push_down_predicate=None, + blob_as_descriptor=False + ) + self.assertIn("Unsupported blob file version", str(context.exception)) + + # ========== Test 2: Truncated blob file (mid-blob) read ========== + + # Create a blob file with substantial content + large_content = b"Large blob content: " + b"X" * 1000 + b" End of content" + large_blob_data = [large_content] + large_schema = pa.schema([pa.field("large_blob", pa.large_binary())]) + large_table = pa.table([large_blob_data], schema=large_schema) + + full_blob_file = Path(self.temp_dir) / "full_blob.blob" + file_io.write_blob(full_blob_file, large_table, False) + + # Read the full file and truncate it in the middle + with open(full_blob_file, 'rb') as f: + full_data = f.read() + + # Truncate to about 50% of original size (mid-blob) + truncated_size = len(full_data) // 2 + truncated_data = full_data[:truncated_size] + + truncated_file = Path(self.temp_dir) / "truncated.blob" + with open(truncated_file, 'wb') as f: + f.write(truncated_data) + + # Try to read truncated file - should fail gracefully + with self.assertRaises((IOError, OSError)) as context: + FormatBlobReader( + file_io=file_io, + file_path=str(truncated_file), + read_fields=["large_blob"], + full_fields=fields, + push_down_predicate=None, + blob_as_descriptor=False + ) + # Should detect truncation/incomplete data (either invalid header or invalid version) + self.assertTrue( + "cannot read header" in str(context.exception) or + "Unsupported blob file version" in str(context.exception) + ) + + # ========== Test 3: Zero-length blob handling ========== + + # Create blob with zero-length content + zero_blob_data = [b""] # Empty blob + zero_schema = pa.schema([pa.field("zero_blob", pa.large_binary())]) + zero_table = pa.table([zero_blob_data], schema=zero_schema) + + zero_blob_file = Path(self.temp_dir) / "zero_length.blob" + file_io.write_blob(zero_blob_file, zero_table, False) + + # Verify file was created + self.assertTrue(file_io.exists(zero_blob_file)) + file_size = file_io.get_file_size(zero_blob_file) + self.assertGreater(file_size, 0) # File should have headers even with empty blob + + # Read zero-length blob + zero_fields = [DataField(0, "zero_blob", AtomicType("BLOB"))] + zero_reader = FormatBlobReader( + file_io=file_io, + file_path=str(zero_blob_file), + read_fields=["zero_blob"], + full_fields=zero_fields, + push_down_predicate=None, + blob_as_descriptor=False + ) + + zero_batch = zero_reader.read_arrow_batch() + self.assertIsNotNone(zero_batch) + self.assertEqual(zero_batch.num_rows, 1) + + # Verify empty blob content + read_zero_blob = zero_batch.column(0)[0].as_py() + self.assertEqual(read_zero_blob, b"") + self.assertEqual(len(read_zero_blob), 0) + zero_reader.close() + + # ========== Test 4: Large blob (multi-GB range) simulation ========== + # Simulate large blob without actually creating multi-GB data + # Test chunked writing and memory-safe reading patterns + + # Create moderately large blob (10MB) to test chunking behavior + chunk_size = 1024 * 1024 # 1MB chunks + large_blob_content = b"LARGE_BLOB_CHUNK:" + b"L" * (chunk_size - 17) # Fill to 1MB + + # Simulate multiple chunks + simulated_large_data = [large_blob_content * 10] # 10MB total + large_sim_schema = pa.schema([pa.field("large_sim_blob", pa.large_binary())]) + large_sim_table = pa.table([simulated_large_data], schema=large_sim_schema) + + large_sim_file = Path(self.temp_dir) / "large_simulation.blob" + file_io.write_blob(large_sim_file, large_sim_table, False) + + # Verify large file was written + large_sim_size = file_io.get_file_size(large_sim_file) + self.assertGreater(large_sim_size, 10 * 1024 * 1024) # Should be > 10MB + + # Read large blob in memory-safe manner + large_sim_fields = [DataField(0, "large_sim_blob", AtomicType("BLOB"))] + large_sim_reader = FormatBlobReader( + file_io=file_io, + file_path=str(large_sim_file), + read_fields=["large_sim_blob"], + full_fields=large_sim_fields, + push_down_predicate=None, + blob_as_descriptor=False + ) + + large_sim_batch = large_sim_reader.read_arrow_batch() + self.assertIsNotNone(large_sim_batch) + self.assertEqual(large_sim_batch.num_rows, 1) + + # Verify large blob content (check prefix to avoid loading all into memory for comparison) + read_large_blob = large_sim_batch.column(0)[0].as_py() + self.assertTrue(read_large_blob.startswith(b"LARGE_BLOB_CHUNK:")) + self.assertEqual(len(read_large_blob), len(large_blob_content) * 10) + large_sim_reader.close() + + # ========== Test 5: Index compression/decompression validation ========== + # Test DeltaVarintCompressor roundtrip + test_indices = [0, 100, 250, 1000, 5000, 10000, 50000] + + # Compress indices + compressed_indices = DeltaVarintCompressor.compress(test_indices) + self.assertIsInstance(compressed_indices, bytes) + self.assertGreater(len(compressed_indices), 0) + + # Decompress indices + decompressed_indices = DeltaVarintCompressor.decompress(compressed_indices) + self.assertEqual(decompressed_indices, test_indices) + + # Test corruption detection in compressed indices + if len(compressed_indices) > 1: + # Corrupt the compressed data + corrupted_indices = bytearray(compressed_indices) + corrupted_indices[-1] = (corrupted_indices[-1] + 1) % 256 # Flip last byte + + # Decompression should fail or produce different results + try: + corrupted_result = DeltaVarintCompressor.decompress(bytes(corrupted_indices)) + # If decompression succeeds, result should be different + self.assertNotEqual(corrupted_result, test_indices) + except Exception: + pass + + # ========== Test 6: Cross-format guard (multi-field tables) ========== + # Test that blob format rejects multi-field tables + multi_field_schema = pa.schema([ + pa.field("blob_field", pa.large_binary()), + pa.field("string_field", pa.string()), + pa.field("int_field", pa.int64()) + ]) + + multi_field_table = pa.table([ + [b"blob_data_1", b"blob_data_2"], + ["string_1", "string_2"], + [100, 200] + ], schema=multi_field_schema) + + multi_field_file = Path(self.temp_dir) / "multi_field.blob" + + # Should reject multi-field table + with self.assertRaises(RuntimeError) as context: + file_io.write_blob(multi_field_file, multi_field_table, False) + self.assertIn("single column", str(context.exception)) + + # Test that blob format rejects non-binary field types + non_binary_schema = pa.schema([pa.field("string_field", pa.string())]) + non_binary_table = pa.table([["not_binary_data"]], schema=non_binary_schema) + + non_binary_file = Path(self.temp_dir) / "non_binary.blob" + + # Should reject non-binary field + with self.assertRaises(RuntimeError) as context: + file_io.write_blob(non_binary_file, non_binary_table, False) + # Should fail due to type conversion issues (non-binary field can't be converted to BLOB) + self.assertTrue( + "large_binary" in str(context.exception) or + "to_paimon_type" in str(context.exception) or + "missing" in str(context.exception) or + "Field must be Blob/BlobData instance" in str(context.exception) + ) + + # Test that blob format rejects tables with null values + null_schema = pa.schema([pa.field("blob_with_null", pa.large_binary())]) + null_table = pa.table([[b"data", None, b"more_data"]], schema=null_schema) + + null_file = Path(self.temp_dir) / "with_nulls.blob" + + # Should reject null values + with self.assertRaises(RuntimeError) as context: + file_io.write_blob(null_file, null_table, False) + self.assertIn("null values", str(context.exception)) + + def test_blob_end_to_end_with_descriptor(self): + # Set up file I/O + file_io = FileIO(self.temp_dir, {}) + + # ========== Step 1: Write data to local file ========== + # Create test data and write it to a local file + test_content = b'This is test blob content stored in an external file for descriptor testing.' + # Write the test content to a local file + local_data_file = Path(self.temp_dir) / "external_blob" + with open(local_data_file, 'wb') as f: + f.write(test_content) + # Verify the file was created and has the correct content + self.assertTrue(local_data_file.exists()) + with open(local_data_file, 'rb') as f: + written_content = f.read() + self.assertEqual(written_content, test_content) + + # ========== Step 2: Use this file as blob descriptor ========== + # Create a BlobDescriptor pointing to the local file + blob_descriptor = BlobDescriptor( + uri=str(local_data_file), + offset=0, + length=len(test_content) + ) + # Serialize the descriptor to bytes (this is what would be stored in the blob column) + descriptor_bytes = blob_descriptor.serialize() + self.assertIsInstance(descriptor_bytes, bytes) + self.assertGreater(len(descriptor_bytes), 0) + + # Create PyArrow table with the serialized descriptor + blob_field_name = "blob_descriptor_field" + schema = pa.schema([pa.field(blob_field_name, pa.large_binary())]) + table = pa.table([[descriptor_bytes]], schema=schema) + + # Write the blob file with blob_as_descriptor=True + blob_file_path = Path(self.temp_dir) / "descriptor_blob.blob" + file_io.write_blob(blob_file_path, table, blob_as_descriptor=True) + # Verify the blob file was created + self.assertTrue(file_io.exists(blob_file_path)) + file_size = file_io.get_file_size(blob_file_path) + self.assertGreater(file_size, 0) + + # ========== Step 3: Read data and check ========== + # Define schema for reading + read_fields = [DataField(0, blob_field_name, AtomicType("BLOB"))] + reader = FormatBlobReader( + file_io=file_io, + file_path=str(blob_file_path), + read_fields=[blob_field_name], + full_fields=read_fields, + push_down_predicate=None, + blob_as_descriptor=True + ) + + # Read the data with blob_as_descriptor=True (should return a descriptor) + batch = reader.read_arrow_batch() + self.assertIsNotNone(batch) + self.assertEqual(batch.num_rows, 1) + self.assertEqual(batch.num_columns, 1) + + read_blob_bytes = batch.column(0)[0].as_py() + self.assertIsInstance(read_blob_bytes, bytes) + + # Deserialize the returned descriptor + returned_descriptor = BlobDescriptor.deserialize(read_blob_bytes) + + # The returned descriptor should point to the blob file (simplified implementation) + # because the current implementation creates a descriptor pointing to the blob file location + self.assertEqual(returned_descriptor.uri, str(blob_file_path)) + self.assertGreater(returned_descriptor.offset, 0) # Should have some offset in the blob file + + reader.close() + + reader_content = FormatBlobReader( + file_io=file_io, + file_path=str(blob_file_path), + read_fields=[blob_field_name], + full_fields=read_fields, + push_down_predicate=None, + blob_as_descriptor=False + ) + batch_content = reader_content.read_arrow_batch() + self.assertIsNotNone(batch_content) + self.assertEqual(batch_content.num_rows, 1) + read_content_bytes = batch_content.column(0)[0].as_py() + self.assertIsInstance(read_content_bytes, bytes) + # When blob_as_descriptor=False, we should get the actual file content + self.assertEqual(read_content_bytes, test_content) + reader_content.close() + + +if __name__ == '__main__': + unittest.main() diff --git a/paimon-python/pypaimon/tests/data_evolution_test.py b/paimon-python/pypaimon/tests/data_evolution_test.py new file mode 100644 index 000000000000..1d6402b327e2 --- /dev/null +++ b/paimon-python/pypaimon/tests/data_evolution_test.py @@ -0,0 +1,535 @@ +""" +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 tempfile +import unittest + +import pyarrow as pa +from pypaimon import Schema, CatalogFactory + + +class DataEvolutionTest(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls.tempdir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.tempdir, 'warehouse') + cls.catalog = CatalogFactory.create({ + 'warehouse': cls.warehouse + }) + cls.catalog.create_database('default', False) + + def test_basic(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int8()), + ('f1', pa.int16()), + ]) + schema = Schema.from_pyarrow_schema(simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'}) + self.catalog.create_table('default.test_row_tracking', schema, False) + table = self.catalog.get_table('default.test_row_tracking') + + # write 1 + write_builder = table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + expect_data = pa.Table.from_pydict({ + 'f0': [-1, 2], + 'f1': [-1001, 1002] + }, schema=simple_pa_schema) + table_write.write_arrow(expect_data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # write 2 + table_write = write_builder.new_write().with_write_type(['f0']) + table_commit = write_builder.new_commit() + data2 = pa.Table.from_pydict({ + 'f0': [3, 4], + }, schema=pa.schema([ + ('f0', pa.int8()), + ])) + table_write.write_arrow(data2) + cmts = table_write.prepare_commit() + cmts[0].new_files[0].first_row_id = 0 + table_commit.commit(cmts) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual_data = table_read.to_arrow(table_scan.plan().splits()) + expect_data = pa.Table.from_pydict({ + 'f0': [3, 4], + 'f1': [-1001, 1002] + }, schema=pa.schema([ + ('f0', pa.int8()), + ('f1', pa.int16()), + ])) + self.assertEqual(actual_data, expect_data) + + def test_with_shard(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int8()), + ('f1', pa.int16()), + ]) + schema = Schema.from_pyarrow_schema(simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'}) + self.catalog.create_table('default.test_with_shard', schema, False) + table = self.catalog.get_table('default.test_with_shard') + + # write 1 + write_builder = table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + expect_data = pa.Table.from_pydict({ + 'f0': [-1, 2], + 'f1': [-1001, 1002] + }, schema=simple_pa_schema) + table_write.write_arrow(expect_data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # write 2 + table_write = write_builder.new_write().with_write_type(['f0']) + table_commit = write_builder.new_commit() + data2 = pa.Table.from_pydict({ + 'f0': [3, 4], + }, schema=pa.schema([ + ('f0', pa.int8()), + ])) + table_write.write_arrow(data2) + cmts = table_write.prepare_commit() + cmts[0].new_files[0].first_row_id = 0 + table_commit.commit(cmts) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan().with_shard(0, 2) + table_read = read_builder.new_read() + splits = table_scan.plan().splits() + actual_data = table_read.to_arrow(splits) + expect_data = pa.Table.from_pydict({ + 'f0': [3], + 'f1': [-1001] + }, schema=pa.schema([ + ('f0', pa.int8()), + ('f1', pa.int16()), + ])) + self.assertEqual(actual_data, expect_data) + + def test_multiple_appends(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_multiple_appends', schema, False) + table = self.catalog.get_table('default.test_multiple_appends') + + write_builder = table.new_batch_write_builder() + + # write 100 rows: (1, "a", "b") + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + init_data = pa.Table.from_pydict({ + 'f0': [1] * 100, + 'f1': ['a'] * 100, + 'f2': ['b'] * 100, + }, schema=simple_pa_schema) + table_write.write_arrow(init_data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + # append:set first_row_id = 100 to modify the row with columns write + write0 = write_builder.new_write().with_write_type(['f0', 'f1']) + write1 = write_builder.new_write().with_write_type(['f2']) + commit = write_builder.new_commit() + data0 = pa.Table.from_pydict({'f0': [2], 'f1': ['x']}, + schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + data1 = pa.Table.from_pydict({'f2': ['y']}, schema=pa.schema([('f2', pa.string())])) + write0.write_arrow(data0) + write1.write_arrow(data1) + cmts = write0.prepare_commit() + write1.prepare_commit() + for c in cmts: + for nf in c.new_files: + nf.first_row_id = 100 + commit.commit(cmts) + write0.close() + write1.close() + commit.close() + + # append:write (3, "c") and ("d"), set first_row_id = 101 + write0 = write_builder.new_write().with_write_type(['f0', 'f1']) + commit0 = write_builder.new_commit() + data0 = pa.Table.from_pydict({'f0': [3], 'f1': ['c']}, + schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + write0.write_arrow(data0) + cmts0 = write0.prepare_commit() + for c in cmts0: + for nf in c.new_files: + nf.first_row_id = 101 + commit0.commit(cmts0) + write0.close() + commit0.close() + + write1 = write_builder.new_write().with_write_type(['f2']) + commit1 = write_builder.new_commit() + data1 = pa.Table.from_pydict({'f2': ['d']}, schema=pa.schema([('f2', pa.string())])) + write1.write_arrow(data1) + cmts1 = write1.prepare_commit() + for c in cmts1: + for nf in c.new_files: + nf.first_row_id = 101 + commit1.commit(cmts1) + write1.close() + commit1.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + + self.assertEqual(actual.num_rows, 102) + expect = pa.Table.from_pydict({ + 'f0': [1] * 100 + [2] + [3], + 'f1': ['a'] * 100 + ['x'] + ['c'], + 'f2': ['b'] * 100 + ['y'] + ['d'], + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) + + def test_disorder_cols_append(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_disorder_cols_append', schema, False) + table = self.catalog.get_table('default.test_disorder_cols_append') + + write_builder = table.new_batch_write_builder() + num_rows = 100 + # write 1 rows: (1, "a", "b") + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + init_data = pa.Table.from_pydict({ + 'f0': [1] * num_rows, + 'f1': ['a'] * num_rows, + 'f2': ['b'] * num_rows, + }, schema=simple_pa_schema) + table_write.write_arrow(init_data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # append:set first_row_id = 0 to modify the row with columns write + write0 = write_builder.new_write().with_write_type(['f0', 'f2']) + write1 = write_builder.new_write().with_write_type(['f1']) + commit = write_builder.new_commit() + data0 = pa.Table.from_pydict({'f0': [2] * num_rows, 'f2': ['y'] * num_rows}, + schema=pa.schema([('f0', pa.int32()), ('f2', pa.string())])) + data1 = pa.Table.from_pydict({'f1': ['x'] * num_rows}, schema=pa.schema([('f1', pa.string())])) + write0.write_arrow(data0) + write1.write_arrow(data1) + cmts = write0.prepare_commit() + write1.prepare_commit() + for c in cmts: + for nf in c.new_files: + nf.first_row_id = 0 + commit.commit(cmts) + write0.close() + write1.close() + commit.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + + self.assertEqual(actual.num_rows, 100) + expect = pa.Table.from_pydict({ + 'f0': [2] * num_rows, + 'f1': ['x'] * num_rows, + 'f2': ['y'] * num_rows, + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) + + def test_only_some_columns(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_only_some_columns', schema, False) + table = self.catalog.get_table('default.test_only_some_columns') + + write_builder = table.new_batch_write_builder() + + # Commit 1: f0 + w0 = write_builder.new_write().with_write_type(['f0']) + c0 = write_builder.new_commit() + d0 = pa.Table.from_pydict({'f0': [1]}, schema=pa.schema([('f0', pa.int32())])) + w0.write_arrow(d0) + c0.commit(w0.prepare_commit()) + w0.close() + c0.close() + + # Commit 2: f1, first_row_id = 0 + w1 = write_builder.new_write().with_write_type(['f1']) + c1 = write_builder.new_commit() + d1 = pa.Table.from_pydict({'f1': ['a']}, schema=pa.schema([('f1', pa.string())])) + w1.write_arrow(d1) + cmts1 = w1.prepare_commit() + for c in cmts1: + for nf in c.new_files: + nf.first_row_id = 0 + c1.commit(cmts1) + w1.close() + c1.close() + + # Commit 3: f2, first_row_id = 0 + w2 = write_builder.new_write().with_write_type(['f2']) + c2 = write_builder.new_commit() + d2 = pa.Table.from_pydict({'f2': ['b']}, schema=pa.schema([('f2', pa.string())])) + w2.write_arrow(d2) + cmts2 = w2.prepare_commit() + for c in cmts2: + for nf in c.new_files: + nf.first_row_id = 0 + c2.commit(cmts2) + w2.close() + c2.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + + expect = pa.Table.from_pydict({ + 'f0': [1], + 'f1': ['a'], + 'f2': ['b'], + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) + + def test_null_values(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_null_values', schema, False) + table = self.catalog.get_table('default.test_null_values') + + write_builder = table.new_batch_write_builder() + + # Commit 1: some cols are null + w0 = write_builder.new_write().with_write_type(['f0', 'f1']) + w1 = write_builder.new_write().with_write_type(['f2']) + c = write_builder.new_commit() + + d0 = pa.Table.from_pydict({'f0': [1], 'f1': [None]}, + schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + d1 = pa.Table.from_pydict({'f2': [None]}, schema=pa.schema([('f2', pa.string())])) + w0.write_arrow(d0) + w1.write_arrow(d1) + cmts = w0.prepare_commit() + w1.prepare_commit() + for msg in cmts: + for nf in msg.new_files: + nf.first_row_id = 0 + c.commit(cmts) + w0.close() + w1.close() + c.close() + + # Commit 2 + w1 = write_builder.new_write().with_write_type(['f2']) + c1 = write_builder.new_commit() + d1 = pa.Table.from_pydict({'f2': ['c']}, schema=pa.schema([('f2', pa.string())])) + w1.write_arrow(d1) + cmts1 = w1.prepare_commit() + for msg in cmts1: + for nf in msg.new_files: + nf.first_row_id = 0 + c1.commit(cmts1) + w1.close() + c1.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + expect = pa.Table.from_pydict({ + 'f0': [1], + 'f1': [None], + 'f2': ['c'], + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) + + # different first_row_id append multiple times + def test_multiple_appends_different_first_row_ids(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_multiple_appends_diff_rowid', schema, False) + table = self.catalog.get_table('default.test_multiple_appends_diff_rowid') + + write_builder = table.new_batch_write_builder() + + # commit 1 + w0 = write_builder.new_write().with_write_type(['f0', 'f1']) + w1 = write_builder.new_write().with_write_type(['f2']) + c = write_builder.new_commit() + d0 = pa.Table.from_pydict({'f0': [1], 'f1': ['a']}, + schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + d1 = pa.Table.from_pydict({'f2': ['b']}, schema=pa.schema([('f2', pa.string())])) + w0.write_arrow(d0) + w1.write_arrow(d1) + cmts = w0.prepare_commit() + w1.prepare_commit() + for msg in cmts: + for nf in msg.new_files: + nf.first_row_id = 0 + c.commit(cmts) + w0.close() + w1.close() + c.close() + + # commit 2 + w0 = write_builder.new_write().with_write_type(['f0', 'f1']) + c0 = write_builder.new_commit() + d0 = pa.Table.from_pydict({'f0': [2], 'f1': ['c']}, + schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + w0.write_arrow(d0) + cmts0 = w0.prepare_commit() + for msg in cmts0: + for nf in msg.new_files: + nf.first_row_id = 1 + c0.commit(cmts0) + w0.close() + c0.close() + + # commit 3 + w1 = write_builder.new_write().with_write_type(['f2']) + c1 = write_builder.new_commit() + d1 = pa.Table.from_pydict({'f2': ['d']}, schema=pa.schema([('f2', pa.string())])) + w1.write_arrow(d1) + cmts1 = w1.prepare_commit() + for msg in cmts1: + for nf in msg.new_files: + nf.first_row_id = 1 + c1.commit(cmts1) + w1.close() + c1.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + + expect = pa.Table.from_pydict({ + 'f0': [1, 2], + 'f1': ['a', 'c'], + 'f2': ['b', 'd'], + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) + + def test_more_data(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_more_data', schema, False) + table = self.catalog.get_table('default.test_more_data') + + write_builder = table.new_batch_write_builder() + + # first commit:100k rows + w0 = write_builder.new_write().with_write_type(['f0', 'f1']) + w1 = write_builder.new_write().with_write_type(['f2']) + c = write_builder.new_commit() + size = 100000 + d0 = pa.Table.from_pydict({ + 'f0': list(range(size)), + 'f1': [f'a{i}' for i in range(size)], + }, schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + d1 = pa.Table.from_pydict({ + 'f2': [f'b{i}' for i in range(size)], + }, schema=pa.schema([('f2', pa.string())])) + w0.write_arrow(d0) + w1.write_arrow(d1) + cmts = w0.prepare_commit() + w1.prepare_commit() + for msg in cmts: + for nf in msg.new_files: + nf.first_row_id = 0 + c.commit(cmts) + w0.close() + w1.close() + c.close() + + # second commit:overwrite f2 to 'c{i}' + w1 = write_builder.new_write().with_write_type(['f2']) + c1 = write_builder.new_commit() + d1 = pa.Table.from_pydict({ + 'f2': [f'c{i}' for i in range(size)], + }, schema=pa.schema([('f2', pa.string())])) + w1.write_arrow(d1) + cmts1 = w1.prepare_commit() + c1.commit(cmts1) + w1.close() + c1.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + + expect = pa.Table.from_pydict({ + 'f0': list(range(size)), + 'f1': [f'a{i}' for i in range(size)], + 'f2': [f'c{i}' for i in range(size)], + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) diff --git a/paimon-python/pypaimon/tests/data_types_test.py b/paimon-python/pypaimon/tests/data_types_test.py new file mode 100755 index 000000000000..53644e24c571 --- /dev/null +++ b/paimon-python/pypaimon/tests/data_types_test.py @@ -0,0 +1,67 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +import unittest +from parameterized import parameterized + +from pypaimon.schema.data_types import DataField, AtomicType, ArrayType, MultisetType, MapType, RowType + + +class DataTypesTest(unittest.TestCase): + def test_atomic_type(self): + self.assertEqual(str(AtomicType("BLOB")), "BLOB") + self.assertEqual(str(AtomicType("TINYINT", nullable=False)), "TINYINT NOT NULL") + self.assertEqual(str(AtomicType("BIGINT", nullable=False)), "BIGINT NOT NULL") + self.assertEqual(str(AtomicType("BOOLEAN", nullable=False)), "BOOLEAN NOT NULL") + self.assertEqual(str(AtomicType("DOUBLE")), "DOUBLE") + self.assertEqual(str(AtomicType("STRING")), "STRING") + self.assertEqual(str(AtomicType("BINARY(12)")), "BINARY(12)") + self.assertEqual(str(AtomicType("DECIMAL(10, 6)")), "DECIMAL(10, 6)") + self.assertEqual(str(AtomicType("BYTES")), "BYTES") + self.assertEqual(str(AtomicType("DATE")), "DATE") + self.assertEqual(str(AtomicType("TIME(0)")), "TIME(0)") + self.assertEqual(str(AtomicType("TIMESTAMP(0)")), "TIMESTAMP(0)") + self.assertEqual(str(AtomicType("SMALLINT", nullable=False)), + str(AtomicType.from_dict(AtomicType("SMALLINT", nullable=False).to_dict()))) + self.assertEqual(str(AtomicType("INT")), + str(AtomicType.from_dict(AtomicType("INT").to_dict()))) + + @parameterized.expand([ + (ArrayType, AtomicType("TIMESTAMP(6)"), "ARRAY", "ARRAY>"), + (MultisetType, AtomicType("TIMESTAMP(6)"), "MULTISET", "MULTISET>") + ]) + def test_complex_types(self, data_type_class, element_type, expected1, expected2): + self.assertEqual(str(data_type_class(True, element_type)), expected1) + self.assertEqual(str(data_type_class(True, data_type_class(True, element_type))), expected2) + self.assertEqual(str(data_type_class(False, element_type)), expected1 + " NOT NULL") + self.assertEqual(str(data_type_class(False, element_type)), + str(data_type_class.from_dict(data_type_class(False, element_type).to_dict()))) + self.assertEqual(str(data_type_class(True, element_type)), + str(data_type_class.from_dict(data_type_class(True, element_type).to_dict()))) + + def test_map_type(self): + self.assertEqual(str(MapType(True, AtomicType("STRING"), AtomicType("TIMESTAMP(6)"))), + "MAP") + + def test_row_type(self): + self.assertEqual(str(RowType(True, [DataField(0, "a", AtomicType("STRING"), "Someone's desc."), + DataField(1, "b", AtomicType("TIMESTAMP(6)"),)])), + "ROW") + row_data = RowType(True, [DataField(0, "a", AtomicType("STRING"), "Someone's desc."), + DataField(1, "b", AtomicType("TIMESTAMP(6)"),)]) + self.assertEqual(str(row_data), + str(RowType.from_dict(row_data.to_dict()))) diff --git a/paimon-python/pypaimon/tests/delta_varint_compressor_test.py b/paimon-python/pypaimon/tests/delta_varint_compressor_test.py new file mode 100644 index 000000000000..5a04becde844 --- /dev/null +++ b/paimon-python/pypaimon/tests/delta_varint_compressor_test.py @@ -0,0 +1,379 @@ +""" +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 random +import sys +import unittest + +from pypaimon.common.delta_varint_compressor import DeltaVarintCompressor + + +class DeltaVarintCompressorTest(unittest.TestCase): + """Tests for DeltaVarintCompressor following org.apache.paimon.utils.DeltaVarintCompressorTest.""" + + def test_normal_case_1(self): + """Test case for normal compression and decompression.""" + original = [80, 50, 90, 80, 70] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) # Verify data integrity + self.assertEqual(6, len(compressed)) # Optimized size for small deltas + + def test_normal_case_2(self): + """Test case for normal compression and decompression.""" + original = [100, 50, 150, 100, 200] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) # Verify data integrity + self.assertEqual(8, len(compressed)) # Optimized size for small deltas + + def test_random(self): + """Test with random data to ensure robustness.""" + # Run multiple iterations to test various random cases + for _ in range(100): # Reduced from 10000 for reasonable test time + original = [] + for i in range(100): + # Use a smaller range than Java's Long.MAX_VALUE for Python compatibility + original.append(random.randint(-sys.maxsize, sys.maxsize)) + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) # Verify data integrity + + def test_empty_array(self): + """Test case for empty array.""" + original = [] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + self.assertEqual(0, len(compressed)) + + def test_single_element(self): + """Test case for single-element array.""" + original = [42] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + # Calculate expected size: Varint encoding for 42 (0x2A -> 1 byte) + self.assertEqual(1, len(compressed)) + + def test_extreme_values(self): + """Test case for extreme values (sys.maxsize and -sys.maxsize).""" + original = [-sys.maxsize, sys.maxsize] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + # The compressed size will depend on the platform's sys.maxsize + # but should be reasonable for the delta encoding + self.assertGreater(len(compressed), 0) + + def test_negative_deltas(self): + """Test case for negative deltas with ZigZag optimization.""" + original = [100, -50, -150, -100] # Negative values + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + # Verify ZigZag optimization: -1 → 1 (1 byte) + # Delta sequence: [100, -150, -100, 50] → ZigZag → + # Each encoded in 1-2 bytes + self.assertLessEqual(len(compressed), 8) # Optimized size + + def test_unsorted_data(self): + """Test case for unsorted data (worse compression ratio).""" + original = [1000, 5, 9999, 12345, 6789] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + # Larger deltas → more bytes (e.g., 9994 → 3 bytes) + self.assertGreater(len(compressed), 5) # Worse than sorted case + + def test_corrupted_input(self): + """Test case for corrupted input (invalid Varint).""" + # Invalid Varint (all continuation flags) + corrupted = bytes([0x80, 0x80, 0x80]) + try: + result = DeltaVarintCompressor.decompress(corrupted) + # If it doesn't raise an exception, the result should be reasonable + self.assertIsInstance(result, list) + except (IndexError, ValueError, RuntimeError): + # It's acceptable to raise an exception for corrupted data + pass + + def test_zero_values(self): + """Test case for arrays with zero values.""" + original = [0, 0, 0, 0, 0] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + # All deltas are 0, so should compress very well + self.assertLessEqual(len(compressed), 5) + + def test_ascending_sequence(self): + """Test case for ascending sequence (optimal for delta compression).""" + original = [1, 2, 3, 4, 5, 6, 7, 8, 9, 10] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + # All deltas are 1, so should compress very well + self.assertLessEqual(len(compressed), 15) # Much smaller than original + + def test_descending_sequence(self): + """Test case for descending sequence.""" + original = [10, 9, 8, 7, 6, 5, 4, 3, 2, 1] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + # All deltas are -1, should still compress well with ZigZag + self.assertLessEqual(len(compressed), 15) + + def test_large_positive_values(self): + """Test case for large positive values.""" + original = [1000000, 2000000, 3000000, 4000000] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + # Large values but consistent deltas should still compress reasonably + self.assertGreater(len(compressed), 4) # Will be larger due to big numbers + + def test_mixed_positive_negative(self): + """Test case for mixed positive and negative values.""" + original = [100, -200, 300, -400, 500] + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + # Mixed signs create larger deltas + self.assertGreater(len(compressed), 5) + + def test_compression_efficiency(self): + """Test that compression actually reduces size for suitable data.""" + # Create a sequence with small deltas + original = [] + base = 1000 + for i in range(100): + base += random.randint(-10, 10) # Small deltas + original.append(base) + + compressed = DeltaVarintCompressor.compress(original) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual(original, decompressed) + # For small deltas, compression should be effective + # Original would need 8 bytes per int (800 bytes), compressed should be much smaller + self.assertLess(len(compressed), len(original) * 4) # At least 50% compression + + def test_round_trip_consistency(self): + """Test that multiple compress/decompress cycles are consistent.""" + original = [1, 10, 100, 1000, 10000] + # First round trip + compressed1 = DeltaVarintCompressor.compress(original) + decompressed1 = DeltaVarintCompressor.decompress(compressed1) + # Second round trip + compressed2 = DeltaVarintCompressor.compress(decompressed1) + decompressed2 = DeltaVarintCompressor.decompress(compressed2) + # All should be identical + self.assertEqual(original, decompressed1) + self.assertEqual(original, decompressed2) + self.assertEqual(compressed1, compressed2) + + def test_boundary_values(self): + """Test boundary values for varint encoding.""" + # Test values around varint boundaries (127, 16383, etc.) + boundary_values = [ + 0, 1, 127, 128, 255, 256, + 16383, 16384, 32767, 32768, + -1, -127, -128, -255, -256, + -16383, -16384, -32767, -32768 + ] + compressed = DeltaVarintCompressor.compress(boundary_values) + decompressed = DeltaVarintCompressor.decompress(compressed) + self.assertEqual(boundary_values, decompressed) + + def test_java_compatibility_zigzag_encoding(self): + """Test ZigZag encoding compatibility with Java implementation.""" + # Test cases that verify ZigZag encoding matches Java's implementation + # ZigZag mapping: 0->0, -1->1, 1->2, -2->3, 2->4, -3->5, 3->6, etc. + zigzag_test_cases = [ + (0, 0), # 0 -> 0 + (-1, 1), # -1 -> 1 + (1, 2), # 1 -> 2 + (-2, 3), # -2 -> 3 + (2, 4), # 2 -> 4 + (-3, 5), # -3 -> 5 + (3, 6), # 3 -> 6 + (-64, 127), # -64 -> 127 + (64, 128), # 64 -> 128 + (-65, 129), # -65 -> 129 + ] + + for original_value, expected_zigzag in zigzag_test_cases: + # Test single value compression to verify ZigZag encoding + compressed = DeltaVarintCompressor.compress([original_value]) + decompressed = DeltaVarintCompressor.decompress(compressed) + + self.assertEqual([original_value], decompressed, + f"ZigZag encoding failed for value {original_value}") + + def test_java_compatibility_known_vectors(self): + """Test with known test vectors that should match Java implementation.""" + # Test vectors with expected compressed output (hexadecimal) + test_vectors = [ + # Simple cases + ([0], "00"), # 0 -> ZigZag(0) = 0 -> Varint(0) = 0x00 + ([1], "02"), # 1 -> ZigZag(1) = 2 -> Varint(2) = 0x02 + ([-1], "01"), # -1 -> ZigZag(-1) = 1 -> Varint(1) = 0x01 + ([2], "04"), # 2 -> ZigZag(2) = 4 -> Varint(4) = 0x04 + ([-2], "03"), # -2 -> ZigZag(-2) = 3 -> Varint(3) = 0x03 + + # Delta encoding cases + ([0, 1], "0002"), # [0, 1] -> [0, delta=1] -> [0x00, 0x02] + ([1, 2], "0202"), # [1, 2] -> [1, delta=1] -> [0x02, 0x02] + ([0, -1], "0001"), # [0, -1] -> [0, delta=-1] -> [0x00, 0x01] + ([1, 0], "0201"), # [1, 0] -> [1, delta=-1] -> [0x02, 0x01] + + # Larger values + ([127], "fe01"), # 127 -> ZigZag(127) = 254 -> Varint(254) = 0xfe01 + ([-127], "fd01"), # -127 -> ZigZag(-127) = 253 -> Varint(253) = 0xfd01 + ([128], "8002"), # 128 -> ZigZag(128) = 256 -> Varint(256) = 0x8002 + ([-128], "ff01"), # -128 -> ZigZag(-128) = 255 -> Varint(255) = 0xff01 + ] + + for original, expected_hex in test_vectors: + compressed = DeltaVarintCompressor.compress(original) + actual_hex = compressed.hex() + + self.assertEqual(expected_hex, actual_hex, + f"Binary compatibility failed for {original}. " + f"Expected: {expected_hex}, Got: {actual_hex}") + + # Also verify round-trip + decompressed = DeltaVarintCompressor.decompress(compressed) + self.assertEqual(original, decompressed, + f"Round-trip failed for {original}") + + def test_java_compatibility_large_numbers(self): + """Test compatibility with Java for large numbers (64-bit range).""" + # Test cases covering the full 64-bit signed integer range + large_number_cases = [ + 2147483647, # Integer.MAX_VALUE + -2147483648, # Integer.MIN_VALUE + 9223372036854775807, # Long.MAX_VALUE + -9223372036854775808 + 1, # Long.MIN_VALUE + 1 (avoid overflow in Python) + 4294967295, # 2^32 - 1 + -4294967296, # -2^32 + ] + + for value in large_number_cases: + # Test individual values + compressed = DeltaVarintCompressor.compress([value]) + decompressed = DeltaVarintCompressor.decompress(compressed) + self.assertEqual([value], decompressed, + f"Large number compatibility failed for {value}") + + # Test as a sequence to verify delta encoding with large numbers + compressed_seq = DeltaVarintCompressor.compress(large_number_cases) + decompressed_seq = DeltaVarintCompressor.decompress(compressed_seq) + self.assertEqual(large_number_cases, decompressed_seq, + "Large number sequence compatibility failed") + + def test_java_compatibility_varint_boundaries(self): + """Test Varint encoding boundaries that match Java implementation.""" + # Test values at Varint encoding boundaries + varint_boundary_cases = [ + # 1-byte Varint boundary + 63, # ZigZag(63) = 126, fits in 1 byte + 64, # ZigZag(64) = 128, needs 2 bytes + -64, # ZigZag(-64) = 127, fits in 1 byte + -65, # ZigZag(-65) = 129, needs 2 bytes + + # 2-byte Varint boundary + 8191, # ZigZag(8191) = 16382, fits in 2 bytes + 8192, # ZigZag(8192) = 16384, needs 3 bytes + -8192, # ZigZag(-8192) = 16383, fits in 2 bytes + -8193, # ZigZag(-8193) = 16385, needs 3 bytes + + # 3-byte Varint boundary + 1048575, # ZigZag(1048575) = 2097150, fits in 3 bytes + 1048576, # ZigZag(1048576) = 2097152, needs 4 bytes + ] + + for value in varint_boundary_cases: + compressed = DeltaVarintCompressor.compress([value]) + decompressed = DeltaVarintCompressor.decompress(compressed) + self.assertEqual([value], decompressed, + f"Varint boundary compatibility failed for {value}") + + def test_java_compatibility_delta_edge_cases(self): + """Test delta encoding edge cases for Java compatibility.""" + # Edge cases that test delta encoding behavior + delta_edge_cases = [ + # Maximum positive delta + [0, sys.maxsize], + # Maximum negative delta + [sys.maxsize, 0], + # Alternating large deltas + [0, 1000000, -1000000, 2000000, -2000000], + # Sequence with zero deltas + [42, 42, 42, 42], + # Mixed small and large deltas + [0, 1, 1000000, 1000001, 0], + ] + + for case in delta_edge_cases: + compressed = DeltaVarintCompressor.compress(case) + decompressed = DeltaVarintCompressor.decompress(compressed) + self.assertEqual(case, decompressed, + f"Delta edge case compatibility failed for {case}") + + def test_java_compatibility_error_conditions(self): + """Test error conditions that should match Java behavior.""" + # Test cases for error handling - our implementation gracefully handles + # truncated data by returning empty lists, which is acceptable behavior + + # Test with various truncated/invalid byte sequences + invalid_cases = [ + bytes([0x80]), # Single incomplete byte + bytes([0x80, 0x80]), # Incomplete 3-byte varint + bytes([0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x80]), # Long sequence + ] + + for invalid_data in invalid_cases: + # Our implementation handles invalid data gracefully by returning empty list + # This is acceptable behavior for robustness + result = DeltaVarintCompressor.decompress(invalid_data) + self.assertIsInstance(result, list, + f"Should return a list for invalid data: {invalid_data.hex()}") + # Empty result is acceptable for invalid/truncated data + + # Test that valid empty input returns empty list + empty_result = DeltaVarintCompressor.decompress(b'') + self.assertEqual([], empty_result, "Empty input should return empty list") + + +if __name__ == '__main__': + unittest.main() diff --git a/paimon-python/pypaimon/tests/e2e/__init__.py b/paimon-python/pypaimon/tests/e2e/__init__.py new file mode 100644 index 000000000000..a67d5ea255b2 --- /dev/null +++ b/paimon-python/pypaimon/tests/e2e/__init__.py @@ -0,0 +1,16 @@ +# 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/e2e/java_py_read_write_test.py b/paimon-python/pypaimon/tests/e2e/java_py_read_write_test.py new file mode 100644 index 000000000000..405484d505b5 --- /dev/null +++ b/paimon-python/pypaimon/tests/e2e/java_py_read_write_test.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. +################################################################################ + +import os +import unittest + +import pandas as pd +import pyarrow as pa +from pypaimon.catalog.catalog_factory import CatalogFactory +from pypaimon.schema.schema import Schema + + +class JavaPyReadWriteTest(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls.tempdir = os.path.abspath(".") + cls.warehouse = os.path.join(cls.tempdir, 'warehouse') + cls.catalog = CatalogFactory.create({ + 'warehouse': cls.warehouse + }) + cls.catalog.create_database('default', True) + + def test_py_write_read(self): + pa_schema = pa.schema([ + ('id', pa.int32()), + ('name', pa.string()), + ('category', pa.string()), + ('value', pa.float64()) + ]) + + schema = Schema.from_pyarrow_schema( + pa_schema, + partition_keys=['category'], + options={'dynamic-partition-overwrite': 'false'} + ) + + self.catalog.create_table('default.mixed_test_tablep', schema, False) + table = self.catalog.get_table('default.mixed_test_tablep') + + initial_data = pd.DataFrame({ + 'id': [1, 2, 3, 4, 5, 6], + 'name': ['Apple', 'Banana', 'Carrot', 'Broccoli', 'Chicken', 'Beef'], + 'category': ['Fruit', 'Fruit', 'Vegetable', 'Vegetable', 'Meat', 'Meat'], + 'value': [1.5, 0.8, 0.6, 1.2, 5.0, 8.0] + }) + # Write initial data + write_builder = table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + + table_write.write_pandas(initial_data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # Verify initial data + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + initial_result = table_read.to_pandas(table_scan.plan().splits()) + print(initial_result) + self.assertEqual(len(initial_result), 6) + self.assertListEqual( + initial_result['name'].tolist(), + ['Apple', 'Banana', 'Carrot', 'Broccoli', 'Chicken', 'Beef'] + ) + + def test_read(self): + table = self.catalog.get_table('default.mixed_test_tablej') + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + res = table_read.to_pandas(table_scan.plan().splits()) + print(res) diff --git a/paimon-python/pypaimon/tests/file_store_commit_test.py b/paimon-python/pypaimon/tests/file_store_commit_test.py index ac7ce95094c1..ab566c3e5202 100644 --- a/paimon-python/pypaimon/tests/file_store_commit_test.py +++ b/paimon-python/pypaimon/tests/file_store_commit_test.py @@ -76,7 +76,10 @@ def test_generate_partition_statistics_single_partition_single_file( schema_id=0, level=0, extra_files=None, - creation_time=creation_time + creation_time=creation_time, + external_path=None, + first_row_id=None, + write_cols=None ) commit_message = CommitMessage( @@ -182,7 +185,10 @@ def test_generate_partition_statistics_multiple_partitions( schema_id=0, level=0, extra_files=None, - creation_time=creation_time + creation_time=creation_time, + external_path=None, + first_row_id=None, + write_cols=None ) # File for partition 2 @@ -199,7 +205,10 @@ def test_generate_partition_statistics_multiple_partitions( schema_id=0, level=0, extra_files=None, - creation_time=creation_time + creation_time=creation_time, + external_path=None, + first_row_id=None, + write_cols=None ) commit_message_1 = CommitMessage( @@ -261,7 +270,10 @@ def test_generate_partition_statistics_unpartitioned_table( schema_id=0, level=0, extra_files=None, - creation_time=creation_time + creation_time=creation_time, + external_path=None, + first_row_id=None, + write_cols=None ) commit_message = CommitMessage( @@ -389,7 +401,3 @@ def _to_entries(commit_messages): file=file )) return commit_entries - - -if __name__ == '__main__': - unittest.main() diff --git a/paimon-python/pypaimon/tests/filesystem_catalog_test.py b/paimon-python/pypaimon/tests/filesystem_catalog_test.py index d6b9433cba3e..530e33aa78ee 100644 --- a/paimon-python/pypaimon/tests/filesystem_catalog_test.py +++ b/paimon-python/pypaimon/tests/filesystem_catalog_test.py @@ -19,13 +19,12 @@ import tempfile import unittest +from pypaimon import CatalogFactory, Schema from pypaimon.catalog.catalog_exception import (DatabaseAlreadyExistException, DatabaseNotExistException, TableAlreadyExistException, TableNotExistException) -from pypaimon import CatalogFactory from pypaimon.schema.data_types import AtomicType, DataField -from pypaimon import Schema from pypaimon.table.file_store_table import FileStoreTable diff --git a/paimon-python/pypaimon/tests/manifest/__init__.py b/paimon-python/pypaimon/tests/manifest/__init__.py new file mode 100644 index 000000000000..53ed4d36c2ce --- /dev/null +++ b/paimon-python/pypaimon/tests/manifest/__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/manifest/manifest_schema_test.py b/paimon-python/pypaimon/tests/manifest/manifest_schema_test.py new file mode 100644 index 000000000000..13890f783c14 --- /dev/null +++ b/paimon-python/pypaimon/tests/manifest/manifest_schema_test.py @@ -0,0 +1,151 @@ +""" +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.manifest.schema import data_file_meta + +from pypaimon.manifest.schema.data_file_meta import DATA_FILE_META_SCHEMA +from pypaimon.manifest.schema.manifest_file_meta import MANIFEST_FILE_META_SCHEMA +from pypaimon.manifest.schema.simple_stats import ( + KEY_STATS_SCHEMA, + VALUE_STATS_SCHEMA, + PARTITION_STATS_SCHEMA +) + + +class ManifestSchemaTest(unittest.TestCase): + def test_file_source_field_type_and_default(self): + schema = data_file_meta.DATA_FILE_META_SCHEMA + fields = schema.get("fields", []) + file_source_field = next((f for f in fields if f.get("name") == "_FILE_SOURCE"), None) + + self.assertIsNotNone(file_source_field, "_FILE_SOURCE field not found in DATA_FILE_META_SCHEMA") + self.assertEqual(file_source_field.get("type"), ["null", "int"]) + self.assertIsNone(file_source_field.get("default")) + + def test_data_file_meta_schema_structure(self): + """Test that DATA_FILE_META_SCHEMA has the correct structure.""" + self.assertIsInstance(DATA_FILE_META_SCHEMA, dict) + self.assertEqual(DATA_FILE_META_SCHEMA["type"], "record") + self.assertEqual(DATA_FILE_META_SCHEMA["name"], "DataFileMeta") + self.assertIn("fields", DATA_FILE_META_SCHEMA) + + fields = DATA_FILE_META_SCHEMA["fields"] + self.assertIsInstance(fields, list) + + # Create a mapping of field names to field definitions for easier testing + field_map = {field["name"]: field for field in fields} + + # Check that all expected fields are present + expected_fields = [ + "_FILE_NAME", "_FILE_SIZE", "_ROW_COUNT", "_MIN_KEY", "_MAX_KEY", + "_KEY_STATS", "_VALUE_STATS", "_MIN_SEQUENCE_NUMBER", + "_MAX_SEQUENCE_NUMBER", "_SCHEMA_ID", "_LEVEL", "_EXTRA_FILES", + "_CREATION_TIME", "_DELETE_ROW_COUNT", "_EMBEDDED_FILE_INDEX", + "_FILE_SOURCE", "_VALUE_STATS_COLS", "_EXTERNAL_PATH", + "_FIRST_ROW_ID", "_WRITE_COLS" + ] + + for field_name in expected_fields: + self.assertIn(field_name, field_map, f"Field {field_name} is missing") + + # Check specific field types + self.assertEqual(field_map["_FILE_NAME"]["type"], "string") + self.assertEqual(field_map["_FILE_SIZE"]["type"], "long") + self.assertEqual(field_map["_ROW_COUNT"]["type"], "long") + self.assertEqual(field_map["_MIN_KEY"]["type"], "bytes") + self.assertEqual(field_map["_MAX_KEY"]["type"], "bytes") + self.assertEqual(field_map["_KEY_STATS"]["type"], KEY_STATS_SCHEMA) + self.assertEqual(field_map["_VALUE_STATS"]["type"], VALUE_STATS_SCHEMA) + self.assertEqual(field_map["_MIN_SEQUENCE_NUMBER"]["type"], "long") + self.assertEqual(field_map["_MAX_SEQUENCE_NUMBER"]["type"], "long") + self.assertEqual(field_map["_SCHEMA_ID"]["type"], "long") + self.assertEqual(field_map["_LEVEL"]["type"], "int") + self.assertEqual(field_map["_EXTRA_FILES"]["type"], {"type": "array", "items": "string"}) + self.assertEqual(field_map["_CREATION_TIME"]["type"], + ["null", {"type": "long", "logicalType": "timestamp-millis"}]) + self.assertEqual(field_map["_DELETE_ROW_COUNT"]["type"], ["null", "long"]) + self.assertEqual(field_map["_EMBEDDED_FILE_INDEX"]["type"], ["null", "bytes"]) + self.assertEqual(field_map["_FILE_SOURCE"]["type"], ["null", "int"]) + self.assertEqual(field_map["_VALUE_STATS_COLS"]["type"], ["null", {"type": "array", "items": "string"}]) + self.assertEqual(field_map["_EXTERNAL_PATH"]["type"], ["null", "string"]) + self.assertEqual(field_map["_FIRST_ROW_ID"]["type"], ["null", "long"]) + self.assertEqual(field_map["_WRITE_COLS"]["type"], ["null", {"type": "array", "items": "string"}]) + + def test_manifest_file_meta_schema_structure(self): + """Test that MANIFEST_FILE_META_SCHEMA has the correct structure.""" + self.assertIsInstance(MANIFEST_FILE_META_SCHEMA, dict) + self.assertEqual(MANIFEST_FILE_META_SCHEMA["type"], "record") + self.assertEqual(MANIFEST_FILE_META_SCHEMA["name"], "ManifestFileMeta") + self.assertIn("fields", MANIFEST_FILE_META_SCHEMA) + + fields = MANIFEST_FILE_META_SCHEMA["fields"] + self.assertIsInstance(fields, list) + + # Create a mapping of field names to field definitions for easier testing + field_map = {field["name"]: field for field in fields} + + # Check that all expected fields are present + expected_fields = [ + "_VERSION", "_FILE_NAME", "_FILE_SIZE", "_NUM_ADDED_FILES", + "_NUM_DELETED_FILES", "_PARTITION_STATS", "_SCHEMA_ID" + ] + + for field_name in expected_fields: + self.assertIn(field_name, field_map, f"Field {field_name} is missing") + + # Check specific field types + self.assertEqual(field_map["_VERSION"]["type"], "int") + self.assertEqual(field_map["_FILE_NAME"]["type"], "string") + self.assertEqual(field_map["_FILE_SIZE"]["type"], "long") + self.assertEqual(field_map["_NUM_ADDED_FILES"]["type"], "long") + self.assertEqual(field_map["_NUM_DELETED_FILES"]["type"], "long") + self.assertEqual(field_map["_PARTITION_STATS"]["type"], PARTITION_STATS_SCHEMA) + self.assertEqual(field_map["_SCHEMA_ID"]["type"], "long") + + def test_schema_references(self): + """Test that schema references are correctly used.""" + data_file_fields = {field["name"]: field for field in DATA_FILE_META_SCHEMA["fields"]} + manifest_file_fields = {field["name"]: field for field in MANIFEST_FILE_META_SCHEMA["fields"]} + + # Check that _KEY_STATS references KEY_STATS_SCHEMA + key_stats_field = data_file_fields["_KEY_STATS"] + self.assertEqual(key_stats_field["type"], KEY_STATS_SCHEMA) + + # Check that _VALUE_STATS references VALUE_STATS_SCHEMA + value_stats_field = data_file_fields["_VALUE_STATS"] + self.assertEqual(value_stats_field["type"], VALUE_STATS_SCHEMA) + + # Check that _PARTITION_STATS references PARTITION_STATS_SCHEMA + partition_stats_field = manifest_file_fields["_PARTITION_STATS"] + self.assertEqual(partition_stats_field["type"], PARTITION_STATS_SCHEMA) + + def test_schema_consistency(self): + """Test that schema definitions are consistent.""" + # Verify that all stats schemas have the same structure + self.assertEqual(KEY_STATS_SCHEMA["type"], "record") + self.assertEqual(VALUE_STATS_SCHEMA["type"], "record") + self.assertEqual(PARTITION_STATS_SCHEMA["type"], "record") + + # Verify that all stats schemas have different names + names = [ + KEY_STATS_SCHEMA["name"], + VALUE_STATS_SCHEMA["name"], + PARTITION_STATS_SCHEMA["name"] + ] + self.assertEqual(len(names), len(set(names)), "Schema names should be unique") diff --git a/paimon-python/pypaimon/tests/predicates_test.py b/paimon-python/pypaimon/tests/predicates_test.py index 0ddebc34516c..6e6de2fcae1e 100644 --- a/paimon-python/pypaimon/tests/predicates_test.py +++ b/paimon-python/pypaimon/tests/predicates_test.py @@ -23,8 +23,8 @@ import pandas as pd import pyarrow as pa -from pypaimon import CatalogFactory -from pypaimon import Schema +from pypaimon import CatalogFactory, Schema +from pypaimon.table.row.generic_row import GenericRowDeserializer def _check_filtered_result(read_builder, expected_df): @@ -81,14 +81,14 @@ def setUpClass(cls): cls.df = df - def testWrongFieldName(self): + def test_wrong_field_name(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() with self.assertRaises(ValueError) as e: predicate_builder.equal('f2', 'a') self.assertEqual(str(e.exception), "The field f2 is not in field list ['f0', 'f1'].") - def testAppendWithDuplicate(self): + def test_append_with_duplicate(self): pa_schema = pa.schema([ ('f0', pa.int64()), ('f1', pa.string()), @@ -121,7 +121,7 @@ def testAppendWithDuplicate(self): actual_df = read.to_pandas(scan.plan().splits()) self.assertEqual(len(actual_df), 0) - def testAllFieldTypesWithEqual(self): + def test_all_field_types_with_equal(self): pa_schema = pa.schema([ # int ('_tinyint', pa.int8()), @@ -194,169 +194,169 @@ def testAllFieldTypesWithEqual(self): predicate = predicate_builder.equal('_boolean', True) _check_filtered_result(table.new_read_builder().with_filter(predicate), df.loc[[0]]) - def testEqualPk(self): + def test_equal_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.equal('f0', 1) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[[0]]) - def testNotEqualAppend(self): + def test_not_equal_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.not_equal('f0', 1) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[1:4]) - def testNotEqualPk(self): + def test_not_equal_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.not_equal('f0', 1) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[1:4]) - def testLessThanAppend(self): + def test_less_than_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.less_than('f0', 3) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:1]) - def testLessThanPk(self): + def test_less_than_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.less_than('f0', 3) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:1]) - def testLessOrEqualAppend(self): + def test_less_or_equal_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.less_or_equal('f0', 3) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:2]) - def testLessOrEqualPk(self): + def test_less_or_equal_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.less_or_equal('f0', 3) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:2]) - def testGreaterThanAppend(self): + def test_greater_than_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.greater_than('f0', 3) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[3:4]) - def testGreaterThanPk(self): + def test_greater_than_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.greater_than('f0', 3) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[3:4]) - def testGreaterOrEqualAppend(self): + def test_greater_or_equal_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.greater_or_equal('f0', 3) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[2:4]) - def testGreaterOrEqualPk(self): + def test_greater_or_equal_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.greater_or_equal('f0', 3) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[2:4]) - def testIsNullAppend(self): + def test_is_null_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.is_null('f1') _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[[4]]) - def testIsNullPk(self): + def test_is_null_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.is_null('f1') _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[[4]]) - def testIsNotNullAppend(self): + def test_is_not_null_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.is_not_null('f1') _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:3]) - def testIsNotNullPk(self): + def test_is_not_null_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.is_not_null('f1') _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:3]) - def testStartswithAppend(self): + def test_startswith_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.startswith('f1', 'ab') _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:1]) - def testStartswithPk(self): + def test_startswith_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.startswith('f1', 'ab') _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:1]) - def testEndswithAppend(self): + def test_endswith_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.endswith('f1', 'bc') _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:2]) - def testEndswithPk(self): + def test_endswith_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.endswith('f1', 'bc') _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:2]) - def testContainsAppend(self): + def test_contains_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.contains('f1', 'bb') _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[[1]]) - def testContainsPk(self): + def test_contains_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.contains('f1', 'bb') _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[[1]]) - def testIsInAppend(self): + def test_is_in_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.is_in('f0', [1, 2]) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:1]) - def testIsInPk(self): + def test_is_in_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.is_in('f1', ['abc', 'd']) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[[0, 3]]) - def testIsNotInAppend(self): + def test_is_not_in_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.is_not_in('f0', [1, 2]) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[2:4]) - def testIsNotInPk(self): + def test_is_not_in_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.is_not_in('f1', ['abc', 'abbc']) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[2:4]) - def testBetweenAppend(self): + def test_between_append(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.between('f0', 1, 3) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:2]) - def testBetweenPk(self): + def test_between_pk(self): table = self.catalog.get_table('default.test_pk') predicate_builder = table.new_read_builder().new_predicate_builder() predicate = predicate_builder.between('f0', 1, 3) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[0:2]) - def testAndPredicates(self): + def test_and_predicates(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate1 = predicate_builder.greater_than('f0', 1) @@ -364,7 +364,7 @@ def testAndPredicates(self): predicate = predicate_builder.and_predicates([predicate1, predicate2]) _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[[1]]) - def testOrPredicates(self): + def test_or_predicates(self): table = self.catalog.get_table('default.test_append') predicate_builder = table.new_read_builder().new_predicate_builder() predicate1 = predicate_builder.greater_than('f0', 3) @@ -373,7 +373,7 @@ def testOrPredicates(self): _check_filtered_result(table.new_read_builder().with_filter(predicate), self.df.loc[[0, 3, 4]]) - def testPkReaderWithFilter(self): + def test_pk_reader_with_filter(self): pa_schema = pa.schema([ pa.field('key1', pa.int32(), nullable=False), pa.field('key2', pa.string(), nullable=False), @@ -420,7 +420,7 @@ def testPkReaderWithFilter(self): table_commit.close() # test filter by partition - predicate_builder: PredicateBuilder = table.new_read_builder().new_predicate_builder() + predicate_builder = table.new_read_builder().new_predicate_builder() p1 = predicate_builder.startswith('dt1', "p1") p2 = predicate_builder.is_in('dt1', ["p2"]) p3 = predicate_builder.or_predicates([p1, p2]) @@ -434,7 +434,7 @@ def testPkReaderWithFilter(self): self.assertEqual(splits[1].partition.to_dict()["dt2"], 2) # test filter by stats - predicate_builder: PredicateBuilder = table.new_read_builder().new_predicate_builder() + predicate_builder = table.new_read_builder().new_predicate_builder() p1 = predicate_builder.equal('key1', 7) p2 = predicate_builder.is_in('key2', ["e", "f"]) p3 = predicate_builder.or_predicates([p1, p2]) @@ -455,20 +455,26 @@ def testPkReaderWithFilter(self): if split.partition.values == ["p1", 2]: count += 1 self.assertEqual(len(split.files), 1) - min_values = split.files[0].value_stats.min_values.to_dict() - max_values = split.files[0].value_stats.max_values.to_dict() + min_values = GenericRowDeserializer.from_bytes(split.files[0].key_stats.min_values.data, + table.primary_keys_fields).to_dict() + max_values = GenericRowDeserializer.from_bytes(split.files[0].key_stats.max_values.data, + table.primary_keys_fields).to_dict() self.assertTrue(min_values["key1"] == 1 and min_values["key2"] == "e" and max_values["key1"] == 4 and max_values["key2"] == "h") elif split.partition.values == ["p2", 2]: count += 1 - min_values = split.files[0].value_stats.min_values.to_dict() - max_values = split.files[0].value_stats.max_values.to_dict() + min_values = GenericRowDeserializer.from_bytes(split.files[0].key_stats.min_values.data, + table.primary_keys_fields).to_dict() + max_values = GenericRowDeserializer.from_bytes(split.files[0].key_stats.max_values.data, + table.primary_keys_fields).to_dict() self.assertTrue(min_values["key1"] == 5 and min_values["key2"] == "a" and max_values["key1"] == 8 and max_values["key2"] == "d") elif split.partition.values == ["p1", 1]: count += 1 - min_values = split.files[0].value_stats.min_values.to_dict() - max_values = split.files[0].value_stats.max_values.to_dict() + min_values = GenericRowDeserializer.from_bytes(split.files[0].key_stats.min_values.data, + table.primary_keys_fields).to_dict() + max_values = GenericRowDeserializer.from_bytes(split.files[0].key_stats.max_values.data, + table.primary_keys_fields).to_dict() self.assertTrue(min_values["key1"] == max_values["key1"] == 7 and max_values["key2"] == max_values["key2"] == "b") self.assertEqual(count, 3) diff --git a/paimon-python/pypaimon/tests/pvfs_test.py b/paimon-python/pypaimon/tests/pvfs_test.py index 89c96a6228e3..7bebceb96e4b 100644 --- a/paimon-python/pypaimon/tests/pvfs_test.py +++ b/paimon-python/pypaimon/tests/pvfs_test.py @@ -26,9 +26,6 @@ from pypaimon import PaimonVirtualFileSystem from pypaimon.api.api_response import ConfigResponse from pypaimon.api.auth import BearTokenAuthProvider -from pypaimon.catalog.rest.table_metadata import TableMetadata -from pypaimon.schema.data_types import AtomicType, DataField -from pypaimon.schema.table_schema import TableSchema from pypaimon.tests.rest.api_test import RESTCatalogServer @@ -67,17 +64,6 @@ def setUp(self): self.test_databases = { self.database: self.server.mock_database(self.database, {"k1": "v1", "k2": "v2"}), } - data_fields = [ - DataField(0, "id", AtomicType('INT'), 'id'), - DataField(1, "name", AtomicType('STRING'), 'name') - ] - schema = TableSchema(TableSchema.CURRENT_VERSION, len(data_fields), data_fields, len(data_fields), - [], [], {}, "") - self.server.database_store.update(self.test_databases) - self.test_tables = { - f"{self.database}.{self.table}": TableMetadata(uuid=str(uuid.uuid4()), is_external=True, schema=schema), - } - self.server.table_metadata_store.update(self.test_tables) def tearDown(self): if self.temp_path.exists(): @@ -165,7 +151,8 @@ def test_api(self): self.assertEqual(table_virtual_path, self.pvfs.info(table_virtual_path).get('name')) self.assertEqual(True, self.pvfs.exists(database_virtual_path)) user_dirs = self.pvfs.ls(f"pvfs://{self.catalog}/{self.database}/{self.table}", detail=False) - self.assertSetEqual(set(user_dirs), {f'pvfs://{self.catalog}/{self.database}/{self.table}/{data_file_name}'}) + self.assertSetEqual(set(user_dirs), {f'pvfs://{self.catalog}/{self.database}/{self.table}/{data_file_name}', + f'pvfs://{self.catalog}/{self.database}/{self.table}/schema'}) data_file_name = 'data.txt' data_file_path = f'pvfs://{self.catalog}/{self.database}/{self.table}/{data_file_name}' diff --git a/paimon-python/pypaimon/tests/py36/ao_predicate_test.py b/paimon-python/pypaimon/tests/py36/ao_predicate_test.py index 92a3e9601ae9..b06a69baa8fb 100644 --- a/paimon-python/pypaimon/tests/py36/ao_predicate_test.py +++ b/paimon-python/pypaimon/tests/py36/ao_predicate_test.py @@ -22,12 +22,12 @@ import pandas as pd import pyarrow as pa -from pypaimon import CatalogFactory -from pypaimon import Schema -from pypaimon.tests.predicates_test import _random_format, _check_filtered_result +from pypaimon import CatalogFactory, Schema +from pypaimon.tests.predicates_test import (_check_filtered_result, + _random_format) -class PredicatePy36Test(unittest.TestCase): +class AOPredicatePy36Test(unittest.TestCase): @classmethod def setUpClass(cls): diff --git a/paimon-python/pypaimon/tests/py36/ao_simple_test.py b/paimon-python/pypaimon/tests/py36/ao_simple_test.py new file mode 100644 index 000000000000..b14fbcf8dbd7 --- /dev/null +++ b/paimon-python/pypaimon/tests/py36/ao_simple_test.py @@ -0,0 +1,479 @@ +""" +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 unittest.mock import patch + +import pyarrow as pa + +from pypaimon import Schema +from pypaimon.catalog.catalog_exception import (DatabaseAlreadyExistException, + DatabaseNotExistException, + TableAlreadyExistException, + TableNotExistException) +from pypaimon.common.config import OssOptions +from pypaimon.common.file_io import FileIO +from pypaimon.tests.py36.pyarrow_compat import table_sort_by +from pypaimon.tests.rest.rest_base_test import RESTBaseTest + + +class AOSimpleTest(RESTBaseTest): + def setUp(self): + super().setUp() + self.pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('behavior', pa.string()), + ('dt', pa.string()), + ]) + self.data = { + 'user_id': [2, 4, 6, 8, 10], + 'item_id': [1001, 1002, 1003, 1004, 1005], + 'behavior': ['a', 'b', 'c', 'd', 'e'], + 'dt': ['2000-10-10', '2025-08-10', '2025-08-11', '2025-08-12', '2025-08-13'] + } + self.expected = pa.Table.from_pydict(self.data, schema=self.pa_schema) + + def test_with_shard_ao_unaware_bucket(self): + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_with_shard_ao_unaware_bucket', schema, False) + table = self.rest_catalog.get_table('default.test_with_shard_ao_unaware_bucket') + write_builder = table.new_batch_write_builder() + # first write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010, 1011, 1012, 1013, 1014], + 'behavior': ['a', 'b', 'c', None, 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm'], + 'dt': ['p1', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + # second write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data2 = { + 'user_id': [5, 6, 7, 8, 18], + 'item_id': [1005, 1006, 1007, 1008, 1018], + 'behavior': ['e', 'f', 'g', 'h', 'z'], + 'dt': ['p2', 'p1', 'p2', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data2, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + splits = read_builder.new_scan().with_shard(2, 3).plan().splits() + actual = table_sort_by(table_read.to_arrow(splits), 'user_id') + expected = pa.Table.from_pydict({ + 'user_id': [5, 7, 7, 8, 9, 11, 13], + 'item_id': [1005, 1007, 1007, 1008, 1009, 1011, 1013], + 'behavior': ['e', 'f', 'g', 'h', 'h', 'j', 'l'], + 'dt': ['p2', 'p2', 'p2', 'p2', 'p2', 'p2', 'p2'], + }, schema=self.pa_schema) + self.assertEqual(actual, expected) + + # Get the three actual tables + splits1 = read_builder.new_scan().with_shard(0, 3).plan().splits() + actual1 = table_sort_by(table_read.to_arrow(splits1), 'user_id') + splits2 = read_builder.new_scan().with_shard(1, 3).plan().splits() + actual2 = table_sort_by(table_read.to_arrow(splits2), 'user_id') + splits3 = read_builder.new_scan().with_shard(2, 3).plan().splits() + actual3 = table_sort_by(table_read.to_arrow(splits3), 'user_id') + + # Concatenate the three tables + actual = table_sort_by(pa.concat_tables([actual1, actual2, actual3]), 'user_id') + expected = table_sort_by(self._read_test_table(read_builder), 'user_id') + self.assertEqual(actual, expected) + + def test_with_shard_ao_fixed_bucket(self): + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt'], + options={'bucket': '5', 'bucket-key': 'item_id'}) + self.rest_catalog.create_table('default.test_with_slice_ao_fixed_bucket', schema, False) + table = self.rest_catalog.get_table('default.test_with_slice_ao_fixed_bucket') + write_builder = table.new_batch_write_builder() + # first write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010, 1011, 1012, 1013, 1014], + 'behavior': ['a', 'b', 'c', None, 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm'], + 'dt': ['p1', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + # second write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data2 = { + 'user_id': [5, 6, 7, 8], + 'item_id': [1005, 1006, 1007, 1008], + 'behavior': ['e', 'f', 'g', 'h'], + 'dt': ['p2', 'p1', 'p2', 'p2'], + } + pa_table = pa.Table.from_pydict(data2, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + splits = read_builder.new_scan().with_shard(0, 3).plan().splits() + actual = table_sort_by(table_read.to_arrow(splits), 'user_id') + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 3, 5, 8, 12], + 'item_id': [1001, 1002, 1003, 1005, 1008, 1012], + 'behavior': ['a', 'b', 'c', 'd', 'g', 'k'], + 'dt': ['p1', 'p1', 'p2', 'p2', 'p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual, expected) + + # Get the three actual tables + splits1 = read_builder.new_scan().with_shard(0, 3).plan().splits() + actual1 = table_sort_by(table_read.to_arrow(splits1), 'user_id') + splits2 = read_builder.new_scan().with_shard(1, 3).plan().splits() + actual2 = table_sort_by(table_read.to_arrow(splits2), 'user_id') + splits3 = read_builder.new_scan().with_shard(2, 3).plan().splits() + actual3 = table_sort_by(table_read.to_arrow(splits3), 'user_id') + + # Concatenate the three tables + actual = table_sort_by(pa.concat_tables([actual1, actual2, actual3]), 'user_id') + expected = table_sort_by(self._read_test_table(read_builder), 'user_id') + self.assertEqual(actual, expected) + + def test_shard_single_partition(self): + """Test sharding with single partition - tests _filter_by_shard with simple data""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_shard_single_partition', schema, False) + table = self.rest_catalog.get_table('default.test_shard_single_partition') + write_builder = table.new_batch_write_builder() + + # Write data with single partition + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = { + 'user_id': [1, 2, 3, 4, 5, 6], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006], + 'behavior': ['a', 'b', 'c', 'd', 'e', 'f'], + 'dt': ['p1', 'p1', 'p1', 'p1', 'p1', 'p1'], + } + pa_table = pa.Table.from_pydict(data, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + + # Test first shard (0, 2) - should get first 3 rows + splits = read_builder.new_scan().with_shard(0, 2).plan().splits() + actual = table_sort_by(table_read.to_arrow(splits), 'user_id') + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 3], + 'item_id': [1001, 1002, 1003], + 'behavior': ['a', 'b', 'c'], + 'dt': ['p1', 'p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual, expected) + + # Test second shard (1, 2) - should get last 3 rows + splits = read_builder.new_scan().with_shard(1, 2).plan().splits() + actual = table_sort_by(table_read.to_arrow(splits), 'user_id') + expected = pa.Table.from_pydict({ + 'user_id': [4, 5, 6], + 'item_id': [1004, 1005, 1006], + 'behavior': ['d', 'e', 'f'], + 'dt': ['p1', 'p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual, expected) + + def test_shard_uneven_distribution(self): + """Test sharding with uneven row distribution across shards""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_shard_uneven', schema, False) + table = self.rest_catalog.get_table('default.test_shard_uneven') + write_builder = table.new_batch_write_builder() + + # Write data with 7 rows (not evenly divisible by 3) + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = { + 'user_id': [1, 2, 3, 4, 5, 6, 7], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006, 1007], + 'behavior': ['a', 'b', 'c', 'd', 'e', 'f', 'g'], + 'dt': ['p1', 'p1', 'p1', 'p1', 'p1', 'p1', 'p1'], + } + pa_table = pa.Table.from_pydict(data, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + + # Test sharding into 3 parts: 2, 2, 3 rows + splits = read_builder.new_scan().with_shard(0, 3).plan().splits() + actual1 = table_sort_by(table_read.to_arrow(splits), 'user_id') + expected1 = pa.Table.from_pydict({ + 'user_id': [1, 2], + 'item_id': [1001, 1002], + 'behavior': ['a', 'b'], + 'dt': ['p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual1, expected1) + + splits = read_builder.new_scan().with_shard(1, 3).plan().splits() + actual2 = table_sort_by(table_read.to_arrow(splits), 'user_id') + expected2 = pa.Table.from_pydict({ + 'user_id': [3, 4], + 'item_id': [1003, 1004], + 'behavior': ['c', 'd'], + 'dt': ['p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual2, expected2) + + splits = read_builder.new_scan().with_shard(2, 3).plan().splits() + actual3 = table_sort_by(table_read.to_arrow(splits), 'user_id') + expected3 = pa.Table.from_pydict({ + 'user_id': [5, 6, 7], + 'item_id': [1005, 1006, 1007], + 'behavior': ['e', 'f', 'g'], + 'dt': ['p1', 'p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual3, expected3) + + def test_shard_many_small_shards(self): + """Test sharding with many small shards""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_shard_many_small', schema, False) + table = self.rest_catalog.get_table('default.test_shard_many_small') + write_builder = table.new_batch_write_builder() + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = { + 'user_id': [1, 2, 3, 4, 5, 6], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006], + 'behavior': ['a', 'b', 'c', 'd', 'e', 'f'], + 'dt': ['p1', 'p1', 'p1', 'p1', 'p1', 'p1'], + } + pa_table = pa.Table.from_pydict(data, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + + # Test with 6 shards (one row per shard) + for i in range(6): + splits = read_builder.new_scan().with_shard(i, 6).plan().splits() + actual = table_read.to_arrow(splits) + self.assertEqual(len(actual), 1) + self.assertEqual(actual['user_id'][0].as_py(), i + 1) + + def test_shard_boundary_conditions(self): + """Test sharding boundary conditions with edge cases""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_shard_boundary', schema, False) + table = self.rest_catalog.get_table('default.test_shard_boundary') + write_builder = table.new_batch_write_builder() + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = { + 'user_id': [1, 2, 3, 4, 5], + 'item_id': [1001, 1002, 1003, 1004, 1005], + 'behavior': ['a', 'b', 'c', 'd', 'e'], + 'dt': ['p1', 'p1', 'p1', 'p1', 'p1'], + } + pa_table = pa.Table.from_pydict(data, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + + # Test first shard (0, 4) - should get 1 row (5//4 = 1) + splits = read_builder.new_scan().with_shard(0, 4).plan().splits() + actual = table_read.to_arrow(splits) + self.assertEqual(len(actual), 1) + + # Test middle shard (1, 4) - should get 1 row + splits = read_builder.new_scan().with_shard(1, 4).plan().splits() + actual = table_read.to_arrow(splits) + self.assertEqual(len(actual), 1) + + # Test last shard (3, 4) - should get 2 rows (remainder goes to last shard) + splits = read_builder.new_scan().with_shard(3, 4).plan().splits() + actual = table_read.to_arrow(splits) + self.assertEqual(len(actual), 2) + + def test_create_drop_database_table(self): + # test create database + self.rest_catalog.create_database("db1", False) + + with self.assertRaises(DatabaseAlreadyExistException) as context: + self.rest_catalog.create_database("db1", False) + + self.assertEqual("db1", context.exception.database) + + try: + self.rest_catalog.create_database("db1", True) + except DatabaseAlreadyExistException: + self.fail("create_database with ignore_if_exists=True should not raise DatabaseAlreadyExistException") + + # test create table + self.rest_catalog.create_table("db1.tbl1", + Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']), + False) + with self.assertRaises(TableAlreadyExistException) as context: + self.rest_catalog.create_table("db1.tbl1", + Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']), + False) + self.assertEqual("db1.tbl1", context.exception.identifier.get_full_name()) + + try: + self.rest_catalog.create_table("db1.tbl1", + Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']), + True) + except TableAlreadyExistException: + self.fail("create_table with ignore_if_exists=True should not raise TableAlreadyExistException") + + # test drop table + self.rest_catalog.drop_table("db1.tbl1", False) + with self.assertRaises(TableNotExistException) as context: + self.rest_catalog.drop_table("db1.tbl1", False) + self.assertEqual("db1.tbl1", context.exception.identifier.get_full_name()) + + try: + self.rest_catalog.drop_table("db1.tbl1", True) + except TableNotExistException: + self.fail("drop_table with ignore_if_exists=True should not raise TableNotExistException") + + # test drop database + self.rest_catalog.drop_database("db1", False) + with self.assertRaises(DatabaseNotExistException) as context: + self.rest_catalog.drop_database("db1", False) + self.assertEqual("db1", context.exception.database) + + try: + self.rest_catalog.drop_database("db1", True) + except DatabaseNotExistException: + self.fail("drop_database with ignore_if_exists=True should not raise DatabaseNotExistException") + + def test_initialize_oss_fs_pyarrow_lt_7(self): + props = { + OssOptions.OSS_ACCESS_KEY_ID: "AKID", + OssOptions.OSS_ACCESS_KEY_SECRET: "SECRET", + OssOptions.OSS_SECURITY_TOKEN: "TOKEN", + OssOptions.OSS_REGION: "cn-hangzhou", + OssOptions.OSS_ENDPOINT: "oss-cn-hangzhou.aliyuncs.com", + } + + with patch("pypaimon.common.file_io.pyarrow.__version__", "6.0.0"), \ + patch("pyarrow.fs.S3FileSystem") as mock_s3fs: + FileIO("oss://oss-bucket/paimon-database/paimon-table", props) + mock_s3fs.assert_called_once_with(access_key="AKID", + secret_key="SECRET", + session_token="TOKEN", + region="cn-hangzhou", + endpoint_override="oss-bucket." + props[OssOptions.OSS_ENDPOINT]) + FileIO("oss://oss-bucket.endpoint/paimon-database/paimon-table", props) + mock_s3fs.assert_called_with(access_key="AKID", + secret_key="SECRET", + session_token="TOKEN", + region="cn-hangzhou", + endpoint_override="oss-bucket." + props[OssOptions.OSS_ENDPOINT]) + FileIO("oss://access_id:secret_key@Endpoint/oss-bucket/paimon-database/paimon-table", props) + mock_s3fs.assert_called_with(access_key="AKID", + secret_key="SECRET", + session_token="TOKEN", + region="cn-hangzhou", + endpoint_override="oss-bucket." + props[OssOptions.OSS_ENDPOINT]) + + def test_multi_prepare_commit_ao(self): + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_append_only_parquet', schema, False) + table = self.rest_catalog.get_table('default.test_append_only_parquet') + write_builder = table.new_stream_write_builder() + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + # write 1 + data1 = { + 'user_id': [1, 2, 3, 4], + 'item_id': [1001, 1002, 1003, 1004], + 'behavior': ['a', 'b', 'c', None], + 'dt': ['p1', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_write.prepare_commit(0) + # write 2 + data2 = { + 'user_id': [5, 6, 7, 8], + 'item_id': [1005, 1006, 1007, 1008], + 'behavior': ['e', 'f', 'g', 'h'], + 'dt': ['p2', 'p1', 'p2', 'p2'], + } + pa_table = pa.Table.from_pydict(data2, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_write.prepare_commit(1) + # write 3 + data3 = { + 'user_id': [9, 10], + 'item_id': [1009, 1010], + 'behavior': ['i', 'j'], + 'dt': ['p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data3, schema=self.pa_schema) + table_write.write_arrow(pa_table) + cm = table_write.prepare_commit(2) + # commit + table_commit.commit(cm, 2) + table_write.close() + table_commit.close() + self.assertEqual(2, table_write.file_store_write.commit_identifier) + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + splits = read_builder.new_scan().plan().splits() + actual = table_sort_by(table_read.to_arrow(splits), 'user_id') + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 3, 4, 5, 6, 7, 8, 9, 10], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010], + 'behavior': ['a', 'b', 'c', None, 'e', 'f', 'g', 'h', 'i', 'j'], + 'dt': ['p1', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p2', 'p2', 'p1'] + }, schema=self.pa_schema) + self.assertEqual(expected, actual) diff --git a/paimon-python/pypaimon/tests/py36/data_evolution_test.py b/paimon-python/pypaimon/tests/py36/data_evolution_test.py new file mode 100644 index 000000000000..90abd2f9168c --- /dev/null +++ b/paimon-python/pypaimon/tests/py36/data_evolution_test.py @@ -0,0 +1,483 @@ +""" +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 tempfile +import unittest + +import pyarrow as pa +from pypaimon import Schema, CatalogFactory + + +class DataEvolutionTest(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls.tempdir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.tempdir, 'warehouse') + cls.catalog = CatalogFactory.create({ + 'warehouse': cls.warehouse + }) + cls.catalog.create_database('default', False) + + def test_basic(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int8()), + ('f1', pa.int16()), + ]) + schema = Schema.from_pyarrow_schema(simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'}) + self.catalog.create_table('default.test_row_tracking', schema, False) + table = self.catalog.get_table('default.test_row_tracking') + + # write 1 + write_builder = table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + expect_data = pa.Table.from_pydict({ + 'f0': [-1, 2], + 'f1': [-1001, 1002] + }, schema=simple_pa_schema) + table_write.write_arrow(expect_data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # write 2 + table_write = write_builder.new_write().with_write_type(['f0']) + table_commit = write_builder.new_commit() + data2 = pa.Table.from_pydict({ + 'f0': [3, 4], + }, schema=pa.schema([ + ('f0', pa.int8()), + ])) + table_write.write_arrow(data2) + cmts = table_write.prepare_commit() + cmts[0].new_files[0].first_row_id = 0 + table_commit.commit(cmts) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual_data = table_read.to_arrow(table_scan.plan().splits()) + expect_data = pa.Table.from_pydict({ + 'f0': [3, 4], + 'f1': [-1001, 1002] + }, schema=pa.schema([ + ('f0', pa.int8()), + ('f1', pa.int16()), + ])) + self.assertEqual(actual_data, expect_data) + + def test_multiple_appends(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_multiple_appends', schema, False) + table = self.catalog.get_table('default.test_multiple_appends') + + write_builder = table.new_batch_write_builder() + + # write 100 rows: (1, "a", "b") + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + init_data = pa.Table.from_pydict({ + 'f0': [1] * 100, + 'f1': ['a'] * 100, + 'f2': ['b'] * 100, + }, schema=simple_pa_schema) + table_write.write_arrow(init_data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + # append:set first_row_id = 100 to modify the row with columns write + write0 = write_builder.new_write().with_write_type(['f0', 'f1']) + write1 = write_builder.new_write().with_write_type(['f2']) + commit = write_builder.new_commit() + data0 = pa.Table.from_pydict({'f0': [2], 'f1': ['x']}, + schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + data1 = pa.Table.from_pydict({'f2': ['y']}, schema=pa.schema([('f2', pa.string())])) + write0.write_arrow(data0) + write1.write_arrow(data1) + cmts = write0.prepare_commit() + write1.prepare_commit() + for c in cmts: + for nf in c.new_files: + nf.first_row_id = 100 + commit.commit(cmts) + write0.close() + write1.close() + commit.close() + + # append:write (3, "c") and ("d"), set first_row_id = 101 + write0 = write_builder.new_write().with_write_type(['f0', 'f1']) + commit0 = write_builder.new_commit() + data0 = pa.Table.from_pydict({'f0': [3], 'f1': ['c']}, + schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + write0.write_arrow(data0) + cmts0 = write0.prepare_commit() + for c in cmts0: + for nf in c.new_files: + nf.first_row_id = 101 + commit0.commit(cmts0) + write0.close() + commit0.close() + + write1 = write_builder.new_write().with_write_type(['f2']) + commit1 = write_builder.new_commit() + data1 = pa.Table.from_pydict({'f2': ['d']}, schema=pa.schema([('f2', pa.string())])) + write1.write_arrow(data1) + cmts1 = write1.prepare_commit() + for c in cmts1: + for nf in c.new_files: + nf.first_row_id = 101 + commit1.commit(cmts1) + write1.close() + commit1.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + + self.assertEqual(actual.num_rows, 102) + expect = pa.Table.from_pydict({ + 'f0': [1] * 100 + [2] + [3], + 'f1': ['a'] * 100 + ['x'] + ['c'], + 'f2': ['b'] * 100 + ['y'] + ['d'], + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) + + def test_disorder_cols_append(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_disorder_cols_append', schema, False) + table = self.catalog.get_table('default.test_disorder_cols_append') + + write_builder = table.new_batch_write_builder() + num_rows = 100 + # write 1 rows: (1, "a", "b") + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + init_data = pa.Table.from_pydict({ + 'f0': [1] * num_rows, + 'f1': ['a'] * num_rows, + 'f2': ['b'] * num_rows, + }, schema=simple_pa_schema) + table_write.write_arrow(init_data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # append:set first_row_id = 0 to modify the row with columns write + write0 = write_builder.new_write().with_write_type(['f0', 'f2']) + write1 = write_builder.new_write().with_write_type(['f1']) + commit = write_builder.new_commit() + data0 = pa.Table.from_pydict({'f0': [2] * num_rows, 'f2': ['y'] * num_rows}, + schema=pa.schema([('f0', pa.int32()), ('f2', pa.string())])) + data1 = pa.Table.from_pydict({'f1': ['x'] * num_rows}, schema=pa.schema([('f1', pa.string())])) + write0.write_arrow(data0) + write1.write_arrow(data1) + cmts = write0.prepare_commit() + write1.prepare_commit() + for c in cmts: + for nf in c.new_files: + nf.first_row_id = 0 + commit.commit(cmts) + write0.close() + write1.close() + commit.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + + self.assertEqual(actual.num_rows, 100) + expect = pa.Table.from_pydict({ + 'f0': [2] * num_rows, + 'f1': ['x'] * num_rows, + 'f2': ['y'] * num_rows, + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) + + def test_only_some_columns(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_only_some_columns', schema, False) + table = self.catalog.get_table('default.test_only_some_columns') + + write_builder = table.new_batch_write_builder() + + # Commit 1: f0 + w0 = write_builder.new_write().with_write_type(['f0']) + c0 = write_builder.new_commit() + d0 = pa.Table.from_pydict({'f0': [1]}, schema=pa.schema([('f0', pa.int32())])) + w0.write_arrow(d0) + c0.commit(w0.prepare_commit()) + w0.close() + c0.close() + + # Commit 2: f1, first_row_id = 0 + w1 = write_builder.new_write().with_write_type(['f1']) + c1 = write_builder.new_commit() + d1 = pa.Table.from_pydict({'f1': ['a']}, schema=pa.schema([('f1', pa.string())])) + w1.write_arrow(d1) + cmts1 = w1.prepare_commit() + for c in cmts1: + for nf in c.new_files: + nf.first_row_id = 0 + c1.commit(cmts1) + w1.close() + c1.close() + + # Commit 3: f2, first_row_id = 0 + w2 = write_builder.new_write().with_write_type(['f2']) + c2 = write_builder.new_commit() + d2 = pa.Table.from_pydict({'f2': ['b']}, schema=pa.schema([('f2', pa.string())])) + w2.write_arrow(d2) + cmts2 = w2.prepare_commit() + for c in cmts2: + for nf in c.new_files: + nf.first_row_id = 0 + c2.commit(cmts2) + w2.close() + c2.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + + expect = pa.Table.from_pydict({ + 'f0': [1], + 'f1': ['a'], + 'f2': ['b'], + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) + + def test_null_values(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_null_values', schema, False) + table = self.catalog.get_table('default.test_null_values') + + write_builder = table.new_batch_write_builder() + + # Commit 1: some cols are null + w0 = write_builder.new_write().with_write_type(['f0', 'f1']) + w1 = write_builder.new_write().with_write_type(['f2']) + c = write_builder.new_commit() + + d0 = pa.Table.from_pydict({'f0': [1], 'f1': [None]}, + schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + d1 = pa.Table.from_pydict({'f2': [None]}, schema=pa.schema([('f2', pa.string())])) + w0.write_arrow(d0) + w1.write_arrow(d1) + cmts = w0.prepare_commit() + w1.prepare_commit() + for msg in cmts: + for nf in msg.new_files: + nf.first_row_id = 0 + c.commit(cmts) + w0.close() + w1.close() + c.close() + + # Commit 2 + w1 = write_builder.new_write().with_write_type(['f2']) + c1 = write_builder.new_commit() + d1 = pa.Table.from_pydict({'f2': ['c']}, schema=pa.schema([('f2', pa.string())])) + w1.write_arrow(d1) + cmts1 = w1.prepare_commit() + for msg in cmts1: + for nf in msg.new_files: + nf.first_row_id = 0 + c1.commit(cmts1) + w1.close() + c1.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + expect = pa.Table.from_pydict({ + 'f0': [1], + 'f1': [None], + 'f2': ['c'], + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) + + # different first_row_id append multiple times + def test_multiple_appends_different_first_row_ids(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_multiple_appends_diff_rowid', schema, False) + table = self.catalog.get_table('default.test_multiple_appends_diff_rowid') + + write_builder = table.new_batch_write_builder() + + # commit 1 + w0 = write_builder.new_write().with_write_type(['f0', 'f1']) + w1 = write_builder.new_write().with_write_type(['f2']) + c = write_builder.new_commit() + d0 = pa.Table.from_pydict({'f0': [1], 'f1': ['a']}, + schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + d1 = pa.Table.from_pydict({'f2': ['b']}, schema=pa.schema([('f2', pa.string())])) + w0.write_arrow(d0) + w1.write_arrow(d1) + cmts = w0.prepare_commit() + w1.prepare_commit() + for msg in cmts: + for nf in msg.new_files: + nf.first_row_id = 0 + c.commit(cmts) + w0.close() + w1.close() + c.close() + + # commit 2 + w0 = write_builder.new_write().with_write_type(['f0', 'f1']) + c0 = write_builder.new_commit() + d0 = pa.Table.from_pydict({'f0': [2], 'f1': ['c']}, + schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + w0.write_arrow(d0) + cmts0 = w0.prepare_commit() + for msg in cmts0: + for nf in msg.new_files: + nf.first_row_id = 1 + c0.commit(cmts0) + w0.close() + c0.close() + + # commit 3 + w1 = write_builder.new_write().with_write_type(['f2']) + c1 = write_builder.new_commit() + d1 = pa.Table.from_pydict({'f2': ['d']}, schema=pa.schema([('f2', pa.string())])) + w1.write_arrow(d1) + cmts1 = w1.prepare_commit() + for msg in cmts1: + for nf in msg.new_files: + nf.first_row_id = 1 + c1.commit(cmts1) + w1.close() + c1.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + + expect = pa.Table.from_pydict({ + 'f0': [1, 2], + 'f1': ['a', 'c'], + 'f2': ['b', 'd'], + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) + + def test_more_data(self): + simple_pa_schema = pa.schema([ + ('f0', pa.int32()), + ('f1', pa.string()), + ('f2', pa.string()), + ]) + schema = Schema.from_pyarrow_schema( + simple_pa_schema, + options={'row-tracking.enabled': 'true', 'data-evolution.enabled': 'true'} + ) + self.catalog.create_table('default.test_more_data', schema, False) + table = self.catalog.get_table('default.test_more_data') + + write_builder = table.new_batch_write_builder() + + # first commit:100k rows + w0 = write_builder.new_write().with_write_type(['f0', 'f1']) + w1 = write_builder.new_write().with_write_type(['f2']) + c = write_builder.new_commit() + size = 100000 + d0 = pa.Table.from_pydict({ + 'f0': list(range(size)), + 'f1': [f'a{i}' for i in range(size)], + }, schema=pa.schema([('f0', pa.int32()), ('f1', pa.string())])) + d1 = pa.Table.from_pydict({ + 'f2': [f'b{i}' for i in range(size)], + }, schema=pa.schema([('f2', pa.string())])) + w0.write_arrow(d0) + w1.write_arrow(d1) + cmts = w0.prepare_commit() + w1.prepare_commit() + for msg in cmts: + for nf in msg.new_files: + nf.first_row_id = 0 + c.commit(cmts) + w0.close() + w1.close() + c.close() + + # second commit:overwrite f2 to 'c{i}' + w1 = write_builder.new_write().with_write_type(['f2']) + c1 = write_builder.new_commit() + d1 = pa.Table.from_pydict({ + 'f2': [f'c{i}' for i in range(size)], + }, schema=pa.schema([('f2', pa.string())])) + w1.write_arrow(d1) + cmts1 = w1.prepare_commit() + c1.commit(cmts1) + w1.close() + c1.close() + + read_builder = table.new_read_builder() + table_scan = read_builder.new_scan() + table_read = read_builder.new_read() + actual = table_read.to_arrow(table_scan.plan().splits()) + + expect = pa.Table.from_pydict({ + 'f0': list(range(size)), + 'f1': [f'a{i}' for i in range(size)], + 'f2': [f'c{i}' for i in range(size)], + }, schema=simple_pa_schema) + self.assertEqual(actual, expect) diff --git a/paimon-python/pypaimon/tests/py36/reader_predicate_test.py b/paimon-python/pypaimon/tests/py36/reader_predicate_test.py new file mode 100644 index 000000000000..4c52fcdc4152 --- /dev/null +++ b/paimon-python/pypaimon/tests/py36/reader_predicate_test.py @@ -0,0 +1,93 @@ +################################################################################ +# 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 +from pypaimon import Schema +from pypaimon.read import push_down_utils +from pypaimon.read.split import Split + + +class ReaderPredicateTest(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls.tempdir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.tempdir, 'warehouse') + cls.catalog = CatalogFactory.create({ + 'warehouse': cls.warehouse + }) + cls.catalog.create_database('default', False) + + cls.pa_schema = pa.schema([ + ('a', pa.int64()), + ('pt', pa.int64()) + ]) + schema = Schema.from_pyarrow_schema(cls.pa_schema, partition_keys=['pt']) + cls.catalog.create_table('default.test_reader_predicate', schema, False) + cls.table = cls.catalog.get_table('default.test_reader_predicate') + + data1 = pa.Table.from_pydict({ + 'a': [1, 2], + 'pt': [1001, 1002]}, schema=cls.pa_schema) + write_builder = cls.table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + table_write.write_arrow(data1) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + data2 = pa.Table.from_pydict({ + 'a': [3, 4], + 'pt': [1003, 1004]}, schema=cls.pa_schema) + write_builder = cls.table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + table_write.write_arrow(data2) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + @classmethod + def tearDownClass(cls): + shutil.rmtree(cls.tempdir, ignore_errors=True) + + def test_partition_predicate(self): + predicate_builder = self.table.new_read_builder().new_predicate_builder() + predicate = predicate_builder.equal('pt', 1003) + read_builder = self.table.new_read_builder() + read_builder.with_filter(predicate) + splits: list[Split] = read_builder.new_scan().plan().splits() + self.assertEqual(len(splits), 1) + self.assertEqual(splits[0].partition.to_dict().get("pt"), 1003) + + def test_trim_predicate(self): + predicate_builder = self.table.new_read_builder().new_predicate_builder() + p1 = predicate_builder.between('pt', 1002, 1003) + p2 = predicate_builder.and_predicates([predicate_builder.equal('pt', 1003), predicate_builder.equal('a', 3)]) + predicate = predicate_builder.and_predicates([p1, p2]) + pred = push_down_utils.trim_predicate_by_fields(predicate, self.table.partition_keys) + self.assertEqual(len(pred.literals), 2) + self.assertEqual(pred.literals[0].field, 'pt') + self.assertEqual(pred.literals[1].field, 'pt') diff --git a/paimon-python/pypaimon/tests/py36/rest_ao_read_write_test.py b/paimon-python/pypaimon/tests/py36/rest_ao_read_write_test.py index 20e6a2c2d85f..6e6d57f963da 100644 --- a/paimon-python/pypaimon/tests/py36/rest_ao_read_write_test.py +++ b/paimon-python/pypaimon/tests/py36/rest_ao_read_write_test.py @@ -16,34 +16,36 @@ limitations under the License. """ import logging -from datetime import datetime, date +import time +from datetime import date, datetime from decimal import Decimal from unittest.mock import Mock -import pandas as pd import numpy as np +import pandas as pd import pyarrow as pa +from pypaimon import CatalogFactory, Schema from pypaimon.api.options import Options from pypaimon.catalog.catalog_context import CatalogContext -from pypaimon import CatalogFactory from pypaimon.catalog.rest.rest_catalog import RESTCatalog +from pypaimon.common.core_options import CoreOptions from pypaimon.common.identifier import Identifier from pypaimon.manifest.manifest_file_manager import ManifestFileManager from pypaimon.manifest.schema.data_file_meta import DataFileMeta from pypaimon.manifest.schema.manifest_entry import ManifestEntry from pypaimon.manifest.schema.simple_stats import SimpleStats -from pypaimon.schema.data_types import DataField, AtomicType -from pypaimon import Schema -from pypaimon.table.row.generic_row import GenericRow, GenericRowSerializer, GenericRowDeserializer +from pypaimon.schema.data_types import AtomicType, DataField +from pypaimon.snapshot.snapshot_manager import SnapshotManager +from pypaimon.table.row.generic_row import (GenericRow, GenericRowDeserializer, + GenericRowSerializer) from pypaimon.table.row.row_kind import RowKind from pypaimon.tests.py36.pyarrow_compat import table_sort_by from pypaimon.tests.rest.rest_base_test import RESTBaseTest - from pypaimon.write.file_store_commit import FileStoreCommit -class RESTReadWritePy36Test(RESTBaseTest): +class RESTAOReadWritePy36Test(RESTBaseTest): def test_overwrite(self): simple_pa_schema = pa.schema([ @@ -175,12 +177,16 @@ def test_full_data_types(self): self.assertEqual(actual_data, expect_data) # to test GenericRow ability - latest_snapshot = table_scan.snapshot_manager.get_latest_snapshot() - manifest_files = table_scan.manifest_list_manager.read_all(latest_snapshot) - manifest_entries = table_scan.manifest_file_manager.read(manifest_files[0].file_name, - lambda row: table_scan._bucket_filter(row)) - min_value_stats = manifest_entries[0].file.value_stats.min_values.values - max_value_stats = manifest_entries[0].file.value_stats.max_values.values + latest_snapshot = SnapshotManager(table).get_latest_snapshot() + manifest_files = table_scan.starting_scanner.manifest_list_manager.read_all(latest_snapshot) + manifest_entries = table_scan.starting_scanner.manifest_file_manager.read( + manifest_files[0].file_name, + lambda row: table_scan.starting_scanner._filter_manifest_entry(row), + drop_stats=False) + min_value_stats = GenericRowDeserializer.from_bytes(manifest_entries[0].file.value_stats.min_values.data, + table.fields).values + max_value_stats = GenericRowDeserializer.from_bytes(manifest_entries[0].file.value_stats.max_values.data, + table.fields).values expected_min_values = [col[0].as_py() for col in expect_data] expected_max_values = [col[1].as_py() for col in expect_data] self.assertEqual(min_value_stats, expected_min_values) @@ -550,7 +556,7 @@ def test_write_wrong_schema(self): with self.assertRaises(ValueError) as e: table_write.write_arrow_batch(record_batch) - self.assertTrue(str(e.exception).startswith("Input schema isn't consistent with table schema.")) + self.assertTrue(str(e.exception).startswith("Input schema isn't consistent with table schema and write cols.")) def test_write_wide_table_large_data(self): logging.basicConfig(level=logging.INFO) @@ -737,6 +743,33 @@ def test_value_stats_cols_logic(self): test_name="specific_case" ) + def test_incremental_timestamp(self): + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_incremental_parquet', schema, False) + table = self.rest_catalog.get_table('default.test_incremental_parquet') + timestamp = int(time.time() * 1000) + self._write_test_table(table) + + snapshot_manager = SnapshotManager(table) + t1 = snapshot_manager.get_snapshot_by_id(1).time_millis + t2 = snapshot_manager.get_snapshot_by_id(2).time_millis + # test 1 + table = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: str(timestamp - 1) + ',' + str(timestamp)}) + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder) + self.assertEqual(len(actual), 0) + # test 2 + table = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: str(timestamp) + ',' + str(t2)}) + read_builder = table.new_read_builder() + actual = table_sort_by(self._read_test_table(read_builder), 'user_id') + self.assertEqual(self.expected, actual) + # test 3 + table = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: str(t1) + ',' + str(t2)}) + read_builder = table.new_read_builder() + actual = table_sort_by(self._read_test_table(read_builder), 'user_id') + expected = self.expected.slice(4, 4) + self.assertEqual(expected, actual) + def _test_value_stats_cols_case(self, manifest_manager, table, value_stats_cols, expected_fields_count, test_name): """Helper method to test a specific _VALUE_STATS_COLS case.""" @@ -801,7 +834,9 @@ def _test_value_stats_cols_case(self, manifest_manager, table, value_stats_cols, embedded_index=None, file_source=None, value_stats_cols=value_stats_cols, # This is the key field we're testing - external_path=None + external_path=None, + first_row_id=None, + write_cols=None ) # Create ManifestEntry @@ -818,7 +853,7 @@ def _test_value_stats_cols_case(self, manifest_manager, table, value_stats_cols, manifest_manager.write(manifest_file_name, [entry]) # Read the manifest entry back - entries = manifest_manager.read(manifest_file_name) + entries = manifest_manager.read(manifest_file_name, drop_stats=False) # Verify we have exactly one entry self.assertEqual(len(entries), 1) @@ -832,23 +867,27 @@ def _test_value_stats_cols_case(self, manifest_manager, table, value_stats_cols, # Verify value_stats structure based on the logic if value_stats_cols is None: # Should use all table fields - verify we have data for all fields - self.assertEqual(len(read_entry.file.value_stats.min_values.values), expected_fields_count) - self.assertEqual(len(read_entry.file.value_stats.max_values.values), expected_fields_count) + self.assertEqual(read_entry.file.value_stats.min_values.arity, expected_fields_count) + self.assertEqual(read_entry.file.value_stats.min_values.arity, expected_fields_count) self.assertEqual(len(read_entry.file.value_stats.null_counts), expected_fields_count) elif not value_stats_cols: # Empty list # Should use empty fields - verify we have no field data - self.assertEqual(len(read_entry.file.value_stats.min_values.values), 0) - self.assertEqual(len(read_entry.file.value_stats.max_values.values), 0) + self.assertEqual(read_entry.file.value_stats.min_values.arity, 0) + self.assertEqual(read_entry.file.value_stats.max_values.arity, 0) self.assertEqual(len(read_entry.file.value_stats.null_counts), 0) else: # Should use specified fields - verify we have data for specified fields only - self.assertEqual(len(read_entry.file.value_stats.min_values.values), expected_fields_count) - self.assertEqual(len(read_entry.file.value_stats.max_values.values), expected_fields_count) + self.assertEqual(read_entry.file.value_stats.min_values.arity, expected_fields_count) + self.assertEqual(read_entry.file.value_stats.max_values.arity, expected_fields_count) self.assertEqual(len(read_entry.file.value_stats.null_counts), expected_fields_count) # Verify the actual values match what we expect if expected_fields_count > 0: - self.assertEqual(read_entry.file.value_stats.min_values.values, min_values) - self.assertEqual(read_entry.file.value_stats.max_values.values, max_values) + self.assertEqual( + GenericRowDeserializer.from_bytes(read_entry.file.value_stats.min_values.data, test_fields).values, + min_values) + self.assertEqual( + GenericRowDeserializer.from_bytes(read_entry.file.value_stats.max_values.data, test_fields).values, + max_values) self.assertEqual(read_entry.file.value_stats.null_counts, null_counts) diff --git a/paimon-python/pypaimon/tests/reader_append_only_test.py b/paimon-python/pypaimon/tests/reader_append_only_test.py index db0cbcccd1d9..3a9919685454 100644 --- a/paimon-python/pypaimon/tests/reader_append_only_test.py +++ b/paimon-python/pypaimon/tests/reader_append_only_test.py @@ -18,14 +18,16 @@ import os import tempfile +import time import unittest -import pyarrow as pa import numpy as np import pandas as pd +import pyarrow as pa -from pypaimon import CatalogFactory -from pypaimon import Schema +from pypaimon import CatalogFactory, Schema +from pypaimon.common.core_options import CoreOptions +from pypaimon.snapshot.snapshot_manager import SnapshotManager class AoReaderTest(unittest.TestCase): @@ -36,7 +38,7 @@ def setUpClass(cls): cls.catalog = CatalogFactory.create({ 'warehouse': cls.warehouse }) - cls.catalog.create_database('default', False) + cls.catalog.create_database('default', True) cls.pa_schema = pa.schema([ ('user_id', pa.int32()), @@ -277,6 +279,69 @@ def test_ao_reader_with_limit(self): # might be split of "dt=1" or split of "dt=2" self.assertEqual(actual.num_rows, 4) + def test_incremental_timestamp(self): + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_incremental_parquet', schema, False) + table = self.catalog.get_table('default.test_incremental_parquet') + timestamp = int(time.time() * 1000) + self._write_test_table(table) + + snapshot_manager = SnapshotManager(table) + t1 = snapshot_manager.get_snapshot_by_id(1).time_millis + t2 = snapshot_manager.get_snapshot_by_id(2).time_millis + # test 1 + table = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: str(timestamp - 1) + ',' + str(timestamp)}) + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder) + self.assertEqual(len(actual), 0) + # test 2 + table = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: str(timestamp) + ',' + str(t2)}) + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + self.assertEqual(self.expected, actual) + # test 3 + table = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: str(t1) + ',' + str(t2)}) + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + expected = self.expected.slice(4, 4) + self.assertEqual(expected, actual) + + def test_incremental_read_multi_snapshots(self): + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_incremental_100', schema, False) + table = self.catalog.get_table('default.test_incremental_100') + + write_builder = table.new_batch_write_builder() + for i in range(1, 101): + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + pa_table = pa.Table.from_pydict({ + 'user_id': [i], + 'item_id': [1000 + i], + 'behavior': [f'snap{i}'], + 'dt': ['p1' if i % 2 == 1 else 'p2'], + }, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + snapshot_manager = SnapshotManager(table) + t10 = snapshot_manager.get_snapshot_by_id(10).time_millis + t20 = snapshot_manager.get_snapshot_by_id(20).time_millis + + table_inc = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: f"{t10},{t20}"}) + read_builder = table_inc.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + + expected = pa.Table.from_pydict({ + 'user_id': list(range(11, 21)), + 'item_id': [1000 + i for i in range(11, 21)], + 'behavior': [f'snap{i}' for i in range(11, 21)], + 'dt': ['p1' if i % 2 == 1 else 'p2' for i in range(11, 21)], + }, schema=self.pa_schema).sort_by('user_id') + self.assertEqual(expected, actual) + def _write_test_table(self, table): write_builder = table.new_batch_write_builder() diff --git a/paimon-python/pypaimon/tests/reader_base_test.py b/paimon-python/pypaimon/tests/reader_base_test.py index ccf06d55971e..fdcb9f1ea2e0 100644 --- a/paimon-python/pypaimon/tests/reader_base_test.py +++ b/paimon-python/pypaimon/tests/reader_base_test.py @@ -16,29 +16,28 @@ # limitations under the License. ################################################################################ -import os import glob +import os import shutil import tempfile import unittest -from datetime import datetime, date, time +from datetime import date, datetime, time from decimal import Decimal from unittest.mock import Mock import pandas as pd import pyarrow as pa -from pypaimon.table.row.generic_row import GenericRow - -from pypaimon.schema.data_types import (ArrayType, AtomicType, DataField, - MapType, PyarrowFieldParser) -from pypaimon.schema.table_schema import TableSchema -from pypaimon import CatalogFactory -from pypaimon import Schema +from pypaimon import CatalogFactory, Schema from pypaimon.manifest.manifest_file_manager import ManifestFileManager -from pypaimon.manifest.schema.simple_stats import SimpleStats from pypaimon.manifest.schema.data_file_meta import DataFileMeta from pypaimon.manifest.schema.manifest_entry import ManifestEntry +from pypaimon.manifest.schema.simple_stats import SimpleStats +from pypaimon.schema.data_types import (ArrayType, AtomicType, DataField, + MapType, PyarrowFieldParser) +from pypaimon.schema.table_schema import TableSchema +from pypaimon.snapshot.snapshot_manager import SnapshotManager +from pypaimon.table.row.generic_row import GenericRow, GenericRowDeserializer from pypaimon.write.file_store_commit import FileStoreCommit @@ -211,16 +210,26 @@ def test_full_data_types(self): read_builder = table.new_read_builder() table_scan = read_builder.new_scan() table_read = read_builder.new_read() - actual_data = table_read.to_arrow(table_scan.plan().splits()) + splits = table_scan.plan().splits() + + # assert data file without stats + first_file = splits[0].files[0] + self.assertEqual(first_file.value_stats_cols, []) + self.assertEqual(first_file.value_stats, SimpleStats.empty_stats()) + + # assert equal + actual_data = table_read.to_arrow(splits) self.assertEqual(actual_data, expect_data) # to test GenericRow ability - latest_snapshot = table_scan.snapshot_manager.get_latest_snapshot() - manifest_files = table_scan.manifest_list_manager.read_all(latest_snapshot) - manifest_entries = table_scan.manifest_file_manager.read(manifest_files[0].file_name, - lambda row: table_scan._bucket_filter(row)) - min_value_stats = manifest_entries[0].file.value_stats.min_values.values - max_value_stats = manifest_entries[0].file.value_stats.max_values.values + latest_snapshot = SnapshotManager(table).get_latest_snapshot() + manifest_files = table_scan.starting_scanner.manifest_list_manager.read_all(latest_snapshot) + manifest_entries = table_scan.starting_scanner.manifest_file_manager.read( + manifest_files[0].file_name, lambda row: table_scan.starting_scanner._filter_manifest_entry(row), False) + min_value_stats = GenericRowDeserializer.from_bytes(manifest_entries[0].file.value_stats.min_values.data, + table.fields).values + max_value_stats = GenericRowDeserializer.from_bytes(manifest_entries[0].file.value_stats.max_values.data, + table.fields).values expected_min_values = [col[0].as_py() for col in expect_data] expected_max_values = [col[1].as_py() for col in expect_data] self.assertEqual(min_value_stats, expected_min_values) @@ -248,7 +257,7 @@ def test_write_wrong_schema(self): with self.assertRaises(ValueError) as e: table_write.write_arrow_batch(record_batch) - self.assertTrue(str(e.exception).startswith("Input schema isn't consistent with table schema.")) + self.assertTrue(str(e.exception).startswith("Input schema isn't consistent with table schema and write cols.")) def test_reader_iterator(self): read_builder = self.table.new_read_builder() @@ -609,7 +618,9 @@ def _test_value_stats_cols_case(self, manifest_manager, table, value_stats_cols, embedded_index=None, file_source=None, value_stats_cols=value_stats_cols, # This is the key field we're testing - external_path=None + external_path=None, + first_row_id=None, + write_cols=None ) # Create ManifestEntry @@ -626,7 +637,7 @@ def _test_value_stats_cols_case(self, manifest_manager, table, value_stats_cols, manifest_manager.write(manifest_file_name, [entry]) # Read the manifest entry back - entries = manifest_manager.read(manifest_file_name) + entries = manifest_manager.read(manifest_file_name, drop_stats=False) # Verify we have exactly one entry self.assertEqual(len(entries), 1) @@ -640,23 +651,159 @@ def _test_value_stats_cols_case(self, manifest_manager, table, value_stats_cols, # Verify value_stats structure based on the logic if value_stats_cols is None: # Should use all table fields - verify we have data for all fields - self.assertEqual(len(read_entry.file.value_stats.min_values.values), expected_fields_count) - self.assertEqual(len(read_entry.file.value_stats.max_values.values), expected_fields_count) + self.assertEqual(read_entry.file.value_stats.min_values.arity, expected_fields_count) + self.assertEqual(read_entry.file.value_stats.min_values.arity, expected_fields_count) self.assertEqual(len(read_entry.file.value_stats.null_counts), expected_fields_count) elif not value_stats_cols: # Empty list # Should use empty fields - verify we have no field data - self.assertEqual(len(read_entry.file.value_stats.min_values.values), 0) - self.assertEqual(len(read_entry.file.value_stats.max_values.values), 0) + self.assertEqual(read_entry.file.value_stats.min_values.arity, 0) + self.assertEqual(read_entry.file.value_stats.max_values.arity, 0) self.assertEqual(len(read_entry.file.value_stats.null_counts), 0) else: # Should use specified fields - verify we have data for specified fields only - self.assertEqual(len(read_entry.file.value_stats.min_values.values), expected_fields_count) - self.assertEqual(len(read_entry.file.value_stats.max_values.values), expected_fields_count) + self.assertEqual(read_entry.file.value_stats.min_values.arity, expected_fields_count) + self.assertEqual(read_entry.file.value_stats.max_values.arity, expected_fields_count) self.assertEqual(len(read_entry.file.value_stats.null_counts), expected_fields_count) # Verify the actual values match what we expect if expected_fields_count > 0: - self.assertEqual(read_entry.file.value_stats.min_values.values, min_values) - self.assertEqual(read_entry.file.value_stats.max_values.values, max_values) + self.assertEqual( + GenericRowDeserializer.from_bytes(read_entry.file.value_stats.min_values.data, test_fields).values, + min_values) + self.assertEqual( + GenericRowDeserializer.from_bytes(read_entry.file.value_stats.max_values.data, test_fields).values, + max_values) self.assertEqual(read_entry.file.value_stats.null_counts, null_counts) + + def test_split_target_size(self): + """Test source.split.target-size configuration effect on split generation.""" + from pypaimon.common.core_options import CoreOptions + + pa_schema = pa.schema([ + ('f0', pa.int64()), + ('f1', pa.string()) + ]) + + # Test with small target_split_size (512B) - should generate more splits + schema_small = Schema.from_pyarrow_schema( + pa_schema, + options={CoreOptions.SOURCE_SPLIT_TARGET_SIZE: '512b'} + ) + self.catalog.create_table('default.test_split_target_size_small', schema_small, False) + table_small = self.catalog.get_table('default.test_split_target_size_small') + + for i in range(10): + write_builder = table_small.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = pa.Table.from_pydict({ + 'f0': list(range(i * 100, (i + 1) * 100)), + 'f1': [f'value_{j}' for j in range(i * 100, (i + 1) * 100)] + }, schema=pa_schema) + table_write.write_arrow(data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table_small.new_read_builder() + splits_small = read_builder.new_scan().plan().splits() + + schema_default = Schema.from_pyarrow_schema(pa_schema) + self.catalog.create_table('default.test_split_target_size_default', schema_default, False) + table_default = self.catalog.get_table('default.test_split_target_size_default') + + for i in range(10): + write_builder = table_default.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = pa.Table.from_pydict({ + 'f0': list(range(i * 100, (i + 1) * 100)), + 'f1': [f'value_{j}' for j in range(i * 100, (i + 1) * 100)] + }, schema=pa_schema) + table_write.write_arrow(data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # Generate splits with default target_split_size + read_builder = table_default.new_read_builder() + splits_default = read_builder.new_scan().plan().splits() + + self.assertGreater( + len(splits_small), len(splits_default), + f"Small target_split_size should generate more splits. " + f"Got {len(splits_small)} splits with 512B vs " + f"{len(splits_default)} splits with default") + + def test_split_open_file_cost(self): + """Test source.split.open-file-cost configuration effect on split generation.""" + from pypaimon.common.core_options import CoreOptions + + pa_schema = pa.schema([ + ('f0', pa.int64()), + ('f1', pa.string()) + ]) + + # Test with large open_file_cost (64MB) - should generate more splits + schema_large_cost = Schema.from_pyarrow_schema( + pa_schema, + options={ + CoreOptions.SOURCE_SPLIT_TARGET_SIZE: '128mb', + CoreOptions.SOURCE_SPLIT_OPEN_FILE_COST: '64mb' + } + ) + self.catalog.create_table('default.test_split_open_file_cost_large', schema_large_cost, False) + table_large_cost = self.catalog.get_table('default.test_split_open_file_cost_large') + + # Write multiple batches to create multiple files + # Write 10 batches, each with 100 rows + for i in range(10): + write_builder = table_large_cost.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = pa.Table.from_pydict({ + 'f0': list(range(i * 100, (i + 1) * 100)), + 'f1': [f'value_{j}' for j in range(i * 100, (i + 1) * 100)] + }, schema=pa_schema) + table_write.write_arrow(data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # Generate splits with large open_file_cost + read_builder = table_large_cost.new_read_builder() + splits_large_cost = read_builder.new_scan().plan().splits() + + # Test with default open_file_cost (4MB) - should generate fewer splits + schema_default = Schema.from_pyarrow_schema( + pa_schema, + options={CoreOptions.SOURCE_SPLIT_TARGET_SIZE: '128mb'} + ) + self.catalog.create_table('default.test_split_open_file_cost_default', schema_default, False) + table_default = self.catalog.get_table('default.test_split_open_file_cost_default') + + # Write same amount of data + for i in range(10): + write_builder = table_default.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = pa.Table.from_pydict({ + 'f0': list(range(i * 100, (i + 1) * 100)), + 'f1': [f'value_{j}' for j in range(i * 100, (i + 1) * 100)] + }, schema=pa_schema) + table_write.write_arrow(data) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # Generate splits with default open_file_cost + read_builder = table_default.new_read_builder() + splits_default = read_builder.new_scan().plan().splits() + + # With default open_file_cost (4MB), more files can be packed into each split + self.assertGreater( + len(splits_large_cost), len(splits_default), + f"Large open_file_cost should generate more splits. " + f"Got {len(splits_large_cost)} splits with 64MB cost vs " + f"{len(splits_default)} splits with default") diff --git a/paimon-python/pypaimon/tests/reader_predicate_test.py b/paimon-python/pypaimon/tests/reader_predicate_test.py new file mode 100644 index 000000000000..4c52fcdc4152 --- /dev/null +++ b/paimon-python/pypaimon/tests/reader_predicate_test.py @@ -0,0 +1,93 @@ +################################################################################ +# 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 +from pypaimon import Schema +from pypaimon.read import push_down_utils +from pypaimon.read.split import Split + + +class ReaderPredicateTest(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls.tempdir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.tempdir, 'warehouse') + cls.catalog = CatalogFactory.create({ + 'warehouse': cls.warehouse + }) + cls.catalog.create_database('default', False) + + cls.pa_schema = pa.schema([ + ('a', pa.int64()), + ('pt', pa.int64()) + ]) + schema = Schema.from_pyarrow_schema(cls.pa_schema, partition_keys=['pt']) + cls.catalog.create_table('default.test_reader_predicate', schema, False) + cls.table = cls.catalog.get_table('default.test_reader_predicate') + + data1 = pa.Table.from_pydict({ + 'a': [1, 2], + 'pt': [1001, 1002]}, schema=cls.pa_schema) + write_builder = cls.table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + table_write.write_arrow(data1) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + data2 = pa.Table.from_pydict({ + 'a': [3, 4], + 'pt': [1003, 1004]}, schema=cls.pa_schema) + write_builder = cls.table.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + table_write.write_arrow(data2) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + @classmethod + def tearDownClass(cls): + shutil.rmtree(cls.tempdir, ignore_errors=True) + + def test_partition_predicate(self): + predicate_builder = self.table.new_read_builder().new_predicate_builder() + predicate = predicate_builder.equal('pt', 1003) + read_builder = self.table.new_read_builder() + read_builder.with_filter(predicate) + splits: list[Split] = read_builder.new_scan().plan().splits() + self.assertEqual(len(splits), 1) + self.assertEqual(splits[0].partition.to_dict().get("pt"), 1003) + + def test_trim_predicate(self): + predicate_builder = self.table.new_read_builder().new_predicate_builder() + p1 = predicate_builder.between('pt', 1002, 1003) + p2 = predicate_builder.and_predicates([predicate_builder.equal('pt', 1003), predicate_builder.equal('a', 3)]) + predicate = predicate_builder.and_predicates([p1, p2]) + pred = push_down_utils.trim_predicate_by_fields(predicate, self.table.partition_keys) + self.assertEqual(len(pred.literals), 2) + self.assertEqual(pred.literals[0].field, 'pt') + self.assertEqual(pred.literals[1].field, 'pt') diff --git a/paimon-python/pypaimon/tests/reader_primary_key_test.py b/paimon-python/pypaimon/tests/reader_primary_key_test.py index b992595fc9b5..bcbc94bd68e7 100644 --- a/paimon-python/pypaimon/tests/reader_primary_key_test.py +++ b/paimon-python/pypaimon/tests/reader_primary_key_test.py @@ -19,12 +19,14 @@ import os import shutil import tempfile +import time import unittest import pyarrow as pa -from pypaimon import CatalogFactory -from pypaimon import Schema +from pypaimon import CatalogFactory, Schema +from pypaimon.common.core_options import CoreOptions +from pypaimon.snapshot.snapshot_manager import SnapshotManager class PkReaderTest(unittest.TestCase): @@ -184,6 +186,79 @@ def test_pk_reader_with_projection(self): expected = self.expected.select(['dt', 'user_id', 'behavior']) self.assertEqual(actual, expected) + def test_incremental_timestamp(self): + schema = Schema.from_pyarrow_schema(self.pa_schema, + partition_keys=['dt'], + primary_keys=['user_id', 'dt'], + options={'bucket': '2'}) + self.catalog.create_table('default.test_incremental_parquet', schema, False) + table = self.catalog.get_table('default.test_incremental_parquet') + timestamp = int(time.time() * 1000) + self._write_test_table(table) + + snapshot_manager = SnapshotManager(table) + t1 = snapshot_manager.get_snapshot_by_id(1).time_millis + t2 = snapshot_manager.get_snapshot_by_id(2).time_millis + # test 1 + table = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: str(timestamp - 1) + ',' + str(timestamp)}) + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder) + self.assertEqual(len(actual), 0) + # test 2 + table = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: str(timestamp) + ',' + str(t2)}) + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + self.assertEqual(self.expected, actual) + # test 3 + table = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: str(t1) + ',' + str(t2)}) + read_builder = table.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + expected = pa.Table.from_pydict({ + "user_id": [2, 5, 7, 8], + "item_id": [1002, 1005, 1007, 1008], + "behavior": ["b-new", "e", "g", "h"], + "dt": ["p1", "p2", "p1", "p2"] + }, schema=self.pa_schema) + self.assertEqual(expected, actual) + + def test_incremental_read_multi_snapshots(self): + schema = Schema.from_pyarrow_schema(self.pa_schema, + partition_keys=['dt'], + primary_keys=['user_id', 'dt'], + options={'bucket': '2'}) + self.catalog.create_table('default.test_incremental_read_multi_snapshots', schema, False) + table = self.catalog.get_table('default.test_incremental_read_multi_snapshots') + write_builder = table.new_batch_write_builder() + for i in range(1, 101): + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + pa_table = pa.Table.from_pydict({ + 'user_id': [i], + 'item_id': [1000 + i], + 'behavior': [f'snap{i}'], + 'dt': ['p1' if i % 2 == 1 else 'p2'], + }, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + snapshot_manager = SnapshotManager(table) + t10 = snapshot_manager.get_snapshot_by_id(10).time_millis + t20 = snapshot_manager.get_snapshot_by_id(20).time_millis + + table_inc = table.copy({CoreOptions.INCREMENTAL_BETWEEN_TIMESTAMP: f"{t10},{t20}"}) + read_builder = table_inc.new_read_builder() + actual = self._read_test_table(read_builder).sort_by('user_id') + + expected = pa.Table.from_pydict({ + 'user_id': list(range(11, 21)), + 'item_id': [1000 + i for i in range(11, 21)], + 'behavior': [f'snap{i}' for i in range(11, 21)], + 'dt': ['p1' if i % 2 == 1 else 'p2' for i in range(11, 21)], + }, schema=self.pa_schema).sort_by('user_id') + self.assertEqual(expected, actual) + def _write_test_table(self, table): write_builder = table.new_batch_write_builder() diff --git a/paimon-python/pypaimon/tests/rest/rest_base_test.py b/paimon-python/pypaimon/tests/rest/rest_base_test.py index 3a83ccb28592..e45fac5cde61 100644 --- a/paimon-python/pypaimon/tests/rest/rest_base_test.py +++ b/paimon-python/pypaimon/tests/rest/rest_base_test.py @@ -25,17 +25,16 @@ import pyarrow as pa +from pypaimon import CatalogFactory, Schema from pypaimon.api.api_response import ConfigResponse from pypaimon.api.auth import BearTokenAuthProvider from pypaimon.api.options import Options from pypaimon.catalog.catalog_context import CatalogContext -from pypaimon import CatalogFactory from pypaimon.catalog.rest.rest_catalog import RESTCatalog from pypaimon.catalog.rest.table_metadata import TableMetadata from pypaimon.common.identifier import Identifier from pypaimon.schema.data_types import (ArrayType, AtomicType, DataField, MapType) -from pypaimon import Schema from pypaimon.schema.table_schema import TableSchema from pypaimon.tests.rest.rest_server import RESTCatalogServer diff --git a/paimon-python/pypaimon/tests/rest/rest_read_write_test.py b/paimon-python/pypaimon/tests/rest/rest_read_write_test.py index dc6c47e7788f..d05942a256cb 100644 --- a/paimon-python/pypaimon/tests/rest/rest_read_write_test.py +++ b/paimon-python/pypaimon/tests/rest/rest_read_write_test.py @@ -339,7 +339,7 @@ def test_write_wrong_schema(self): with self.assertRaises(ValueError) as e: table_write.write_arrow_batch(record_batch) - self.assertTrue(str(e.exception).startswith("Input schema isn't consistent with table schema.")) + self.assertTrue(str(e.exception).startswith("Input schema isn't consistent with table schema and write cols.")) def test_reader_iterator(self): read_builder = self.table.new_read_builder() diff --git a/paimon-python/pypaimon/tests/rest/rest_server.py b/paimon-python/pypaimon/tests/rest/rest_server.py old mode 100644 new mode 100755 index 7290863eb936..1970b79dd067 --- a/paimon-python/pypaimon/tests/rest/rest_server.py +++ b/paimon-python/pypaimon/tests/rest/rest_server.py @@ -38,7 +38,8 @@ from pypaimon.catalog.catalog_exception import (DatabaseNoPermissionException, DatabaseNotExistException, TableNoPermissionException, - TableNotExistException) + TableNotExistException, DatabaseAlreadyExistException, + TableAlreadyExistException) from pypaimon.catalog.rest.table_metadata import TableMetadata from pypaimon.common.identifier import Identifier from pypaimon.common.json_util import JSON @@ -252,9 +253,9 @@ def _route_request(self, method: str, resource_path: str, parameters: Dict[str, source = self.table_metadata_store.get(source_table.get_full_name()) self.table_metadata_store.update({destination_table.get_full_name(): source}) source_table_dir = (Path(self.data_path) / self.warehouse - / source_table.database_name / source_table.object_name) + / source_table.get_database_name() / source_table.get_object_name()) destination_table_dir = (Path(self.data_path) / self.warehouse - / destination_table.database_name / destination_table.object_name) + / destination_table.get_database_name() / destination_table.get_object_name()) if not source_table_dir.exists(): destination_table_dir.mkdir(parents=True) else: @@ -321,6 +322,16 @@ def _route_request(self, method: str, resource_path: str, parameters: Dict[str, ErrorResponse.RESOURCE_TYPE_TABLE, e.identifier.get_table_name(), str(e), 403 ) return self._mock_response(response, 403) + except DatabaseAlreadyExistException as e: + response = ErrorResponse( + ErrorResponse.RESOURCE_TYPE_DATABASE, e.database, str(e), 409 + ) + return self._mock_response(response, 409) + except TableAlreadyExistException as e: + response = ErrorResponse( + ErrorResponse.RESOURCE_TYPE_TABLE, e.identifier.get_full_name(), str(e), 409 + ) + return self._mock_response(response, 409) except Exception as e: self.logger.error(f"Unexpected error: {e}") response = ErrorResponse(None, None, str(e), 500) @@ -341,7 +352,7 @@ def _handle_table_resource(self, method: str, path_parts: List[str], table_name_part = table_parts[0] branch_part = table_parts[1] # Recreate identifier without branch for lookup - lookup_identifier = Identifier.create(identifier.database_name, table_name_part) + lookup_identifier = Identifier.create(identifier.get_database_name(), table_name_part) else: lookup_identifier = identifier branch_part = None @@ -377,6 +388,8 @@ def _databases_api_handler(self, method: str, data: str, return self._generate_final_list_databases_response(parameters, databases) if method == "POST": create_database = JSON.from_json(data, CreateDatabaseRequest) + if create_database.name in self.database_store: + raise DatabaseAlreadyExistException(create_database.name) self.database_store.update({ create_database.name: self.mock_database(create_database.name, create_database.options) }) @@ -412,13 +425,20 @@ def _tables_handle(self, method: str = None, data: str = None, database_name: st return self._generate_final_list_tables_response(parameters, tables) elif method == "POST": create_table = JSON.from_json(data, CreateTableRequest) + if create_table.identifier.get_full_name() in self.table_metadata_store: + raise TableAlreadyExistException(create_table.identifier) table_metadata = self._create_table_metadata( - create_table.identifier, 1, create_table.schema, str(uuid.uuid4()), False + create_table.identifier, 0, create_table.schema, str(uuid.uuid4()), False ) self.table_metadata_store.update({create_table.identifier.get_full_name(): table_metadata}) - table_dir = Path(self.data_path) / self.warehouse / database_name / create_table.identifier.object_name + table_dir = ( + Path(self.data_path) / self.warehouse / database_name / + create_table.identifier.get_object_name() / 'schema' + ) if not table_dir.exists(): table_dir.mkdir(parents=True) + with open(table_dir / "schema-0", "w") as f: + f.write(JSON.to_json(table_metadata.schema, indent=2)) return self._mock_response("", 200) return self._mock_response(ErrorResponse(None, None, "Method Not Allowed", 405), 405) @@ -428,7 +448,8 @@ def _table_handle(self, method: str, data: str, identifier: Identifier) -> Tuple if identifier.get_full_name() not in self.table_metadata_store: raise TableNotExistException(identifier) table_metadata = self.table_metadata_store[identifier.get_full_name()] - table_path = f'file://{self.data_path}/{self.warehouse}/{identifier.database_name}/{identifier.object_name}' + table_path = (f'file://{self.data_path}/{self.warehouse}/' + f'{identifier.get_database_name()}/{identifier.get_object_name()}') schema = table_metadata.schema.to_schema() response = self.mock_table(identifier, table_metadata, table_path, schema) return self._mock_response(response, 200) @@ -441,7 +462,9 @@ def _table_handle(self, method: str, data: str, identifier: Identifier) -> Tuple elif method == "DELETE": # Drop table - if identifier.get_full_name() in self.table_metadata_store: + if identifier.get_full_name() not in self.table_metadata_store: + raise TableNotExistException(identifier) + else: del self.table_metadata_store[identifier.get_full_name()] if identifier.get_full_name() in self.table_latest_snapshot_store: del self.table_latest_snapshot_store[identifier.get_full_name()] @@ -502,7 +525,8 @@ def _write_snapshot_files(self, identifier: Identifier, snapshot, statistics): import uuid # Construct table path: {warehouse}/{database}/{table} - table_path = os.path.join(self.data_path, self.warehouse, identifier.database_name, identifier.object_name) + table_path = os.path.join(self.data_path, self.warehouse, identifier.get_database_name(), + identifier.get_object_name()) # Create directory structure snapshot_dir = os.path.join(table_path, "snapshot") diff --git a/paimon-python/pypaimon/tests/rest/rest_simple_test.py b/paimon-python/pypaimon/tests/rest/rest_simple_test.py index 95a20345b099..19aec430fd96 100644 --- a/paimon-python/pypaimon/tests/rest/rest_simple_test.py +++ b/paimon-python/pypaimon/tests/rest/rest_simple_test.py @@ -21,10 +21,11 @@ import pyarrow as pa from pypaimon import Schema +from pypaimon.catalog.catalog_exception import DatabaseAlreadyExistException, TableAlreadyExistException, \ + DatabaseNotExistException, TableNotExistException from pypaimon.tests.rest.rest_base_test import RESTBaseTest -from pypaimon.write.row_key_extractor import (DynamicBucketRowKeyExtractor, - FixedBucketRowKeyExtractor, - UnawareBucketRowKeyExtractor) +from pypaimon.write.row_key_extractor import FixedBucketRowKeyExtractor, DynamicBucketRowKeyExtractor, \ + UnawareBucketRowKeyExtractor class RESTSimpleTest(RESTBaseTest): @@ -45,58 +46,475 @@ def setUp(self): self.expected = pa.Table.from_pydict(self.data, schema=self.pa_schema) def test_with_shard_ao_unaware_bucket(self): - schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['user_id']) - self.rest_catalog.create_table('default.test_with_shard', schema, False) - table = self.rest_catalog.get_table('default.test_with_shard') - + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_with_shard_ao_unaware_bucket', schema, False) + table = self.rest_catalog.get_table('default.test_with_shard_ao_unaware_bucket') write_builder = table.new_batch_write_builder() + # first write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010, 1011, 1012, 1013, 1014], + 'behavior': ['a', 'b', 'c', None, 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm'], + 'dt': ['p1', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + # second write table_write = write_builder.new_write() table_commit = write_builder.new_commit() + data2 = { + 'user_id': [5, 6, 7, 8, 18], + 'item_id': [1005, 1006, 1007, 1008, 1018], + 'behavior': ['e', 'f', 'g', 'h', 'z'], + 'dt': ['p2', 'p1', 'p2', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data2, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + splits = read_builder.new_scan().with_shard(2, 3).plan().splits() + actual = table_read.to_arrow(splits).sort_by('user_id') + expected = pa.Table.from_pydict({ + 'user_id': [5, 7, 7, 8, 9, 11, 13], + 'item_id': [1005, 1007, 1007, 1008, 1009, 1011, 1013], + 'behavior': ['e', 'f', 'g', 'h', 'h', 'j', 'l'], + 'dt': ['p2', 'p2', 'p2', 'p2', 'p2', 'p2', 'p2'], + }, schema=self.pa_schema) + self.assertEqual(actual, expected) + + # Get the three actual tables + splits1 = read_builder.new_scan().with_shard(0, 3).plan().splits() + actual1 = table_read.to_arrow(splits1).sort_by('user_id') + splits2 = read_builder.new_scan().with_shard(1, 3).plan().splits() + actual2 = table_read.to_arrow(splits2).sort_by('user_id') + splits3 = read_builder.new_scan().with_shard(2, 3).plan().splits() + actual3 = table_read.to_arrow(splits3).sort_by('user_id') + + # Concatenate the three tables + actual = pa.concat_tables([actual1, actual2, actual3]).sort_by('user_id') + expected = self._read_test_table(read_builder).sort_by('user_id') + self.assertEqual(actual, expected) + + def test_with_shard_ao_unaware_bucket_manual(self): + """Test shard_ao_unaware_bucket with setting bucket -1 manually""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt'], + options={'bucket': '-1'}) + self.rest_catalog.create_table('default.test_with_shard_ao_unaware_bucket_manual', schema, False) + table = self.rest_catalog.get_table('default.test_with_shard_ao_unaware_bucket_manual') + write_builder = table.new_batch_write_builder() + + # Write data with single partition + table_write = write_builder.new_write() self.assertIsInstance(table_write.row_key_extractor, UnawareBucketRowKeyExtractor) - pa_table = pa.Table.from_pydict(self.data, schema=self.pa_schema) + table_commit = write_builder.new_commit() + data = { + 'user_id': [1, 2, 3, 4, 5, 6], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006], + 'behavior': ['a', 'b', 'c', 'd', 'e', 'f'], + 'dt': ['p1', 'p1', 'p1', 'p1', 'p1', 'p1'], + } + pa_table = pa.Table.from_pydict(data, schema=self.pa_schema) table_write.write_arrow(pa_table) table_commit.commit(table_write.prepare_commit()) table_write.close() table_commit.close() - splits = [] read_builder = table.new_read_builder() - splits.extend(read_builder.new_scan().with_shard(0, 3).plan().splits()) - splits.extend(read_builder.new_scan().with_shard(1, 3).plan().splits()) - splits.extend(read_builder.new_scan().with_shard(2, 3).plan().splits()) - table_read = read_builder.new_read() - actual = table_read.to_arrow(splits) - self.assertEqual(actual.sort_by('user_id'), self.expected) + # Test first shard (0, 2) - should get first 3 rows + plan = read_builder.new_scan().with_shard(0, 2).plan() + actual = table_read.to_arrow(plan.splits()).sort_by('user_id') + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 3], + 'item_id': [1001, 1002, 1003], + 'behavior': ['a', 'b', 'c'], + 'dt': ['p1', 'p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual, expected) + + # Test second shard (1, 2) - should get last 3 rows + plan = read_builder.new_scan().with_shard(1, 2).plan() + actual = table_read.to_arrow(plan.splits()).sort_by('user_id') + expected = pa.Table.from_pydict({ + 'user_id': [4, 5, 6], + 'item_id': [1004, 1005, 1006], + 'behavior': ['d', 'e', 'f'], + 'dt': ['p1', 'p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual, expected) def test_with_shard_ao_fixed_bucket(self): - schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['user_id'], + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt'], options={'bucket': '5', 'bucket-key': 'item_id'}) - self.rest_catalog.create_table('default.test_with_shard', schema, False) - table = self.rest_catalog.get_table('default.test_with_shard') + self.rest_catalog.create_table('default.test_with_slice_ao_fixed_bucket', schema, False) + table = self.rest_catalog.get_table('default.test_with_slice_ao_fixed_bucket') + write_builder = table.new_batch_write_builder() + # first write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010, 1011, 1012, 1013, 1014], + 'behavior': ['a', 'b', 'c', None, 'd', 'e', 'f', 'g', 'h', 'i', 'j', 'k', 'l', 'm'], + 'dt': ['p1', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + # second write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data2 = { + 'user_id': [5, 6, 7, 8], + 'item_id': [1005, 1006, 1007, 1008], + 'behavior': ['e', 'f', 'g', 'h'], + 'dt': ['p2', 'p1', 'p2', 'p2'], + } + pa_table = pa.Table.from_pydict(data2, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + splits = read_builder.new_scan().with_shard(0, 3).plan().splits() + actual = table_read.to_arrow(splits).sort_by('user_id') + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 3, 5, 8, 12], + 'item_id': [1001, 1002, 1003, 1005, 1008, 1012], + 'behavior': ['a', 'b', 'c', 'd', 'g', 'k'], + 'dt': ['p1', 'p1', 'p2', 'p2', 'p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual, expected) + + # Get the three actual tables + splits1 = read_builder.new_scan().with_shard(0, 3).plan().splits() + actual1 = table_read.to_arrow(splits1).sort_by('user_id') + splits2 = read_builder.new_scan().with_shard(1, 3).plan().splits() + actual2 = table_read.to_arrow(splits2).sort_by('user_id') + splits3 = read_builder.new_scan().with_shard(2, 3).plan().splits() + actual3 = table_read.to_arrow(splits3).sort_by('user_id') + + # Concatenate the three tables + actual = pa.concat_tables([actual1, actual2, actual3]).sort_by('user_id') + expected = self._read_test_table(read_builder).sort_by('user_id') + self.assertEqual(actual, expected) + + def test_shard_single_partition(self): + """Test sharding with single partition - tests _filter_by_shard with simple data""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_shard_single_partition', schema, False) + table = self.rest_catalog.get_table('default.test_shard_single_partition') write_builder = table.new_batch_write_builder() + + # Write data with single partition table_write = write_builder.new_write() table_commit = write_builder.new_commit() - self.assertIsInstance(table_write.row_key_extractor, FixedBucketRowKeyExtractor) + data = { + 'user_id': [1, 2, 3, 4, 5, 6], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006], + 'behavior': ['a', 'b', 'c', 'd', 'e', 'f'], + 'dt': ['p1', 'p1', 'p1', 'p1', 'p1', 'p1'], + } + pa_table = pa.Table.from_pydict(data, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() - pa_table = pa.Table.from_pydict(self.data, schema=self.pa_schema) + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + + # Test first shard (0, 2) - should get first 3 rows + plan = read_builder.new_scan().with_shard(0, 2).plan() + actual = table_read.to_arrow(plan.splits()).sort_by('user_id') + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 3], + 'item_id': [1001, 1002, 1003], + 'behavior': ['a', 'b', 'c'], + 'dt': ['p1', 'p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual, expected) + + # Test second shard (1, 2) - should get last 3 rows + plan = read_builder.new_scan().with_shard(1, 2).plan() + actual = table_read.to_arrow(plan.splits()).sort_by('user_id') + expected = pa.Table.from_pydict({ + 'user_id': [4, 5, 6], + 'item_id': [1004, 1005, 1006], + 'behavior': ['d', 'e', 'f'], + 'dt': ['p1', 'p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual, expected) + + def test_shard_uneven_distribution(self): + """Test sharding with uneven row distribution across shards""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_shard_uneven', schema, False) + table = self.rest_catalog.get_table('default.test_shard_uneven') + write_builder = table.new_batch_write_builder() + + # Write data with 7 rows (not evenly divisible by 3) + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = { + 'user_id': [1, 2, 3, 4, 5, 6, 7], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006, 1007], + 'behavior': ['a', 'b', 'c', 'd', 'e', 'f', 'g'], + 'dt': ['p1', 'p1', 'p1', 'p1', 'p1', 'p1', 'p1'], + } + pa_table = pa.Table.from_pydict(data, schema=self.pa_schema) table_write.write_arrow(pa_table) table_commit.commit(table_write.prepare_commit()) table_write.close() table_commit.close() - splits = [] read_builder = table.new_read_builder() - splits.extend(read_builder.new_scan().with_shard(0, 3).plan().splits()) - splits.extend(read_builder.new_scan().with_shard(1, 3).plan().splits()) - splits.extend(read_builder.new_scan().with_shard(2, 3).plan().splits()) + table_read = read_builder.new_read() + + # Test sharding into 3 parts: 2, 2, 3 rows + plan1 = read_builder.new_scan().with_shard(0, 3).plan() + actual1 = table_read.to_arrow(plan1.splits()).sort_by('user_id') + expected1 = pa.Table.from_pydict({ + 'user_id': [1, 2], + 'item_id': [1001, 1002], + 'behavior': ['a', 'b'], + 'dt': ['p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual1, expected1) + + plan2 = read_builder.new_scan().with_shard(1, 3).plan() + actual2 = table_read.to_arrow(plan2.splits()).sort_by('user_id') + expected2 = pa.Table.from_pydict({ + 'user_id': [3, 4], + 'item_id': [1003, 1004], + 'behavior': ['c', 'd'], + 'dt': ['p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual2, expected2) + + plan3 = read_builder.new_scan().with_shard(2, 3).plan() + actual3 = table_read.to_arrow(plan3.splits()).sort_by('user_id') + expected3 = pa.Table.from_pydict({ + 'user_id': [5, 6, 7], + 'item_id': [1005, 1006, 1007], + 'behavior': ['e', 'f', 'g'], + 'dt': ['p1', 'p1', 'p1'], + }, schema=self.pa_schema) + self.assertEqual(actual3, expected3) + + def test_shard_single_shard(self): + """Test sharding with only one shard - should return all data""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_shard_single', schema, False) + table = self.rest_catalog.get_table('default.test_shard_single') + write_builder = table.new_batch_write_builder() + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = { + 'user_id': [1, 2, 3, 4], + 'item_id': [1001, 1002, 1003, 1004], + 'behavior': ['a', 'b', 'c', 'd'], + 'dt': ['p1', 'p1', 'p2', 'p2'], + } + pa_table = pa.Table.from_pydict(data, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + read_builder = table.new_read_builder() table_read = read_builder.new_read() - actual = table_read.to_arrow(splits) - self.assertEqual(actual.sort_by("user_id"), self.expected) + + # Test single shard (0, 1) - should get all data + plan = read_builder.new_scan().with_shard(0, 1).plan() + actual = table_read.to_arrow(plan.splits()).sort_by('user_id') + expected = pa.Table.from_pydict(data, schema=self.pa_schema) + self.assertEqual(actual, expected) + + def test_shard_many_small_shards(self): + """Test sharding with many small shards""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_shard_many_small', schema, False) + table = self.rest_catalog.get_table('default.test_shard_many_small') + write_builder = table.new_batch_write_builder() + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = { + 'user_id': [1, 2, 3, 4, 5, 6], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006], + 'behavior': ['a', 'b', 'c', 'd', 'e', 'f'], + 'dt': ['p1', 'p1', 'p1', 'p1', 'p1', 'p1'], + } + pa_table = pa.Table.from_pydict(data, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + + # Test with 6 shards (one row per shard) + for i in range(6): + plan = read_builder.new_scan().with_shard(i, 6).plan() + actual = table_read.to_arrow(plan.splits()) + self.assertEqual(len(actual), 1) + self.assertEqual(actual['user_id'][0].as_py(), i + 1) + + def test_shard_boundary_conditions(self): + """Test sharding boundary conditions with edge cases""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_shard_boundary', schema, False) + table = self.rest_catalog.get_table('default.test_shard_boundary') + write_builder = table.new_batch_write_builder() + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data = { + 'user_id': [1, 2, 3, 4, 5], + 'item_id': [1001, 1002, 1003, 1004, 1005], + 'behavior': ['a', 'b', 'c', 'd', 'e'], + 'dt': ['p1', 'p1', 'p1', 'p1', 'p1'], + } + pa_table = pa.Table.from_pydict(data, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + + # Test first shard (0, 4) - should get 1 row (5//4 = 1) + plan = read_builder.new_scan().with_shard(0, 4).plan() + actual = table_read.to_arrow(plan.splits()) + self.assertEqual(len(actual), 1) + + # Test middle shard (1, 4) - should get 1 row + plan = read_builder.new_scan().with_shard(1, 4).plan() + actual = table_read.to_arrow(plan.splits()) + self.assertEqual(len(actual), 1) + + # Test last shard (3, 4) - should get 2 rows (remainder goes to last shard) + plan = read_builder.new_scan().with_shard(3, 4).plan() + actual = table_read.to_arrow(plan.splits()) + self.assertEqual(len(actual), 2) + + def test_with_shard_large_dataset(self): + """Test with_shard method using 50000 rows of data to verify performance and correctness""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt'], + options={'bucket': '5', 'bucket-key': 'item_id'}) + self.rest_catalog.create_table('default.test_with_shard_large_dataset', schema, False) + table = self.rest_catalog.get_table('default.test_with_shard_large_dataset') + write_builder = table.new_batch_write_builder() + + # Generate 50000 rows of test data + num_rows = 50000 + batch_size = 5000 # Write in batches to avoid memory issues + + for batch_start in range(0, num_rows, batch_size): + batch_end = min(batch_start + batch_size, num_rows) + batch_data = { + 'user_id': list(range(batch_start + 1, batch_end + 1)), + 'item_id': [2000 + i for i in range(batch_start, batch_end)], + 'behavior': [chr(ord('a') + (i % 26)) for i in range(batch_start, batch_end)], + 'dt': [f'p{(i % 5) + 1}' for i in range(batch_start, batch_end)], + } + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + pa_table = pa.Table.from_pydict(batch_data, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + + # Test with 6 shards + num_shards = 6 + shard_results = [] + total_rows_from_shards = 0 + + for shard_idx in range(num_shards): + splits = read_builder.new_scan().with_shard(shard_idx, num_shards).plan().splits() + shard_result = table_read.to_arrow(splits) + shard_results.append(shard_result) + shard_rows = len(shard_result) if shard_result else 0 + total_rows_from_shards += shard_rows + print(f"Shard {shard_idx}/{num_shards}: {shard_rows} rows") + + # Verify that all shards together contain all the data + concatenated_result = pa.concat_tables(shard_results).sort_by('user_id') + + # Read all data without sharding for comparison + all_splits = read_builder.new_scan().plan().splits() + all_data = table_read.to_arrow(all_splits).sort_by('user_id') + + # Verify total row count + self.assertEqual(len(concatenated_result), len(all_data)) + self.assertEqual(len(all_data), num_rows) + self.assertEqual(total_rows_from_shards, num_rows) + + # Verify data integrity - check first and last few rows + self.assertEqual(concatenated_result['user_id'][0].as_py(), 1) + self.assertEqual(concatenated_result['user_id'][-1].as_py(), num_rows) + self.assertEqual(concatenated_result['item_id'][0].as_py(), 2000) + self.assertEqual(concatenated_result['item_id'][-1].as_py(), 2000 + num_rows - 1) + + # Verify that concatenated result equals all data + self.assertEqual(concatenated_result, all_data) + # Test with different shard configurations + # Test with 10 shards + shard_10_results = [] + for shard_idx in range(10): + splits = read_builder.new_scan().with_shard(shard_idx, 10).plan().splits() + shard_result = table_read.to_arrow(splits) + if shard_result: + shard_10_results.append(shard_result) + + if shard_10_results: + concatenated_10_shards = pa.concat_tables(shard_10_results).sort_by('user_id') + self.assertEqual(len(concatenated_10_shards), num_rows) + self.assertEqual(concatenated_10_shards, all_data) + + # Test with single shard (should return all data) + single_shard_splits = read_builder.new_scan().with_shard(0, 1).plan().splits() + single_shard_result = table_read.to_arrow(single_shard_splits).sort_by('user_id') + self.assertEqual(len(single_shard_result), num_rows) + self.assertEqual(single_shard_result, all_data) + + print(f"Successfully tested with_shard method using {num_rows} rows of data") + + def test_shard_parameter_validation(self): + """Test edge cases for parameter validation""" + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.rest_catalog.create_table('default.test_shard_validation_edge', schema, False) + table = self.rest_catalog.get_table('default.test_shard_validation_edge') + + read_builder = table.new_read_builder() + # Test invalid case with number_of_para_subtasks = 1 + with self.assertRaises(Exception) as context: + read_builder.new_scan().with_shard(1, 1).plan() + self.assertEqual(str(context.exception), "idx_of_this_subtask must be less than number_of_para_subtasks") def test_with_shard_pk_dynamic_bucket(self): schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['user_id'], primary_keys=['user_id', 'dt']) @@ -193,3 +611,56 @@ def test_postpone_read_write(self): splits = read_builder.new_scan().plan().splits() actual = table_read.to_arrow(splits) self.assertTrue(not actual) + + def test_create_drop_database_table(self): + # test create database + self.rest_catalog.create_database("db1", False) + + with self.assertRaises(DatabaseAlreadyExistException) as context: + self.rest_catalog.create_database("db1", False) + + self.assertEqual("db1", context.exception.database) + + try: + self.rest_catalog.create_database("db1", True) + except DatabaseAlreadyExistException: + self.fail("create_database with ignore_if_exists=True should not raise DatabaseAlreadyExistException") + + # test create table + self.rest_catalog.create_table("db1.tbl1", + Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']), + False) + with self.assertRaises(TableAlreadyExistException) as context: + self.rest_catalog.create_table("db1.tbl1", + Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']), + False) + self.assertEqual("db1.tbl1", context.exception.identifier.get_full_name()) + + try: + self.rest_catalog.create_table("db1.tbl1", + Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']), + True) + except TableAlreadyExistException: + self.fail("create_table with ignore_if_exists=True should not raise TableAlreadyExistException") + + # test drop table + self.rest_catalog.drop_table("db1.tbl1", False) + with self.assertRaises(TableNotExistException) as context: + self.rest_catalog.drop_table("db1.tbl1", False) + self.assertEqual("db1.tbl1", context.exception.identifier.get_full_name()) + + try: + self.rest_catalog.drop_table("db1.tbl1", True) + except TableNotExistException: + self.fail("drop_table with ignore_if_exists=True should not raise TableNotExistException") + + # test drop database + self.rest_catalog.drop_database("db1", False) + with self.assertRaises(DatabaseNotExistException) as context: + self.rest_catalog.drop_database("db1", False) + self.assertEqual("db1", context.exception.database) + + try: + self.rest_catalog.drop_database("db1", True) + except DatabaseNotExistException: + self.fail("drop_database with ignore_if_exists=True should not raise DatabaseNotExistException") diff --git a/paimon-python/pypaimon/tests/schema_evolution_read_test.py b/paimon-python/pypaimon/tests/schema_evolution_read_test.py new file mode 100644 index 000000000000..2ff4b09e53b7 --- /dev/null +++ b/paimon-python/pypaimon/tests/schema_evolution_read_test.py @@ -0,0 +1,431 @@ +################################################################################ +# 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 pandas +import pyarrow as pa + +from pypaimon import CatalogFactory, Schema + +from pypaimon.schema.schema_manager import SchemaManager +from pypaimon.schema.table_schema import TableSchema + + +class SchemaEvolutionReadTest(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls.tempdir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.tempdir, 'warehouse') + cls.catalog = CatalogFactory.create({ + 'warehouse': cls.warehouse + }) + cls.catalog.create_database('default', False) + + cls.pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('behavior', pa.string()), + ('dt', pa.string()) + ]) + cls.raw_data = { + 'user_id': [1, 2, 3, 4, 5], + 'item_id': [1001, 1002, 1003, 1004, 1005], + 'behavior': ['a', 'b', 'c', None, 'e'], + 'dt': ['p1', 'p1', 'p1', 'p1', 'p2'], + } + cls.expected = pa.Table.from_pydict(cls.raw_data, schema=cls.pa_schema) + + @classmethod + def tearDownClass(cls): + shutil.rmtree(cls.tempdir, ignore_errors=True) + + def test_schema_evolution(self): + # schema 0 + pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('dt', pa.string()) + ]) + schema = Schema.from_pyarrow_schema(pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_sample', schema, False) + table1 = self.catalog.get_table('default.test_sample') + write_builder = table1.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4], + 'item_id': [1001, 1002, 1003, 1004], + 'dt': ['p1', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # schema 1 add behavior column + pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('dt', pa.string()), + ('behavior', pa.string()) + ]) + schema2 = Schema.from_pyarrow_schema(pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_schema_evolution', schema2, False) + table2 = self.catalog.get_table('default.test_schema_evolution') + table2.table_schema.id = 1 + write_builder = table2.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data2 = { + 'user_id': [5, 6, 7, 8], + 'item_id': [1005, 1006, 1007, 1008], + 'dt': ['p2', 'p1', 'p2', 'p2'], + 'behavior': ['e', 'f', 'g', 'h'], + } + pa_table = pa.Table.from_pydict(data2, schema=pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # write schema-0 and schema-1 to table2 + schema_manager = SchemaManager(table2.file_io, table2.table_path) + schema_manager.commit(TableSchema.from_schema(schema_id=0, schema=schema)) + schema_manager.commit(TableSchema.from_schema(schema_id=1, schema=schema2)) + + splits = self._scan_table(table1.new_read_builder()) + read_builder = table2.new_read_builder() + splits2 = self._scan_table(read_builder) + splits.extend(splits2) + + table_read = read_builder.new_read() + actual = table_read.to_arrow(splits) + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 4, 3, 5, 7, 8, 6], + 'item_id': [1001, 1002, 1004, 1003, 1005, 1007, 1008, 1006], + 'dt': ["p1", "p1", "p1", "p2", "p2", "p2", "p2", "p1"], + 'behavior': [None, None, None, None, "e", "g", "h", "f"], + }, schema=pa_schema) + self.assertEqual(expected, actual) + + def test_schema_evolution_type(self): + # schema 0 + pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('time', pa.timestamp('s')), + ('dt', pa.string()) + ]) + schema = Schema.from_pyarrow_schema(pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.schema_evolution_type', schema, False) + table1 = self.catalog.get_table('default.schema_evolution_type') + write_builder = table1.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4], + 'time': [pandas.Timestamp("2025-01-01 00:00:00"), pandas.Timestamp("2025-01-02 00:02:00"), + pandas.Timestamp("2025-01-03 00:03:00"), pandas.Timestamp("2025-01-04 00:04:00")], + 'dt': ['p1', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # schema 1 add behavior column + pa_schema = pa.schema([ + ('user_id', pa.int8()), + ('time', pa.timestamp('ms')), + ('dt', pa.string()), + ('behavior', pa.string()) + ]) + schema2 = Schema.from_pyarrow_schema(pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.schema_evolution_type2', schema2, False) + table2 = self.catalog.get_table('default.schema_evolution_type2') + table2.table_schema.id = 1 + write_builder = table2.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data2 = { + 'user_id': [5, 6, 7, 8], + 'time': [pandas.Timestamp("2025-01-05 00:05:00"), pandas.Timestamp("2025-01-06 00:06:00"), + pandas.Timestamp("2025-01-07 00:07:00"), pandas.Timestamp("2025-01-08 00:08:00")], + 'dt': ['p2', 'p1', 'p2', 'p2'], + 'behavior': ['e', 'f', 'g', 'h'], + } + pa_table = pa.Table.from_pydict(data2, schema=pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # write schema-0 and schema-1 to table2 + schema_manager = SchemaManager(table2.file_io, table2.table_path) + schema_manager.commit(TableSchema.from_schema(schema_id=0, schema=schema)) + schema_manager.commit(TableSchema.from_schema(schema_id=1, schema=schema2)) + + splits = self._scan_table(table1.new_read_builder()) + read_builder = table2.new_read_builder() + splits2 = self._scan_table(read_builder) + splits.extend(splits2) + + table_read = read_builder.new_read() + actual = table_read.to_arrow(splits) + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 4, 3, 5, 7, 8, 6], + 'time': [pandas.Timestamp("2025-01-01 00:00:00"), pandas.Timestamp("2025-01-02 00:02:00"), + pandas.Timestamp("2025-01-04 00:04:00"), pandas.Timestamp("2025-01-03 00:03:00"), + pandas.Timestamp("2025-01-05 00:05:00"), pandas.Timestamp("2025-01-07 00:07:00"), + pandas.Timestamp("2025-01-08 00:08:00"), pandas.Timestamp("2025-01-06 00:06:00"), ], + 'dt': ["p1", "p1", "p1", "p2", "p2", "p2", "p2", "p1"], + 'behavior': [None, None, None, None, "e", "g", "h", "f"], + }, schema=pa_schema) + self.assertEqual(expected, actual) + + def test_schema_evolution_with_scan_filter(self): + # schema 0 + pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('dt', pa.string()) + ]) + schema = Schema.from_pyarrow_schema(pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_schema_evolution1', schema, False) + table1 = self.catalog.get_table('default.test_schema_evolution1') + write_builder = table1.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4], + 'item_id': [1001, 1002, 1003, 1004], + 'dt': ['p1', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # schema 1 add behavior column + pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('behavior', pa.string()), + ('dt', pa.string()) + ]) + schema2 = Schema.from_pyarrow_schema(pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_schema_evolution2', schema2, False) + table2 = self.catalog.get_table('default.test_schema_evolution2') + table2.table_schema.id = 1 + write_builder = table2.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data2 = { + 'user_id': [5, 6, 7, 8], + 'item_id': [1005, 1006, 1007, 1008], + 'behavior': ['e', 'f', 'g', 'h'], + 'dt': ['p2', 'p1', 'p2', 'p2'], + } + pa_table = pa.Table.from_pydict(data2, schema=pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # write schema-0 and schema-1 to table2 + schema_manager = SchemaManager(table2.file_io, table2.table_path) + schema_manager.commit(TableSchema.from_schema(schema_id=0, schema=schema)) + schema_manager.commit(TableSchema.from_schema(schema_id=1, schema=schema2)) + # scan filter for schema evolution + latest_snapshot = table1.new_read_builder().new_scan().starting_scanner.snapshot_manager.get_latest_snapshot() + table2.table_path = table1.table_path + new_read_buidler = table2.new_read_builder() + predicate_builder = new_read_buidler.new_predicate_builder() + predicate = predicate_builder.less_than('user_id', 3) + new_scan = new_read_buidler.with_filter(predicate).new_scan() + manifest_files = new_scan.starting_scanner.manifest_list_manager.read_all(latest_snapshot) + entries = new_scan.starting_scanner.read_manifest_entries(manifest_files) + self.assertEqual(1, len(entries)) # verify scan filter success for schema evolution + + def test_schema_evolution_with_read_filter(self): + # schema 0 + pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('dt', pa.string()) + ]) + schema = Schema.from_pyarrow_schema(pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_schema_evolution_with_filter', schema, False) + table1 = self.catalog.get_table('default.test_schema_evolution_with_filter') + write_builder = table1.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4], + 'item_id': [1001, 1002, 1003, 1004], + 'dt': ['p1', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # schema 1 add behavior column + pa_schema = pa.schema([ + ('user_id', pa.int64()), + ('item_id', pa.int64()), + ('dt', pa.string()), + ('behavior', pa.string()) + ]) + schema2 = Schema.from_pyarrow_schema(pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_schema_evolution_with_filter2', schema2, False) + table2 = self.catalog.get_table('default.test_schema_evolution_with_filter2') + table2.table_schema.id = 1 + write_builder = table2.new_batch_write_builder() + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data2 = { + 'user_id': [5, 6, 7, 8], + 'item_id': [1005, 1006, 1007, 1008], + 'dt': ['p2', 'p1', 'p2', 'p2'], + 'behavior': ['e', 'f', 'g', 'h'], + } + pa_table = pa.Table.from_pydict(data2, schema=pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # write schema-0 and schema-1 to table2 + schema_manager = SchemaManager(table2.file_io, table2.table_path) + schema_manager.commit(TableSchema.from_schema(schema_id=0, schema=schema)) + schema_manager.commit(TableSchema.from_schema(schema_id=1, schema=schema2)) + + # behavior or user_id filter + splits = self._scan_table(table1.new_read_builder()) + read_builder = table2.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + ne_predicate = predicate_builder.equal('behavior', "g") + lt_predicate = predicate_builder.less_than('user_id', 6) + and_predicate = predicate_builder.or_predicates([ne_predicate, lt_predicate]) + splits2 = self._scan_table(read_builder.with_filter(and_predicate)) + for split in splits2: + for file in split.files: + file.schema_id = 1 + splits.extend(splits2) + + table_read = read_builder.new_read() + actual = table_read.to_arrow(splits) + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 4, 3, 5, 7], + 'item_id': [1001, 1002, 1004, 1003, 1005, 1007], + 'dt': ["p1", "p1", "p1", "p2", "p2", "p2"], + 'behavior': [None, None, None, None, "e", "g"], + }, schema=pa_schema) + self.assertEqual(expected, actual) + + # behavior and user_id filter + splits = self._scan_table(table1.new_read_builder()) + + read_builder = table2.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + ne_predicate = predicate_builder.equal('behavior', "g") + lt_predicate = predicate_builder.less_than('user_id', 8) + and_predicate = predicate_builder.and_predicates([ne_predicate, lt_predicate]) + splits2 = self._scan_table(read_builder.with_filter(and_predicate)) + for split in splits2: + for file in split.files: + file.schema_id = 1 + splits.extend(splits2) + + table_read = read_builder.new_read() + actual = table_read.to_arrow(splits) + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 4, 3, 7], + 'item_id': [1001, 1002, 1004, 1003, 1007], + 'dt': ["p1", "p1", "p1", "p2", "p2"], + 'behavior': [None, None, None, None, "g"], + }, schema=pa_schema) + self.assertEqual(expected, actual) + + # user_id filter + splits = self._scan_table(table1.new_read_builder()) + + read_builder = table2.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + predicate = predicate_builder.less_than('user_id', 6) + splits2 = self._scan_table(read_builder.with_filter(predicate)) + self.assertEqual(1, len(splits2)) + for split in splits2: + for file in split.files: + file.schema_id = 1 + splits.extend(splits2) + + table_read = read_builder.new_read() + actual = table_read.to_arrow(splits) + expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 4, 3, 5], + 'item_id': [1001, 1002, 1004, 1003, 1005], + 'dt': ["p1", "p1", "p1", "p2", "p2"], + 'behavior': [None, None, None, None, "e"], + }, schema=pa_schema) + self.assertEqual(expected, actual) + + def _write_test_table(self, table): + write_builder = table.new_batch_write_builder() + + # first write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data1 = { + 'user_id': [1, 2, 3, 4], + 'item_id': [1001, 1002, 1003, 1004], + 'behavior': ['a', 'b', 'c', None], + 'dt': ['p1', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + # second write + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + data2 = { + 'user_id': [5, 6, 7, 8], + 'item_id': [1005, 1006, 1007, 1008], + 'behavior': ['e', 'f', 'g', 'h'], + 'dt': ['p2', 'p1', 'p2', 'p2'], + } + pa_table = pa.Table.from_pydict(data2, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_commit.commit(table_write.prepare_commit()) + table_write.close() + table_commit.close() + + def _scan_table(self, read_builder): + splits = read_builder.new_scan().plan().splits() + return splits diff --git a/paimon-python/pypaimon/tests/serializable_test.py b/paimon-python/pypaimon/tests/serializable_test.py index eed351e33039..20812701204f 100644 --- a/paimon-python/pypaimon/tests/serializable_test.py +++ b/paimon-python/pypaimon/tests/serializable_test.py @@ -54,7 +54,7 @@ def setUpClass(cls): def tearDownClass(cls): shutil.rmtree(cls.tempdir, ignore_errors=True) - def testPickleSerializable(self): + def test_pickle_serializable(self): schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt'], primary_keys=['user_id', 'dt'], diff --git a/paimon-python/pypaimon/tests/uri_reader_factory_test.py b/paimon-python/pypaimon/tests/uri_reader_factory_test.py new file mode 100644 index 000000000000..12088d746c61 --- /dev/null +++ b/paimon-python/pypaimon/tests/uri_reader_factory_test.py @@ -0,0 +1,227 @@ +""" +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 tempfile +import unittest +from pathlib import Path +from pypaimon.common.file_io import FileIO +from pypaimon.common.uri_reader import UriReaderFactory, HttpUriReader, FileUriReader, UriReader + + +class MockFileIO: + """Mock FileIO for testing.""" + + def __init__(self, file_io: FileIO): + self._file_io = file_io + + def get_file_size(self, path: str) -> int: + """Get file size.""" + return self._file_io.get_file_size(Path(path)) + + def new_input_stream(self, path: Path): + """Create new input stream for reading.""" + return self._file_io.new_input_stream(path) + + +class UriReaderFactoryTest(unittest.TestCase): + + def setUp(self): + self.factory = UriReaderFactory({}) + self.temp_dir = tempfile.mkdtemp() + self.temp_file = os.path.join(self.temp_dir, "test.txt") + with open(self.temp_file, 'w') as f: + f.write("test content") + + def tearDown(self): + """Clean up temporary files.""" + try: + if os.path.exists(self.temp_file): + os.remove(self.temp_file) + os.rmdir(self.temp_dir) + except OSError: + pass # Ignore cleanup errors + + def test_create_http_uri_reader(self): + """Test creating HTTP URI reader.""" + reader = self.factory.create("http://example.com/file.txt") + self.assertIsInstance(reader, HttpUriReader) + + def test_create_https_uri_reader(self): + """Test creating HTTPS URI reader.""" + reader = self.factory.create("https://example.com/file.txt") + self.assertIsInstance(reader, HttpUriReader) + + def test_create_file_uri_reader(self): + """Test creating file URI reader.""" + reader = self.factory.create(f"file://{self.temp_file}") + self.assertIsInstance(reader, FileUriReader) + + def test_create_uri_reader_with_authority(self): + """Test creating URI readers with different authorities.""" + reader1 = self.factory.create("http://my_bucket1/path/to/file.txt") + reader2 = self.factory.create("http://my_bucket2/path/to/file.txt") + + # Different authorities should create different readers + self.assertNotEqual(reader1, reader2) + self.assertIsNot(reader1, reader2) + + def test_cached_readers_with_same_scheme_and_authority(self): + """Test that readers with same scheme and authority are cached.""" + reader1 = self.factory.create("http://my_bucket/path/to/file1.txt") + reader2 = self.factory.create("http://my_bucket/path/to/file2.txt") + + # Same scheme and authority should return the same cached reader + self.assertIs(reader1, reader2) + + def test_cached_readers_with_null_authority(self): + """Test that readers with null authority are cached.""" + reader1 = self.factory.create(f"file://{self.temp_file}") + reader2 = self.factory.create(f"file://{self.temp_dir}/another_file.txt") + + # Same scheme with null authority should return the same cached reader + self.assertIs(reader1, reader2) + + def test_create_uri_reader_with_local_path(self): + """Test creating URI reader with local path (no scheme).""" + reader = self.factory.create(self.temp_file) + self.assertIsInstance(reader, FileUriReader) + + def test_cache_size_tracking(self): + """Test that cache size is tracked correctly.""" + initial_size = self.factory.get_cache_size() + + # Create readers with different schemes/authorities + self.factory.create("http://example.com/file.txt") + self.assertEqual(self.factory.get_cache_size(), initial_size + 1) + + self.factory.create("https://example.com/file.txt") + self.assertEqual(self.factory.get_cache_size(), initial_size + 2) + + self.factory.create(f"file://{self.temp_file}") + self.assertEqual(self.factory.get_cache_size(), initial_size + 3) + + # Same scheme/authority should not increase cache size + self.factory.create("http://example.com/another_file.txt") + self.assertEqual(self.factory.get_cache_size(), initial_size + 3) + + def test_uri_reader_functionality(self): + """Test that created URI readers actually work.""" + # Test file URI reader + reader = self.factory.create(f"file://{self.temp_file}") + stream = reader.new_input_stream(self.temp_file) + content = stream.read().decode('utf-8') + self.assertEqual(content, "test content") + stream.close() + + def test_invalid_uri_handling(self): + """Test handling of invalid URIs.""" + # This should not raise an exception as urlparse is quite permissive + # But we can test edge cases + reader = self.factory.create("") + self.assertIsInstance(reader, (HttpUriReader, FileUriReader)) + + def test_uri_key_equality(self): + """Test UriKey equality and hashing behavior.""" + from pypaimon.common.uri_reader import UriKey + + key1 = UriKey("http", "example.com") + key2 = UriKey("http", "example.com") + key3 = UriKey("https", "example.com") + key4 = UriKey("http", "other.com") + + # Same scheme and authority should be equal + self.assertEqual(key1, key2) + self.assertEqual(hash(key1), hash(key2)) + + # Different scheme or authority should not be equal + self.assertNotEqual(key1, key3) + self.assertNotEqual(key1, key4) + + # Test with None values + key_none1 = UriKey(None, None) + key_none2 = UriKey(None, None) + self.assertEqual(key_none1, key_none2) + + def test_uri_key_string_representation(self): + """Test UriKey string representation.""" + from pypaimon.common.uri_reader import UriKey + + key = UriKey("http", "example.com") + repr_str = repr(key) + self.assertIn("http", repr_str) + self.assertIn("example.com", repr_str) + + def test_thread_safety_simulation(self): + """Test thread safety by creating multiple readers concurrently.""" + import threading + import time + + results = [] + + def create_reader(): + reader = self.factory.create("http://example.com/file.txt") + results.append(reader) + time.sleep(0.01) # Small delay to increase chance of race conditions + + # Create multiple threads + threads = [] + for _ in range(10): + thread = threading.Thread(target=create_reader) + threads.append(thread) + thread.start() + + # Wait for all threads to complete + for thread in threads: + thread.join() + + # All results should be the same cached reader + first_reader = results[0] + for reader in results[1:]: + self.assertIs(reader, first_reader) + + def test_different_file_schemes(self): + """Test different file-based schemes.""" + # Test absolute path without scheme + reader1 = self.factory.create(os.path.abspath(self.temp_file)) + self.assertIsInstance(reader1, FileUriReader) + + # Test file:// scheme + reader2 = self.factory.create(f"file://{self.temp_file}") + self.assertIsInstance(reader2, FileUriReader) + + # Different schemes (empty vs "file") should create different cache entries + self.assertIsNot(reader1, reader2) + + # But same scheme should be cached + reader3 = self.factory.create(f"file://{self.temp_dir}/another_file.txt") + self.assertIs(reader2, reader3) # Same file:// scheme + + def test_get_file_path_with_file_uri(self): + file_uri = f"file://{self.temp_file}" + path = UriReader.get_file_path(file_uri) + self.assertEqual(str(path), self.temp_file) + oss_file_path = "bucket/tmp/another_file.txt" + file_uri = f"oss://{oss_file_path}" + path = UriReader.get_file_path(file_uri) + self.assertEqual(str(path), oss_file_path) + path = UriReader.get_file_path(self.temp_file) + self.assertEqual(str(path), self.temp_file) + + +if __name__ == '__main__': + unittest.main() diff --git a/paimon-python/pypaimon/tests/write/table_write_test.py b/paimon-python/pypaimon/tests/write/table_write_test.py new file mode 100644 index 000000000000..21b76731acd0 --- /dev/null +++ b/paimon-python/pypaimon/tests/write/table_write_test.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. +""" + +import os +import shutil + +import tempfile +import unittest + +from pypaimon import CatalogFactory, Schema +import pyarrow as pa + + +class TableWriteTest(unittest.TestCase): + @classmethod + def setUpClass(cls): + cls.tempdir = tempfile.mkdtemp() + cls.warehouse = os.path.join(cls.tempdir, 'warehouse') + cls.catalog = CatalogFactory.create({ + 'warehouse': cls.warehouse + }) + cls.catalog.create_database('default', True) + + cls.pa_schema = pa.schema([ + ('user_id', pa.int32()), + ('item_id', pa.int64()), + ('behavior', pa.string()), + ('dt', pa.string()) + ]) + cls.expected = pa.Table.from_pydict({ + 'user_id': [1, 2, 3, 4, 5, 6, 7, 8, 9, 10], + 'item_id': [1001, 1002, 1003, 1004, 1005, 1006, 1007, 1008, 1009, 1010], + 'behavior': ['a', 'b', 'c', None, 'e', 'f', 'g', 'h', 'i', 'j'], + 'dt': ['p1', 'p1', 'p2', 'p1', 'p2', 'p1', 'p2', 'p2', 'p2', 'p1'] + }, schema=cls.pa_schema) + + @classmethod + def tearDownClass(cls): + shutil.rmtree(cls.tempdir, ignore_errors=True) + + def test_multi_prepare_commit_ao(self): + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt']) + self.catalog.create_table('default.test_append_only_parquet', schema, False) + table = self.catalog.get_table('default.test_append_only_parquet') + write_builder = table.new_stream_write_builder() + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + # write 1 + data1 = { + 'user_id': [1, 2, 3, 4], + 'item_id': [1001, 1002, 1003, 1004], + 'behavior': ['a', 'b', 'c', None], + 'dt': ['p1', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_write.prepare_commit(0) + # write 2 + data2 = { + 'user_id': [5, 6, 7, 8], + 'item_id': [1005, 1006, 1007, 1008], + 'behavior': ['e', 'f', 'g', 'h'], + 'dt': ['p2', 'p1', 'p2', 'p2'], + } + pa_table = pa.Table.from_pydict(data2, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_write.prepare_commit(1) + # write 3 + data3 = { + 'user_id': [9, 10], + 'item_id': [1009, 1010], + 'behavior': ['i', 'j'], + 'dt': ['p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data3, schema=self.pa_schema) + table_write.write_arrow(pa_table) + cm = table_write.prepare_commit(2) + # commit + table_commit.commit(cm, 2) + table_write.close() + table_commit.close() + self.assertEqual(2, table_write.file_store_write.commit_identifier) + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + splits = read_builder.new_scan().plan().splits() + actual = table_read.to_arrow(splits).sort_by('user_id') + self.assertEqual(self.expected, actual) + + def test_multi_prepare_commit_pk(self): + schema = Schema.from_pyarrow_schema(self.pa_schema, partition_keys=['dt'], primary_keys=['user_id', 'dt'], + options={'bucket': '2'}) + self.catalog.create_table('default.test_primary_key_parquet', schema, False) + table = self.catalog.get_table('default.test_primary_key_parquet') + write_builder = table.new_stream_write_builder() + + table_write = write_builder.new_write() + table_commit = write_builder.new_commit() + # write 1 + data1 = { + 'user_id': [1, 2, 3, 4], + 'item_id': [1001, 1002, 1003, 1004], + 'behavior': ['a', 'b', 'c', None], + 'dt': ['p1', 'p1', 'p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data1, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_write.prepare_commit(0) + # write 2 + data2 = { + 'user_id': [5, 6, 7, 8], + 'item_id': [1005, 1006, 1007, 1008], + 'behavior': ['e', 'f', 'g', 'h'], + 'dt': ['p2', 'p1', 'p2', 'p2'], + } + pa_table = pa.Table.from_pydict(data2, schema=self.pa_schema) + table_write.write_arrow(pa_table) + table_write.prepare_commit(1) + # write 3 + data3 = { + 'user_id': [9, 10], + 'item_id': [1009, 1010], + 'behavior': ['i', 'j'], + 'dt': ['p2', 'p1'], + } + pa_table = pa.Table.from_pydict(data3, schema=self.pa_schema) + table_write.write_arrow(pa_table) + cm = table_write.prepare_commit(2) + # commit + table_commit.commit(cm, 2) + table_write.close() + table_commit.close() + self.assertEqual(2, table_write.file_store_write.commit_identifier) + + read_builder = table.new_read_builder() + table_read = read_builder.new_read() + splits = read_builder.new_scan().plan().splits() + actual = table_read.to_arrow(splits).sort_by('user_id') + self.assertEqual(self.expected, actual) diff --git a/paimon-python/pypaimon/write/blob_format_writer.py b/paimon-python/pypaimon/write/blob_format_writer.py new file mode 100644 index 000000000000..07b29bbc4e46 --- /dev/null +++ b/paimon-python/pypaimon/write/blob_format_writer.py @@ -0,0 +1,107 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import struct +import zlib +from typing import BinaryIO, List + +from pypaimon.table.row.blob import Blob, BlobData +from pypaimon.common.delta_varint_compressor import DeltaVarintCompressor + + +class BlobFormatWriter: + VERSION = 1 + MAGIC_NUMBER = 1481511375 + BUFFER_SIZE = 4096 + METADATA_SIZE = 12 # 8-byte length + 4-byte CRC + + def __init__(self, output_stream: BinaryIO): + self.output_stream = output_stream + self.lengths: List[int] = [] + self.position = 0 + + def add_element(self, row) -> None: + if not hasattr(row, 'values') or len(row.values) != 1: + raise ValueError("BlobFormatWriter only supports one field") + + blob_value = row.values[0] + if blob_value is None: + raise ValueError("BlobFormatWriter only supports non-null blob") + + if not isinstance(blob_value, Blob): + raise ValueError("Field must be Blob/BlobData instance") + + previous_pos = self.position + crc32 = 0 # Initialize CRC32 + + # Write magic number + magic_bytes = struct.pack(' int: + crc32 = zlib.crc32(data, crc32) + self.output_stream.write(data) + self.position += len(data) + return crc32 + + def reach_target_size(self, suggested_check: bool, target_size: int) -> bool: + return self.position >= target_size + + def close(self) -> None: + index_bytes = DeltaVarintCompressor.compress(self.lengths) + self.output_stream.write(index_bytes) + + # Write header (index length + version) + header = struct.pack(' 0: - predicate_builder = PredicateBuilder(self.table.table_schema.get_partition_key_fields()) + predicate_builder = PredicateBuilder(self.table.partition_keys_fields) sub_predicates = [] for key, value in overwrite_partition.items(): sub_predicates.append(predicate_builder.equal(key, value)) @@ -101,12 +102,12 @@ def overwrite(self, overwrite_partition, commit_messages: List[CommitMessage], c f"in {msg.partition} does not belong to this partition") commit_entries = [] - current_entries = TableScan(self.table, partition_filter, None, []).plan().files() + current_entries = FullStartingScanner(self.table, partition_filter, None).plan_files() for entry in current_entries: entry.kind = 1 commit_entries.append(entry) for msg in commit_messages: - partition = GenericRow(list(msg.partition), self.table.table_schema.get_partition_key_fields()) + partition = GenericRow(list(msg.partition), self.table.partition_keys_fields) for file in msg.new_files: commit_entries.append(ManifestEntry( kind=0, @@ -130,6 +131,24 @@ def _try_commit(self, commit_kind, commit_entries, commit_identifier): added_file_count = 0 deleted_file_count = 0 delta_record_count = 0 + # process snapshot + new_snapshot_id = self._generate_snapshot_id() + + # Check if row tracking is enabled + row_tracking_enabled = self.table.options.get(CoreOptions.ROW_TRACKING_ENABLED, 'false').lower() == 'true' + + # Apply row tracking logic if enabled + next_row_id = None + if row_tracking_enabled: + # Assign snapshot ID to delta files + commit_entries = self._assign_snapshot_id(new_snapshot_id, commit_entries) + + # Get the next row ID start from the latest snapshot + first_row_id_start = self._get_next_row_id_start() + + # Assign row IDs to new files and get the next row ID for the snapshot + commit_entries, next_row_id = self._assign_row_tracking_meta(first_row_id_start, commit_entries) + for entry in commit_entries: if entry.kind == 0: added_file_count += 1 @@ -155,11 +174,11 @@ def _try_commit(self, commit_kind, commit_entries, commit_identifier): partition_stats=SimpleStats( min_values=GenericRow( values=partition_min_stats, - fields=self.table.table_schema.get_partition_key_fields(), + fields=self.table.partition_keys_fields ), max_values=GenericRow( values=partition_max_stats, - fields=self.table.table_schema.get_partition_key_fields(), + fields=self.table.partition_keys_fields ), null_counts=partition_null_counts, ), @@ -194,6 +213,7 @@ def _try_commit(self, commit_kind, commit_entries, commit_identifier): commit_identifier=commit_identifier, commit_kind=commit_kind, time_millis=int(time.time() * 1000), + next_row_id=next_row_id, ) # Generate partition statistics for the commit @@ -314,3 +334,59 @@ def _generate_partition_statistics(self, commit_entries: List[ManifestEntry]) -> ) for stats in partition_stats.values() ] + + def _assign_snapshot_id(self, snapshot_id: int, commit_entries: List[ManifestEntry]) -> List[ManifestEntry]: + """Assign snapshot ID to all commit entries.""" + return [entry.assign_sequence_number(snapshot_id, snapshot_id) for entry in commit_entries] + + def _get_next_row_id_start(self) -> int: + """Get the next row ID start from the latest snapshot.""" + latest_snapshot = self.snapshot_manager.get_latest_snapshot() + if latest_snapshot and hasattr(latest_snapshot, 'next_row_id') and latest_snapshot.next_row_id is not None: + return latest_snapshot.next_row_id + return 0 + + def _assign_row_tracking_meta(self, first_row_id_start: int, commit_entries: List[ManifestEntry]): + """ + Assign row tracking metadata (first_row_id) to new files. + This follows the Java implementation logic from FileStoreCommitImpl.assignRowTrackingMeta. + """ + if not commit_entries: + return commit_entries, first_row_id_start + + row_id_assigned = [] + start = first_row_id_start + blob_start = first_row_id_start + + for entry in commit_entries: + # Check if this is an append file that needs row ID assignment + if (entry.kind == 0 and # ADD kind + entry.file.file_source == 0 and # APPEND file source + entry.file.first_row_id is None): # No existing first_row_id + + if self._is_blob_file(entry.file.file_name): + # Handle blob files specially + if blob_start >= start: + raise RuntimeError( + f"This is a bug, blobStart {blob_start} should be less than start {start} " + f"when assigning a blob entry file." + ) + row_count = entry.file.row_count + row_id_assigned.append(entry.assign_first_row_id(blob_start)) + blob_start += row_count + else: + # Handle regular files + row_count = entry.file.row_count + row_id_assigned.append(entry.assign_first_row_id(start)) + blob_start = start + start += row_count + else: + # For compact files or files that already have first_row_id, don't assign + row_id_assigned.append(entry) + + return row_id_assigned, start + + @staticmethod + def _is_blob_file(file_name: str) -> bool: + """Check if a file is a blob file based on its extension.""" + return file_name.endswith('.blob') diff --git a/paimon-python/pypaimon/write/file_store_write.py b/paimon-python/pypaimon/write/file_store_write.py index bcef10a4c346..c100b64966d8 100644 --- a/paimon-python/pypaimon/write/file_store_write.py +++ b/paimon-python/pypaimon/write/file_store_write.py @@ -21,8 +21,10 @@ from pypaimon.write.commit_message import CommitMessage 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 from pypaimon.write.writer.key_value_data_writer import KeyValueDataWriter +from pypaimon.table.bucket_mode import BucketMode class FileStoreWrite: @@ -33,7 +35,9 @@ def __init__(self, table): self.table: FileStoreTable = table self.data_writers: Dict[Tuple, DataWriter] = {} - self.max_seq_numbers = self._seq_number_stats() # TODO: build this on-demand instead of on all + self.max_seq_numbers: dict = {} + self.write_cols = None + self.commit_identifier = 0 def write(self, partition: Tuple, bucket: int, data: pa.RecordBatch): key = (partition, bucket) @@ -43,22 +47,46 @@ def write(self, partition: Tuple, bucket: int, data: pa.RecordBatch): writer.write(data) def _create_data_writer(self, partition: Tuple, bucket: int) -> DataWriter: - if self.table.is_primary_key_table: - return KeyValueDataWriter( + def max_seq_number(): + return self._seq_number_stats(partition).get(bucket, 1) + + # Check if table has blob columns + if self._has_blob_columns(): + return DataBlobWriter( table=self.table, partition=partition, bucket=bucket, - max_seq_number=self.max_seq_numbers.get((partition, bucket), 1), + max_seq_number=0, ) + elif self.table.is_primary_key_table: + return KeyValueDataWriter( + table=self.table, + partition=partition, + bucket=bucket, + max_seq_number=max_seq_number()) else: + seq_number = 0 if self.table.bucket_mode() == BucketMode.BUCKET_UNAWARE else max_seq_number() return AppendOnlyDataWriter( table=self.table, partition=partition, bucket=bucket, - max_seq_number=self.max_seq_numbers.get((partition, bucket), 1), + max_seq_number=seq_number, + write_cols=self.write_cols ) - def prepare_commit(self) -> List[CommitMessage]: + def _has_blob_columns(self) -> bool: + """Check if the table schema contains blob columns.""" + for field in self.table.table_schema.fields: + # Check if field type is blob + if hasattr(field.type, 'type') and field.type.type == 'BLOB': + return True + # Alternative: check for specific blob type class + elif hasattr(field.type, '__class__') and 'blob' in field.type.__class__.__name__.lower(): + return True + return False + + def prepare_commit(self, commit_identifier) -> List[CommitMessage]: + self.commit_identifier = commit_identifier commit_messages = [] for (partition, bucket), writer in self.data_writers.items(): committed_files = writer.prepare_commit() @@ -77,32 +105,28 @@ def close(self): writer.close() self.data_writers.clear() - def _seq_number_stats(self) -> dict: - from pypaimon.manifest.manifest_file_manager import ManifestFileManager - from pypaimon.manifest.manifest_list_manager import ManifestListManager - from pypaimon.snapshot.snapshot_manager import SnapshotManager - - snapshot_manager = SnapshotManager(self.table) - manifest_list_manager = ManifestListManager(self.table) - manifest_file_manager = ManifestFileManager(self.table) + def _seq_number_stats(self, partition: Tuple) -> Dict[int, int]: + buckets = self.max_seq_numbers.get(partition) + if buckets is None: + buckets = self._load_seq_number_stats(partition) + self.max_seq_numbers[partition] = buckets + return buckets - latest_snapshot = snapshot_manager.get_latest_snapshot() - if not latest_snapshot: - return {} - manifest_files = manifest_list_manager.read_all(latest_snapshot) + def _load_seq_number_stats(self, partition: Tuple) -> dict: + read_builder = self.table.new_read_builder() + predicate_builder = read_builder.new_predicate_builder() + sub_predicates = [] + for key, value in zip(self.table.partition_keys, partition): + sub_predicates.append(predicate_builder.equal(key, value)) + partition_filter = predicate_builder.and_predicates(sub_predicates) - file_entries = [] - for manifest_file in manifest_files: - manifest_entries = manifest_file_manager.read(manifest_file.file_name) - for entry in manifest_entries: - if entry.kind == 0: - file_entries.append(entry) + scan = read_builder.with_filter(partition_filter).new_scan() + splits = scan.plan().splits() max_seq_numbers = {} - for entry in file_entries: - partition_key = (tuple(entry.partition.values), entry.bucket) - current_seq_num = entry.file.max_sequence_number - existing_max = max_seq_numbers.get(partition_key, -1) + for split in splits: + current_seq_num = max([file.max_sequence_number for file in split.files]) + existing_max = max_seq_numbers.get(split.bucket, -1) if current_seq_num > existing_max: - max_seq_numbers[partition_key] = current_seq_num + max_seq_numbers[split.bucket] = current_seq_num return max_seq_numbers diff --git a/paimon-python/pypaimon/write/row_key_extractor.py b/paimon-python/pypaimon/write/row_key_extractor.py index c1b7ebf7d01f..db36d50e910e 100644 --- a/paimon-python/pypaimon/write/row_key_extractor.py +++ b/paimon-python/pypaimon/write/row_key_extractor.py @@ -101,7 +101,7 @@ class UnawareBucketRowKeyExtractor(RowKeyExtractor): def __init__(self, table_schema: TableSchema): super().__init__(table_schema) - num_buckets = table_schema.options.get(CoreOptions.BUCKET, -1) + num_buckets = int(table_schema.options.get(CoreOptions.BUCKET, -1)) if num_buckets != -1: raise ValueError(f"Unaware bucket mode requires bucket = -1, got {num_buckets}") @@ -118,7 +118,7 @@ class DynamicBucketRowKeyExtractor(RowKeyExtractor): def __init__(self, table_schema: 'TableSchema'): super().__init__(table_schema) - num_buckets = table_schema.options.get(CoreOptions.BUCKET, -1) + num_buckets = int(table_schema.options.get(CoreOptions.BUCKET, -1)) if num_buckets != -1: raise ValueError( diff --git a/paimon-python/pypaimon/write/batch_table_commit.py b/paimon-python/pypaimon/write/table_commit.py similarity index 86% rename from paimon-python/pypaimon/write/batch_table_commit.py rename to paimon-python/pypaimon/write/table_commit.py index 7f42e1cef113..eab7606128e3 100644 --- a/paimon-python/pypaimon/write/batch_table_commit.py +++ b/paimon-python/pypaimon/write/table_commit.py @@ -16,14 +16,14 @@ # limitations under the License. ################################################################################ -import time from typing import List, Optional +from pypaimon.snapshot.snapshot import BATCH_COMMIT_IDENTIFIER from pypaimon.write.commit_message import CommitMessage from pypaimon.write.file_store_commit import FileStoreCommit -class BatchTableCommit: +class TableCommit: """Python implementation of BatchTableCommit for batch writing scenarios.""" def __init__(self, table, commit_user: str, static_partition: Optional[dict]): @@ -41,15 +41,13 @@ def __init__(self, table, commit_user: str, static_partition: Optional[dict]): self.file_store_commit = FileStoreCommit(snapshot_commit, table, commit_user) self.batch_committed = False - def commit(self, commit_messages: List[CommitMessage]): + def _commit(self, commit_messages: List[CommitMessage], commit_identifier: int = BATCH_COMMIT_IDENTIFIER): self._check_committed() non_empty_messages = [msg for msg in commit_messages if not msg.is_empty()] if not non_empty_messages: return - commit_identifier = int(time.time() * 1000) - try: if self.overwrite_partition is not None: self.file_store_commit.overwrite( @@ -76,3 +74,14 @@ def _check_committed(self): if self.batch_committed: raise RuntimeError("BatchTableCommit only supports one-time committing.") self.batch_committed = True + + +class BatchTableCommit(TableCommit): + def commit(self, commit_messages: List[CommitMessage]): + self._commit(commit_messages, BATCH_COMMIT_IDENTIFIER) + + +class StreamTableCommit(TableCommit): + + def commit(self, commit_messages: List[CommitMessage], commit_identifier: int = BATCH_COMMIT_IDENTIFIER): + self._commit(commit_messages, commit_identifier) diff --git a/paimon-python/pypaimon/write/batch_table_write.py b/paimon-python/pypaimon/write/table_write.py similarity index 68% rename from paimon-python/pypaimon/write/batch_table_write.py rename to paimon-python/pypaimon/write/table_write.py index c2e533cef5a6..7f415a0ba7d5 100644 --- a/paimon-python/pypaimon/write/batch_table_write.py +++ b/paimon-python/pypaimon/write/table_write.py @@ -15,18 +15,18 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ - from collections import defaultdict from typing import List import pyarrow as pa from pypaimon.schema.data_types import PyarrowFieldParser +from pypaimon.snapshot.snapshot import BATCH_COMMIT_IDENTIFIER from pypaimon.write.commit_message import CommitMessage from pypaimon.write.file_store_write import FileStoreWrite -class BatchTableWrite: +class TableWrite: def __init__(self, table): from pypaimon.table.file_store_table import FileStoreTable @@ -34,16 +34,13 @@ def __init__(self, table): self.table_pyarrow_schema = PyarrowFieldParser.from_paimon_schema(self.table.table_schema.fields) self.file_store_write = FileStoreWrite(self.table) self.row_key_extractor = self.table.create_row_key_extractor() - self.batch_committed = False - def write_arrow(self, table: pa.Table, row_kind: List[int] = None): - # TODO: support row_kind + def write_arrow(self, table: pa.Table): batches_iterator = table.to_batches() for batch in batches_iterator: self.write_arrow_batch(batch) - def write_arrow_batch(self, data: pa.RecordBatch, row_kind: List[int] = None): - # TODO: support row_kind + def write_arrow_batch(self, data: pa.RecordBatch): self._validate_pyarrow_schema(data.schema) partitions, buckets = self.row_key_extractor.extract_partition_bucket_batch(data) @@ -61,16 +58,39 @@ def write_pandas(self, dataframe): record_batch = pa.RecordBatch.from_pandas(dataframe, schema=pa_schema) return self.write_arrow_batch(record_batch) + def with_write_type(self, write_cols: List[str]): + for col in write_cols: + if col not in self.table_pyarrow_schema.names: + raise ValueError(f"Column {col} is not in table schema.") + if len(write_cols) == len(self.table_pyarrow_schema.names): + write_cols = None + self.file_store_write.write_cols = write_cols + return self + + def close(self): + self.file_store_write.close() + + def _validate_pyarrow_schema(self, data_schema: pa.Schema): + if data_schema != self.table_pyarrow_schema and data_schema.names != self.file_store_write.write_cols: + raise ValueError(f"Input schema isn't consistent with table schema and write cols. " + f"Input schema is: {data_schema} " + f"Table schema is: {self.table_pyarrow_schema} " + f"Write cols is: {self.file_store_write.write_cols}") + + +class BatchTableWrite(TableWrite): + def __init__(self, table): + super().__init__(table) + self.batch_committed = False + def prepare_commit(self) -> List[CommitMessage]: if self.batch_committed: raise RuntimeError("BatchTableWrite only supports one-time committing.") self.batch_committed = True - return self.file_store_write.prepare_commit() + return self.file_store_write.prepare_commit(BATCH_COMMIT_IDENTIFIER) - def close(self): - self.file_store_write.close() - def _validate_pyarrow_schema(self, data_schema): - if data_schema != self.table_pyarrow_schema: - raise ValueError(f"Input schema isn't consistent with table schema. " - f"Table schema is: {data_schema} Input schema is: {self.table_pyarrow_schema}") +class StreamTableWrite(TableWrite): + + def prepare_commit(self, commit_identifier) -> List[CommitMessage]: + return self.file_store_write.prepare_commit(commit_identifier) diff --git a/paimon-python/pypaimon/write/batch_write_builder.py b/paimon-python/pypaimon/write/write_builder.py similarity index 73% rename from paimon-python/pypaimon/write/batch_write_builder.py rename to paimon-python/pypaimon/write/write_builder.py index 2380530fbc29..8c9ed725f5e6 100644 --- a/paimon-python/pypaimon/write/batch_write_builder.py +++ b/paimon-python/pypaimon/write/write_builder.py @@ -17,14 +17,15 @@ ################################################################################ import uuid +from abc import ABC from typing import Optional from pypaimon.common.core_options import CoreOptions -from pypaimon.write.batch_table_commit import BatchTableCommit -from pypaimon.write.batch_table_write import BatchTableWrite +from pypaimon.write.table_commit import BatchTableCommit, StreamTableCommit, TableCommit +from pypaimon.write.table_write import BatchTableWrite, StreamTableWrite, TableWrite -class BatchWriteBuilder: +class WriteBuilder(ABC): def __init__(self, table): from pypaimon.table.file_store_table import FileStoreTable @@ -36,15 +37,33 @@ def overwrite(self, static_partition: Optional[dict] = None): self.static_partition = static_partition if static_partition is not None else {} return self - def new_write(self) -> BatchTableWrite: - return BatchTableWrite(self.table) + def new_write(self) -> TableWrite: + """Returns a table write.""" - def new_commit(self) -> BatchTableCommit: - commit = BatchTableCommit(self.table, self.commit_user, self.static_partition) - return commit + def new_commit(self) -> TableCommit: + """Returns a table commit.""" def _create_commit_user(self): if CoreOptions.COMMIT_USER_PREFIX in self.table.options: return f"{self.table.options.get(CoreOptions.COMMIT_USER_PREFIX)}_{uuid.uuid4()}" else: return str(uuid.uuid4()) + + +class BatchWriteBuilder(WriteBuilder): + + def new_write(self) -> BatchTableWrite: + return BatchTableWrite(self.table) + + def new_commit(self) -> BatchTableCommit: + commit = BatchTableCommit(self.table, self.commit_user, self.static_partition) + return commit + + +class StreamWriteBuilder(WriteBuilder): + def new_write(self) -> StreamTableWrite: + return StreamTableWrite(self.table) + + def new_commit(self) -> StreamTableCommit: + commit = StreamTableCommit(self.table, self.commit_user, self.static_partition) + return commit diff --git a/paimon-python/pypaimon/write/writer/blob_writer.py b/paimon-python/pypaimon/write/writer/blob_writer.py new file mode 100644 index 000000000000..ff153da8437b --- /dev/null +++ b/paimon-python/pypaimon/write/writer/blob_writer.py @@ -0,0 +1,46 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +import logging +from typing import Tuple + +from pypaimon.common.core_options import CoreOptions +from pypaimon.write.writer.append_only_data_writer import AppendOnlyDataWriter + +logger = logging.getLogger(__name__) + + +class BlobWriter(AppendOnlyDataWriter): + + def __init__(self, table, partition: Tuple, bucket: int, max_seq_number: int, blob_column: str): + super().__init__(table, partition, bucket, max_seq_number, [blob_column]) + + # Override file format to "blob" + self.file_format = CoreOptions.FILE_FORMAT_BLOB + + logger.info("Initialized BlobWriter with blob file format") + + @staticmethod + def _get_column_stats(record_batch, column_name: str): + column_array = record_batch.column(column_name) + # For blob data, don't generate min/max values + return { + "min_values": None, + "max_values": None, + "null_counts": column_array.null_count, + } diff --git a/paimon-python/pypaimon/write/writer/data_blob_writer.py b/paimon-python/pypaimon/write/writer/data_blob_writer.py new file mode 100644 index 000000000000..9d2e0982a4c2 --- /dev/null +++ b/paimon-python/pypaimon/write/writer/data_blob_writer.py @@ -0,0 +1,321 @@ +################################################################################ +# 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 datetime import datetime +from pathlib import Path +from typing import List, Optional, Tuple + +import pyarrow as pa + +from pypaimon.common.core_options import CoreOptions +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 +from pypaimon.write.writer.data_writer import DataWriter + +logger = logging.getLogger(__name__) + + +class DataBlobWriter(DataWriter): + """ + A rolling file writer that handles both normal data and blob data. This writer creates separate + files for normal columns and blob columns, managing their lifecycle independently. + + For example, given a table schema with normal columns (id INT, name STRING) and a blob column + (data BLOB), this writer will create separate files for (id, name) and (data). + + Key features: + - Blob data can roll independently when normal data doesn't need rolling + - When normal data rolls, blob data MUST also be closed (Java behavior) + - Blob data uses more aggressive rolling (smaller target size) to prevent memory issues + - One normal data file may correspond to multiple blob data files + - Blob data is written immediately to disk to prevent memory corruption + - Blob file metadata is stored as separate DataFileMeta objects after normal file metadata + + Rolling behavior: + - Normal data rolls: Both normal and blob writers are closed together, blob metadata added after normal metadata + - Blob data rolls independently: Only blob writer is closed, blob metadata is cached until normal data rolls + + Metadata organization: + - Normal file metadata is added first to committed_files + - Blob file metadata is added after normal file metadata in committed_files + - When blob rolls independently, metadata is cached until normal data rolls + - Result: [normal_meta, blob_meta1, blob_meta2, blob_meta3, ...] + + Example file organization: + committed_files = [ + normal_file1_meta, # f1.parquet metadata + blob_file1_meta, # b1.blob metadata + blob_file2_meta, # b2.blob metadata + blob_file3_meta, # b3.blob metadata + normal_file2_meta, # f1-2.parquet metadata + blob_file4_meta, # b4.blob metadata + blob_file5_meta, # b5.blob metadata + ] + + This matches the Java RollingBlobFileWriter behavior exactly. + """ + + # Constant for checking rolling condition periodically + CHECK_ROLLING_RECORD_CNT = 1000 + + def __init__(self, table, partition: Tuple, bucket: int, max_seq_number: int): + super().__init__(table, partition, bucket, max_seq_number) + + # Determine blob column from table schema + self.blob_column_name = self._get_blob_columns_from_schema() + + # Split schema into normal and blob columns + all_column_names = self.table.field_names + self.normal_column_names = [col for col in all_column_names if col != self.blob_column_name] + self.write_cols = self.normal_column_names + + # State management for blob writer + self.record_count = 0 + self.closed = False + + # Track pending data for normal data only + self.pending_normal_data: Optional[pa.Table] = None + + # Initialize blob writer with blob column name + from pypaimon.write.writer.blob_writer import BlobWriter + self.blob_writer = BlobWriter( + table=self.table, + partition=self.partition, + bucket=self.bucket, + max_seq_number=max_seq_number, + blob_column=self.blob_column_name + ) + + logger.info(f"Initialized DataBlobWriter with blob column: {self.blob_column_name}") + + def _get_blob_columns_from_schema(self) -> str: + blob_columns = [] + for field in self.table.table_schema.fields: + type_str = str(field.type).lower() + if 'blob' in type_str: + blob_columns.append(field.name) + + # Validate blob column count (matching Java constraint) + if len(blob_columns) == 0: + raise ValueError("No blob field found in table schema.") + elif len(blob_columns) > 1: + raise ValueError("Limit exactly one blob field in one paimon table yet.") + + return blob_columns[0] # Return single blob column name + + def _process_data(self, data: pa.RecordBatch) -> pa.RecordBatch: + normal_data, _ = self._split_data(data) + return normal_data + + def _merge_data(self, existing_data: pa.Table, new_data: pa.Table) -> pa.Table: + return self._merge_normal_data(existing_data, new_data) + + def write(self, data: pa.RecordBatch): + try: + # Split data into normal and blob parts + normal_data, blob_data = self._split_data(data) + + # Process and accumulate normal data + processed_normal = self._process_normal_data(normal_data) + if self.pending_normal_data is None: + self.pending_normal_data = processed_normal + else: + self.pending_normal_data = self._merge_normal_data(self.pending_normal_data, processed_normal) + + # Write blob data directly to blob writer (handles its own rolling) + if blob_data is not None and blob_data.num_rows > 0: + # Write blob data directly to blob writer + self.blob_writer.write(blob_data) + + self.record_count += data.num_rows + + # Check if normal data rolling is needed + if self._should_roll_normal(): + # When normal data rolls, close both writers and fetch blob metadata + self._close_current_writers() + + except Exception as e: + logger.error("Exception occurs when writing data. Cleaning up.", exc_info=e) + self.abort() + raise e + + def prepare_commit(self) -> List[DataFileMeta]: + # Close any remaining data + self._close_current_writers() + + return self.committed_files.copy() + + def close(self): + if self.closed: + return + + try: + if self.pending_normal_data is not None and self.pending_normal_data.num_rows > 0: + self._close_current_writers() + except Exception as e: + logger.error("Exception occurs when closing writer. Cleaning up.", exc_info=e) + self.abort() + finally: + self.closed = True + self.pending_normal_data = None + + def abort(self): + """Abort all writers and clean up resources.""" + self.blob_writer.abort() + self.pending_normal_data = None + self.committed_files.clear() + + def _split_data(self, data: pa.RecordBatch) -> Tuple[pa.RecordBatch, pa.RecordBatch]: + """Split data into normal and blob parts based on column names.""" + # Use the pre-computed column names + normal_columns = self.normal_column_names + blob_columns = [self.blob_column_name] # Single blob column + + # Create projected batches + normal_data = data.select(normal_columns) if normal_columns else None + blob_data = data.select(blob_columns) if blob_columns else None + + return normal_data, blob_data + + def _process_normal_data(self, data: pa.RecordBatch) -> pa.Table: + """Process normal data (similar to base DataWriter).""" + if data is None or data.num_rows == 0: + return pa.Table.from_batches([]) + return pa.Table.from_batches([data]) + + def _merge_normal_data(self, existing_data: pa.Table, new_data: pa.Table) -> pa.Table: + return pa.concat_tables([existing_data, new_data]) + + def _should_roll_normal(self) -> bool: + if self.pending_normal_data is None: + return False + + # Check rolling condition periodically (every CHECK_ROLLING_RECORD_CNT records) + if self.record_count % self.CHECK_ROLLING_RECORD_CNT != 0: + return False + + # Check if normal data exceeds target size + current_size = self.pending_normal_data.nbytes + return current_size > self.target_file_size + + def _close_current_writers(self): + """Close both normal and blob writers and add blob metadata after normal metadata (Java behavior).""" + if self.pending_normal_data is None or self.pending_normal_data.num_rows == 0: + return + + # Close normal writer and get metadata + normal_meta = self._write_normal_data_to_file(self.pending_normal_data) + + # Fetch blob metadata from blob writer + blob_metas = self.blob_writer.prepare_commit() + + # Validate consistency between normal and blob files (Java behavior) + self._validate_consistency(normal_meta, blob_metas) + + # Add normal file metadata first + self.committed_files.append(normal_meta) + + # Add blob file metadata after normal metadata + self.committed_files.extend(blob_metas) + + # Reset pending data + self.pending_normal_data = None + + logger.info(f"Closed both writers - normal: {normal_meta.file_name}, " + f"added {len(blob_metas)} blob file metadata after normal metadata") + + def _write_normal_data_to_file(self, data: pa.Table) -> DataFileMeta: + if data.num_rows == 0: + return None + + file_name = f"data-{uuid.uuid4()}-0.{self.file_format}" + file_path = self._generate_file_path(file_name) + + # Write file based on format + if self.file_format == CoreOptions.FILE_FORMAT_PARQUET: + self.file_io.write_parquet(file_path, data, compression=self.compression) + elif self.file_format == CoreOptions.FILE_FORMAT_ORC: + self.file_io.write_orc(file_path, data, compression=self.compression) + elif self.file_format == CoreOptions.FILE_FORMAT_AVRO: + self.file_io.write_avro(file_path, data) + else: + raise ValueError(f"Unsupported file format: {self.file_format}") + + # Generate metadata + return self._create_data_file_meta(file_name, file_path, data) + + def _create_data_file_meta(self, file_name: str, file_path: Path, data: pa.Table) -> DataFileMeta: + # Column stats (only for normal columns) + column_stats = { + field.name: self._get_column_stats(data, field.name) + for field in self.table.table_schema.fields + if field.name != self.blob_column_name + } + + # Get normal fields only + normal_fields = [field for field in self.table.table_schema.fields + if field.name != self.blob_column_name] + + min_value_stats = [column_stats[field.name]['min_values'] for field in normal_fields] + max_value_stats = [column_stats[field.name]['max_values'] for field in normal_fields] + value_null_counts = [column_stats[field.name]['null_counts'] for field in normal_fields] + + self.sequence_generator.start = self.sequence_generator.current + + return DataFileMeta( + file_name=file_name, + file_size=self.file_io.get_file_size(file_path), + row_count=data.num_rows, + min_key=GenericRow([], []), + max_key=GenericRow([], []), + key_stats=SimpleStats( + GenericRow([], []), + GenericRow([], []), + []), + value_stats=SimpleStats( + GenericRow(min_value_stats, normal_fields), + GenericRow(max_value_stats, normal_fields), + value_null_counts), + min_sequence_number=-1, + max_sequence_number=-1, + schema_id=self.table.table_schema.id, + level=0, + extra_files=[], + creation_time=datetime.now(), + delete_row_count=0, + file_source=0, + value_stats_cols=self.normal_column_names, + file_path=str(file_path), + write_cols=self.write_cols) + + def _validate_consistency(self, normal_meta: DataFileMeta, blob_metas: List[DataFileMeta]): + if normal_meta is None: + return + + normal_row_count = normal_meta.row_count + blob_row_count = sum(meta.row_count for meta in blob_metas) + + if normal_row_count != blob_row_count: + raise RuntimeError( + f"This is a bug: The row count of main file and blob files does not match. " + f"Main file: {normal_meta.file_name} (row count: {normal_row_count}), " + f"blob files: {[meta.file_name for meta in blob_metas]} (total row count: {blob_row_count})" + ) diff --git a/paimon-python/pypaimon/write/writer/data_writer.py b/paimon-python/pypaimon/write/writer/data_writer.py index e2f778b58667..351ff3297903 100644 --- a/paimon-python/pypaimon/write/writer/data_writer.py +++ b/paimon-python/pypaimon/write/writer/data_writer.py @@ -26,6 +26,7 @@ from pypaimon.common.core_options import CoreOptions from pypaimon.manifest.schema.data_file_meta import DataFileMeta from pypaimon.manifest.schema.simple_stats import SimpleStats +from pypaimon.schema.data_types import PyarrowFieldParser from pypaimon.table.bucket_mode import BucketMode from pypaimon.table.row.generic_row import GenericRow @@ -33,7 +34,8 @@ class DataWriter(ABC): """Base class for data writers that handle PyArrow tables directly.""" - def __init__(self, table, partition: Tuple, bucket: int, max_seq_number: int): + def __init__(self, table, partition: Tuple, bucket: int, max_seq_number: int, + write_cols: Optional[List[str]] = None): from pypaimon.table.file_store_table import FileStoreTable self.table: FileStoreTable = table @@ -41,8 +43,8 @@ def __init__(self, table, partition: Tuple, bucket: int, max_seq_number: int): self.bucket = bucket self.file_io = self.table.file_io - self.trimmed_primary_key_fields = self.table.table_schema.get_trimmed_primary_key_fields() - self.trimmed_primary_key = [field.name for field in self.trimmed_primary_key_fields] + self.trimmed_primary_keys_fields = self.table.trimmed_primary_keys_fields + self.trimmed_primary_keys = self.table.trimmed_primary_keys options = self.table.options self.target_file_size = 256 * 1024 * 1024 @@ -55,16 +57,25 @@ def __init__(self, table, partition: Tuple, bucket: int, max_seq_number: int): self.pending_data: Optional[pa.Table] = None self.committed_files: List[DataFileMeta] = [] + self.write_cols = write_cols + self.blob_as_descriptor = CoreOptions.get_blob_as_descriptor(options) def write(self, data: pa.RecordBatch): - processed_data = self._process_data(data) + try: + processed_data = self._process_data(data) - if self.pending_data is None: - self.pending_data = processed_data - else: - self.pending_data = self._merge_data(self.pending_data, processed_data) + if self.pending_data is None: + self.pending_data = processed_data + else: + self.pending_data = self._merge_data(self.pending_data, processed_data) - self._check_and_roll_if_needed() + self._check_and_roll_if_needed() + except Exception as e: + import logging + logger = logging.getLogger(__name__) + logger.warning("Exception occurs when writing data. Cleaning up.", exc_info=e) + self.abort() + raise e def prepare_commit(self) -> List[DataFileMeta]: if self.pending_data is not None and self.pending_data.num_rows > 0: @@ -74,6 +85,36 @@ def prepare_commit(self) -> List[DataFileMeta]: return self.committed_files.copy() def close(self): + try: + if self.pending_data is not None and self.pending_data.num_rows > 0: + self._write_data_to_file(self.pending_data) + except Exception as e: + import logging + logger = logging.getLogger(__name__) + logger.warning("Exception occurs when closing writer. Cleaning up.", exc_info=e) + self.abort() + raise e + finally: + self.pending_data = None + # Note: Don't clear committed_files in close() - they should be returned by prepare_commit() + + def abort(self): + """ + Abort all writers and clean up resources. This method should be called when an error occurs + during writing. It deletes any files that were written and cleans up resources. + """ + # Delete any files that were written + for file_meta in self.committed_files: + try: + if file_meta.file_path: + self.file_io.delete_quietly(file_meta.file_path) + except Exception as e: + # Log but don't raise - we want to clean up as much as possible + import logging + logger = logging.getLogger(__name__) + logger.warning(f"Failed to delete file {file_meta.file_path} during abort: {e}") + + # Clean up resources self.pending_data = None self.committed_files.clear() @@ -111,12 +152,14 @@ def _write_data_to_file(self, data: pa.Table): self.file_io.write_orc(file_path, data, compression=self.compression) elif self.file_format == CoreOptions.FILE_FORMAT_AVRO: self.file_io.write_avro(file_path, data) + elif self.file_format == CoreOptions.FILE_FORMAT_BLOB: + self.file_io.write_blob(file_path, data, self.blob_as_descriptor) else: raise ValueError(f"Unsupported file format: {self.file_format}") # min key & max key - selected_table = data.select(self.trimmed_primary_key) + selected_table = data.select(self.trimmed_primary_keys) key_columns_batch = selected_table.to_batches()[0] min_key_row_batch = key_columns_batch.slice(0, 1) max_key_row_batch = key_columns_batch.slice(key_columns_batch.num_rows - 1, 1) @@ -124,15 +167,17 @@ def _write_data_to_file(self, data: pa.Table): max_key = [col.to_pylist()[0] for col in max_key_row_batch.columns] # key stats & value stats + data_fields = self.table.fields if self.table.is_primary_key_table \ + else PyarrowFieldParser.to_paimon_schema(data.schema) column_stats = { field.name: self._get_column_stats(data, field.name) - for field in self.table.table_schema.fields + for field in data_fields } - all_fields = self.table.table_schema.fields + all_fields = data_fields min_value_stats = [column_stats[field.name]['min_values'] for field in all_fields] max_value_stats = [column_stats[field.name]['max_values'] for field in all_fields] value_null_counts = [column_stats[field.name]['null_counts'] for field in all_fields] - key_fields = self.trimmed_primary_key_fields + key_fields = self.trimmed_primary_keys_fields min_key_stats = [column_stats[field.name]['min_values'] for field in key_fields] max_key_stats = [column_stats[field.name]['max_values'] for field in key_fields] key_null_counts = [column_stats[field.name]['null_counts'] for field in key_fields] @@ -146,16 +191,16 @@ def _write_data_to_file(self, data: pa.Table): file_name=file_name, file_size=self.file_io.get_file_size(file_path), row_count=data.num_rows, - min_key=GenericRow(min_key, self.trimmed_primary_key_fields), - max_key=GenericRow(max_key, self.trimmed_primary_key_fields), + min_key=GenericRow(min_key, self.trimmed_primary_keys_fields), + max_key=GenericRow(max_key, self.trimmed_primary_keys_fields), key_stats=SimpleStats( - GenericRow(min_key_stats, self.trimmed_primary_key_fields), - GenericRow(max_key_stats, self.trimmed_primary_key_fields), + GenericRow(min_key_stats, self.trimmed_primary_keys_fields), + GenericRow(max_key_stats, self.trimmed_primary_keys_fields), key_null_counts, ), value_stats=SimpleStats( - GenericRow(min_value_stats, self.table.table_schema.fields), - GenericRow(max_value_stats, self.table.table_schema.fields), + GenericRow(min_value_stats, data_fields), + GenericRow(max_value_stats, data_fields), value_null_counts, ), min_sequence_number=min_seq, @@ -165,7 +210,12 @@ def _write_data_to_file(self, data: pa.Table): extra_files=[], creation_time=datetime.now(), delete_row_count=0, - value_stats_cols=None, # None means all columns have statistics + file_source=0, + value_stats_cols=None, # None means all columns in the data have statistics + external_path=None, + first_row_id=None, + write_cols=self.write_cols, + # None means all columns in the table have been written file_path=str(file_path), )) diff --git a/paimon-python/pypaimon/write/writer/key_value_data_writer.py b/paimon-python/pypaimon/write/writer/key_value_data_writer.py index fb929710e8b2..05cad9bca95e 100644 --- a/paimon-python/pypaimon/write/writer/key_value_data_writer.py +++ b/paimon-python/pypaimon/write/writer/key_value_data_writer.py @@ -38,23 +38,23 @@ def _add_system_fields(self, data: pa.RecordBatch) -> pa.RecordBatch: num_rows = data.num_rows enhanced_table = data - for pk_key in reversed(self.trimmed_primary_key): + for pk_key in reversed(self.trimmed_primary_keys): if pk_key in data.column_names: key_column = data.column(pk_key) enhanced_table = enhanced_table.add_column(0, f'_KEY_{pk_key}', key_column) sequence_column = pa.array([self.sequence_generator.next() for _ in range(num_rows)], type=pa.int64()) - enhanced_table = enhanced_table.add_column(len(self.trimmed_primary_key), '_SEQUENCE_NUMBER', sequence_column) + enhanced_table = enhanced_table.add_column(len(self.trimmed_primary_keys), '_SEQUENCE_NUMBER', sequence_column) # TODO: support real row kind here value_kind_column = pa.array([0] * num_rows, type=pa.int32()) - enhanced_table = enhanced_table.add_column(len(self.trimmed_primary_key) + 1, '_VALUE_KIND', + enhanced_table = enhanced_table.add_column(len(self.trimmed_primary_keys) + 1, '_VALUE_KIND', value_kind_column) return enhanced_table def _sort_by_primary_key(self, data: pa.RecordBatch) -> pa.RecordBatch: - sort_keys = [(key, 'ascending') for key in self.trimmed_primary_key] + sort_keys = [(key, 'ascending') for key in self.trimmed_primary_keys] if '_SEQUENCE_NUMBER' in data.column_names: sort_keys.append(('_SEQUENCE_NUMBER', 'ascending')) diff --git a/paimon-python/setup.py b/paimon-python/setup.py index 4761a7396970..4744eba0b71c 100644 --- a/paimon-python/setup.py +++ b/paimon-python/setup.py @@ -15,33 +15,31 @@ # See the License for the specific language governing permissions and # limitations under the License. ########################################################################## +import os from setuptools import find_packages, setup -VERSION = "0.3.dev" # noqa +VERSION = "1.3.0" PACKAGES = find_packages(include=["pypaimon*"]) -install_requires = [ - 'readerwriterlock==1.0.9', - 'fsspec==2024.3.1; python_version>"3.6"', - 'fsspec==2021.10.1; python_version=="3.6"', - 'cachetools==5.3.3; python_version>"3.6"', - 'cachetools==4.2.4; python_version=="3.6"', - 'ossfs==2023.12.0; python_version>"3.6"', - 'ossfs==2021.8.0; python_version=="3.6"', - 'pyarrow>=16; python_version >= "3.8"', - 'pyarrow==6.0.1; python_version < "3.8"', - 'pandas==2.3.2; python_version >= "3.7"', - 'pandas==1.1.5; python_version < "3.7"', - 'polars==1.32.0; python_version>"3.6"', - 'polars==0.9.12; python_version=="3.6"', - 'fastavro==1.11.1; python_version>"3.6"', - 'fastavro==1.4.7; python_version=="3.6"', - 'zstandard==0.24.0; python_version>="3.7"', - 'zstandard==0.19.0; python_version<"3.7"', - 'dataclasses==0.8.0; python_version < "3.7"', - 'pip==21.3.1' -] + +def read_requirements(): + """Read requirements from dev/requirements.txt file.""" + requirements_path = os.path.join(os.path.dirname(__file__), 'dev', 'requirements.txt') + requirements = [] + + if os.path.exists(requirements_path): + with open(requirements_path, 'r', encoding='utf-8') as f: + for line in f: + line = line.strip() + # Skip empty lines and comments + if line and not line.startswith('#'): + requirements.append(line) + + return requirements + + +install_requires = read_requirements() long_description = "See Apache Paimon Python API \ [Doc](https://paimon.apache.org/docs/master/program-api/python-api/) for usage." diff --git a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala index 6cc1ce479411..56223c36cd75 100644 --- a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala +++ b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala @@ -23,11 +23,7 @@ import org.apache.paimon.spark.SparkFilterConverter import org.apache.paimon.types.RowType import org.apache.spark.sql.PaimonUtils.{normalizeExprs, translateFilter} -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.optimizer.ConstantFolding -import org.apache.spark.sql.catalyst.plans.logical.Filter -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation trait ExpressionHelper extends ExpressionHelperBase { @@ -56,23 +52,4 @@ trait ExpressionHelper extends ExpressionHelperBase { Some(PredicateBuilder.and(predicates: _*)) } } - - def resolveFilter( - spark: SparkSession, - relation: DataSourceV2Relation, - conditionSql: String): Expression = { - val unResolvedExpression = spark.sessionState.sqlParser.parseExpression(conditionSql) - val filter = Filter(unResolvedExpression, relation) - spark.sessionState.analyzer.execute(filter) match { - case filter: Filter => - try { - ConstantFolding.apply(filter).asInstanceOf[Filter].condition - } catch { - case _: Throwable => filter.condition - } - case _ => - throw new RuntimeException( - s"Could not resolve expression $conditionSql in relation: $relation") - } - } } diff --git a/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala b/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala index f1f0d8c06567..5ab343c41c1d 100644 --- a/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala +++ b/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala @@ -32,8 +32,10 @@ class MergeIntoPrimaryKeyNonBucketTableTest class MergeIntoAppendBucketedTableTest extends MergeIntoTableTestBase + with MergeIntoAppendTableTest with PaimonAppendBucketedTableTest {} class MergeIntoAppendNonBucketedTableTest extends MergeIntoTableTestBase + with MergeIntoAppendTableTest with PaimonAppendNonBucketTableTest {} diff --git a/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala b/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala similarity index 94% rename from paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala rename to paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala index f7d911c31cb4..9f96840a7788 100644 --- a/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala +++ b/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala @@ -18,4 +18,4 @@ package org.apache.paimon.spark.sql -class RowLineageTest extends RowLineageTestBase {} +class RowTrackingTest extends RowTrackingTestBase {} diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala index f1f0d8c06567..5ab343c41c1d 100644 --- a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala +++ b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala @@ -32,8 +32,10 @@ class MergeIntoPrimaryKeyNonBucketTableTest class MergeIntoAppendBucketedTableTest extends MergeIntoTableTestBase + with MergeIntoAppendTableTest with PaimonAppendBucketedTableTest {} class MergeIntoAppendNonBucketedTableTest extends MergeIntoTableTestBase + with MergeIntoAppendTableTest with PaimonAppendNonBucketTableTest {} diff --git a/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala similarity index 94% rename from paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala rename to paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala index f7d911c31cb4..9f96840a7788 100644 --- a/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala +++ b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala @@ -18,4 +18,4 @@ package org.apache.paimon.spark.sql -class RowLineageTest extends RowLineageTestBase {} +class RowTrackingTest extends RowTrackingTestBase {} diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala index e1cfe3a3960f..b9a85b147eea 100644 --- a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala @@ -34,10 +34,12 @@ class MergeIntoPrimaryKeyNonBucketTableTest class MergeIntoAppendBucketedTableTest extends MergeIntoTableTestBase + with MergeIntoAppendTableTest with MergeIntoNotMatchedBySourceTest with PaimonAppendBucketedTableTest {} class MergeIntoAppendNonBucketedTableTest extends MergeIntoTableTestBase + with MergeIntoAppendTableTest with MergeIntoNotMatchedBySourceTest with PaimonAppendNonBucketTableTest {} diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala similarity index 94% rename from paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala rename to paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala index f7d911c31cb4..9f96840a7788 100644 --- a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala @@ -18,4 +18,4 @@ package org.apache.paimon.spark.sql -class RowLineageTest extends RowLineageTestBase {} +class RowTrackingTest extends RowTrackingTestBase {} diff --git a/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala index e1cfe3a3960f..b9a85b147eea 100644 --- a/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala +++ b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala @@ -34,10 +34,12 @@ class MergeIntoPrimaryKeyNonBucketTableTest class MergeIntoAppendBucketedTableTest extends MergeIntoTableTestBase + with MergeIntoAppendTableTest with MergeIntoNotMatchedBySourceTest with PaimonAppendBucketedTableTest {} class MergeIntoAppendNonBucketedTableTest extends MergeIntoTableTestBase + with MergeIntoAppendTableTest with MergeIntoNotMatchedBySourceTest with PaimonAppendNonBucketTableTest {} diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala similarity index 94% rename from paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala rename to paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala index f7d911c31cb4..9f96840a7788 100644 --- a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala +++ b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala @@ -18,4 +18,4 @@ package org.apache.paimon.spark.sql -class RowLineageTest extends RowLineageTestBase {} +class RowTrackingTest extends RowTrackingTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala index e1cfe3a3960f..b9a85b147eea 100644 --- a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala @@ -34,10 +34,12 @@ class MergeIntoPrimaryKeyNonBucketTableTest class MergeIntoAppendBucketedTableTest extends MergeIntoTableTestBase + with MergeIntoAppendTableTest with MergeIntoNotMatchedBySourceTest with PaimonAppendBucketedTableTest {} class MergeIntoAppendNonBucketedTableTest extends MergeIntoTableTestBase + with MergeIntoAppendTableTest with MergeIntoNotMatchedBySourceTest with PaimonAppendNonBucketTableTest {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala deleted file mode 100644 index f7d911c31cb4..000000000000 --- a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/RowLineageTest.scala +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.spark.sql - -class RowLineageTest extends RowLineageTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala new file mode 100644 index 000000000000..9f96840a7788 --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.sql + +class RowTrackingTest extends RowTrackingTestBase {} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index 1df1d66f8077..db7281de92d2 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -524,34 +524,33 @@ public void renameTable(Identifier oldIdent, Identifier newIdent) @Override public Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceException { - if (isFunctionNamespace(namespace)) { - List functionIdentifiers = new ArrayList<>(); - PaimonFunctions.names() - .forEach(name -> functionIdentifiers.add(Identifier.of(namespace, name))); - if (namespace.length > 0) { - String databaseName = getDatabaseNameFromNamespace(namespace); - try { - catalog.listFunctions(databaseName) - .forEach( - name -> - functionIdentifiers.add( - Identifier.of(namespace, name))); - } catch (Catalog.DatabaseNotExistException e) { - throw new NoSuchNamespaceException(namespace); - } + if (isSystemFunctionNamespace(namespace)) { + List result = new ArrayList<>(); + PaimonFunctions.names().forEach(name -> result.add(Identifier.of(namespace, name))); + return result.toArray(new Identifier[0]); + } else if (isDatabaseFunctionNamespace(namespace)) { + List result = new ArrayList<>(); + String databaseName = getDatabaseNameFromNamespace(namespace); + try { + catalog.listFunctions(databaseName) + .forEach(name -> result.add(Identifier.of(namespace, name))); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchNamespaceException(namespace); } - return functionIdentifiers.toArray(new Identifier[0]); + return result.toArray(new Identifier[0]); } throw new NoSuchNamespaceException(namespace); } @Override public UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionException { - if (isFunctionNamespace(ident.namespace())) { + String[] namespace = ident.namespace(); + if (isSystemFunctionNamespace(namespace)) { UnboundFunction func = PaimonFunctions.load(ident.name()); if (func != null) { return func; } + } else if (isDatabaseFunctionNamespace(namespace)) { try { Function paimonFunction = catalog.getFunction(toIdentifier(ident)); FunctionDefinition functionDefinition = @@ -582,11 +581,14 @@ public UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionExcep throw new NoSuchFunctionException(ident); } - private boolean isFunctionNamespace(String[] namespace) { + private boolean isSystemFunctionNamespace(String[] namespace) { // Allow for empty namespace, as Spark's bucket join will use `bucket` function with empty // namespace to generate transforms for partitioning. - // Otherwise, check if it is paimon namespace. - return namespace.length == 0 || (namespace.length == 1 && namespaceExists(namespace)); + return namespace.length == 0 || isSystemNamespace(namespace); + } + + private boolean isDatabaseFunctionNamespace(String[] namespace) { + return namespace.length == 1 && namespaceExists(namespace); } private PaimonV1FunctionRegistry v1FunctionRegistry() { diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SparkBaseCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SparkBaseCatalog.java index 1cb3035fad96..ac6736e2e1c0 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SparkBaseCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SparkBaseCatalog.java @@ -18,7 +18,6 @@ package org.apache.paimon.spark.catalog; -import org.apache.paimon.catalog.Catalog; import org.apache.paimon.spark.SparkProcedures; import org.apache.paimon.spark.SparkSource; import org.apache.paimon.spark.analysis.NoSuchProcedureException; @@ -35,6 +34,7 @@ import java.util.Collections; import java.util.Set; +import static org.apache.paimon.catalog.Catalog.SYSTEM_DATABASE_NAME; import static org.apache.spark.sql.connector.catalog.TableCatalogCapability.SUPPORT_COLUMN_DEFAULT_VALUE; /** Spark base catalog. */ @@ -54,7 +54,7 @@ public Set capabilities() { @Override public Procedure loadProcedure(Identifier identifier) throws NoSuchProcedureException { - if (Catalog.SYSTEM_DATABASE_NAME.equals(identifier.namespace()[0])) { + if (isSystemNamespace(identifier.namespace())) { ProcedureBuilder builder = SparkProcedures.newBuilder(identifier.name()); if (builder != null) { return builder.withTableCatalog(this).build(); @@ -63,7 +63,11 @@ public Procedure loadProcedure(Identifier identifier) throws NoSuchProcedureExce throw new NoSuchProcedureException(identifier); } - public boolean usePaimon(@Nullable String provider) { + public static boolean usePaimon(@Nullable String provider) { return provider == null || SparkSource.NAME().equalsIgnoreCase(provider); } + + public static boolean isSystemNamespace(String[] namespace) { + return namespace.length == 1 && namespace[0].equalsIgnoreCase(SYSTEM_DATABASE_NAME); + } } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java index dd36e2f570b2..e66f1ae7d5b9 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java @@ -23,8 +23,14 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.append.AppendCompactCoordinator; import org.apache.paimon.append.AppendCompactTask; +import org.apache.paimon.append.cluster.IncrementalClusterManager; +import org.apache.paimon.compact.CompactUnit; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.io.CompactIncrement; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.io.DataIncrement; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.operation.BaseAppendFileStoreWrite; import org.apache.paimon.partition.PartitionPredicate; @@ -42,6 +48,7 @@ import org.apache.paimon.table.sink.BatchTableWrite; import org.apache.paimon.table.sink.BatchWriteBuilder; import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.table.sink.CommitMessageSerializer; import org.apache.paimon.table.sink.TableCommitImpl; import org.apache.paimon.table.source.DataSplit; @@ -92,6 +99,9 @@ import java.util.Set; import java.util.stream.Collectors; +import scala.collection.JavaConverters; +import scala.collection.Seq; + import static org.apache.paimon.CoreOptions.createCommitUser; import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.spark.sql.types.DataTypes.StringType; @@ -146,8 +156,7 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); String partitions = blank(args, 1) ? null : args.getString(1); - // make full compact strategy as default. - String compactStrategy = blank(args, 2) ? FULL : args.getString(2); + String compactStrategy = blank(args, 2) ? null : args.getString(2); String sortType = blank(args, 3) ? OrderType.NONE.name() : args.getString(3); List sortColumns = blank(args, 4) @@ -166,7 +175,9 @@ public InternalRow[] call(InternalRow args) { "sort compact do not support 'partition_idle_time'."); } - if (!(compactStrategy.equalsIgnoreCase(FULL) || compactStrategy.equalsIgnoreCase(MINOR))) { + if (!(compactStrategy == null + || compactStrategy.equalsIgnoreCase(FULL) + || compactStrategy.equalsIgnoreCase(MINOR))) { throw new IllegalArgumentException( String.format( "The compact strategy only supports 'full' or 'minor', but '%s' is configured.", @@ -205,6 +216,12 @@ public InternalRow[] call(InternalRow args) { dynamicOptions, CoreOptions.WRITE_ONLY.key(), "false"); ProcedureUtils.putAllOptions(dynamicOptions, options); table = table.copy(dynamicOptions); + if (((FileStoreTable) table).coreOptions().clusteringIncrementalEnabled() + && (!OrderType.NONE.name().equals(sortType))) { + throw new IllegalArgumentException( + "The table has enabled incremental clustering, do not support sort compact."); + } + InternalRow internalRow = newInternalRow( execute( @@ -238,9 +255,16 @@ private boolean execute( @Nullable Duration partitionIdleTime) { BucketMode bucketMode = table.bucketMode(); OrderType orderType = OrderType.of(sortType); + + boolean clusterIncrementalEnabled = table.coreOptions().clusteringIncrementalEnabled(); + if (compactStrategy == null) { + // make full compact strategy as default for compact. + // make non-full compact strategy as default for incremental clustering. + compactStrategy = clusterIncrementalEnabled ? MINOR : FULL; + } boolean fullCompact = compactStrategy.equalsIgnoreCase(FULL); RowType partitionType = table.schema().logicalPartitionType(); - Predicate filter = + Predicate partitionFilter = condition == null ? null : ExpressionUtils.convertConditionToPaimonPredicate( @@ -250,7 +274,8 @@ private boolean execute( false) .getOrElse(null); PartitionPredicate partitionPredicate = - PartitionPredicate.fromPredicate(partitionType, filter); + PartitionPredicate.fromPredicate(partitionType, partitionFilter); + if (orderType.equals(OrderType.NONE)) { JavaSparkContext javaSparkContext = new JavaSparkContext(spark().sparkContext()); switch (bucketMode) { @@ -264,8 +289,12 @@ private boolean execute( javaSparkContext); break; case BUCKET_UNAWARE: - compactUnAwareBucketTable( - table, partitionPredicate, partitionIdleTime, javaSparkContext); + if (clusterIncrementalEnabled) { + clusterIncrementalUnAwareBucketTable(table, fullCompact, relation); + } else { + compactUnAwareBucketTable( + table, partitionPredicate, partitionIdleTime, javaSparkContext); + } break; default: throw new UnsupportedOperationException( @@ -274,7 +303,8 @@ private boolean execute( } else { switch (bucketMode) { case BUCKET_UNAWARE: - sortCompactUnAwareBucketTable(table, orderType, sortColumns, relation, filter); + sortCompactUnAwareBucketTable( + table, orderType, sortColumns, relation, partitionFilter); break; default: throw new UnsupportedOperationException( @@ -424,7 +454,7 @@ private void compactUnAwareBucketTable( CoreOptions coreOptions = table.coreOptions(); if (coreOptions.rowTrackingEnabled()) { write.withWriteType( - SpecialFields.rowTypeWithRowLineage( + SpecialFields.rowTypeWithRowTracking( table.rowType())); } AppendCompactTaskSerializer ser = @@ -493,10 +523,10 @@ private void sortCompactUnAwareBucketTable( OrderType orderType, List sortColumns, DataSourceV2Relation relation, - @Nullable Predicate filter) { + @Nullable Predicate partitionFilter) { SnapshotReader snapshotReader = table.newSnapshotReader(); - if (filter != null) { - snapshotReader.withFilter(filter); + if (partitionFilter != null) { + snapshotReader.withPartitionFilter(partitionFilter); } Map packedSplits = packForSort(snapshotReader.read().dataSplits()); TableSorter sorter = TableSorter.getSorter(table, orderType, sortColumns); @@ -521,6 +551,107 @@ private void sortCompactUnAwareBucketTable( } } + private void clusterIncrementalUnAwareBucketTable( + FileStoreTable table, boolean fullCompaction, DataSourceV2Relation relation) { + IncrementalClusterManager incrementalClusterManager = new IncrementalClusterManager(table); + Map compactUnits = + incrementalClusterManager.prepareForCluster(fullCompaction); + + Map, CommitMessage>> partitionSplits = + incrementalClusterManager.toSplitsAndRewriteDvFiles(compactUnits); + + // sort in partition + TableSorter sorter = + TableSorter.getSorter( + table, + incrementalClusterManager.clusterCurve(), + incrementalClusterManager.clusterKeys()); + LOG.info( + "Start to sort in partition, cluster curve is {}, cluster keys is {}", + incrementalClusterManager.clusterCurve(), + incrementalClusterManager.clusterKeys()); + + Dataset datasetForWrite = + partitionSplits.values().stream() + .map(Pair::getKey) + .map( + splits -> { + Dataset dataset = + PaimonUtils.createDataset( + spark(), + ScanPlanHelper$.MODULE$.createNewScanPlan( + splits.toArray(new DataSplit[0]), + relation)); + return sorter.sort(dataset); + }) + .reduce(Dataset::union) + .orElse(null); + if (datasetForWrite != null) { + // set to write only to prevent invoking compaction + // do not use overwrite, we don't need to overwrite the whole partition + PaimonSparkWriter writer = PaimonSparkWriter.apply(table).writeOnly(); + Seq commitMessages = writer.write(datasetForWrite); + if (LOG.isDebugEnabled()) { + LOG.debug("Commit messages after writing:{}", commitMessages); + } + + // re-organize the commit messages to generate the compact messages + Map> partitionClustered = new HashMap<>(); + for (CommitMessage commitMessage : JavaConverters.seqAsJavaList(commitMessages)) { + checkArgument(commitMessage.bucket() == 0); + partitionClustered + .computeIfAbsent(commitMessage.partition(), k -> new ArrayList<>()) + .addAll(((CommitMessageImpl) commitMessage).newFilesIncrement().newFiles()); + } + + List clusterMessages = new ArrayList<>(); + for (Map.Entry> entry : partitionClustered.entrySet()) { + BinaryRow partition = entry.getKey(); + CommitMessageImpl dvCommitMessage = + (CommitMessageImpl) partitionSplits.get(partition).getValue(); + List clusterBefore = compactUnits.get(partition).files(); + // upgrade the clustered file to outputLevel + List clusterAfter = + IncrementalClusterManager.upgrade( + entry.getValue(), compactUnits.get(partition).outputLevel()); + LOG.info( + "Partition {}: upgrade file level to {}", + partition, + compactUnits.get(partition).outputLevel()); + + List newIndexFiles = new ArrayList<>(); + List deletedIndexFiles = new ArrayList<>(); + if (dvCommitMessage != null) { + newIndexFiles = dvCommitMessage.compactIncrement().newIndexFiles(); + deletedIndexFiles = dvCommitMessage.compactIncrement().deletedIndexFiles(); + } + + // get the dv index messages + CompactIncrement compactIncrement = + new CompactIncrement( + clusterBefore, + clusterAfter, + Collections.emptyList(), + newIndexFiles, + deletedIndexFiles); + clusterMessages.add( + new CommitMessageImpl( + partition, + // bucket 0 is bucket for unaware-bucket table + // for compatibility with the old design + 0, + table.coreOptions().bucket(), + DataIncrement.emptyIncrement(), + compactIncrement)); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Commit messages after reorganizing:{}", clusterMessages); + } + + writer.commit(JavaConverters.asScalaBuffer(clusterMessages).toSeq()); + } + } + private Map packForSort(List dataSplits) { // Make a single partition as a compact group return dataSplits.stream() diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ColumnPruningAndPushDown.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ColumnPruningAndPushDown.scala index ce1fe2e25fdf..db28b4f4efe4 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ColumnPruningAndPushDown.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ColumnPruningAndPushDown.scala @@ -40,7 +40,7 @@ trait ColumnPruningAndPushDown extends Scan with Logging { lazy val tableRowType: RowType = { val coreOptions: CoreOptions = CoreOptions.fromMap(table.options()) if (coreOptions.rowTrackingEnabled()) { - SpecialFields.rowTypeWithRowLineage(table.rowType()) + SpecialFields.rowTypeWithRowTracking(table.rowType()) } else { table.rowType() } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala index 802fbfdfa649..2bf95df1897b 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala @@ -43,10 +43,10 @@ case class PaimonPartitionReader( ) extends PartitionReader[InternalRow] { private val splits: Iterator[Split] = partition.splits.toIterator - @Nullable private var currentRecordReader = readSplit() private var advanced = false private var currentRow: PaimonInternalRow = _ private val ioManager: IOManager = createIOManager() + @Nullable private var currentRecordReader = readSplit() private val sparkRow: SparkInternalRow = { val dataFields = new JList(readBuilder.readType().getFields) dataFields.addAll(metadataColumns.map(_.toPaimonDataField).asJava) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTableWrite.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTableWrite.scala index 1ac9aab47ba3..704a6c2d5bde 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTableWrite.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTableWrite.scala @@ -33,7 +33,7 @@ case class SparkTableWrite( writeBuilder: BatchWriteBuilder, writeType: RowType, rowKindColIdx: Int = -1, - writeRowLineage: Boolean = false) + writeRowTracking: Boolean = false) extends SparkTableWriteTrait { private val ioManager: IOManager = SparkUtils.createIOManager @@ -41,7 +41,7 @@ case class SparkTableWrite( private val write: BatchTableWrite = { val _write = writeBuilder.newWrite() _write.withIOManager(ioManager) - if (writeRowLineage) { + if (writeRowTracking) { _write.withWriteType(writeType) } _write diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala index fb401c78e12b..ae62c37c4f78 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala @@ -24,7 +24,8 @@ import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.stats.SimpleStatsEvolutions import org.apache.paimon.table.FileStoreTable import org.apache.paimon.table.source.DataSplit -import org.apache.paimon.utils.{InternalRowUtils, ProjectedRow} +import org.apache.paimon.types.RowType +import org.apache.paimon.utils.ProjectedRow import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.JoinedRow @@ -40,7 +41,8 @@ class LocalAggregator(table: FileStoreTable) { private val partitionType = SparkTypeUtils.toPartitionType(table) private val groupByEvaluatorMap = new mutable.HashMap[InternalRow, Seq[AggFuncEvaluator[_]]]() private var requiredGroupByType: Seq[DataType] = _ - private var requiredGroupByIndexMapping: Seq[Int] = _ + private var requiredGroupByIndexMapping: Array[Int] = _ + private var requiredGroupByPaimonType: RowType = _ private var aggFuncEvaluatorGetter: () => Seq[AggFuncEvaluator[_]] = _ private var isInitialized = false private lazy val simpleStatsEvolutions = { @@ -78,15 +80,14 @@ class LocalAggregator(table: FileStoreTable) { partitionType.getFieldIndex(r.fieldNames().head) } + requiredGroupByPaimonType = partitionType.project(requiredGroupByIndexMapping) + isInitialized = true } private def requiredGroupByRow(partitionRow: BinaryRow): InternalRow = { - val projectedRow = - ProjectedRow.from(requiredGroupByIndexMapping.toArray).replaceRow(partitionRow) - // `ProjectedRow` does not support `hashCode`, so do a deep copy - val genericRow = InternalRowUtils.copyInternalRow(projectedRow, partitionType) - SparkInternalRow.create(partitionType).replace(genericRow) + val projectedRow = ProjectedRow.from(requiredGroupByIndexMapping).replaceRow(partitionRow) + SparkInternalRow.create(requiredGroupByPaimonType).replace(projectedRow) } def update(dataSplit: DataSplit): Unit = { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonAnalysis.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonAnalysis.scala index f44d9274e428..1ee0687faa69 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonAnalysis.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonAnalysis.scala @@ -54,7 +54,8 @@ class PaimonAnalysis(session: SparkSession) extends Rule[LogicalPlan] { case o @ PaimonDynamicPartitionOverwrite(r, d) if o.resolved => PaimonDynamicPartitionOverwriteCommand(r, d, o.query, o.writeOptions, o.isByName) - case merge: MergeIntoTable if isPaimonTable(merge.targetTable) && merge.childrenResolved => + case merge: MergeIntoTable + if !merge.resolved && isPaimonTable(merge.targetTable) && merge.childrenResolved => PaimonMergeIntoResolver(merge, session) case s @ ShowColumns(PaimonRelation(table), _, _) if s.resolved => diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonFunctionResolver.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonFunctionResolver.scala index 507cbf79deca..3fb76fa521c5 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonFunctionResolver.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonFunctionResolver.scala @@ -34,7 +34,7 @@ case class PaimonFunctionResolver(spark: SparkSession) extends Rule[LogicalPlan] plan.resolveOperatorsUpWithPruning(_.containsAnyPattern(UNRESOLVED_FUNCTION)) { case l: LogicalPlan => l.transformExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_FUNCTION)) { - case u: UnResolvedPaimonV1Function => + case u: UnResolvedPaimonV1Function if u.arguments.forall(_.resolved) => u.funcIdent.catalog match { case Some(catalog) => catalogManager.catalog(catalog) match { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala index fbb1aa7d8f56..8bc976b8669d 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala @@ -69,25 +69,6 @@ trait ExpressionHelper extends ExpressionHelperBase { Some(PredicateBuilder.and(predicates: _*)) } } - - def resolveFilter( - spark: SparkSession, - relation: DataSourceV2Relation, - conditionSql: String): Expression = { - val unResolvedExpression = spark.sessionState.sqlParser.parseExpression(conditionSql) - val filter = Filter(unResolvedExpression, relation) - spark.sessionState.analyzer.executeAndCheck(filter, new QueryPlanningTracker) match { - case filter: Filter => - try { - ConstantFolding.apply(filter).asInstanceOf[Filter].condition - } catch { - case _: Throwable => filter.condition - } - case _ => - throw new RuntimeException( - s"Could not resolve expression $conditionSql in relation: $relation") - } - } } trait ExpressionHelperBase extends PredicateHelper { @@ -210,6 +191,25 @@ trait ExpressionHelperBase extends PredicateHelper { s"Unsupported update expression: $other, only support update with PrimitiveType and StructType.") } + def resolveFilter( + spark: SparkSession, + relation: DataSourceV2Relation, + conditionSql: String): Expression = { + val unResolvedExpression = spark.sessionState.sqlParser.parseExpression(conditionSql) + val filter = Filter(unResolvedExpression, relation) + spark.sessionState.analyzer.execute(filter) match { + case filter: Filter => + try { + ConstantFolding.apply(filter).asInstanceOf[Filter].condition + } catch { + case _: Throwable => filter.condition + } + case _ => + throw new RuntimeException( + s"Could not resolve expression $conditionSql in relation: $relation") + } + } + def splitPruePartitionAndOtherPredicates( condition: Expression, partitionColumns: Seq[String], diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/DeleteFromPaimonTableCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/DeleteFromPaimonTableCommand.scala index 6080cd0378eb..461aca90ef2f 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/DeleteFromPaimonTableCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/DeleteFromPaimonTableCommand.scala @@ -119,6 +119,7 @@ case class DeleteFromPaimonTableCommand( } private def performNonPrimaryKeyDelete(sparkSession: SparkSession): Seq[CommitMessage] = { + val readSnapshot = table.snapshotManager().latestSnapshot() // Step1: the candidate data splits which are filtered by Paimon Predicate. val candidateDataSplits = findCandidateDataSplits(condition, relation.output) val dataFilePathToMeta = candidateFileMap(candidateDataSplits) @@ -133,7 +134,7 @@ case class DeleteFromPaimonTableCommand( sparkSession) // Step3: update the touched deletion vectors and index files - writer.persistDeletionVectors(deletionVectors) + writer.persistDeletionVectors(deletionVectors, readSnapshot) } else { // Step2: extract out the exactly files, which must have at least one record to be updated. val touchedFilePaths = @@ -147,11 +148,11 @@ case class DeleteFromPaimonTableCommand( val toRewriteScanRelation = Filter(Not(condition), newRelation) var data = createDataset(sparkSession, toRewriteScanRelation) if (coreOptions.rowTrackingEnabled()) { - data = selectWithRowLineage(data) + data = selectWithRowTracking(data) } // only write new files, should have no compaction - val addCommitMessage = writer.writeOnly().withRowLineage().write(data) + val addCommitMessage = writer.writeOnly().withRowTracking().write(data) // Step5: convert the deleted files that need to be written to commit message. val deletedCommitMessage = buildDeletedCommitMessage(touchedFiles) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala index 3e8d4deabecd..d956a9472f11 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala @@ -88,6 +88,8 @@ case class MergeIntoPaimonTable( } private def performMergeForNonPkTable(sparkSession: SparkSession): Seq[CommitMessage] = { + // todo: find a more universal way to make read snapshot consistent. + val readSnapshot = table.snapshotManager().latestSnapshot() val targetDS = createDataset(sparkSession, filteredTargetPlan) val sourceDS = createDataset(sparkSession, sourceTable) @@ -113,7 +115,7 @@ case class MergeIntoPaimonTable( val dvDS = ds.where( s"$ROW_KIND_COL = ${RowKind.DELETE.toByteValue} or $ROW_KIND_COL = ${RowKind.UPDATE_AFTER.toByteValue}") val deletionVectors = collectDeletionVectors(dataFilePathToMeta, dvDS, sparkSession) - val indexCommitMsg = writer.persistDeletionVectors(deletionVectors) + val indexCommitMsg = writer.persistDeletionVectors(deletionVectors, readSnapshot) // Step4: filter rows that should be written as the inserted/updated data. val toWriteDS = ds @@ -164,32 +166,32 @@ case class MergeIntoPaimonTable( val (_, filesToReadScan) = extractFilesAndCreateNewScan(filePathsToRead.toArray, dataFilePathToMeta, relation) - // If no files need to be rewritten, no need to write row lineage - val writeRowLineage = coreOptions.rowTrackingEnabled() && filesToRewritten.nonEmpty + // If no files need to be rewritten, no need to write row tracking + val writeRowTracking = coreOptions.rowTrackingEnabled() && filesToRewritten.nonEmpty // Add FILE_TOUCHED_COL to mark the row as coming from the touched file, if the row has not been // modified and was from touched file, it should be kept too. var filesToRewrittenDS = createDataset(sparkSession, filesToRewrittenScan).withColumn(FILE_TOUCHED_COL, lit(true)) - if (writeRowLineage) { - filesToRewrittenDS = selectWithRowLineage(filesToRewrittenDS) + if (writeRowTracking) { + filesToRewrittenDS = selectWithRowTracking(filesToRewrittenDS) } var filesToReadDS = createDataset(sparkSession, filesToReadScan).withColumn(FILE_TOUCHED_COL, lit(false)) - if (writeRowLineage) { - // For filesToReadScan we don't need to read row lineage meta cols, just add placeholders - ROW_LINEAGE_META_COLUMNS.foreach( + if (writeRowTracking) { + // For filesToReadScan we don't need to read row tracking meta cols, just add placeholders + ROW_TRACKING_META_COLUMNS.foreach( c => filesToReadDS = filesToReadDS.withColumn(c, lit(null))) } val toWriteDS = constructChangedRows( sparkSession, filesToRewrittenDS.union(filesToReadDS), - writeRowLineage = writeRowLineage).drop(ROW_KIND_COL) + writeRowTracking = writeRowTracking).drop(ROW_KIND_COL) - val finalWriter = if (writeRowLineage) { - writer.withRowLineage() + val finalWriter = if (writeRowTracking) { + writer.withRowTracking() } else { writer } @@ -207,7 +209,7 @@ case class MergeIntoPaimonTable( remainDeletedRow: Boolean = false, deletionVectorEnabled: Boolean = false, extraMetadataCols: Seq[PaimonMetadataColumn] = Seq.empty, - writeRowLineage: Boolean = false): Dataset[Row] = { + writeRowTracking: Boolean = false): Dataset[Row] = { val targetDS = targetDataset .withColumn(TARGET_ROW_COL, lit(true)) @@ -233,7 +235,7 @@ case class MergeIntoPaimonTable( def attribute(name: String) = joinedPlan.output.find(attr => resolver(name, attr.name)) val extraMetadataAttributes = extraMetadataCols.flatMap(metadataCol => attribute(metadataCol.name)) - val (rowIdAttr, sequenceNumberAttr) = if (writeRowLineage) { + val (rowIdAttr, sequenceNumberAttr) = if (writeRowTracking) { ( attribute(SpecialFields.ROW_ID.name()).get, attribute(SpecialFields.SEQUENCE_NUMBER.name()).get) @@ -241,7 +243,7 @@ case class MergeIntoPaimonTable( (null, null) } - val targetOutput = if (writeRowLineage) { + val targetOutput = if (writeRowTracking) { filteredTargetPlan.output ++ Seq(rowIdAttr, sequenceNumberAttr) } else { filteredTargetPlan.output @@ -253,7 +255,7 @@ case class MergeIntoPaimonTable( val columnExprs = actions.map { case UpdateAction(_, assignments) => var exprs = assignments.map(_.value) - if (writeRowLineage) { + if (writeRowTracking) { exprs ++= Seq(rowIdAttr, Literal(null)) } exprs :+ Literal(RowKind.UPDATE_AFTER.toByteValue) @@ -267,7 +269,7 @@ case class MergeIntoPaimonTable( } case InsertAction(_, assignments) => var exprs = assignments.map(_.value) - if (writeRowLineage) { + if (writeRowTracking) { exprs ++= Seq(rowIdAttr, sequenceNumberAttr) } exprs :+ Literal(RowKind.INSERT.toByteValue) @@ -280,7 +282,7 @@ case class MergeIntoPaimonTable( val notMatchedBySourceOutputs = processMergeActions(notMatchedBySourceActions) val notMatchedOutputs = processMergeActions(notMatchedActions) val outputFields = mutable.ArrayBuffer(targetTable.schema.fields: _*) - if (writeRowLineage) { + if (writeRowTracking) { outputFields += PaimonMetadataColumn.ROW_ID.toStructField outputFields += PaimonMetadataColumn.SEQUENCE_NUMBER.toStructField } @@ -324,7 +326,7 @@ case class MergeIntoPaimonTable( if (count > 0) { throw new RuntimeException( "Can't execute this MergeInto when there are some target rows that each of " + - "them match more then one source rows. It may lead to an unexpected result.") + "them match more than one source rows. It may lead to an unexpected result.") } } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonRowLevelCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonRowLevelCommand.scala index 7aea75404034..11dd94700942 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonRowLevelCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonRowLevelCommand.scala @@ -21,8 +21,7 @@ package org.apache.paimon.spark.commands import org.apache.paimon.CoreOptions import org.apache.paimon.deletionvectors.{Bitmap64DeletionVector, BitmapDeletionVector, DeletionVector} import org.apache.paimon.fs.Path -import org.apache.paimon.index.IndexFileMeta -import org.apache.paimon.io.{CompactIncrement, DataFileMeta, DataIncrement, IndexIncrement} +import org.apache.paimon.io.{CompactIncrement, DataFileMeta, DataIncrement} import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper import org.apache.paimon.spark.commands.SparkDataFileMeta.convertToSparkDataFileMeta import org.apache.paimon.spark.leafnode.PaimonLeafRunnableCommand @@ -72,12 +71,6 @@ trait PaimonRowLevelCommand } } - /** Gets a relative path against the table path. */ - protected def relativePath(absolutePath: String): String = { - val location = table.location().toUri - location.relativize(new URI(absolutePath)).toString - } - protected def findCandidateDataSplits( condition: Expression, output: Seq[Attribute]): Seq[DataSplit] = { @@ -122,7 +115,6 @@ trait PaimonRowLevelCommand .distinct() .as[String] .collect() - .map(relativePath) } protected def extractFilesAndCreateNewScan( @@ -137,15 +129,14 @@ trait PaimonRowLevelCommand (files, newRelation) } - /** Notice that, the key is a relative path, not just the file name. */ + /** Notice that, the key is a file path, not just the file name. */ protected def candidateFileMap( candidateDataSplits: Seq[DataSplit]): Map[String, SparkDataFileMeta] = { val totalBuckets = coreOptions.bucket() val candidateDataFiles = candidateDataSplits .flatMap(dataSplit => convertToSparkDataFileMeta(dataSplit, totalBuckets)) - val fileStorePathFactory = fileStore.pathFactory() candidateDataFiles - .map(file => (file.relativePath(fileStorePathFactory), file)) + .map(file => (file.filePath(), file)) .toMap } @@ -166,11 +157,12 @@ trait PaimonRowLevelCommand dataset: Dataset[Row], sparkSession: SparkSession): Dataset[SparkDeletionVector] = { import sparkSession.implicits._ - val dataFileToPartitionAndBucket = - dataFilePathToMeta.mapValues(meta => (meta.partition, meta.bucket)).toArray + // convert to a serializable map + val dataFileToPartitionAndBucket = dataFilePathToMeta.map { + case (k, v) => k -> (v.bucketPath, v.partition, v.bucket) + } val my_table = table - val location = my_table.location val dvBitmap64 = my_table.coreOptions().deletionVectorBitmap64() dataset .select(DV_META_COLUMNS.map(col): _*) @@ -184,18 +176,12 @@ trait PaimonRowLevelCommand dv.delete(iter.next()._2) } - val relativeFilePath = location.toUri.relativize(new URI(filePath)).toString - val (partition, bucket) = dataFileToPartitionAndBucket.toMap.apply(relativeFilePath) - val pathFactory = my_table.store().pathFactory() - val relativeBucketPath = pathFactory - .relativeBucketPath(partition, bucket) - .toString - + val (bucketPath, partition, bucket) = dataFileToPartitionAndBucket.apply(filePath) SparkDeletionVector( - relativeBucketPath, + bucketPath, SerializationUtils.serializeBinaryRow(partition), bucket, - new Path(filePath).getName, + filePath, DeletionVector.serializeToBytes(dv) ) } @@ -220,8 +206,7 @@ trait PaimonRowLevelCommand new CompactIncrement( Collections.emptyList[DataFileMeta], Collections.emptyList[DataFileMeta], - Collections.emptyList[DataFileMeta]), - new IndexIncrement(Collections.emptyList[IndexFileMeta]) + Collections.emptyList[DataFileMeta]) ) } .toSeq diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala index a440ad353a22..a130fde5278d 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonSparkWriter.scala @@ -18,15 +18,16 @@ package org.apache.paimon.spark.commands -import org.apache.paimon.CoreOptions +import org.apache.paimon.{CoreOptions, Snapshot} import org.apache.paimon.CoreOptions.{PartitionSinkStrategy, WRITE_ONLY} import org.apache.paimon.codegen.CodeGenUtils import org.apache.paimon.crosspartition.{IndexBootstrap, KeyPartOrRow} import org.apache.paimon.data.serializer.InternalSerializers import org.apache.paimon.deletionvectors.DeletionVector import org.apache.paimon.deletionvectors.append.BaseAppendDeleteFileMaintainer +import org.apache.paimon.fs.Path import org.apache.paimon.index.{BucketAssigner, SimpleHashBucketAssigner} -import org.apache.paimon.io.{CompactIncrement, DataIncrement, IndexIncrement} +import org.apache.paimon.io.{CompactIncrement, DataIncrement} import org.apache.paimon.manifest.FileKind import org.apache.paimon.spark.{SparkRow, SparkTableWrite, SparkTypeUtils} import org.apache.paimon.spark.catalog.functions.BucketFunction @@ -51,7 +52,7 @@ import java.util.Collections.singletonMap import scala.collection.JavaConverters._ -case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = false) +case class PaimonSparkWriter(table: FileStoreTable, writeRowTracking: Boolean = false) extends WriteHelper { private lazy val tableSchema = table.schema @@ -61,8 +62,8 @@ case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = f @transient private lazy val serializer = new CommitMessageSerializer private val writeType = { - if (writeRowLineage) { - SpecialFields.rowTypeWithRowLineage(table.rowType(), true) + if (writeRowTracking) { + SpecialFields.rowTypeWithRowTracking(table.rowType(), true) } else { table.rowType() } @@ -74,9 +75,9 @@ case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = f PaimonSparkWriter(table.copy(singletonMap(WRITE_ONLY.key(), "true"))) } - def withRowLineage(): PaimonSparkWriter = { + def withRowTracking(): PaimonSparkWriter = { if (coreOptions.rowTrackingEnabled()) { - PaimonSparkWriter(table, writeRowLineage = true) + PaimonSparkWriter(table, writeRowTracking = true) } else { this } @@ -88,7 +89,7 @@ case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = f val withInitBucketCol = bucketMode match { case BUCKET_UNAWARE => data - case CROSS_PARTITION if !data.schema.fieldNames.contains(ROW_KIND_COL) => + case KEY_DYNAMIC if !data.schema.fieldNames.contains(ROW_KIND_COL) => data .withColumn(ROW_KIND_COL, lit(RowKind.INSERT.toByteValue)) .withColumn(BUCKET_COL, lit(-1)) @@ -98,7 +99,7 @@ case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = f val bucketColIdx = SparkRowUtils.getFieldIndex(withInitBucketCol.schema, BUCKET_COL) val encoderGroupWithBucketCol = EncoderSerDeGroup(withInitBucketCol.schema) - def newWrite() = SparkTableWrite(writeBuilder, writeType, rowKindColIdx, writeRowLineage) + def newWrite() = SparkTableWrite(writeBuilder, writeType, rowKindColIdx, writeRowTracking) def sparkParallelism = { val defaultParallelism = sparkSession.sparkContext.defaultParallelism @@ -165,17 +166,14 @@ case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = f } val written: Dataset[Array[Byte]] = bucketMode match { - case CROSS_PARTITION => + case KEY_DYNAMIC => // Topology: input -> bootstrap -> shuffle by key hash -> bucket-assigner -> shuffle by partition & bucket val rowType = SparkTypeUtils.toPaimonType(withInitBucketCol.schema).asInstanceOf[RowType] val assignerParallelism = Option(coreOptions.dynamicBucketAssignerParallelism) .map(_.toInt) .getOrElse(sparkParallelism) - val bootstrapped = bootstrapAndRepartitionByKeyHash( - withInitBucketCol, - assignerParallelism, - rowKindColIdx, - rowType) + val bootstrapped = + bootstrapAndRepartitionByKeyHash(withInitBucketCol, assignerParallelism, rowKindColIdx) val globalDynamicBucketProcessor = GlobalDynamicBucketProcessor( @@ -258,7 +256,7 @@ case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = f } } val clusteringColumns = coreOptions.clusteringColumns() - if (!clusteringColumns.isEmpty) { + if ((!coreOptions.clusteringIncrementalEnabled()) && (!clusteringColumns.isEmpty)) { val strategy = coreOptions.clusteringStrategy(tableSchema.fields().size()) val sorter = TableSorter.getSorter(table, strategy, clusteringColumns) input = sorter.sort(data) @@ -304,12 +302,13 @@ case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = f * deletion vectors; else, one index file will contain all deletion vector with the same partition * and bucket. */ - def persistDeletionVectors(deletionVectors: Dataset[SparkDeletionVector]): Seq[CommitMessage] = { + def persistDeletionVectors( + deletionVectors: Dataset[SparkDeletionVector], + snapshot: Snapshot): Seq[CommitMessage] = { val sparkSession = deletionVectors.sparkSession import sparkSession.implicits._ - val snapshot = table.snapshotManager().latestSnapshotFromFileSystem() val serializedCommits = deletionVectors - .groupByKey(_.partitionAndBucket) + .groupByKey(_.bucketPath) .mapGroups { (_, iter: Iterator[SparkDeletionVector]) => val indexHandler = table.store().newIndexFileHandler() @@ -333,7 +332,7 @@ case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = f } dvIndexFileMaintainer.notifyNewDeletionVector( - sdv.dataFileName, + new Path(sdv.dataFilePath).getName, DeletionVector.deserializeFromBytes(sdv.deletionVector)) } val indexEntries = dvIndexFileMaintainer.persist() @@ -344,9 +343,13 @@ case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = f dvIndexFileMaintainer.getPartition, dvIndexFileMaintainer.getBucket, null, - DataIncrement.emptyIncrement(), - CompactIncrement.emptyIncrement(), - new IndexIncrement(added.map(_.indexFile).asJava, deleted.map(_.indexFile).asJava) + new DataIncrement( + java.util.Collections.emptyList(), + java.util.Collections.emptyList(), + java.util.Collections.emptyList(), + added.map(_.indexFile).asJava, + deleted.map(_.indexFile).asJava), + CompactIncrement.emptyIncrement() ) val serializer = new CommitMessageSerializer serializer.serialize(commitMessage) @@ -372,11 +375,11 @@ case class PaimonSparkWriter(table: FileStoreTable, writeRowLineage: Boolean = f private def bootstrapAndRepartitionByKeyHash( data: DataFrame, parallelism: Int, - rowKindColIdx: Int, - rowType: RowType): RDD[(KeyPartOrRow, Array[Byte])] = { + rowKindColIdx: Int): RDD[(KeyPartOrRow, Array[Byte])] = { val numSparkPartitions = data.rdd.getNumPartitions val primaryKeys = table.schema().primaryKeys() val bootstrapType = IndexBootstrap.bootstrapType(table.schema()) + val rowType = table.rowType() data.rdd .mapPartitions { iter => diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala index 569a84a74cf5..921d2e473505 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala @@ -27,17 +27,19 @@ import org.apache.paimon.utils.FileStorePathFactory import scala.collection.JavaConverters._ case class SparkDataFileMeta( + bucketPath: String, partition: BinaryRow, bucket: Int, totalBuckets: Int, dataFileMeta: DataFileMeta, deletionFile: Option[DeletionFile] = None) { - def relativePath(fileStorePathFactory: FileStorePathFactory): String = { - fileStorePathFactory - .relativeBucketPath(partition, bucket) - .toUri - .toString + "/" + dataFileMeta.fileName() + def filePath(): String = { + if (dataFileMeta.externalPath().isPresent) { + dataFileMeta.externalPath().get() + } else { + bucketPath + "/" + dataFileMeta.fileName() + } } } @@ -52,6 +54,7 @@ object SparkDataFileMeta { dataSplit.dataFiles().asScala.map { file => SparkDataFileMeta( + dataSplit.bucketPath(), dataSplit.partition, dataSplit.bucket, totalBuckets, diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDeletionVector.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDeletionVector.scala index 9fc7fdadcb33..53453af9135f 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDeletionVector.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDeletionVector.scala @@ -18,9 +18,8 @@ package org.apache.paimon.spark.commands -import org.apache.paimon.fs.Path import org.apache.paimon.table.source.DataSplit -import org.apache.paimon.utils.{FileStorePathFactory, SerializationUtils} +import org.apache.paimon.utils.SerializationUtils import scala.collection.JavaConverters._ @@ -29,31 +28,21 @@ import scala.collection.JavaConverters._ * or DeletionVector. */ case class SparkDeletionVector( - partitionAndBucket: String, + bucketPath: String, partition: Array[Byte], bucket: Int, - dataFileName: String, + dataFilePath: String, deletionVector: Array[Byte] -) { - def relativePath(pathFactory: FileStorePathFactory): String = { - val prefix = pathFactory - .relativeBucketPath(SerializationUtils.deserializeBinaryRow(partition), bucket) - .toUri - .toString + "/" - prefix + dataFileName - } -} +) object SparkDeletionVector { def toDataSplit( deletionVector: SparkDeletionVector, - root: Path, - pathFactory: FileStorePathFactory, dataFilePathToMeta: Map[String, SparkDataFileMeta]): DataSplit = { - val meta = dataFilePathToMeta(deletionVector.relativePath(pathFactory)) + val meta = dataFilePathToMeta(deletionVector.dataFilePath) DataSplit .builder() - .withBucketPath(root + "/" + deletionVector.partitionAndBucket) + .withBucketPath(deletionVector.bucketPath) .withPartition(SerializationUtils.deserializeBinaryRow(deletionVector.partition)) .withBucket(deletionVector.bucket) .withTotalBuckets(meta.totalBuckets) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala index e304480bae9a..1258ebc44955 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/UpdatePaimonTableCommand.scala @@ -71,6 +71,7 @@ case class UpdatePaimonTableCommand( /** Update for table without primary keys */ private def performUpdateForNonPkTable(sparkSession: SparkSession): Seq[CommitMessage] = { + val readSnapshot = table.snapshotManager().latestSnapshot() // Step1: the candidate data splits which are filtered by Paimon Predicate. val candidateDataSplits = findCandidateDataSplits(condition, relation.output) val dataFilePathToMeta = candidateFileMap(candidateDataSplits) @@ -80,8 +81,6 @@ case class UpdatePaimonTableCommand( logDebug("No file need to rewrote. It's an empty Commit.") Seq.empty[CommitMessage] } else { - val pathFactory = fileStore.pathFactory() - if (deletionVectorsEnabled) { // Step2: collect all the deletion vectors that marks the deleted rows. val deletionVectors = collectDeletionVectors( @@ -94,13 +93,12 @@ case class UpdatePaimonTableCommand( deletionVectors.cache() try { // Step3: write these updated data - val touchedDataSplits = deletionVectors.collect().map { - SparkDeletionVector.toDataSplit(_, root, pathFactory, dataFilePathToMeta) - } + val touchedDataSplits = + deletionVectors.collect().map(SparkDeletionVector.toDataSplit(_, dataFilePathToMeta)) val addCommitMessage = writeOnlyUpdatedData(sparkSession, touchedDataSplits) // Step4: write these deletion vectors. - val indexCommitMsg = writer.persistDeletionVectors(deletionVectors) + val indexCommitMsg = writer.persistDeletionVectors(deletionVectors, readSnapshot) addCommitMessage ++ indexCommitMsg } finally { @@ -161,7 +159,7 @@ case class UpdatePaimonTableCommand( } val data = createDataset(sparkSession, toUpdateScanRelation).select(updateColumns: _*) - writer.withRowLineage().write(data) + writer.withRowTracking().write(data) } private def optimizedIf( diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/schema/PaimonMetadataColumn.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/schema/PaimonMetadataColumn.scala index 0acb25e58b59..fa71c7530fa5 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/schema/PaimonMetadataColumn.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/schema/PaimonMetadataColumn.scala @@ -53,7 +53,7 @@ object PaimonMetadataColumn { val SEQUENCE_NUMBER_COLUMN: String = SpecialFields.SEQUENCE_NUMBER.name() val DV_META_COLUMNS: Seq[String] = Seq(FILE_PATH_COLUMN, ROW_INDEX_COLUMN) - val ROW_LINEAGE_META_COLUMNS: Seq[String] = Seq(ROW_ID_COLUMN, SEQUENCE_NUMBER_COLUMN) + val ROW_TRACKING_META_COLUMNS: Seq[String] = Seq(ROW_ID_COLUMN, SEQUENCE_NUMBER_COLUMN) val SUPPORTED_METADATA_COLUMNS: Seq[String] = Seq( ROW_INDEX_COLUMN, diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/OptionUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/OptionUtils.scala index dd35dd394d9c..d2dbdfd64dac 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/OptionUtils.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/OptionUtils.scala @@ -47,6 +47,21 @@ object OptionUtils extends SQLConfHelper { conf.getConfString(s"$PAIMON_OPTION_PREFIX${option.key()}", option.defaultValue().toString) } + private def getSparkVersionSpecificDefault(option: ConfigOption[_]): String = { + val sparkVersion = org.apache.spark.SPARK_VERSION + + option.key() match { + case key if key == SparkConnectorOptions.USE_V2_WRITE.key() => + if (sparkVersion >= "3.4") { + "false" + } else { + option.defaultValue().toString + } + case _ => + option.defaultValue().toString + } + } + def checkRequiredConfigurations(): Unit = { if (getOptionString(SparkConnectorOptions.REQUIRED_SPARK_CONFS_CHECK_ENABLED).toBoolean) { if (!paimonExtensionEnabled) { @@ -60,7 +75,13 @@ object OptionUtils extends SQLConfHelper { } def useV2Write(): Boolean = { - getOptionString(SparkConnectorOptions.USE_V2_WRITE).toBoolean + val defaultValue = getSparkVersionSpecificDefault(SparkConnectorOptions.USE_V2_WRITE) + conf + .getConfString( + s"$PAIMON_OPTION_PREFIX${SparkConnectorOptions.USE_V2_WRITE.key()}", + defaultValue + ) + .toBoolean } def writeMergeSchemaEnabled(): Boolean = { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/ScanPlanHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/ScanPlanHelper.scala index 8b2964cd5505..1799df48fe2a 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/ScanPlanHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/util/ScanPlanHelper.scala @@ -19,7 +19,7 @@ package org.apache.paimon.spark.util import org.apache.paimon.spark.SparkTable -import org.apache.paimon.spark.schema.PaimonMetadataColumn.{DV_META_COLUMNS, ROW_LINEAGE_META_COLUMNS} +import org.apache.paimon.spark.schema.PaimonMetadataColumn.{DV_META_COLUMNS, ROW_TRACKING_META_COLUMNS} import org.apache.paimon.table.{InnerTable, KnownSplitsTable} import org.apache.paimon.table.source.DataSplit @@ -60,8 +60,8 @@ trait ScanPlanHelper extends SQLConfHelper { selectWithAdditionalCols(data, DV_META_COLUMNS) } - def selectWithRowLineage(data: DataFrame): DataFrame = { - selectWithAdditionalCols(data, ROW_LINEAGE_META_COLUMNS) + def selectWithRowTracking(data: DataFrame): DataFrame = { + selectWithAdditionalCols(data, ROW_TRACKING_META_COLUMNS) } private def selectWithAdditionalCols(data: DataFrame, additionalCols: Seq[String]): DataFrame = { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonFunctionCommands.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonFunctionCommands.scala index bad7fe3c62c4..ddbd9df5ac1b 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonFunctionCommands.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonFunctionCommands.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.parser.extensions +import org.apache.paimon.catalog.Catalog.SYSTEM_DATABASE_NAME import org.apache.paimon.function.{Function => PaimonFunction} import org.apache.paimon.function.FunctionDefinition import org.apache.paimon.spark.SparkCatalog.FUNCTION_DEFINITION_NAME @@ -50,8 +51,17 @@ case class RewritePaimonFunctionCommands(spark: SparkSession) } val applied = plan.resolveOperatorsUp { + case CreateFunction(CatalogAndFunctionIdentifier(_, funcIdent, true), _, _, _, replace) => + if (replace) { + throw new UnsupportedOperationException( + s"$funcIdent is a temporary function, you should use `CREATE OR REPLACE TEMPORARY FUNCTION $funcIdent` or DROP TEMPORARY FUNCTION $funcIdent`.") + } else { + throw new UnsupportedOperationException( + s"$funcIdent is a temporary function and already exists.") + } + case CreateFunction( - CatalogAndFunctionIdentifier(v1FunctionCatalog: SupportV1Function, funcIdent), + CatalogAndFunctionIdentifier(v1FunctionCatalog: SupportV1Function, funcIdent, false), className, resources, ifExists, @@ -63,7 +73,7 @@ case class RewritePaimonFunctionCommands(spark: SparkSession) CreatePaimonV1FunctionCommand(v1FunctionCatalog, v1Function, ifExists, replace) case DropFunction( - CatalogAndFunctionIdentifier(v1FunctionCatalog: SupportV1Function, funcIdent), + CatalogAndFunctionIdentifier(v1FunctionCatalog: SupportV1Function, funcIdent, false), ifExists) => if (isPaimonBuildInFunction(funcIdent)) { throw new UnsupportedOperationException(s"Can't drop build-in function: $funcIdent") @@ -72,7 +82,7 @@ case class RewritePaimonFunctionCommands(spark: SparkSession) DropPaimonV1FunctionCommand(v1FunctionCatalog, funcIdent, ifExists) case d @ DescribeFunction( - CatalogAndFunctionIdentifier(v1FunctionCatalog: SupportV1Function, funcIdent), + CatalogAndFunctionIdentifier(v1FunctionCatalog: SupportV1Function, funcIdent, false), isExtended) // For Paimon built-in functions, Spark will resolve them by itself. if !isPaimonBuildInFunction(funcIdent) => @@ -98,7 +108,7 @@ case class RewritePaimonFunctionCommands(spark: SparkSession) l.transformExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_FUNCTION)) { case u: UnresolvedFunction => CatalogAndFunctionIdentifier.unapply(u.nameParts) match { - case Some((v1FunctionCatalog: SupportV1Function, funcIdent)) + case Some((v1FunctionCatalog: SupportV1Function, funcIdent, false)) // For Paimon built-in functions, Spark will resolve them by itself. if !isPaimonBuildInFunction(funcIdent) => // If the function is already registered, avoid redundant lookup in the catalog to reduce overhead. @@ -120,7 +130,7 @@ case class RewritePaimonFunctionCommands(spark: SparkSession) private object CatalogAndFunctionIdentifier { - def unapply(unresolved: LogicalPlan): Option[(CatalogPlugin, FunctionIdentifier)] = + def unapply(unresolved: LogicalPlan): Option[(CatalogPlugin, FunctionIdentifier, Boolean)] = unresolved match { case ui: UnresolvedIdentifier => unapply(ui.nameParts) @@ -130,11 +140,13 @@ case class RewritePaimonFunctionCommands(spark: SparkSession) None } - def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, FunctionIdentifier)] = { + def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, FunctionIdentifier, Boolean)] = { nameParts match { // Spark's built-in or tmp functions is without database name or catalog name. - case Seq(funName) if isSparkBuiltInOrTmpFunction(FunctionIdentifier(funName)) => + case Seq(funName) if isSparkBuiltInFunction(FunctionIdentifier(funName)) => None + case Seq(funName) if isSparkTmpFunc(FunctionIdentifier(funName)) => + Some(null, FunctionIdentifier(funName), true) case CatalogAndIdentifier(v1FunctionCatalog: SupportV1Function, ident) if v1FunctionCatalog.v1FunctionEnabled() => Some( @@ -142,7 +154,8 @@ case class RewritePaimonFunctionCommands(spark: SparkSession) FunctionIdentifier( ident.name(), Some(ident.namespace().last), - Some(v1FunctionCatalog.name))) + Some(v1FunctionCatalog.name)), + false) case _ => None } @@ -150,12 +163,20 @@ case class RewritePaimonFunctionCommands(spark: SparkSession) } private def isPaimonBuildInFunction(funcIdent: FunctionIdentifier): Boolean = { - PaimonFunctions.names.contains(funcIdent.funcName) + funcIdent.database match { + case Some(db) + if db == SYSTEM_DATABASE_NAME && PaimonFunctions.names.contains(funcIdent.funcName) => + true + case _ => false + } + } + + private def isSparkBuiltInFunction(funcIdent: FunctionIdentifier): Boolean = { + catalogManager.v1SessionCatalog.isBuiltinFunction(funcIdent) } - private def isSparkBuiltInOrTmpFunction(funcIdent: FunctionIdentifier): Boolean = { - catalogManager.v1SessionCatalog.isBuiltinFunction(funcIdent) || catalogManager.v1SessionCatalog - .isTemporaryFunction(funcIdent) + private def isSparkTmpFunc(funcIdent: FunctionIdentifier): Boolean = { + catalogManager.v1SessionCatalog.isTemporaryFunction(funcIdent) } private def isPaimonV1Function(fun: PaimonFunction): Boolean = { diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithRestTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithRestTest.java index 575eb727118e..ee8978c68767 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithRestTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithRestTest.java @@ -165,6 +165,9 @@ public void testFunction() throws Exception { .get(0) .toString()) .isEqualTo("[3]"); + assertThat(spark.sql("show user functions").collectAsList().toString()) + .contains("[paimon.db2.area_func]"); + paimonCatalog.dropFunction(identifier, false); cleanFunction(functionName); } diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/catalog/functions/BucketFunctionTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/catalog/functions/BucketFunctionTest.java index b8fbcdae4248..214965bf15a2 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/catalog/functions/BucketFunctionTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/catalog/functions/BucketFunctionTest.java @@ -237,7 +237,7 @@ private static void validateSparkBucketFunction(String... bucketColumns) { setupTable(bucketColumns); spark.sql( String.format( - "SELECT id_col, __paimon_bucket as expected_bucket, paimon.bucket(%s, %s) FROM %s", + "SELECT id_col, __paimon_bucket as expected_bucket, paimon.sys.bucket(%s, %s) FROM %s", NUM_BUCKETS, String.join(",", bucketColumns), TABLE_NAME)) .collectAsList() .forEach(row -> Assertions.assertThat(row.getInt(2)).isEqualTo(row.get(1))); @@ -328,7 +328,7 @@ public void testTimestampPrecisionNotEqualToSpark() { setupTable(TIMESTAMP_COL_PRECISION_3); spark.sql( String.format( - "SELECT id_col, __paimon_bucket as expected_bucket, paimon.bucket(%s, %s) FROM %s", + "SELECT id_col, __paimon_bucket as expected_bucket, paimon.sys.bucket(%s, %s) FROM %s", NUM_BUCKETS, String.join(",", TIMESTAMP_COL_PRECISION_3), TABLE_NAME)) diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/benchmark/BucketFunctionBenchmark.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/benchmark/BucketFunctionBenchmark.scala deleted file mode 100644 index 1ba618c6a1eb..000000000000 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/benchmark/BucketFunctionBenchmark.scala +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.spark.benchmark - -import org.apache.spark.sql.paimon.PaimonBenchmark - -object BucketFunctionBenchmark extends PaimonSqlBasedBenchmark { - - private val N = 20L * 1000 * 1000 - - override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - val benchmark = PaimonBenchmark(s"Bucket function", N, output = output) - - benchmark.addCase("Single int column", 3) { - _ => spark.range(N).selectExpr("fixed_bucket(10, id)").noop() - } - - benchmark.addCase("Single string column", 3) { - _ => spark.range(N).selectExpr("fixed_bucket(10, uuid())").noop() - } - - benchmark.addCase("Multiple columns", 3) { - _ => spark.range(N).selectExpr("fixed_bucket(10, id, uuid(), uuid())").noop() - } - - benchmark.run() - } -} diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala index 8d1b35cc1202..74f80befed81 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala @@ -24,15 +24,17 @@ import org.apache.paimon.spark.PaimonSparkTestBase import org.apache.paimon.table.FileStoreTable import org.apache.paimon.table.source.DataSplit -import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd, SparkListenerStageCompleted, SparkListenerStageSubmitted} +import org.apache.spark.scheduler.{SparkListener, SparkListenerStageSubmitted} import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.streaming.StreamTest import org.assertj.core.api.Assertions +import org.scalatest.time.Span import java.util import scala.collection.JavaConverters._ +import scala.util.Random /** Test compact procedure. See [[CompactProcedure]]. */ abstract class CompactProcedureTestBase extends PaimonSparkTestBase with StreamTest { @@ -342,6 +344,19 @@ abstract class CompactProcedureTestBase extends PaimonSparkTestBase with StreamT } } + test("Paimon Procedure: sort compact with partition filter") { + withTable("t") { + sql("CREATE TABLE t (a INT, pt INT) PARTITIONED BY (pt)") + sql("INSERT INTO t VALUES (1, 1)") + sql("INSERT INTO t VALUES (2, 1)") + sql( + "CALL sys.compact(table => 't', order_strategy => 'order', where => 'pt = 1', order_by => 'a')") + val table = loadTable("t") + assert(table.latestSnapshot().get().commitKind.equals(CommitKind.OVERWRITE)) + checkAnswer(sql("SELECT * FROM t ORDER BY a"), Seq(Row(1, 1), Row(2, 1))) + } + } + test("Paimon Procedure: compact for pk") { failAfter(streamingTimeout) { withTempDir { @@ -795,6 +810,475 @@ abstract class CompactProcedureTestBase extends PaimonSparkTestBase with StreamT } } + test("Paimon Procedure: cluster for unpartitioned table") { + failAfter(streamingTimeout) { + withTempDir { + checkpointDir => + spark.sql( + s""" + |CREATE TABLE T (a INT, b INT, c STRING) + |TBLPROPERTIES ('bucket'='-1','num-levels'='6', 'num-sorted-run.compaction-trigger'='2', 'clustering.columns'='a,b', 'clustering.strategy'='zorder', 'clustering.incremental' = 'true') + |""".stripMargin) + val location = loadTable("T").location().toString + + val inputData = MemoryStream[(Int, Int, String)] + val stream = inputData + .toDS() + .toDF("a", "b", "c") + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .foreachBatch { + (batch: Dataset[Row], _: Long) => + batch.write.format("paimon").mode("append").save(location) + } + .start() + + val query = () => spark.sql("SELECT * FROM T") + + try { + val random = new Random() + val randomStr = random.nextString(40) + // first write + inputData.addData((0, 0, randomStr)) + inputData.addData((0, 1, randomStr)) + inputData.addData((0, 2, randomStr)) + inputData.addData((1, 0, randomStr)) + inputData.addData((1, 1, randomStr)) + inputData.addData((1, 2, randomStr)) + inputData.addData((2, 0, randomStr)) + inputData.addData((2, 1, randomStr)) + inputData.addData((2, 2, randomStr)) + stream.processAllAvailable() + + val result = new util.ArrayList[Row]() + for (a <- 0 until 3) { + for (b <- 0 until 3) { + result.add(Row(a, b, randomStr)) + } + } + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result) + + // first cluster, the outputLevel should be 5 + checkAnswer(spark.sql("CALL paimon.sys.compact(table => 'T')"), Row(true) :: Nil) + + // first cluster result + val result2 = new util.ArrayList[Row]() + result2.add(0, Row(0, 0, randomStr)) + result2.add(1, Row(0, 1, randomStr)) + result2.add(2, Row(1, 0, randomStr)) + result2.add(3, Row(1, 1, randomStr)) + result2.add(4, Row(0, 2, randomStr)) + result2.add(5, Row(1, 2, randomStr)) + result2.add(6, Row(2, 0, randomStr)) + result2.add(7, Row(2, 1, randomStr)) + result2.add(8, Row(2, 2, randomStr)) + + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result2) + + var clusteredTable = loadTable("T") + checkSnapshot(clusteredTable) + var dataSplits = clusteredTable.newSnapshotReader().read().dataSplits() + Assertions.assertThat(dataSplits.size()).isEqualTo(1) + Assertions.assertThat(dataSplits.get(0).dataFiles().size()).isEqualTo(1) + Assertions.assertThat(dataSplits.get(0).dataFiles().get(0).level()).isEqualTo(5) + + // second write + inputData.addData((0, 3, null), (1, 3, null), (2, 3, null)) + inputData.addData((3, 0, null), (3, 1, null), (3, 2, null), (3, 3, null)) + stream.processAllAvailable() + + val result3 = new util.ArrayList[Row]() + result3.addAll(result2) + for (a <- 0 until 3) { + result3.add(Row(a, 3, null)) + } + for (b <- 0 until 4) { + result3.add(Row(3, b, null)) + } + + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result3) + + // second cluster, the outputLevel should be 4 + checkAnswer(spark.sql("CALL paimon.sys.compact(table => 'T')"), Row(true) :: Nil) + // second cluster result, level-5 and level-4 are individually ordered + val result4 = new util.ArrayList[Row]() + result4.addAll(result2) + result4.add(Row(0, 3, null)) + result4.add(Row(1, 3, null)) + result4.add(Row(3, 0, null)) + result4.add(Row(3, 1, null)) + result4.add(Row(2, 3, null)) + result4.add(Row(3, 2, null)) + result4.add(Row(3, 3, null)) + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result4) + + clusteredTable = loadTable("T") + checkSnapshot(clusteredTable) + dataSplits = clusteredTable.newSnapshotReader().read().dataSplits() + Assertions.assertThat(dataSplits.size()).isEqualTo(1) + Assertions.assertThat(dataSplits.get(0).dataFiles().size()).isEqualTo(2) + Assertions.assertThat(dataSplits.get(0).dataFiles().get(0).level()).isEqualTo(5) + Assertions.assertThat(dataSplits.get(0).dataFiles().get(1).level()).isEqualTo(4) + + // full cluster + checkAnswer( + spark.sql("CALL paimon.sys.compact(table => 'T', compact_strategy => 'full')"), + Row(true) :: Nil) + val result5 = new util.ArrayList[Row]() + result5.add(Row(0, 0, randomStr)) + result5.add(Row(0, 1, randomStr)) + result5.add(Row(1, 0, randomStr)) + result5.add(Row(1, 1, randomStr)) + result5.add(Row(0, 2, randomStr)) + result5.add(Row(0, 3, null)) + result5.add(Row(1, 2, randomStr)) + result5.add(Row(1, 3, null)) + result5.add(Row(2, 0, randomStr)) + result5.add(Row(2, 1, randomStr)) + result5.add(Row(3, 0, null)) + result5.add(Row(3, 1, null)) + result5.add(Row(2, 2, randomStr)) + result5.add(Row(2, 3, null)) + result5.add(Row(3, 2, null)) + result5.add(Row(3, 3, null)) + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result5) + + clusteredTable = loadTable("T") + checkSnapshot(clusteredTable) + dataSplits = clusteredTable.newSnapshotReader().read().dataSplits() + Assertions.assertThat(dataSplits.size()).isEqualTo(1) + Assertions.assertThat(dataSplits.get(0).dataFiles().size()).isEqualTo(1) + Assertions.assertThat(dataSplits.get(0).dataFiles().get(0).level()).isEqualTo(5) + + } finally { + stream.stop() + } + } + } + } + + test("Paimon Procedure: cluster for partitioned table") { + failAfter(streamingTimeout) { + withTempDir { + checkpointDir => + spark.sql( + s""" + |CREATE TABLE T (a INT, b INT, c STRING, pt INT) + |PARTITIONED BY (pt) + |TBLPROPERTIES ('bucket'='-1', 'num-levels'='6', 'num-sorted-run.compaction-trigger'='2', 'clustering.columns'='a,b', 'clustering.strategy'='zorder', 'clustering.incremental' = 'true') + |""".stripMargin) + val location = loadTable("T").location().toString + + val inputData = MemoryStream[(Int, Int, String, Int)] + val stream = inputData + .toDS() + .toDF("a", "b", "c", "pt") + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .foreachBatch { + (batch: Dataset[Row], _: Long) => + batch.write.format("paimon").mode("append").save(location) + } + .start() + + val query = () => spark.sql("SELECT * FROM T ORDER BY pt") + + try { + val random = new Random() + val randomStr = random.nextString(50) + // first write + for (pt <- 0 until 2) { + val c = if (pt == 0) randomStr else null + inputData.addData((0, 0, c, pt)) + inputData.addData((0, 1, c, pt)) + inputData.addData((0, 2, c, pt)) + inputData.addData((1, 0, c, pt)) + inputData.addData((1, 1, c, pt)) + inputData.addData((1, 2, c, pt)) + inputData.addData((2, 0, c, pt)) + inputData.addData((2, 1, c, pt)) + inputData.addData((2, 2, c, pt)) + } + stream.processAllAvailable() + + val result = new util.ArrayList[Row]() + for (pt <- 0 until 2) { + for (a <- 0 until 3) { + for (b <- 0 until 3) { + val c = if (pt == 0) randomStr else null + result.add(Row(a, b, c, pt)) + } + } + } + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result) + + // first cluster, the outputLevel should be 5 + checkAnswer(spark.sql("CALL paimon.sys.compact(table => 'T')"), Row(true) :: Nil) + + // first cluster result + val result2 = new util.ArrayList[Row]() + for (pt <- 0 until 2) { + val c = if (pt == 0) randomStr else null + result2.add(Row(0, 0, c, pt)) + result2.add(Row(0, 1, c, pt)) + result2.add(Row(1, 0, c, pt)) + result2.add(Row(1, 1, c, pt)) + result2.add(Row(0, 2, c, pt)) + result2.add(Row(1, 2, c, pt)) + result2.add(Row(2, 0, c, pt)) + result2.add(Row(2, 1, c, pt)) + result2.add(Row(2, 2, c, pt)) + } + + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result2) + + var clusteredTable = loadTable("T") + checkSnapshot(clusteredTable) + var dataSplits = clusteredTable.newSnapshotReader().read().dataSplits() + Assertions.assertThat(dataSplits.size()).isEqualTo(2) + dataSplits.forEach( + dataSplit => { + Assertions.assertThat(dataSplit.dataFiles().size()).isEqualTo(1) + Assertions.assertThat(dataSplit.dataFiles().get(0).level()).isEqualTo(5) + }) + + // second write + for (pt <- 0 until 2) { + inputData.addData((0, 3, null, pt), (1, 3, null, pt), (2, 3, null, pt)) + inputData.addData( + (3, 0, null, pt), + (3, 1, null, pt), + (3, 2, null, pt), + (3, 3, null, pt)) + } + stream.processAllAvailable() + + val result3 = new util.ArrayList[Row]() + for (pt <- 0 until 2) { + val c = if (pt == 0) randomStr else null + result3.add(Row(0, 0, c, pt)) + result3.add(Row(0, 1, c, pt)) + result3.add(Row(1, 0, c, pt)) + result3.add(Row(1, 1, c, pt)) + result3.add(Row(0, 2, c, pt)) + result3.add(Row(1, 2, c, pt)) + result3.add(Row(2, 0, c, pt)) + result3.add(Row(2, 1, c, pt)) + result3.add(Row(2, 2, c, pt)) + for (a <- 0 until 3) { + result3.add(Row(a, 3, null, pt)) + } + for (b <- 0 until 4) { + result3.add(Row(3, b, null, pt)) + } + } + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result3) + + // second cluster + checkAnswer(spark.sql("CALL paimon.sys.compact(table => 'T')"), Row(true) :: Nil) + val result4 = new util.ArrayList[Row]() + // for partition-0: only file in level-0 will be picked for clustering, outputLevel is 4 + result4.add(Row(0, 0, randomStr, 0)) + result4.add(Row(0, 1, randomStr, 0)) + result4.add(Row(1, 0, randomStr, 0)) + result4.add(Row(1, 1, randomStr, 0)) + result4.add(Row(0, 2, randomStr, 0)) + result4.add(Row(1, 2, randomStr, 0)) + result4.add(Row(2, 0, randomStr, 0)) + result4.add(Row(2, 1, randomStr, 0)) + result4.add(Row(2, 2, randomStr, 0)) + result4.add(Row(0, 3, null, 0)) + result4.add(Row(1, 3, null, 0)) + result4.add(Row(3, 0, null, 0)) + result4.add(Row(3, 1, null, 0)) + result4.add(Row(2, 3, null, 0)) + result4.add(Row(3, 2, null, 0)) + result4.add(Row(3, 3, null, 0)) + // for partition-1:all files will be picked for clustering, outputLevel is 5 + result4.add(Row(0, 0, null, 1)) + result4.add(Row(0, 1, null, 1)) + result4.add(Row(1, 0, null, 1)) + result4.add(Row(1, 1, null, 1)) + result4.add(Row(0, 2, null, 1)) + result4.add(Row(0, 3, null, 1)) + result4.add(Row(1, 2, null, 1)) + result4.add(Row(1, 3, null, 1)) + result4.add(Row(2, 0, null, 1)) + result4.add(Row(2, 1, null, 1)) + result4.add(Row(3, 0, null, 1)) + result4.add(Row(3, 1, null, 1)) + result4.add(Row(2, 2, null, 1)) + result4.add(Row(2, 3, null, 1)) + result4.add(Row(3, 2, null, 1)) + result4.add(Row(3, 3, null, 1)) + + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result4) + + clusteredTable = loadTable("T") + checkSnapshot(clusteredTable) + dataSplits = clusteredTable.newSnapshotReader().read().dataSplits() + Assertions.assertThat(dataSplits.size()).isEqualTo(2) + dataSplits.forEach( + dataSplit => { + if (dataSplit.partition().getInt(0) == 1) { + // partition-1 + Assertions.assertThat(dataSplit.dataFiles().size()).isEqualTo(1) + Assertions.assertThat(dataSplit.dataFiles().get(0).level()).isEqualTo(5) + } else { + // partition-0 + Assertions.assertThat(dataSplit.dataFiles().size()).isEqualTo(2) + Assertions.assertThat(dataSplit.dataFiles().get(0).level()).isEqualTo(5) + Assertions.assertThat(dataSplit.dataFiles().get(1).level()).isEqualTo(4) + } + }) + } finally { + stream.stop() + } + } + } + } + + test("Paimon Procedure: cluster with deletion vectors") { + failAfter(Span(5, org.scalatest.time.Minutes)) { + withTempDir { + checkpointDir => + spark.sql( + s""" + |CREATE TABLE T (a INT, b INT, c STRING) + |TBLPROPERTIES ('bucket'='-1', 'deletion-vectors.enabled'='true','num-levels'='6', 'num-sorted-run.compaction-trigger'='2', 'clustering.columns'='a,b', 'clustering.strategy'='zorder', 'clustering.incremental' = 'true') + |""".stripMargin) + val location = loadTable("T").location().toString + + val inputData = MemoryStream[(Int, Int, String)] + val stream = inputData + .toDS() + .toDF("a", "b", "c") + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .foreachBatch { + (batch: Dataset[Row], _: Long) => + batch.write.format("paimon").mode("append").save(location) + } + .start() + + val query = () => spark.sql("SELECT * FROM T") + + try { + val random = new Random() + val randomStr = random.nextString(40) + // first write + inputData.addData((0, 0, randomStr)) + inputData.addData((0, 1, randomStr)) + inputData.addData((0, 2, randomStr)) + inputData.addData((1, 0, randomStr)) + inputData.addData((1, 1, randomStr)) + inputData.addData((1, 2, randomStr)) + inputData.addData((2, 0, randomStr)) + inputData.addData((2, 1, randomStr)) + inputData.addData((2, 2, randomStr)) + stream.processAllAvailable() + + val result = new util.ArrayList[Row]() + for (a <- 0 until 3) { + for (b <- 0 until 3) { + result.add(Row(a, b, randomStr)) + } + } + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result) + + // first cluster, the outputLevel should be 5 + checkAnswer(spark.sql("CALL paimon.sys.compact(table => 'T')"), Row(true) :: Nil) + + // first cluster result + val result2 = new util.ArrayList[Row]() + result2.add(0, Row(0, 0, randomStr)) + result2.add(1, Row(0, 1, randomStr)) + result2.add(2, Row(1, 0, randomStr)) + result2.add(3, Row(1, 1, randomStr)) + result2.add(4, Row(0, 2, randomStr)) + result2.add(5, Row(1, 2, randomStr)) + result2.add(6, Row(2, 0, randomStr)) + result2.add(7, Row(2, 1, randomStr)) + result2.add(8, Row(2, 2, randomStr)) + + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result2) + + var clusteredTable = loadTable("T") + checkSnapshot(clusteredTable) + var dataSplits = clusteredTable.newSnapshotReader().read().dataSplits() + Assertions.assertThat(dataSplits.size()).isEqualTo(1) + Assertions.assertThat(dataSplits.get(0).dataFiles().size()).isEqualTo(1) + Assertions.assertThat(dataSplits.get(0).dataFiles().get(0).level()).isEqualTo(5) + + // second write + inputData.addData((0, 3, null), (1, 3, null), (2, 3, null)) + inputData.addData((3, 0, null), (3, 1, null), (3, 2, null), (3, 3, null)) + stream.processAllAvailable() + + // delete (0,0), which is in level-5 file + spark.sql("DELETE FROM T WHERE a=0 and b=0;").collect() + // delete (0,3), which is in level-0 file + spark.sql("DELETE FROM T WHERE a=0 and b=3;").collect() + + val result3 = new util.ArrayList[Row]() + result3.addAll(result2.subList(1, result2.size())) + for (a <- 1 until 3) { + result3.add(Row(a, 3, null)) + } + for (b <- 0 until 4) { + result3.add(Row(3, b, null)) + } + + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result3) + + // second cluster, the outputLevel should be 4. dv index for level-0 will be updated + // and dv index for level-5 will be retained + checkAnswer(spark.sql("CALL paimon.sys.compact(table => 'T')"), Row(true) :: Nil) + // second cluster result, level-5 and level-4 are individually ordered + val result4 = new util.ArrayList[Row]() + result4.addAll(result2.subList(1, result2.size())) + result4.add(Row(1, 3, null)) + result4.add(Row(3, 0, null)) + result4.add(Row(3, 1, null)) + result4.add(Row(2, 3, null)) + result4.add(Row(3, 2, null)) + result4.add(Row(3, 3, null)) + Assertions.assertThat(query().collect()).containsExactlyElementsOf(result4) + + clusteredTable = loadTable("T") + checkSnapshot(clusteredTable) + dataSplits = clusteredTable.newSnapshotReader().read().dataSplits() + Assertions.assertThat(dataSplits.size()).isEqualTo(1) + Assertions.assertThat(dataSplits.get(0).dataFiles().size()).isEqualTo(2) + Assertions.assertThat(dataSplits.get(0).dataFiles().get(0).level()).isEqualTo(5) + Assertions.assertThat(dataSplits.get(0).deletionFiles().get().get(0)).isNotNull + Assertions.assertThat(dataSplits.get(0).dataFiles().get(1).level()).isEqualTo(4) + Assertions.assertThat(dataSplits.get(0).deletionFiles().get().get(1)).isNull() + + // full cluster + checkAnswer( + spark.sql("CALL paimon.sys.compact(table => 'T', compact_strategy => 'full')"), + Row(true) :: Nil) + clusteredTable = loadTable("T") + checkSnapshot(clusteredTable) + dataSplits = clusteredTable.newSnapshotReader().read().dataSplits() + Assertions.assertThat(dataSplits.size()).isEqualTo(1) + Assertions.assertThat(dataSplits.get(0).dataFiles().size()).isEqualTo(1) + Assertions.assertThat(dataSplits.get(0).deletionFiles().get().get(0)).isNull() + + } finally { + stream.stop() + } + } + } + } + + def checkSnapshot(table: FileStoreTable): Unit = { + Assertions + .assertThat(table.latestSnapshot().get().commitKind().toString) + .isEqualTo(CommitKind.COMPACT.toString) + } + def lastSnapshotCommand(table: FileStoreTable): CommitKind = { table.snapshotManager().latestSnapshot().commitKind() } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala index 6b38c1f5f027..ea05d94dd380 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala @@ -18,7 +18,7 @@ package org.apache.paimon.spark.sql -import org.apache.paimon.CoreOptions +import org.apache.paimon.{CoreOptions, Snapshot} import org.apache.paimon.CoreOptions.MergeEngine import org.apache.paimon.spark.PaimonSparkTestBase @@ -435,4 +435,40 @@ abstract class DeleteFromTableTestBase extends PaimonSparkTestBase { } } } + + test("Paimon delete: non pk table commit kind") { + for (dvEnabled <- Seq(true, false)) { + withTable("t") { + sql( + s"CREATE TABLE t (id INT, data INT) TBLPROPERTIES ('deletion-vectors.enabled' = '$dvEnabled')") + sql("INSERT INTO t SELECT /*+ REPARTITION(1) */ id, id AS data FROM range(1, 4)") + + sql("DELETE FROM t WHERE id = 1") + checkAnswer(sql("SELECT * FROM t ORDER BY id"), Seq(Row(2, 2), Row(3, 3))) + val table = loadTable("t") + var latestSnapshot = table.latestSnapshot().get() + assert(latestSnapshot.id == 2) + assert(latestSnapshot.commitKind.equals(Snapshot.CommitKind.OVERWRITE)) + + sql("DELETE FROM t WHERE id = 2") + checkAnswer(sql("SELECT * FROM t ORDER BY id"), Seq(Row(3, 3))) + latestSnapshot = table.latestSnapshot().get() + assert(latestSnapshot.id == 3) + assert(latestSnapshot.commitKind.equals(Snapshot.CommitKind.OVERWRITE)) + } + } + } + + test("Paimon delete: pk dv table commit kind") { + withTable("t") { + sql( + s"CREATE TABLE t (id INT, data INT) TBLPROPERTIES ('deletion-vectors.enabled' = 'true', 'primary-key' = 'id')") + sql("INSERT INTO t SELECT /*+ REPARTITION(1) */ id, id AS data FROM range(1, 4)") + sql("DELETE FROM t WHERE id = 1") + val table = loadTable("t") + val latestSnapshot = table.latestSnapshot().get() + assert(latestSnapshot.id == 4) + assert(latestSnapshot.commitKind.equals(Snapshot.CommitKind.COMPACT)) + } + } } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala index 68e741fb13c1..e3a5896ab165 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala @@ -660,7 +660,7 @@ class DeletionVectorTest extends PaimonSparkTestBase with AdaptiveSparkPlanHelpe assert(dvMeta.cardinality() == 334) } - test("Paimon deletionVector: delete from non-pk table with data file path") { + test("Paimon deletionVector: delete from non-pk table with data file directory") { sql(s""" |CREATE TABLE T (id INT) |TBLPROPERTIES ( @@ -677,6 +677,37 @@ class DeletionVectorTest extends PaimonSparkTestBase with AdaptiveSparkPlanHelpe checkAnswer(sql("SELECT count(*) FROM T"), Row(49665)) } + test("Paimon deletionVector: delete from non-pk table with data file external paths") { + withTempDir { + tmpDir => + { + sql(s""" + |CREATE TABLE T (id INT, v INT) + |TBLPROPERTIES ( + | 'deletion-vectors.enabled' = 'true', + | 'deletion-vectors.bitmap64' = '${Random.nextBoolean()}', + | 'bucket-key' = 'id', + | 'bucket' = '1', + | 'data-file.external-paths' = 'file://${tmpDir.getCanonicalPath}', + | 'data-file.external-paths.strategy' = 'round-robin' + |) + |""".stripMargin) + sql("INSERT INTO T SELECT /*+ REPARTITION(1) */ id, id FROM range (1, 50000)") + sql("DELETE FROM T WHERE id >= 111 and id <= 444") + checkAnswer(sql("SELECT count(*) FROM T"), Row(49665)) + checkAnswer(sql("SELECT sum(v) FROM T"), Row(1249882315L)) + + sql("UPDATE T SET v = v + 1 WHERE id >= 555 and id <= 666") + checkAnswer(sql("SELECT count(*) FROM T"), Row(49665)) + checkAnswer(sql("SELECT sum(v) FROM T"), Row(1249882427L)) + + sql("UPDATE T SET v = v + 1 WHERE id >= 600 and id <= 800") + checkAnswer(sql("SELECT count(*) FROM T"), Row(49665)) + checkAnswer(sql("SELECT sum(v) FROM T"), Row(1249882628L)) + } + } + } + test("Paimon deletionVector: work v1 with v2") { sql(s""" |CREATE TABLE T (id INT) diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala index f005156182b4..fee7973acabe 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala @@ -200,4 +200,25 @@ class DynamicBucketTableTest extends PaimonSparkTestBase { checkAnswer(sql("SELECT DISTINCT bucket FROM `T$FILES`"), Seq(Row(0), Row(1), Row(2))) } + + test("Paimon cross partition table: write multiple cols") { + withTable("t") { + val columns = (0 until 118).map(i => s"c$i STRING").mkString(", ") + sql(s""" + |CREATE TABLE t ( + | $columns, + | pt STRING + |) + |PARTITIONED BY (pt) + |TBLPROPERTIES ( + | 'primary-key' = 'c0'); + |""".stripMargin) + + sql("INSERT INTO t (c0, c1, c2, pt) VALUES ('c0','c1','c2','pt')") + checkAnswer( + sql("SELECT c0, c1, c2, pt from t"), + Row("c0", "c1", "c2", "pt") + ) + } + } } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala index 467cd72f7777..92e2c3ee1983 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala @@ -20,13 +20,16 @@ package org.apache.paimon.spark.sql import org.apache.paimon.spark.PaimonSparkTestBase -import org.apache.spark.sql.Row +import org.apache.spark.sql.{Row, SaveMode} +import org.apache.spark.sql.functions.lit import org.apache.spark.sql.types._ import java.sql.{Date, Timestamp} abstract class InsertOverwriteTableTestBase extends PaimonSparkTestBase { + import testImplicits._ + fileFormats.foreach { fileFormat => Seq(true, false).foreach { @@ -639,4 +642,16 @@ abstract class InsertOverwriteTableTestBase extends PaimonSparkTestBase { } } } + + test("Paimon Insert: insert unsupported type") { + val rows = Seq(("1", "2")).toDF("f1", "f2") + assert(intercept[Exception] { + rows + .withColumn("bad", lit(null)) + .write + .format("paimon") + .mode(SaveMode.Overwrite) + .saveAsTable("badTable") + }.getMessage.contains("Not a supported type: void")) + } } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala index 4564e1646cf1..9866f4712997 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala @@ -18,10 +18,15 @@ package org.apache.paimon.spark.sql -import org.apache.paimon.spark.{PaimonPrimaryKeyTable, PaimonSparkTestBase, PaimonTableTest} +import org.apache.paimon.Snapshot +import org.apache.paimon.spark.{PaimonAppendTable, PaimonPrimaryKeyTable, PaimonSparkTestBase, PaimonTableTest} import org.apache.spark.sql.Row +import scala.concurrent.{Await, Future} +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration.DurationInt + abstract class MergeIntoTableTestBase extends PaimonSparkTestBase with PaimonTableTest { import testImplicits._ @@ -532,7 +537,7 @@ abstract class MergeIntoTableTestBase extends PaimonSparkTestBase with PaimonTab |THEN INSERT (a, b, c) values (a, b, c) |""".stripMargin) }.getMessage - assert(error.contains("match more then one source rows")) + assert(error.contains("match more than one source rows")) } } @@ -714,3 +719,90 @@ trait MergeIntoPrimaryKeyTableTest extends PaimonSparkTestBase with PaimonPrimar } } } + +trait MergeIntoAppendTableTest extends PaimonSparkTestBase with PaimonAppendTable { + + test("Paimon MergeInto: non pk table commit kind") { + withTable("s", "t") { + createTable("s", "id INT, b INT, c INT", Seq("id")) + sql("INSERT INTO s VALUES (1, 1, 1)") + + createTable("t", "id INT, b INT, c INT", Seq("id")) + sql("INSERT INTO t VALUES (2, 2, 2)") + + sql(""" + |MERGE INTO t + |USING s + |ON t.id = s.id + |WHEN NOT MATCHED THEN + |INSERT (id, b, c) VALUES (s.id, s.b, s.c); + |""".stripMargin) + + val table = loadTable("t") + var latestSnapshot = table.latestSnapshot().get() + assert(latestSnapshot.id == 2) + // no old data is deleted, so the commit kind is APPEND + assert(latestSnapshot.commitKind.equals(Snapshot.CommitKind.APPEND)) + + sql("INSERT INTO s VALUES (2, 22, 22)") + sql(""" + |MERGE INTO t + |USING s + |ON t.id = s.id + |WHEN MATCHED THEN + |UPDATE SET id = s.id, b = s.b, c = s.c + |""".stripMargin) + latestSnapshot = table.latestSnapshot().get() + assert(latestSnapshot.id == 3) + // new data is updated, so the commit kind is OVERWRITE + assert(latestSnapshot.commitKind.equals(Snapshot.CommitKind.OVERWRITE)) + } + } + + test("Paimon MergeInto: concurrent merge and compact") { + for (dvEnabled <- Seq("true", "false")) { + withTable("s", "t") { + sql("CREATE TABLE s (id INT, b INT, c INT)") + sql("INSERT INTO s VALUES (1, 1, 1)") + + sql( + s"CREATE TABLE t (id INT, b INT, c INT) TBLPROPERTIES ('deletion-vectors.enabled' = '$dvEnabled')") + sql("INSERT INTO t VALUES (1, 1, 1)") + + val mergeInto = Future { + for (_ <- 1 to 10) { + try { + sql(""" + |MERGE INTO t + |USING s + |ON t.id = s.id + |WHEN MATCHED THEN + |UPDATE SET t.id = s.id, t.b = s.b + t.b, t.c = s.c + t.c + |""".stripMargin) + } catch { + case a: Throwable => + assert( + a.getMessage.contains("Conflicts during commits") || a.getMessage.contains( + "Missing file")) + } + checkAnswer(sql("SELECT count(*) FROM t"), Seq(Row(1))) + } + } + + val compact = Future { + for (_ <- 1 to 10) { + try { + sql("CALL sys.compact(table => 't', order_strategy => 'order', order_by => 'id')") + } catch { + case a: Throwable => assert(a.getMessage.contains("Conflicts during commits")) + } + checkAnswer(sql("SELECT count(*) FROM t"), Seq(Row(1))) + } + } + + Await.result(mergeInto, 60.seconds) + Await.result(compact, 60.seconds) + } + } + } +} diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala index b1263cd8d2e4..765eb136bba4 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala @@ -69,7 +69,7 @@ class PaimonFunctionTest extends PaimonHiveTestBase { Seq("paimon", paimonHiveCatalogName).foreach { catalogName => sql(s"use $catalogName") - val functions = sql("show user functions").collect() + val functions = sql("show user functions in sys").collect() assert(functions.exists(_.getString(0).contains("max_pt")), catalogName) } } @@ -120,9 +120,9 @@ class PaimonFunctionTest extends PaimonHiveTestBase { { sql(s"use $catalogName") val maxPt = if (catalogName == sparkCatalogName) { - "paimon.max_pt" + "paimon.sys.max_pt" } else { - "max_pt" + "sys.max_pt" } intercept[Exception] { diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonV1FunctionTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonV1FunctionTestBase.scala index 61fe87758da6..09aa98d93afa 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonV1FunctionTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonV1FunctionTestBase.scala @@ -106,7 +106,7 @@ abstract class PaimonV1FunctionTestBase extends PaimonSparkTestWithRestCatalogBa sql("INSERT INTO t VALUES (1, 2), (3, 4)") checkAnswer( sql( - "SELECT a, udf_add2(pow(a, pt), max_pt('t')), pow(a, udf_add2(a, pt)) FROM t ORDER BY a"), + "SELECT a, udf_add2(pow(a, pt), sys.max_pt('t')), pow(a, udf_add2(a, pt)) FROM t ORDER BY a"), Seq(Row(1, 5.0d, 1.0d), Row(3, 85.0d, 2187.0d)) ) } @@ -145,17 +145,17 @@ abstract class PaimonV1FunctionTestBase extends PaimonSparkTestWithRestCatalogBa test("Paimon V1 Function: unsupported operation") { // create a build-in function - intercept[Exception] { + assert(intercept[Exception] { sql(s""" - |CREATE FUNCTION max_pt AS '$UDFExampleAdd2Class' + |CREATE FUNCTION sys.max_pt AS '$UDFExampleAdd2Class' |USING JAR '$testUDFJarPath' |""".stripMargin) - } + }.getMessage.contains("Can't create build-in function")) // drop a build-in function - intercept[Exception] { - sql("DROP FUNCTION max_pt") - } + assert(intercept[Exception] { + sql("DROP FUNCTION sys.max_pt") + }.getMessage.contains("Can't drop build-in function")) } test("Paimon V1 Function: user defined aggregate function") { @@ -214,6 +214,58 @@ abstract class PaimonV1FunctionTestBase extends PaimonSparkTestWithRestCatalogBa } } } + + test("Paimon V1 Function: select with view") { + withUserDefinedFunction("udf_add2" -> false) { + sql(s""" + |CREATE FUNCTION udf_add2 AS '$UDFExampleAdd2Class' + |USING JAR '$testUDFJarPath' + |""".stripMargin) + withTable("t") { + withView("v") { + sql("CREATE TABLE t (a INT, b INT)") + sql("INSERT INTO t VALUES (1, 2), (3, 4)") + sql("CREATE VIEW v AS SELECT udf_add2(a, b) AS c1 FROM t") + checkAnswer(sql("SELECT * FROM v"), Seq(Row(3), Row(7))) + } + } + } + } + + test("Paimon V1 Function: create or drop function on an existing temporary function") { + withUserDefinedFunction("udf_add2" -> true) { + sql(s""" + |CREATE TEMPORARY FUNCTION udf_add2 AS '$UDFExampleAdd2Class' + |USING JAR '$testUDFJarPath' + |""".stripMargin) + + assert(intercept[Exception] { + sql(s""" + |CREATE FUNCTION udf_add2 AS '$UDFExampleAdd2Class' + |USING JAR '$testUDFJarPath' + |""".stripMargin) + }.getMessage.contains("udf_add2 is a temporary function and already exists")) + + assert(intercept[Exception] { + sql(s""" + |CREATE OR REPLACE FUNCTION udf_add2 AS '$UDFExampleAdd2Class' + |USING JAR '$testUDFJarPath' + |""".stripMargin) + }.getMessage.contains( + "udf_add2 is a temporary function, you should use `CREATE OR REPLACE TEMPORARY FUNCTION udf_add2`")) + + sql(s""" + |CREATE OR REPLACE TEMPORARY FUNCTION udf_add2 AS '$UDFExampleAdd2Class' + |USING JAR '$testUDFJarPath' + |""".stripMargin) + + assert(intercept[Exception] { + sql(s""" + |DROP FUNCTION udf_add2 + |""".stripMargin) + }.getMessage.contains("udf_add2 is a built-in/temporary function")) + } + } } class DisablePaimonV1FunctionTest extends PaimonSparkTestWithRestCatalogBase { diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala index a60d88aef920..af99a67b24dc 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala @@ -267,4 +267,31 @@ class PushDownAggregatesTest extends PaimonSparkTestBase with AdaptiveSparkPlanH }) }) } + + test("Push down aggregate: group by partial partition of a multi partition table") { + sql(s""" + |CREATE TABLE T ( + |c1 STRING, + |c2 STRING, + |c3 STRING, + |c4 STRING, + |c5 DATE) + |PARTITIONED BY (c5, c1) + |TBLPROPERTIES ('primary-key' = 'c5, c1, c3') + |""".stripMargin) + + sql("INSERT INTO T VALUES ('t1', 'k1', 'v1', 'r1', '2025-01-01')") + checkAnswer( + sql("SELECT COUNT(*) FROM T GROUP BY c1"), + Seq(Row(1)) + ) + checkAnswer( + sql("SELECT c1, COUNT(*) FROM T GROUP BY c1"), + Seq(Row("t1", 1)) + ) + checkAnswer( + sql("SELECT COUNT(*), c1 FROM T GROUP BY c1"), + Seq(Row(1, "t1")) + ) + } } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/RowLineageTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTestBase.scala similarity index 95% rename from paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/RowLineageTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTestBase.scala index d8a3516a8091..c8a54d334292 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/RowLineageTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTestBase.scala @@ -23,9 +23,9 @@ import org.apache.paimon.spark.PaimonSparkTestBase import org.apache.spark.sql.Row -abstract class RowLineageTestBase extends PaimonSparkTestBase { +abstract class RowTrackingTestBase extends PaimonSparkTestBase { - test("Row Lineage: read row lineage") { + test("Row Tracking: read row Tracking") { withTable("t") { sql("CREATE TABLE t (id INT, data STRING) TBLPROPERTIES ('row-tracking.enabled' = 'true')") sql("INSERT INTO t VALUES (11, 'a'), (22, 'b')") @@ -41,7 +41,7 @@ abstract class RowLineageTestBase extends PaimonSparkTestBase { } } - test("Row Lineage: compact table") { + test("Row Tracking: compact table") { withTable("t") { sql( "CREATE TABLE t (id INT, data INT) TBLPROPERTIES ('row-tracking.enabled' = 'true', 'compaction.min.file-num'='2')") @@ -64,7 +64,7 @@ abstract class RowLineageTestBase extends PaimonSparkTestBase { } } - test("Row Lineage: delete table") { + test("Row Tracking: delete table") { withTable("t") { sql("CREATE TABLE t (id INT, data INT) TBLPROPERTIES ('row-tracking.enabled' = 'true')") @@ -77,7 +77,7 @@ abstract class RowLineageTestBase extends PaimonSparkTestBase { } } - test("Row Lineage: update table") { + test("Row Tracking: update table") { withTable("t") { sql("CREATE TABLE t (id INT, data INT) TBLPROPERTIES ('row-tracking.enabled' = 'true')") @@ -95,7 +95,7 @@ abstract class RowLineageTestBase extends PaimonSparkTestBase { } } - test("Row Lineage: update table without condition") { + test("Row Tracking: update table without condition") { withTable("t") { sql("CREATE TABLE t (id INT, data INT) TBLPROPERTIES ('row-tracking.enabled' = 'true')") @@ -109,7 +109,7 @@ abstract class RowLineageTestBase extends PaimonSparkTestBase { } } - test("Row Lineage: merge into table") { + test("Row Tracking: merge into table") { withTable("s", "t") { sql("CREATE TABLE s (id INT, b INT) TBLPROPERTIES ('row-tracking.enabled' = 'true')") sql("INSERT INTO s VALUES (1, 11), (2, 22)") @@ -135,7 +135,7 @@ abstract class RowLineageTestBase extends PaimonSparkTestBase { } } - test("Row Lineage: merge into table with only insert") { + test("Row Tracking: merge into table with only insert") { withTable("s", "t") { sql("CREATE TABLE s (id INT, b INT) TBLPROPERTIES ('row-tracking.enabled' = 'true')") sql("INSERT INTO s VALUES (1, 11), (2, 22)") @@ -156,7 +156,7 @@ abstract class RowLineageTestBase extends PaimonSparkTestBase { } } - test("Row Lineage: merge into table with only delete") { + test("Row Tracking: merge into table with only delete") { withTable("s", "t") { sql("CREATE TABLE s (id INT, b INT) TBLPROPERTIES ('row-tracking.enabled' = 'true')") sql("INSERT INTO s VALUES (1, 11), (2, 22)") @@ -177,7 +177,7 @@ abstract class RowLineageTestBase extends PaimonSparkTestBase { } } - test("Row Lineage: merge into table with only update") { + test("Row Tracking: merge into table with only update") { withTable("s", "t") { sql("CREATE TABLE s (id INT, b INT) TBLPROPERTIES ('row-tracking.enabled' = 'true')") sql("INSERT INTO s VALUES (1, 11), (2, 22)") @@ -342,7 +342,7 @@ abstract class RowLineageTestBase extends PaimonSparkTestBase { } } - test("Row Lineage: merge into table not matched by source") { + test("Row Tracking: merge into table not matched by source") { if (gteqSpark3_4) { withTable("source", "target") { sql( diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala index 5cbefe9faa4d..072324ce3a98 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala @@ -18,13 +18,18 @@ package org.apache.paimon.spark.sql -import org.apache.paimon.CoreOptions +import org.apache.paimon.{CoreOptions, Snapshot} import org.apache.paimon.spark.PaimonSparkTestBase import org.apache.paimon.spark.catalyst.analysis.Update import org.apache.spark.sql.Row import org.assertj.core.api.Assertions.{assertThat, assertThatThrownBy} +import scala.concurrent.{Await, Future} +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration.DurationInt +import scala.util.Random + abstract class UpdateTableTestBase extends PaimonSparkTestBase { import testImplicits._ @@ -357,4 +362,74 @@ abstract class UpdateTableTestBase extends PaimonSparkTestBase { sql("UPDATE T SET c = 'b' WHERE id = 1") checkAnswer(sql("SELECT * FROM T"), Seq(Row(1, "s", "b"))) } + + test("Paimon update: non pk table commit kind") { + for (dvEnabled <- Seq(true, false)) { + withTable("t") { + sql( + s"CREATE TABLE t (id INT, data INT) TBLPROPERTIES ('deletion-vectors.enabled' = '$dvEnabled')") + sql("INSERT INTO t SELECT /*+ REPARTITION(1) */ id, id AS data FROM range(1, 4)") + + sql("UPDATE t SET data = 111 WHERE id = 1") + checkAnswer(sql("SELECT * FROM t ORDER BY id"), Seq(Row(1, 111), Row(2, 2), Row(3, 3))) + val table = loadTable("t") + var latestSnapshot = table.latestSnapshot().get() + assert(latestSnapshot.id == 2) + assert(latestSnapshot.commitKind.equals(Snapshot.CommitKind.OVERWRITE)) + + sql("UPDATE t SET data = 222 WHERE id = 2") + checkAnswer(sql("SELECT * FROM t ORDER BY id"), Seq(Row(1, 111), Row(2, 222), Row(3, 3))) + latestSnapshot = table.latestSnapshot().get() + assert(latestSnapshot.id == 3) + assert(latestSnapshot.commitKind.equals(Snapshot.CommitKind.OVERWRITE)) + } + } + } + + test("Paimon update: pk dv table commit kind") { + withTable("t") { + sql( + s"CREATE TABLE t (id INT, data INT) TBLPROPERTIES ('deletion-vectors.enabled' = 'true', 'primary-key' = 'id')") + sql("INSERT INTO t SELECT /*+ REPARTITION(1) */ id, id AS data FROM range(1, 4)") + sql("UPDATE t SET data = 111 WHERE id = 1") + val table = loadTable("t") + val latestSnapshot = table.latestSnapshot().get() + assert(latestSnapshot.id == 4) + assert(latestSnapshot.commitKind.equals(Snapshot.CommitKind.COMPACT)) + } + } + + test("Paimon update: random concurrent update and dv table") { + withTable("t") { + val recordCount = 10000 + val maxCurrent = Random.nextInt(2) + 1 + + sql(s"CREATE TABLE t (a INT, b INT) TBLPROPERTIES ('deletion-vectors.enabled' = 'true')") + sql(s"INSERT INTO t SELECT id AS a, 0 AS b FROM range(0, $recordCount)") + + def run(): Future[Unit] = Future { + for (_ <- 1 to 20) { + try { + val i = 20 + Random.nextInt(100) + Random.nextInt(2) match { + case 0 => sql(s"UPDATE t SET b = b + 1 WHERE (a % $i) = ${Random.nextInt(i)}") + case 1 => + sql("CALL sys.compact(table => 't', options => 'compaction.min.file-num=1')") + case 2 => + sql("CALL sys.compact(table => 't', order_strategy => 'order', order_by => 'a')") + } + } catch { + case a: Throwable => assert(a.getMessage.contains("Conflicts during commits")) + } + checkAnswer(sql("SELECT count(*) FROM t"), Seq(Row(recordCount))) + } + } + + (1 to maxCurrent) + .map(_ => run()) + .foreach( + Await.result(_, 600.seconds) + ) + } + } } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/util/ScanPlanHelperTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/util/ScanPlanHelperTest.scala index dcbf12e273de..df65a5009589 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/util/ScanPlanHelperTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/util/ScanPlanHelperTest.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.Row class ScanPlanHelperTest extends PaimonSparkTestBase with ScanPlanHelper { - test("ScanPlanHelper: create new scan plan and select with row lineage meta cols") { + test("ScanPlanHelper: create new scan plan and select with row tracking meta cols") { withTable("t") { sql("CREATE TABLE t (id INT, data STRING) TBLPROPERTIES ('row-tracking.enabled' = 'true')") sql("INSERT INTO t VALUES (11, 'a'), (22, 'b')") @@ -38,17 +38,17 @@ class ScanPlanHelperTest extends PaimonSparkTestBase with ScanPlanHelper { // select original df should not contain meta cols checkAnswer(newDf, Seq(Row(11, "a"), Row(22, "b"))) - // select df with row lineage meta cols - checkAnswer(selectWithRowLineage(newDf), Seq(Row(11, "a", 0, 1), Row(22, "b", 1, 1))) + // select df with row tracking meta cols + checkAnswer(selectWithRowTracking(newDf), Seq(Row(11, "a", 0, 1), Row(22, "b", 1, 1))) - // select with row lineage meta cols twice should not add new more meta cols + // select with row tracking meta cols twice should not add new more meta cols checkAnswer( - selectWithRowLineage(selectWithRowLineage(newDf)), + selectWithRowTracking(selectWithRowTracking(newDf)), Seq(Row(11, "a", 0, 1), Row(22, "b", 1, 1))) - // select df already contains meta cols with row lineage + // select df already contains meta cols with row tracking checkAnswer( - selectWithRowLineage(newDf.select("_ROW_ID", "id")), + selectWithRowTracking(newDf.select("_ROW_ID", "id")), Seq(Row(0, 11, 1), Row(1, 22, 1))) } } diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala index 74803626d6ce..b9eafe05444e 100644 --- a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala @@ -103,8 +103,7 @@ class Spark3Shim extends SparkShim { override def toPaimonVariant(o: Object): Variant = throw new UnsupportedOperationException() - override def isSparkVariantType(dataType: org.apache.spark.sql.types.DataType): Boolean = - throw new UnsupportedOperationException() + override def isSparkVariantType(dataType: org.apache.spark.sql.types.DataType): Boolean = false override def SparkVariantType(): org.apache.spark.sql.types.DataType = throw new UnsupportedOperationException() diff --git a/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/PaimonVirtualFileSystem.java b/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/PaimonVirtualFileSystem.java index 3687460d2c15..356d96f6f0d7 100644 --- a/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/PaimonVirtualFileSystem.java +++ b/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/PaimonVirtualFileSystem.java @@ -59,6 +59,7 @@ public class PaimonVirtualFileSystem extends FileSystem { private Configuration conf; private static final String USER_AGENT = "HadoopPVFS"; + private static final long DEFAULT_BLOCK_SIZE = 128 * 1024 * 1024L; @Override public void initialize(URI uri, Configuration conf) throws IOException { @@ -299,7 +300,7 @@ public boolean delete(Path f, boolean recursive) throws IOException { public FileStatus getFileStatus(Path f) throws IOException { VFSIdentifier vfsIdentifier = vfsOperations.getVFSIdentifier(getVirtualPath(f)); if (vfsIdentifier instanceof VFSCatalogIdentifier) { - return new FileStatus(0, true, 1, 1, 0, new Path(this.uri)); + return new FileStatus(0, true, 1, 0, 0, new Path(this.uri)); } else if (vfsIdentifier instanceof VFSDatabaseIdentifier) { String databaseName = ((VFSDatabaseIdentifier) vfsIdentifier).databaseName(); GetDatabaseResponse database = vfsOperations.getDatabase(databaseName); @@ -318,7 +319,7 @@ public FileStatus getFileStatus(Path f) throws IOException { } private FileStatus convertDatabase(GetDatabaseResponse database) { - return new FileStatus(0, true, 1, 1, 0, new Path(new Path(this.uri), database.getName())); + return new FileStatus(0, true, 1, 0, 0, new Path(new Path(this.uri), database.getName())); } private FileStatus convertFileStatus( @@ -338,11 +339,12 @@ private FileStatus convertFileStatus( childPath = "/" + childPath; } Path virtualPath = new Path(new Path(this.uri), databaseName + "/" + tableName + childPath); + long blockSize = fileStatus.isDir() ? 0 : DEFAULT_BLOCK_SIZE; return new FileStatus( fileStatus.getLen(), fileStatus.isDir(), 1, - 1, + blockSize, fileStatus.getModificationTime(), virtualPath); } @@ -375,7 +377,7 @@ private FileStatus[] convertDatabases(List databases) { for (int i = 0; i < databases.size(); i++) { String database = databases.get(i); FileStatus fileStatus = - new FileStatus(0, true, 1, 1, 0, new Path(new Path(this.uri), database)); + new FileStatus(0, true, 1, 0, 0, new Path(new Path(this.uri), database)); fileStatuses[i] = fileStatus; } return fileStatuses; @@ -387,7 +389,7 @@ private FileStatus[] convertTables(String database, List tables) { String table = tables.get(i); FileStatus fileStatus = new FileStatus( - 0, true, 1, 1, 0, new Path(new Path(this.uri), database + "/" + table)); + 0, true, 1, 0, 0, new Path(new Path(this.uri), database + "/" + table)); fileStatuses[i] = fileStatus; } return fileStatuses; diff --git a/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/VFSInputStream.java b/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/VFSInputStream.java index 020a58d318e6..7356e87bb932 100644 --- a/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/VFSInputStream.java +++ b/paimon-vfs/paimon-vfs-hadoop/src/main/java/org/apache/paimon/vfs/hadoop/VFSInputStream.java @@ -77,4 +77,9 @@ public int read() throws IOException { } return (n == -1) ? -1 : oneByteBuf[0] & 0xff; } + + @Override + public void close() throws IOException { + in.close(); + } } diff --git a/paimon-vfs/paimon-vfs-hadoop/src/test/java/org/apache/paimon/vfs/hadoop/VirtualFileSystemTest.java b/paimon-vfs/paimon-vfs-hadoop/src/test/java/org/apache/paimon/vfs/hadoop/VirtualFileSystemTest.java index d6e50d1fb02a..62c893a52b1b 100644 --- a/paimon-vfs/paimon-vfs-hadoop/src/test/java/org/apache/paimon/vfs/hadoop/VirtualFileSystemTest.java +++ b/paimon-vfs/paimon-vfs-hadoop/src/test/java/org/apache/paimon/vfs/hadoop/VirtualFileSystemTest.java @@ -137,6 +137,7 @@ public void testMkdir() throws Exception { FileStatus fileStatus = vfs.getFileStatus(vfsPath); Assert.assertEquals(vfsPath.toString(), fileStatus.getPath().toString()); Assert.assertTrue(fileStatus.isDirectory()); + Assert.assertEquals(0, fileStatus.getBlockSize()); // Mkdir in non-existing table tableName = "object_table2"; @@ -195,6 +196,7 @@ public void testCreate() throws Exception { Assert.assertEquals(vfsPath.toString(), fileStatus.getPath().toString()); Assert.assertTrue(fileStatus.isFile()); Assert.assertEquals(5, fileStatus.getLen()); + Assert.assertEquals(128 * 1024 * 1024L, fileStatus.getBlockSize()); FSDataInputStream in = vfs.open(vfsPath); byte[] buffer = new byte[5]; diff --git a/pom.xml b/pom.xml index 803c94444175..3cbbb3b29e41 100644 --- a/pom.xml +++ b/pom.xml @@ -110,7 +110,7 @@ under the License. 1.19.1 1.6.1 0.15.0 - 1.15.1 + 1.15.2 1.9.2 3.19.6 1.2.1