diff --git a/.github/workflows/unitcase-flink-jdk11.yml b/.github/workflows/utitcase-flink-jdk11.yml similarity index 100% rename from .github/workflows/unitcase-flink-jdk11.yml rename to .github/workflows/utitcase-flink-jdk11.yml diff --git a/.github/workflows/unitcase-jdk11.yml b/.github/workflows/utitcase-jdk11.yml similarity index 81% rename from .github/workflows/unitcase-jdk11.yml rename to .github/workflows/utitcase-jdk11.yml index 1baed87f9027..f1d7c25cbe44 100644 --- a/.github/workflows/unitcase-jdk11.yml +++ b/.github/workflows/utitcase-jdk11.yml @@ -16,7 +16,7 @@ # limitations under the License. ################################################################################ -name: UTCase and ITCase Non Flink on JDK 11 +name: UTCase and ITCase Others on JDK 11 on: issue_comment: @@ -52,6 +52,11 @@ jobs: . .github/workflows/utils.sh jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" - mvn -T 1C -B clean install -pl '!paimon-e2e-tests,!org.apache.paimon:paimon-hive-connector-3.1' -Pskip-paimon-flink-tests -Duser.timezone=$jvm_timezone + test_modules="!paimon-e2e-tests,!org.apache.paimon:paimon-hive-connector-3.1," + for suffix in 3.5 3.4 3.3 3.2 common; do + test_modules+="!org.apache.paimon:paimon-spark-${suffix}," + done + test_modules="${test_modules%,}" + mvn -T 1C -B clean install -pl "${test_modules}" -Pskip-paimon-flink-tests -Duser.timezone=$jvm_timezone env: MAVEN_OPTS: -Xmx4096m \ No newline at end of file diff --git a/.github/workflows/utitcase-spark-3.x.yml b/.github/workflows/utitcase-spark-3.x.yml new file mode 100644 index 000000000000..5edcfe49007a --- /dev/null +++ b/.github/workflows/utitcase-spark-3.x.yml @@ -0,0 +1,63 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +name: UTCase and ITCase Spark 3.x + +on: + push: + pull_request: + paths-ignore: + - 'docs/**' + - '**/*.md' + +env: + JDK_VERSION: 8 + +concurrency: + group: ${{ github.workflow }}-${{ github.event_name }}-${{ github.event.number || github.run_id }} + cancel-in-progress: true + +jobs: + build: + runs-on: ubuntu-latest + + steps: + - name: Checkout code + uses: actions/checkout@v2 + - name: Set up JDK ${{ env.JDK_VERSION }} + uses: actions/setup-java@v2 + with: + java-version: ${{ env.JDK_VERSION }} + distribution: 'adopt' + - name: Build Spark + run: mvn -T 1C -B clean install -DskipTests + - name: Test Spark + timeout-minutes: 60 + run: | + # run tests with random timezone to find out timezone related bugs + . .github/workflows/utils.sh + jvm_timezone=$(random_timezone) + echo "JVM timezone is set to $jvm_timezone" + test_modules="" + for suffix in common_2.12 3.5 3.4 3.3 3.2; do + test_modules+="org.apache.paimon:paimon-spark-${suffix}," + done + test_modules="${test_modules%,}" + mvn -T 1C -B test -pl "${test_modules}" -Duser.timezone=$jvm_timezone + env: + MAVEN_OPTS: -Xmx4096m \ No newline at end of file diff --git a/.github/workflows/utitcase-spark-4.x.yml b/.github/workflows/utitcase-spark-4.x.yml new file mode 100644 index 000000000000..7fbac23dda4f --- /dev/null +++ b/.github/workflows/utitcase-spark-4.x.yml @@ -0,0 +1,63 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ + +name: UTCase and ITCase Spark 4.x + +on: + push: + pull_request: + paths-ignore: + - 'docs/**' + - '**/*.md' + +env: + JDK_VERSION: 17 + +concurrency: + group: ${{ github.workflow }}-${{ github.event_name }}-${{ github.event.number || github.run_id }} + cancel-in-progress: true + +jobs: + build: + runs-on: ubuntu-latest + + steps: + - name: Checkout code + uses: actions/checkout@v2 + - name: Set up JDK ${{ env.JDK_VERSION }} + uses: actions/setup-java@v2 + with: + java-version: ${{ env.JDK_VERSION }} + distribution: 'adopt' + - name: Build Spark + run: mvn -T 1C -B clean install -DskipTests -Pspark4 + - name: Test Spark + timeout-minutes: 60 + run: | + # run tests with random timezone to find out timezone related bugs + . .github/workflows/utils.sh + jvm_timezone=$(random_timezone) + echo "JVM timezone is set to $jvm_timezone" + test_modules="" + for suffix in common_2.13 4.0; do + test_modules+="org.apache.paimon:paimon-spark-${suffix}," + done + test_modules="${test_modules%,}" + mvn -T 1C -B test -pl "${test_modules}" -Duser.timezone=$jvm_timezone -Pspark4 + env: + MAVEN_OPTS: -Xmx4096m \ No newline at end of file diff --git a/.github/workflows/utitcase.yml b/.github/workflows/utitcase.yml index 431b44332232..bde67cb4c203 100644 --- a/.github/workflows/utitcase.yml +++ b/.github/workflows/utitcase.yml @@ -16,7 +16,7 @@ # limitations under the License. ################################################################################ -name: UTCase and ITCase Non Flink +name: UTCase and ITCase Others on: push: @@ -53,6 +53,11 @@ jobs: . .github/workflows/utils.sh jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" - mvn -T 1C -B clean install -pl '!paimon-e2e-tests' -Pskip-paimon-flink-tests -Duser.timezone=$jvm_timezone + test_modules="!paimon-e2e-tests," + for suffix in 3.5 3.4 3.3 3.2 common_2.12; do + test_modules+="!org.apache.paimon:paimon-spark-${suffix}," + done + test_modules="${test_modules%,}" + mvn -T 1C -B clean install -pl "${test_modules}" -Pskip-paimon-flink-tests -Duser.timezone=$jvm_timezone env: MAVEN_OPTS: -Xmx4096m \ No newline at end of file diff --git a/.gitignore b/.gitignore index 3831eb7e9ef1..25ebf232470a 100644 --- a/.gitignore +++ b/.gitignore @@ -17,4 +17,5 @@ target .DS_Store *.ipr *.iws +.java-version dependency-reduced-pom.xml diff --git a/docs/content/append-table/query.md b/docs/content/append-table/query-performance.md similarity index 94% rename from docs/content/append-table/query.md rename to docs/content/append-table/query-performance.md index 4f7e0ab9f66f..7ec745468ef5 100644 --- a/docs/content/append-table/query.md +++ b/docs/content/append-table/query-performance.md @@ -1,9 +1,9 @@ --- -title: "Query" +title: "Query Performance" weight: 3 type: docs aliases: -- /append-table/query.html +- /append-table/query-performance.html --- -# Query +# Query Performance ## Data Skipping By Order @@ -57,8 +57,6 @@ multiple columns. Different file index may be efficient in different scenario. For example bloom filter may speed up query in point lookup scenario. Using a bitmap may consume more space but can result in greater accuracy. -Currently, file index is only supported in append-only table. - `Bloom Filter`: * `file-index.bloom-filter.columns`: specify the columns that need bloom filter index. * `file-index.bloom-filter..fpp` to config false positive probability. @@ -67,6 +65,9 @@ Currently, file index is only supported in append-only table. `Bitmap`: * `file-index.bitmap.columns`: specify the columns that need bitmap index. +`Bit-Slice Index Bitmap` +* `file-index.bsi.columns`: specify the columns that need bsi index. + More filter types will be supported... If you want to add file index to existing table, without any rewrite, you can use `rewrite_file_index` procedure. Before diff --git a/docs/content/flink/cdc-ingestion/_index.md b/docs/content/cdc-ingestion/_index.md similarity index 90% rename from docs/content/flink/cdc-ingestion/_index.md rename to docs/content/cdc-ingestion/_index.md index 0c76825b0f6a..b3d4dce2a0c7 100644 --- a/docs/content/flink/cdc-ingestion/_index.md +++ b/docs/content/cdc-ingestion/_index.md @@ -1,7 +1,9 @@ --- title: CDC Ingestion +icon: +bold: true bookCollapseSection: true -weight: 95 +weight: 91 --- + +# Catalog + +Paimon provides a Catalog abstraction to manage the table of contents and metadata. The Catalog abstraction provides +a series of ways to help you better integrate with computing engines. We always recommend that you use Catalog to +access the Paimon table. + +## Catalogs + +Paimon catalogs currently support three types of metastores: + +* `filesystem` metastore (default), which stores both metadata and table files in filesystems. +* `hive` metastore, which additionally stores metadata in Hive metastore. Users can directly access the tables from Hive. +* `jdbc` metastore, which additionally stores metadata in relational databases such as MySQL, Postgres, etc. + +## Filesystem Catalog + +Metadata and table files are stored under `hdfs:///path/to/warehouse`. + +```sql +-- Flink SQL +CREATE CATALOG my_catalog WITH ( + 'type' = 'paimon', + 'warehouse' = 'hdfs:///path/to/warehouse' +); +``` + +## Hive Catalog + +By using Paimon Hive catalog, changes to the catalog will directly affect the corresponding Hive metastore. Tables +created in such catalog can also be accessed directly from Hive. Metadata and table files are stored under +`hdfs:///path/to/warehouse`. In addition, schema is also stored in Hive metastore. + +```sql +-- Flink SQL +CREATE CATALOG my_hive WITH ( + 'type' = 'paimon', + 'metastore' = 'hive', + -- 'warehouse' = 'hdfs:///path/to/warehouse', default use 'hive.metastore.warehouse.dir' in HiveConf +); +``` + +By default, Paimon does not synchronize newly created partitions into Hive metastore. Users will see an unpartitioned +table in Hive. Partition push-down will be carried out by filter push-down instead. + +If you want to see a partitioned table in Hive and also synchronize newly created partitions into Hive metastore, +please set the table option `metastore.partitioned-table` to true. + +## JDBC Catalog + +By using the Paimon JDBC catalog, changes to the catalog will be directly stored in relational databases such as SQLite, +MySQL, postgres, etc. + +```sql +-- Flink SQL +CREATE CATALOG my_jdbc WITH ( + 'type' = 'paimon', + 'metastore' = 'jdbc', + 'uri' = 'jdbc:mysql://:/', + 'jdbc.user' = '...', + 'jdbc.password' = '...', + 'catalog-key'='jdbc', + 'warehouse' = 'hdfs:///path/to/warehouse' +); +``` diff --git a/docs/content/concepts/spec/_index.md b/docs/content/concepts/spec/_index.md index 3bd8e657ffbc..166ce4eeaa54 100644 --- a/docs/content/concepts/spec/_index.md +++ b/docs/content/concepts/spec/_index.md @@ -1,7 +1,7 @@ --- title: Specification bookCollapseSection: true -weight: 4 +weight: 6 --- + +# Table Types + +Paimon supports table types: + +1. table with pk: Paimon Data Table with Primary key +2. table w/o pk: Paimon Data Table without Primary key +3. view: metastore required, views in SQL are a kind of virtual table +4. format-table: file format table refers to a directory that contains multiple files of the same format, where + operations on this table allow for reading or writing to these files, compatible with Hive tables +5. materialized-table: aimed at simplifying both batch and stream data pipelines, providing a consistent development + experience, see [Flink Materialized Table](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/materialized-table/overview/) + +## Table with PK + +See [Paimon with Primary key]({{< ref "primary-key-table/overview" >}}). + +Primary keys consist of a set of columns that contain unique values for each record. Paimon enforces data ordering by +sorting the primary key within each bucket, allowing streaming update and streaming changelog read. + +The definition of primary key is similar to that of standard SQL, as it ensures that there is only one data entry for +the same primary key during batch queries. + +{{< tabs "primary-table" >}} +{{< tab "Flink SQL" >}} + +```sql +CREATE TABLE my_table ( + a INT PRIMARY KEY NOT ENFORCED, + b STRING +) WITH ( + 'bucket'='8' +) +``` +{{< /tab >}} + +{{< tab "Spark SQL" >}} + +```sql +CREATE TABLE my_table ( + a INT, + b STRING +) TBLPROPERTIES ( + 'primary-key' = 'a', + 'bucket' = '8' +) +``` + +{{< /tab >}} +{{< /tabs >}} + +## Table w/o PK + +See [Paimon w/o Primary key]({{< ref "append-table/overview" >}}). + +If a table does not have a primary key defined, it is an append table. Compared to the primary key table, it does not +have the ability to directly receive changelogs. It cannot be directly updated with data through streaming upsert. It +can only receive incoming data from append data. + +However, it also supports batch sql: DELETE, UPDATE, and MERGE-INTO. + +```sql +CREATE TABLE my_table ( + a INT, + b STRING +) +``` + +## View + +View is supported when the metastore can support view, for example, hive metastore. + +View will currently save the original SQL. If you need to use View across engines, you can write a cross engine +SQL statement. For example: + +```sql +CREATE VIEW my_view AS SELECT a + 1, b FROM my_db.my_source; +``` + +## Format Table + +Format table is supported when the metastore can support format table, for example, hive metastore. The Hive tables +inside the metastore will be mapped to Paimon's Format Table for computing engines (Spark, Hive, Flink) to read and write. + +Format table refers to a directory that contains multiple files of the same format, where operations on this table +allow for reading or writing to these files, facilitating the retrieval of existing data and the addition of new files. + +Partitioned file format table just like the standard hive format. Partitions are discovered and inferred based on +directory structure. + +Format Table is enabled by default, you can disable it by configuring Catalog option: `'format-table.enabled'`. + +Currently only support `CSV`, `Parquet`, `ORC` formats. + +{{< tabs "format-table" >}} +{{< tab "Flink-CSV" >}} + +```sql +CREATE TABLE my_csv_table ( + a INT, + b STRING +) WITH ( + 'type'='format-table', + 'file.format'='csv', + 'field-delimiter'=',' +) +``` +{{< /tab >}} + +{{< tab "Spark-CSV" >}} + +```sql +CREATE TABLE my_csv_table ( + a INT, + b STRING +) USING csv OPTIONS ('field-delimiter' ',') +``` + +{{< /tab >}} + +{{< tab "Flink-Parquet" >}} + +```sql +CREATE TABLE my_parquet_table ( + a INT, + b STRING +) WITH ( + 'type'='format-table', + 'file.format'='parquet' +) +``` +{{< /tab >}} + +{{< tab "Spark-Parquet" >}} + +```sql +CREATE TABLE my_parquet_table ( + a INT, + b STRING +) USING parquet +``` + +{{< /tab >}} + +{{< /tabs >}} + +## Materialized Table + +Materialized Table aimed at simplifying both batch and stream data pipelines, providing a consistent development +experience, see [Flink Materialized Table](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/materialized-table/overview/). + +Now only Flink SQL integrate to Materialized Table, we plan to support it in Spark SQL too. + +```sql +CREATE MATERIALIZED TABLE continuous_users_shops +PARTITIONED BY (ds) +FRESHNESS = INTERVAL '30' SECOND +AS SELECT + user_id, + ds, + SUM (payment_amount_cents) AS payed_buy_fee_sum, + SUM (1) AS PV +FROM ( + SELECT user_id, order_created_at AS ds, payment_amount_cents + FROM json_source + ) AS tmp +GROUP BY user_id, ds; +``` diff --git a/docs/content/engines/doris.md b/docs/content/engines/doris.md index 634e7f7c71da..cd778cd57797 100644 --- a/docs/content/engines/doris.md +++ b/docs/content/engines/doris.md @@ -73,13 +73,13 @@ See [Apache Doris Website](https://doris.apache.org/docs/lakehouse/datalake-anal 1. Query Paimon table with full qualified name - ``` + ```sql SELECT * FROM paimon_hdfs.paimon_db.paimon_table; ``` 2. Switch to Paimon Catalog and query - ``` + ```sql SWITCH paimon_hdfs; USE paimon_db; SELECT * FROM paimon_table; diff --git a/docs/content/engines/overview.md b/docs/content/engines/overview.md index 264071d994f6..7dba9b49fb0f 100644 --- a/docs/content/engines/overview.md +++ b/docs/content/engines/overview.md @@ -28,15 +28,15 @@ under the License. ## Compatibility Matrix -| Engine | Version | Batch Read | Batch Write | Create Table | Alter Table | Streaming Write | Streaming Read | Batch Overwrite | DELETE & UPDATE | MERGE INTO | Time Travel | -|:-------------------------------------------------------------------------------:|:-------------:|:-----------:|:-----------:|:-------------:|:-------------:|:----------------:|:----------------:|:---------------:|:------------------:|:-----------:|:-----------:| -| Flink | 1.15 - 1.20 | ✅ | ✅ | ✅ | ✅(1.17+) | ✅ | ✅ | ✅ | ✅(1.17+) | ❌ | ✅ | -| Spark | 3.1 - 3.5 | ✅ | ✅(3.2+) | ✅ | ✅ | ✅(3.3+) | ✅(3.3+) | ✅(3.2+) | ✅(3.2+) | ✅(3.2+) | ✅(3.3+) | -| Hive | 2.1 - 3.1 | ✅ | ✅ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ✅ | -| Trino | 420 - 439 | ✅ | ✅(427+) | ✅(427+) | ✅(427+) | ❌ | ❌ | ❌ | ❌ | ❌ | ✅ | -| Presto | 0.236 - 0.280 | ✅ | ❌ | ✅ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | -| [StarRocks](https://docs.starrocks.io/docs/data_source/catalog/paimon_catalog/) | 3.1+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ✅ | -| [Doris](https://doris.apache.org/docs/lakehouse/datalake-analytics/paimon) | 2.0.6+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ✅ | +| Engine | Version | Batch Read | Batch Write | Create Table | Alter Table | Streaming Write | Streaming Read | Batch Overwrite | DELETE & UPDATE | MERGE INTO | Time Travel | +|:-------------------------------------------------------------------------------:|:-------------:|:-----------:|:-----------:|:-------------:|:-------------:|:----------------:|:----------------:|:---------------:|:---------------:|:----------:|:-----------:| +| Flink | 1.15 - 1.20 | ✅ | ✅ | ✅ | ✅(1.17+) | ✅ | ✅ | ✅ | ✅(1.17+) | ❌ | ✅ | +| Spark | 3.2 - 3.5 | ✅ | ✅ | ✅ | ✅ | ✅(3.3+) | ✅(3.3+) | ✅ | ✅ | ✅ | ✅(3.3+) | +| Hive | 2.1 - 3.1 | ✅ | ✅ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ✅ | +| Trino | 420 - 439 | ✅ | ✅(427+) | ✅(427+) | ✅(427+) | ❌ | ❌ | ❌ | ❌ | ❌ | ✅ | +| Presto | 0.236 - 0.280 | ✅ | ❌ | ✅ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | +| [StarRocks](https://docs.starrocks.io/docs/data_source/catalog/paimon_catalog/) | 3.1+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ✅ | +| [Doris](https://doris.apache.org/docs/lakehouse/datalake-analytics/paimon) | 2.0.6+ | ✅ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ❌ | ✅ | ## Streaming Engines diff --git a/docs/content/engines/trino.md b/docs/content/engines/trino.md index 0f0fe8b94bf9..05fc47729d15 100644 --- a/docs/content/engines/trino.md +++ b/docs/content/engines/trino.md @@ -34,9 +34,9 @@ Paimon currently supports Trino 420 and above. ## Filesystem -From version 0.8, paimon share trino filesystem for all actions, which means, you should -config trino filesystem before using trino-paimon. You can find information about how to config -filesystems for trino on trino official website. +From version 0.8, Paimon share Trino filesystem for all actions, which means, you should +config Trino filesystem before using trino-paimon. You can find information about how to config +filesystems for Trino on Trino official website. ## Preparing Paimon Jar File @@ -113,7 +113,7 @@ If you are using HDFS, choose one of the following ways to configure your HDFS: - set environment variable HADOOP_CONF_DIR. - configure `hadoop-conf-dir` in the properties. -If you are using a hadoop filesystem, you can still use trino-hdfs and trino-hive to config it. +If you are using a Hadoop filesystem, you can still use trino-hdfs and trino-hive to config it. For example, if you use oss as a storage, you can write in `paimon.properties` according to [Trino Reference](https://trino.io/docs/current/connector/hive.html#hdfs-configuration): ``` diff --git a/docs/content/filesystems/_index.md b/docs/content/filesystems/_index.md deleted file mode 100644 index 37fb9791f078..000000000000 --- a/docs/content/filesystems/_index.md +++ /dev/null @@ -1,25 +0,0 @@ ---- -title: Filesystems -icon: -bold: true -bookCollapseSection: true -weight: 92 ---- - diff --git a/docs/content/filesystems/hdfs.md b/docs/content/filesystems/hdfs.md deleted file mode 100644 index ace26d1a84a0..000000000000 --- a/docs/content/filesystems/hdfs.md +++ /dev/null @@ -1,129 +0,0 @@ ---- -title: "HDFS" -weight: 2 -type: docs -aliases: -- /filesystems/hdfs.html ---- - - -# HDFS - -You don't need any additional dependencies to access HDFS because you have already taken care of the Hadoop dependencies. - -## HDFS Configuration - -For HDFS, the most important thing is to be able to read your HDFS configuration. - -{{< tabs "hdfs conf" >}} - -{{< tab "Flink/Trino/JavaAPI" >}} - -You may not have to do anything, if you are in a hadoop environment. Otherwise pick one of the following ways to -configure your HDFS: - -1. Set environment variable `HADOOP_HOME` or `HADOOP_CONF_DIR`. -2. Configure `'hadoop-conf-dir'` in the paimon catalog. -3. Configure Hadoop options through prefix `'hadoop.'` in the paimon catalog. - -The first approach is recommended. - -If you do not want to include the value of the environment variable, you can configure `hadoop-conf-loader` to `option`. - -{{< /tab >}} - -{{< tab "Hive/Spark" >}} - -HDFS Configuration is available directly through the computation cluster, see cluster configuration of Hive and Spark for details. - -{{< /tab >}} - -{{< /tabs >}} - -## Hadoop-compatible file systems (HCFS) - -All Hadoop file systems are automatically available when the Hadoop libraries are on the classpath. - -This way, Paimon seamlessly supports all of Hadoop file systems implementing the `org.apache.hadoop.fs.FileSystem` -interface, and all Hadoop-compatible file systems (HCFS). - -- HDFS -- Alluxio (see configuration specifics below) -- XtreemFS -- … - -The Hadoop configuration has to have an entry for the required file system implementation in the `core-site.xml` file. - -For Alluxio support add the following entry into the core-site.xml file: - -```shell - - fs.alluxio.impl - alluxio.hadoop.FileSystem - -``` - -## Kerberos - -{{< tabs "Kerberos" >}} - -{{< tab "Flink" >}} - -It is recommended to use [Flink Kerberos Keytab](https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/security/security-kerberos/). - -{{< /tab >}} - -{{< tab "Spark" >}} - -It is recommended to use [Spark Kerberos Keytab](https://spark.apache.org/docs/latest/security.html#using-a-keytab). - -{{< /tab >}} - -{{< tab "Hive" >}} - -An intuitive approach is to configure Hive's kerberos authentication. - -{{< /tab >}} - -{{< tab "Trino/JavaAPI" >}} - -Configure the following three options in your catalog configuration: - -- security.kerberos.login.keytab: Absolute path to a Kerberos keytab file that contains the user credentials. - Please make sure it is copied to each machine. -- security.kerberos.login.principal: Kerberos principal name associated with the keytab. -- security.kerberos.login.use-ticket-cache: True or false, indicates whether to read from your Kerberos ticket cache. - -For JavaAPI: -``` -SecurityContext.install(catalogOptions); -``` - -{{< /tab >}} - -{{< /tabs >}} - -## HDFS HA - -Ensure that `hdfs-site.xml` and `core-site.xml` contain the necessary [HA configuration](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html). - -## HDFS ViewFS - -Ensure that `hdfs-site.xml` and `core-site.xml` contain the necessary [ViewFs configuration](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/ViewFs.html). diff --git a/docs/content/filesystems/oss.md b/docs/content/filesystems/oss.md deleted file mode 100644 index b381350a5c9d..000000000000 --- a/docs/content/filesystems/oss.md +++ /dev/null @@ -1,120 +0,0 @@ ---- -title: "OSS" -weight: 3 -type: docs -aliases: -- /filesystems/oss.html ---- - - -# OSS - -{{< stable >}} - -Download [paimon-oss-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-oss/{{< version >}}/paimon-oss-{{< version >}}.jar). - -{{< /stable >}} - -{{< unstable >}} - -Download [paimon-oss-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-oss/{{< version >}}/). - -{{< /unstable >}} - -{{< tabs "oss" >}} - -{{< tab "Flink" >}} - -{{< hint info >}} -If you have already configured [oss access through Flink](https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/filesystems/oss/) (Via Flink FileSystem), -here you can skip the following configuration. -{{< /hint >}} - -Put `paimon-oss-{{< version >}}.jar` into `lib` directory of your Flink home, and create catalog: - -```sql -CREATE CATALOG my_catalog WITH ( - 'type' = 'paimon', - 'warehouse' = 'oss:///', - 'fs.oss.endpoint' = 'oss-cn-hangzhou.aliyuncs.com', - 'fs.oss.accessKeyId' = 'xxx', - 'fs.oss.accessKeySecret' = 'yyy' -); -``` - -{{< /tab >}} - -{{< tab "Spark" >}} - -{{< hint info >}} -If you have already configured oss access through Spark (Via Hadoop FileSystem), here you can skip the following configuration. -{{< /hint >}} - -Place `paimon-oss-{{< version >}}.jar` together with `paimon-spark-{{< version >}}.jar` under Spark's jars directory, and start like - -```shell -spark-sql \ - --conf spark.sql.catalog.paimon=org.apache.paimon.spark.SparkCatalog \ - --conf spark.sql.catalog.paimon.warehouse=oss:/// \ - --conf spark.sql.catalog.paimon.fs.oss.endpoint=oss-cn-hangzhou.aliyuncs.com \ - --conf spark.sql.catalog.paimon.fs.oss.accessKeyId=xxx \ - --conf spark.sql.catalog.paimon.fs.oss.accessKeySecret=yyy -``` - -{{< /tab >}} - -{{< tab "Hive" >}} - -{{< hint info >}} -If you have already configured oss access through Hive (Via Hadoop FileSystem), here you can skip the following configuration. -{{< /hint >}} - -NOTE: You need to ensure that Hive metastore can access `oss`. - -Place `paimon-oss-{{< version >}}.jar` together with `paimon-hive-connector-{{< version >}}.jar` under Hive's auxlib directory, and start like - -```sql -SET paimon.fs.oss.endpoint=oss-cn-hangzhou.aliyuncs.com; -SET paimon.fs.oss.accessKeyId=xxx; -SET paimon.fs.oss.accessKeySecret=yyy; -``` - -And read table from hive metastore, table can be created by Flink or Spark, see [Catalog with Hive Metastore]({{< ref "flink/sql-ddl" >}}) -```sql -SELECT * FROM test_table; -SELECT COUNT(1) FROM test_table; -``` - -{{< /tab >}} - -{{< tab "Trino" >}} - -From version 0.8, paimon-trino uses trino filesystem as basic file read and write system. We strongly recommend you to use jindo-sdk in trino. - -You can find [How to config jindo sdk on trino](https://github.com/aliyun/alibabacloud-jindodata/blob/master/docs/user/4.x/4.6.x/4.6.12/oss/presto/jindosdk_on_presto.md) here. -Please note that: - * Use paimon to replace hive-hadoop2 when you decompress the plugin jar and find location to put in. - * You can specify the `core-site.xml` in `paimon.properties` on configuration [hive.config.resources](https://trino.io/docs/current/connector/hive.html#hdfs-configuration). - * Presto and Jindo use the same configuration method. - - -{{< /tab >}} - -{{< /tabs >}} diff --git a/docs/content/filesystems/overview.md b/docs/content/filesystems/overview.md deleted file mode 100644 index 3de3c2ec500a..000000000000 --- a/docs/content/filesystems/overview.md +++ /dev/null @@ -1,60 +0,0 @@ ---- -title: "Overview" -weight: 1 -type: docs -aliases: -- /filesystems/overview.html ---- - - -# Overview - -Apache Paimon utilizes the same pluggable file systems as Apache Flink. Users can follow the -[standard plugin mechanism](https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/filesystems/plugins/) -to configure the plugin structure if using Flink as compute engine. However, for other engines like Spark -or Hive, the provided opt jars (by Flink) may get conflicts and cannot be used directly. It is not convenient -for users to fix class conflicts, thus Paimon provides the self-contained and engine-unified -FileSystem pluggable jars for user to query tables from Spark/Hive side. - -## Supported FileSystems - -| FileSystem | URI Scheme | Pluggable | Description | -|:------------------|:-----------|-----------|:-----------------------------------------------------------------------| -| Local File System | file:// | N | Built-in Support | -| HDFS | hdfs:// | N | Built-in Support, ensure that the cluster is in the hadoop environment | -| Aliyun OSS | oss:// | Y | | -| S3 | s3:// | Y | | - -## Dependency - -We recommend you to download the jar directly: [Download Link]({{< ref "project/download#filesystem-jars" >}}). - -You can also manually build bundled jar from the source code. - -To build from source code, [clone the git repository]({{< github_repo >}}). - -Build shaded jar with the following command. - -```bash -mvn clean install -DskipTests -``` - -You can find the shaded jars under -`./paimon-filesystems/paimon-${fs}/target/paimon-${fs}-{{< version >}}.jar`. diff --git a/docs/content/filesystems/s3.md b/docs/content/filesystems/s3.md deleted file mode 100644 index 3085d820b67e..000000000000 --- a/docs/content/filesystems/s3.md +++ /dev/null @@ -1,143 +0,0 @@ ---- -title: "S3" -weight: 4 -type: docs -aliases: -- /filesystems/s3.html ---- - - -# S3 - -{{< stable >}} - -Download [paimon-s3-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-s3/{{< version >}}/paimon-s3-{{< version >}}.jar). - -{{< /stable >}} - -{{< unstable >}} - -Download [paimon-s3-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-s3/{{< version >}}/). - -{{< /unstable >}} - -{{< tabs "oss" >}} - -{{< tab "Flink" >}} - -{{< hint info >}} -If you have already configured [s3 access through Flink](https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/filesystems/s3/) (Via Flink FileSystem), -here you can skip the following configuration. -{{< /hint >}} - -Put `paimon-s3-{{< version >}}.jar` into `lib` directory of your Flink home, and create catalog: - -```sql -CREATE CATALOG my_catalog WITH ( - 'type' = 'paimon', - 'warehouse' = 's3:///', - 's3.endpoint' = 'your-endpoint-hostname', - 's3.access-key' = 'xxx', - 's3.secret-key' = 'yyy' -); -``` - -{{< /tab >}} - -{{< tab "Spark" >}} - -{{< hint info >}} -If you have already configured s3 access through Spark (Via Hadoop FileSystem), here you can skip the following configuration. -{{< /hint >}} - -Place `paimon-s3-{{< version >}}.jar` together with `paimon-spark-{{< version >}}.jar` under Spark's jars directory, and start like - -```shell -spark-sql \ - --conf spark.sql.catalog.paimon=org.apache.paimon.spark.SparkCatalog \ - --conf spark.sql.catalog.paimon.warehouse=s3:/// \ - --conf spark.sql.catalog.paimon.s3.endpoint=your-endpoint-hostname \ - --conf spark.sql.catalog.paimon.s3.access-key=xxx \ - --conf spark.sql.catalog.paimon.s3.secret-key=yyy -``` - -{{< /tab >}} - -{{< tab "Hive" >}} - -{{< hint info >}} -If you have already configured s3 access through Hive ((Via Hadoop FileSystem)), here you can skip the following configuration. -{{< /hint >}} - -NOTE: You need to ensure that Hive metastore can access `s3`. - -Place `paimon-s3-{{< version >}}.jar` together with `paimon-hive-connector-{{< version >}}.jar` under Hive's auxlib directory, and start like - -```sql -SET paimon.s3.endpoint=your-endpoint-hostname; -SET paimon.s3.access-key=xxx; -SET paimon.s3.secret-key=yyy; -``` - -And read table from hive metastore, table can be created by Flink or Spark, see [Catalog with Hive Metastore]({{< ref "flink/sql-ddl" >}}) -```sql -SELECT * FROM test_table; -SELECT COUNT(1) FROM test_table; -``` - -{{< /tab >}} - -{{< tab "Trino" >}} - -Paimon use shared trino filesystem as basic read and write system. - -Please refer to [Trino S3](https://trino.io/docs/current/object-storage/file-system-s3.html) to config s3 filesystem in trino. - -{{< /tab >}} - -{{< /tabs >}} - -## S3 Complaint Object Stores - -The S3 Filesystem also support using S3 compliant object stores such as MinIO, Tencent's COS and IBM’s Cloud Object -Storage. Just configure your endpoint to the provider of the object store service. - -```yaml -s3.endpoint: your-endpoint-hostname -``` - -## Configure Path Style Access - -Some S3 compliant object stores might not have virtual host style addressing enabled by default, for example when using Standalone MinIO for testing purpose. -In such cases, you will have to provide the property to enable path style access. - -```yaml -s3.path.style.access: true -``` - -## S3A Performance - -[Tune Performance](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/performance.html) for `S3AFileSystem`. - -If you encounter the following exception: -```shell -Caused by: org.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool. -``` -Try to configure this in catalog options: `fs.s3a.connection.maximum=1000`. diff --git a/docs/content/flink/_index.md b/docs/content/flink/_index.md index c39ff01d8760..6ec757fa520f 100644 --- a/docs/content/flink/_index.md +++ b/docs/content/flink/_index.md @@ -3,7 +3,7 @@ title: Engine Flink icon: bold: true bookCollapseSection: true -weight: 4 +weight: 5 --- + +# Consumer ID + +Consumer id can help you accomplish the following two things: + +1. Safe consumption: When deciding whether a snapshot has expired, Paimon looks at all the consumers of the table in + the file system, and if there are consumers that still depend on this snapshot, then this snapshot will not be + deleted by expiration. +2. Resume from breakpoint: When previous job is stopped, the newly started job can continue to consume from the previous + progress without resuming from the state. + +## Usage + +You can specify the `consumer-id` when streaming read table. + +The consumer will prevent expiration of the snapshot. In order to prevent too many snapshots caused by mistakes, +you need to specify `'consumer.expiration-time'` to manage the lifetime of consumers. + +```sql +ALTER TABLE t SET ('consumer.expiration-time' = '1 d'); +``` + +Then, restart streaming write job of this table, expiration of consumers will be triggered in writing job. + +```sql +SELECT * FROM t /*+ OPTIONS('consumer-id' = 'myid', 'consumer.mode' = 'at-least-once') */; +``` + +## Ignore Progress + +Sometimes, you only want the feature of 'Safe Consumption'. You want to get a new snapshot progress when restarting the +stream consumption job , you can enable the `'consumer.ignore-progress'` option. + +```sql +SELECT * FROM t /*+ OPTIONS('consumer-id' = 'myid', 'consumer.ignore-progress' = 'true') */; +``` + +The startup of this job will retrieve the snapshot that should be read again. + +## Consumer Mode + +By default, the consumption of snapshots is strictly aligned within the checkpoint to make 'Resume from breakpoint' +feature exactly-once. + +But in some scenarios where you don't need 'Resume from breakpoint', or you don't need strict 'Resume from breakpoint', +you can consider enabling `'consumer.mode' = 'at-least-once'` mode. This mode: +1. Allow readers consume snapshots at different rates and record the slowest snapshot-id among all readers into the + consumer. It doesn't affect the checkpoint time and have good performance. +2. This mode can provide more capabilities, such as watermark alignment. + +{{< hint >}} +About `'consumer.mode'`, since the implementation of `exactly-once` mode and `at-least-once` mode are completely +different, the state of flink is incompatible and cannot be restored from the state when switching modes. +{{< /hint >}} + +## Reset Consumer + +You can reset or delete a consumer with a given consumer ID and next snapshot ID and delete a consumer with a given +consumer ID. First, you need to stop the streaming task using this consumer ID, and then execute the reset consumer +action job. + +Run the following command: + +{{< tabs "reset_consumer" >}} + +{{< tab "Flink SQL" >}} + +```sql +CALL sys.reset_consumer( + `table` => 'database_name.table_name', + consumer_id => 'consumer_id', + next_snapshot_id => +); +-- No next_snapshot_id if you want to delete the consumer +``` +{{< /tab >}} + +{{< tab "Flink Action" >}} + +```bash +/bin/flink run \ + /path/to/paimon-flink-action-{{< version >}}.jar \ + reset-consumer \ + --warehouse \ + --database \ + --table \ + --consumer_id \ + [--next_snapshot ] \ + [--catalog_conf [--catalog_conf ...]] + +## No next_snapshot if you want to delete the consumer +``` +{{< /tab >}} + +{{< /tabs >}} diff --git a/docs/content/flink/expire-partition.md b/docs/content/flink/expire-partition.md index 3acf6e59d58c..226017513fee 100644 --- a/docs/content/flink/expire-partition.md +++ b/docs/content/flink/expire-partition.md @@ -134,7 +134,7 @@ More options:
end-input.check-partition-expire
false Boolean - Whether check partition expire after batch mode or bounded stream job finish. + Whether check partition expire after batch mode or bounded stream job finish. diff --git a/docs/content/flink/procedures.md b/docs/content/flink/procedures.md index 430b42c3bd7f..d55a8daddef6 100644 --- a/docs/content/flink/procedures.md +++ b/docs/content/flink/procedures.md @@ -71,6 +71,7 @@ All available procedures are listed below. -- Use indexed argument
CALL [catalog.]sys.compact('table')

CALL [catalog.]sys.compact('table', 'partitions')

+ CALL [catalog.]sys.compact('table', 'order_strategy', 'order_by')

CALL [catalog.]sys.compact('table', 'partitions', 'order_strategy', 'order_by')

CALL [catalog.]sys.compact('table', 'partitions', 'order_strategy', 'order_by', 'options')

CALL [catalog.]sys.compact('table', 'partitions', 'order_strategy', 'order_by', 'options', 'where')

@@ -221,6 +222,46 @@ All available procedures are listed below. CALL sys.delete_tag(`table` => 'default.T', tag => 'my_tag') + + replace_tag + + -- Use named argument
+ -- replace tag with new time retained
+ CALL [catalog.]sys.replace_tag(`table` => 'identifier', tag => 'tagName', time_retained => 'timeRetained')
+ -- replace tag with new snapshot id and time retained
+ CALL [catalog.]sys.replace_tag(`table` => 'identifier', snapshot_id => 'snapshotId')

+ -- Use indexed argument
+ -- replace tag with new snapshot id and time retained
+ CALL [catalog.]sys.replace_tag('identifier', 'tagName', 'snapshotId', 'timeRetained')
+ + + To replace an existing tag with new tag info. Arguments: +
  • table: the target table identifier. Cannot be empty.
  • +
  • tag: name of the existed tag. Cannot be empty.
  • +
  • snapshot(Long): id of the snapshot which the tag is based on, it is optional.
  • +
  • time_retained: The maximum time retained for the existing tag, it is optional.
  • + + + -- for Flink 1.18
    + CALL sys.replace_tag('default.T', 'my_tag', 5, '1 d')

    + -- for Flink 1.19 and later
    + CALL sys.replace_tag(`table` => 'default.T', tag => 'my_tag', snapshot_id => 5, time_retained => '1 d')

    + + + + expire_tags + + CALL [catalog.]sys.expire_tags('identifier', 'older_than') + + + To expire tags by time. Arguments: +
  • identifier: the target table identifier. Cannot be empty.
  • +
  • older_than: tagCreateTime before which tags will be removed.
  • + + + CALL sys.expire_tags(table => 'default.T', older_than => '2024-09-06 11:00:00') + + merge_into @@ -241,10 +282,13 @@ All available procedures are listed below. matched_upsert_setting => 'matchedUpsertSetting',
    not_matched_insert_condition => 'notMatchedInsertCondition',
    not_matched_insert_values => 'notMatchedInsertValues',
    - matched_delete_condition => 'matchedDeleteCondition')

    + matched_delete_condition => 'matchedDeleteCondition',
    + not_matched_by_source_upsert_condition => 'notMatchedBySourceUpsertCondition',
    + not_matched_by_source_upsert_setting => 'notMatchedBySourceUpsertSetting',
    + not_matched_by_source_delete_condition => 'notMatchedBySourceDeleteCondition')

    - To perform "MERGE INTO" syntax. See merge_into action for + To perform "MERGE INTO" syntax. See merge_into action for details of arguments. @@ -268,12 +312,13 @@ All available procedures are listed below. remove_orphan_files -- Use named argument
    - CALL [catalog.]sys.remove_orphan_files(`table` => 'identifier', older_than => 'olderThan', dry_run => 'dryRun')

    + CALL [catalog.]sys.remove_orphan_files(`table` => 'identifier', older_than => 'olderThan', dry_run => 'dryRun', mode => 'mode')

    -- Use indexed argument
    CALL [catalog.]sys.remove_orphan_files('identifier')

    CALL [catalog.]sys.remove_orphan_files('identifier', 'olderThan')

    CALL [catalog.]sys.remove_orphan_files('identifier', 'olderThan', 'dryRun')

    - CALL [catalog.]sys.remove_orphan_files('identifier', 'olderThan', 'dryRun','parallelism') + CALL [catalog.]sys.remove_orphan_files('identifier', 'olderThan', 'dryRun','parallelism')

    + CALL [catalog.]sys.remove_orphan_files('identifier', 'olderThan', 'dryRun','parallelism','mode') To remove the orphan data files and metadata files. Arguments: @@ -283,11 +328,13 @@ All available procedures are listed below.
  • dryRun: when true, view only orphan files, don't actually remove files. Default is false.
  • parallelism: The maximum number of concurrent deleting files. By default is the number of processors available to the Java virtual machine.
  • +
  • mode: The mode of remove orphan clean procedure (local or distributed) . By default is distributed.
  • CALL remove_orphan_files(`table` => 'default.T', older_than => '2023-10-31 12:00:00')

    CALL remove_orphan_files(`table` => 'default.*', older_than => '2023-10-31 12:00:00')

    CALL remove_orphan_files(`table` => 'default.T', older_than => '2023-10-31 12:00:00', dry_run => true)

    - CALL remove_orphan_files(`table` => 'default.T', older_than => '2023-10-31 12:00:00', dry_run => false, parallelism => '5') + CALL remove_orphan_files(`table` => 'default.T', older_than => '2023-10-31 12:00:00', dry_run => false, parallelism => '5')

    + CALL remove_orphan_files(`table` => 'default.T', older_than => '2023-10-31 12:00:00', dry_run => false, parallelism => '5', mode => 'local') @@ -336,6 +383,28 @@ All available procedures are listed below. CALL sys.rollback_to(`table` => 'default.T', snapshot_id => 10) + + rollback_to_timestamp + + -- for Flink 1.18
    + -- rollback to the snapshot which earlier or equal than timestamp.
    + CALL sys.rollback_to_timestamp('identifier', timestamp)

    + -- for Flink 1.19 and later
    + -- rollback to the snapshot which earlier or equal than timestamp.
    + CALL sys.rollback_to_timestamp(`table` => 'default.T', `timestamp` => timestamp)

    + + + To rollback to the snapshot which earlier or equal than timestamp. Argument: +
  • identifier: the target table identifier. Cannot be empty.
  • +
  • timestamp (Long): Roll back to the snapshot which earlier or equal than timestamp.
  • + + + -- for Flink 1.18
    + CALL sys.rollback_to_timestamp('default.T', 10) + -- for Flink 1.19 and later
    + CALL sys.rollback_to_timestamp(`table` => 'default.T', timestamp => 1730292023000) + + expire_snapshots diff --git a/docs/content/flink/quick-start.md b/docs/content/flink/quick-start.md index 62559065ec9a..e50acfe484e1 100644 --- a/docs/content/flink/quick-start.md +++ b/docs/content/flink/quick-start.md @@ -269,11 +269,16 @@ SELECT * FROM ....; ## Setting dynamic options When interacting with the Paimon table, table options can be tuned without changing the options in the catalog. Paimon will extract job-level dynamic options and take effect in the current session. -The dynamic option's key format is `paimon.${catalogName}.${dbName}.${tableName}.${config_key}`. The catalogName/dbName/tableName can be `*`, which means matching all the specific parts. +The dynamic table option's key format is `paimon.${catalogName}.${dbName}.${tableName}.${config_key}`. The catalogName/dbName/tableName can be `*`, which means matching all the specific parts. +The dynamic global option's key format is `${config_key}`. Global options will take effect for all the tables. Table options will override global options if there are conflicts. For example: ```sql +-- set scan.timestamp-millis=1697018249001 for all tables +SET 'scan.timestamp-millis' = '1697018249001'; +SELECT * FROM T; + -- set scan.timestamp-millis=1697018249000 for the table mycatalog.default.T SET 'paimon.mycatalog.default.T.scan.timestamp-millis' = '1697018249000'; SELECT * FROM T; @@ -281,4 +286,10 @@ SELECT * FROM T; -- set scan.timestamp-millis=1697018249000 for the table default.T in any catalog SET 'paimon.*.default.T.scan.timestamp-millis' = '1697018249000'; SELECT * FROM T; + +-- set scan.timestamp-millis=1697018249000 for the table mycatalog.default.T1 +-- set scan.timestamp-millis=1697018249001 for others tables +SET 'paimon.mycatalog.default.T1.scan.timestamp-millis' = '1697018249000'; +SET 'scan.timestamp-millis' = '1697018249001'; +SELECT * FROM T1 JOIN T2 ON xxxx; ``` diff --git a/docs/content/flink/savepoint.md b/docs/content/flink/savepoint.md index 16139f0b0fc8..a0934df13453 100644 --- a/docs/content/flink/savepoint.md +++ b/docs/content/flink/savepoint.md @@ -41,12 +41,12 @@ metadata left. This is very safe, so we recommend using this feature to stop and ## Tag with Savepoint -In Flink, we may consume from kafka and then write to paimon. Since flink's checkpoint only retains a limited number, +In Flink, we may consume from Kafka and then write to Paimon. Since Flink's checkpoint only retains a limited number, we will trigger a savepoint at certain time (such as code upgrades, data updates, etc.) to ensure that the state can be retained for a longer time, so that the job can be restored incrementally. -Paimon's snapshot is similar to flink's checkpoint, and both will automatically expire, but the tag feature of paimon -allows snapshots to be retained for a long time. Therefore, we can combine the two features of paimon's tag and flink's +Paimon's snapshot is similar to Flink's checkpoint, and both will automatically expire, but the tag feature of Paimon +allows snapshots to be retained for a long time. Therefore, we can combine the two features of Paimon's tag and Flink's savepoint to achieve incremental recovery of job from the specified savepoint. {{< hint warning >}} @@ -64,7 +64,7 @@ You can set `sink.savepoint.auto-tag` to `true` to enable the feature of automat **Step 2: Trigger savepoint.** -You can refer to [flink savepoint](https://nightlies.apache.org/flink/flink-docs-stable/docs/ops/state/savepoints/#operations) +You can refer to [Flink savepoint](https://nightlies.apache.org/flink/flink-docs-stable/docs/ops/state/savepoints/#operations) to learn how to configure and trigger savepoint. **Step 3: Choose the tag corresponding to the savepoint.** @@ -74,7 +74,7 @@ The tag corresponding to the savepoint will be named in the form of `savepoint-$ **Step 4: Rollback the paimon table.** -[Rollback]({{< ref "maintenance/manage-tags#rollback-to-tag" >}}) the paimon table to the specified tag. +[Rollback]({{< ref "maintenance/manage-tags#rollback-to-tag" >}}) the Paimon table to the specified tag. **Step 5: Restart from the savepoint.** diff --git a/docs/content/flink/sql-alter.md b/docs/content/flink/sql-alter.md index fe96ec413796..bee616f047d4 100644 --- a/docs/content/flink/sql-alter.md +++ b/docs/content/flink/sql-alter.md @@ -1,6 +1,6 @@ --- title: "SQL Alter" -weight: 6 +weight: 7 type: docs aliases: - /flink/sql-alter.html @@ -114,6 +114,21 @@ ALTER TABLE my_table DROP PARTITION (`id` = 1), PARTITION (`id` = 2); ``` +## Adding Partitions + +The following SQL adds the partitions of the paimon table. + +For flink sql, you can specify the partial columns of partition columns, and you can also specify multiple partition values at the same time, only with metastore configured metastore.partitioned-table=true. + +```sql +ALTER TABLE my_table ADD PARTITION (`id` = 1); + +ALTER TABLE my_table ADD PARTITION (`id` = 1, `name` = 'paimon'); + +ALTER TABLE my_table ADD PARTITION (`id` = 1), PARTITION (`id` = 2); + +``` + ## Changing Column Nullability The following SQL changes nullability of column `coupon_info`. diff --git a/docs/content/flink/sql-ddl.md b/docs/content/flink/sql-ddl.md index 363d7475761c..0324e6655689 100644 --- a/docs/content/flink/sql-ddl.md +++ b/docs/content/flink/sql-ddl.md @@ -203,6 +203,9 @@ Paimon will automatically collect the statistics of the data file for speeding u The statistics collector mode can be configured by `'metadata.stats-mode'`, by default is `'truncate(16)'`. You can configure the field level by setting `'fields.{field_name}.stats-mode'`. +For the stats mode of `none`, we suggest that you configure `metadata.stats-dense-store` = `true`, which will +significantly reduce the storage size of the manifest. + ### Field Default Value Paimon table currently supports setting default values for fields in table properties by `'fields.item_id.default-value'`, diff --git a/docs/content/flink/sql-lookup.md b/docs/content/flink/sql-lookup.md index 296b9c862eee..8b95c6cb1b15 100644 --- a/docs/content/flink/sql-lookup.md +++ b/docs/content/flink/sql-lookup.md @@ -1,6 +1,6 @@ --- title: "SQL Lookup" -weight: 5 +weight: 6 type: docs aliases: - /flink/sql-lookup.html diff --git a/docs/content/flink/sql-query.md b/docs/content/flink/sql-query.md index bb26d5d3c66a..89136b0b0635 100644 --- a/docs/content/flink/sql-query.md +++ b/docs/content/flink/sql-query.md @@ -172,70 +172,6 @@ prevent you from reading older incremental data. So, Paimon also provides anothe SELECT * FROM t /*+ OPTIONS('scan.file-creation-time-millis' = '1678883047356') */; ``` -### Consumer ID - -You can specify the `consumer-id` when streaming read table: -```sql -SELECT * FROM t /*+ OPTIONS('consumer-id' = 'myid', 'consumer.expiration-time' = '1 d', 'consumer.mode' = 'at-least-once') */; -``` - -When stream read Paimon tables, the next snapshot id to be recorded into the file system. This has several advantages: - -1. When previous job is stopped, the newly started job can continue to consume from the previous progress without - resuming from the state. The newly reading will start reading from next snapshot id found in consumer files. - If you don't want this behavior, you can set `'consumer.ignore-progress'` to true. -2. When deciding whether a snapshot has expired, Paimon looks at all the consumers of the table in the file system, - and if there are consumers that still depend on this snapshot, then this snapshot will not be deleted by expiration. - -{{< hint warning >}} -NOTE 1: The consumer will prevent expiration of the snapshot. You can specify `'consumer.expiration-time'` to manage the -lifetime of consumers. - -NOTE 2: If you don't want to affect the checkpoint time, you need to configure `'consumer.mode' = 'at-least-once'`. -This mode allow readers consume snapshots at different rates and record the slowest snapshot-id among all readers into -the consumer. This mode can provide more capabilities, such as watermark alignment. - -NOTE 3: About `'consumer.mode'`, since the implementation of `exactly-once` mode and `at-least-once` mode are completely -different, the state of flink is incompatible and cannot be restored from the state when switching modes. -{{< /hint >}} - -You can reset a consumer with a given consumer ID and next snapshot ID and delete a consumer with a given consumer ID. -First, you need to stop the streaming task using this consumer ID, and then execute the reset consumer action job. - -Run the following command: - -{{< tabs "reset_consumer" >}} - -{{< tab "Flink SQL" >}} - -```sql -CALL sys.reset_consumer( - `table` => 'database_name.table_name', - consumer_id => 'consumer_id', - next_snapshot_id -> -); -``` -{{< /tab >}} - -{{< tab "Flink Action" >}} - -```bash -/bin/flink run \ - /path/to/paimon-flink-action-{{< version >}}.jar \ - reset-consumer \ - --warehouse \ - --database \ - --table \ - --consumer_id \ - [--next_snapshot ] \ - [--catalog_conf [--catalog_conf ...]] -``` -{{< /tab >}} - -{{< /tabs >}} - -please don't specify --next_snapshot parameter if you want to delete the consumer. - ### Read Overwrite Streaming reading will ignore the commits generated by `INSERT OVERWRITE` by default. If you want to read the diff --git a/docs/content/learn-paimon/understand-files.md b/docs/content/learn-paimon/understand-files.md index b18c259993a7..8f67f85eee2f 100644 --- a/docs/content/learn-paimon/understand-files.md +++ b/docs/content/learn-paimon/understand-files.md @@ -328,9 +328,9 @@ The new file layout as of snapshot-4 looks like Note that `manifest-4-0` contains 20 manifest entries (18 `DELETE` operations and 2 `ADD` operations) 1. For partition `20230503` to `20230510`, two `DELETE` operations for two data files -2. For partition `20230501` to `20230502`, one `DELETE` operation and one `ADD` operation - for the same data file. - +2. For partition `20230501` to `20230502`, one `DELETE` operation and one `ADD` operation for the same data file. + This is because there has been an upgrade of the file from level 0 to the highest level. Please rest assured that + this is only a change in metadata, and the file is still the same. ### Alter Table Execute the following statement to configure full-compaction: diff --git a/docs/content/maintenance/dedicated-compaction.md b/docs/content/maintenance/dedicated-compaction.md index 471bdad22275..c0010bf9cc85 100644 --- a/docs/content/maintenance/dedicated-compaction.md +++ b/docs/content/maintenance/dedicated-compaction.md @@ -1,6 +1,6 @@ --- title: "Dedicated Compaction" -weight: 3 +weight: 4 type: docs aliases: - /maintenance/dedicated-compaction.html diff --git a/docs/content/maintenance/filesystems.md b/docs/content/maintenance/filesystems.md new file mode 100644 index 000000000000..dc030a9ec2bd --- /dev/null +++ b/docs/content/maintenance/filesystems.md @@ -0,0 +1,374 @@ +--- +title: "Filesystems" +weight: 1 +type: docs +aliases: +- /maintenance/filesystems.html +--- + + +# Filesystems + +Apache Paimon utilizes the same pluggable file systems as Apache Flink. Users can follow the +[standard plugin mechanism](https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/filesystems/plugins/) +to configure the plugin structure if using Flink as compute engine. However, for other engines like Spark +or Hive, the provided opt jars (by Flink) may get conflicts and cannot be used directly. It is not convenient +for users to fix class conflicts, thus Paimon provides the self-contained and engine-unified +FileSystem pluggable jars for user to query tables from Spark/Hive side. + +## Supported FileSystems + +| FileSystem | URI Scheme | Pluggable | Description | +|:------------------|:-----------|-----------|:-----------------------------------------------------------------------| +| Local File System | file:// | N | Built-in Support | +| HDFS | hdfs:// | N | Built-in Support, ensure that the cluster is in the hadoop environment | +| Aliyun OSS | oss:// | Y | | +| S3 | s3:// | Y | | + +## Dependency + +We recommend you to download the jar directly: [Download Link]({{< ref "project/download#filesystem-jars" >}}). + +You can also manually build bundled jar from the source code. + +To build from source code, [clone the git repository]({{< github_repo >}}). + +Build shaded jar with the following command. + +```bash +mvn clean install -DskipTests +``` + +You can find the shaded jars under +`./paimon-filesystems/paimon-${fs}/target/paimon-${fs}-{{< version >}}.jar`. + +## HDFS + +You don't need any additional dependencies to access HDFS because you have already taken care of the Hadoop dependencies. + +### HDFS Configuration + +For HDFS, the most important thing is to be able to read your HDFS configuration. + +{{< tabs "hdfs conf" >}} + +{{< tab "Flink" >}} + +You may not have to do anything, if you are in a hadoop environment. Otherwise pick one of the following ways to +configure your HDFS: + +1. Set environment variable `HADOOP_HOME` or `HADOOP_CONF_DIR`. +2. Configure `'hadoop-conf-dir'` in the paimon catalog. +3. Configure Hadoop options through prefix `'hadoop.'` in the paimon catalog. + +The first approach is recommended. + +If you do not want to include the value of the environment variable, you can configure `hadoop-conf-loader` to `option`. + +{{< /tab >}} + +{{< tab "Hive/Spark" >}} + +HDFS Configuration is available directly through the computation cluster, see cluster configuration of Hive and Spark for details. + +{{< /tab >}} + +{{< /tabs >}} + +### Hadoop-compatible file systems (HCFS) + +All Hadoop file systems are automatically available when the Hadoop libraries are on the classpath. + +This way, Paimon seamlessly supports all of Hadoop file systems implementing the `org.apache.hadoop.fs.FileSystem` +interface, and all Hadoop-compatible file systems (HCFS). + +- HDFS +- Alluxio (see configuration specifics below) +- XtreemFS +- … + +The Hadoop configuration has to have an entry for the required file system implementation in the `core-site.xml` file. + +For Alluxio support add the following entry into the core-site.xml file: + +```shell + + fs.alluxio.impl + alluxio.hadoop.FileSystem + +``` + +### Kerberos + +{{< tabs "Kerberos" >}} + +{{< tab "Flink" >}} + +It is recommended to use [Flink Kerberos Keytab](https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/security/security-kerberos/). + +{{< /tab >}} + +{{< tab "Spark" >}} + +It is recommended to use [Spark Kerberos Keytab](https://spark.apache.org/docs/latest/security.html#using-a-keytab). + +{{< /tab >}} + +{{< tab "Hive" >}} + +An intuitive approach is to configure Hive's kerberos authentication. + +{{< /tab >}} + +{{< tab "Trino/JavaAPI" >}} + +Configure the following three options in your catalog configuration: + +- security.kerberos.login.keytab: Absolute path to a Kerberos keytab file that contains the user credentials. + Please make sure it is copied to each machine. +- security.kerberos.login.principal: Kerberos principal name associated with the keytab. +- security.kerberos.login.use-ticket-cache: True or false, indicates whether to read from your Kerberos ticket cache. + +For JavaAPI: +``` +SecurityContext.install(catalogOptions); +``` + +{{< /tab >}} + +{{< /tabs >}} + +### HDFS HA + +Ensure that `hdfs-site.xml` and `core-site.xml` contain the necessary [HA configuration](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html). + +### HDFS ViewFS + +Ensure that `hdfs-site.xml` and `core-site.xml` contain the necessary [ViewFs configuration](https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/ViewFs.html). + +## OSS + +{{< stable >}} + +Download [paimon-oss-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-oss/{{< version >}}/paimon-oss-{{< version >}}.jar). + +{{< /stable >}} + +{{< unstable >}} + +Download [paimon-oss-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-oss/{{< version >}}/). + +{{< /unstable >}} + +{{< tabs "oss" >}} + +{{< tab "Flink" >}} + +{{< hint info >}} +If you have already configured [oss access through Flink](https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/filesystems/oss/) (Via Flink FileSystem), +here you can skip the following configuration. +{{< /hint >}} + +Put `paimon-oss-{{< version >}}.jar` into `lib` directory of your Flink home, and create catalog: + +```sql +CREATE CATALOG my_catalog WITH ( + 'type' = 'paimon', + 'warehouse' = 'oss:///', + 'fs.oss.endpoint' = 'oss-cn-hangzhou.aliyuncs.com', + 'fs.oss.accessKeyId' = 'xxx', + 'fs.oss.accessKeySecret' = 'yyy' +); +``` + +{{< /tab >}} + +{{< tab "Spark" >}} + +{{< hint info >}} +If you have already configured oss access through Spark (Via Hadoop FileSystem), here you can skip the following configuration. +{{< /hint >}} + +Place `paimon-oss-{{< version >}}.jar` together with `paimon-spark-{{< version >}}.jar` under Spark's jars directory, and start like + +```shell +spark-sql \ + --conf spark.sql.catalog.paimon=org.apache.paimon.spark.SparkCatalog \ + --conf spark.sql.catalog.paimon.warehouse=oss:/// \ + --conf spark.sql.catalog.paimon.fs.oss.endpoint=oss-cn-hangzhou.aliyuncs.com \ + --conf spark.sql.catalog.paimon.fs.oss.accessKeyId=xxx \ + --conf spark.sql.catalog.paimon.fs.oss.accessKeySecret=yyy +``` + +{{< /tab >}} + +{{< tab "Hive" >}} + +{{< hint info >}} +If you have already configured oss access through Hive (Via Hadoop FileSystem), here you can skip the following configuration. +{{< /hint >}} + +NOTE: You need to ensure that Hive metastore can access `oss`. + +Place `paimon-oss-{{< version >}}.jar` together with `paimon-hive-connector-{{< version >}}.jar` under Hive's auxlib directory, and start like + +```sql +SET paimon.fs.oss.endpoint=oss-cn-hangzhou.aliyuncs.com; +SET paimon.fs.oss.accessKeyId=xxx; +SET paimon.fs.oss.accessKeySecret=yyy; +``` + +And read table from hive metastore, table can be created by Flink or Spark, see [Catalog with Hive Metastore]({{< ref "flink/sql-ddl" >}}) +```sql +SELECT * FROM test_table; +SELECT COUNT(1) FROM test_table; +``` + +{{< /tab >}} +{{< tab "Trino" >}} + +From version 0.8, paimon-trino uses trino filesystem as basic file read and write system. We strongly recommend you to use jindo-sdk in trino. + +You can find [How to config jindo sdk on trino](https://github.com/aliyun/alibabacloud-jindodata/blob/master/docs/user/4.x/4.6.x/4.6.12/oss/presto/jindosdk_on_presto.md) here. +Please note that: +* Use paimon to replace hive-hadoop2 when you decompress the plugin jar and find location to put in. +* You can specify the `core-site.xml` in `paimon.properties` on configuration [hive.config.resources](https://trino.io/docs/current/connector/hive.html#hdfs-configuration). +* Presto and Jindo use the same configuration method. + +{{< /tab >}} +{{< /tabs >}} + +## S3 + +{{< stable >}} + +Download [paimon-s3-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-s3/{{< version >}}/paimon-s3-{{< version >}}.jar). + +{{< /stable >}} + +{{< unstable >}} + +Download [paimon-s3-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-s3/{{< version >}}/). + +{{< /unstable >}} + +{{< tabs "s3" >}} + +{{< tab "Flink" >}} + +{{< hint info >}} +If you have already configured [s3 access through Flink](https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/filesystems/s3/) (Via Flink FileSystem), +here you can skip the following configuration. +{{< /hint >}} + +Put `paimon-s3-{{< version >}}.jar` into `lib` directory of your Flink home, and create catalog: + +```sql +CREATE CATALOG my_catalog WITH ( + 'type' = 'paimon', + 'warehouse' = 's3:///', + 's3.endpoint' = 'your-endpoint-hostname', + 's3.access-key' = 'xxx', + 's3.secret-key' = 'yyy' +); +``` + +{{< /tab >}} + +{{< tab "Spark" >}} + +{{< hint info >}} +If you have already configured s3 access through Spark (Via Hadoop FileSystem), here you can skip the following configuration. +{{< /hint >}} + +Place `paimon-s3-{{< version >}}.jar` together with `paimon-spark-{{< version >}}.jar` under Spark's jars directory, and start like + +```shell +spark-sql \ + --conf spark.sql.catalog.paimon=org.apache.paimon.spark.SparkCatalog \ + --conf spark.sql.catalog.paimon.warehouse=s3:/// \ + --conf spark.sql.catalog.paimon.s3.endpoint=your-endpoint-hostname \ + --conf spark.sql.catalog.paimon.s3.access-key=xxx \ + --conf spark.sql.catalog.paimon.s3.secret-key=yyy +``` + +{{< /tab >}} + +{{< tab "Hive" >}} + +{{< hint info >}} +If you have already configured s3 access through Hive ((Via Hadoop FileSystem)), here you can skip the following configuration. +{{< /hint >}} + +NOTE: You need to ensure that Hive metastore can access `s3`. + +Place `paimon-s3-{{< version >}}.jar` together with `paimon-hive-connector-{{< version >}}.jar` under Hive's auxlib directory, and start like + +```sql +SET paimon.s3.endpoint=your-endpoint-hostname; +SET paimon.s3.access-key=xxx; +SET paimon.s3.secret-key=yyy; +``` + +And read table from hive metastore, table can be created by Flink or Spark, see [Catalog with Hive Metastore]({{< ref "flink/sql-ddl" >}}) +```sql +SELECT * FROM test_table; +SELECT COUNT(1) FROM test_table; +``` + +{{< /tab >}} + +{{< tab "Trino" >}} + +Paimon use shared trino filesystem as basic read and write system. + +Please refer to [Trino S3](https://trino.io/docs/current/object-storage/file-system-s3.html) to config s3 filesystem in trino. + +{{< /tab >}} + +{{< /tabs >}} + +### S3 Complaint Object Stores + +The S3 Filesystem also support using S3 compliant object stores such as MinIO, Tencent's COS and IBM’s Cloud Object +Storage. Just configure your endpoint to the provider of the object store service. + +```yaml +s3.endpoint: your-endpoint-hostname +``` + +### Configure Path Style Access + +Some S3 compliant object stores might not have virtual host style addressing enabled by default, for example when using Standalone MinIO for testing purpose. +In such cases, you will have to provide the property to enable path style access. + +```yaml +s3.path.style.access: true +``` + +### S3A Performance + +[Tune Performance](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/performance.html) for `S3AFileSystem`. + +If you encounter the following exception: +```shell +Caused by: org.apache.http.conn.ConnectionPoolTimeoutException: Timeout waiting for connection from pool. +``` +Try to configure this in catalog options: `fs.s3a.connection.maximum=1000`. diff --git a/docs/content/maintenance/manage-snapshots.md b/docs/content/maintenance/manage-snapshots.md index 00c0322de8c2..eed0f72c08c2 100644 --- a/docs/content/maintenance/manage-snapshots.md +++ b/docs/content/maintenance/manage-snapshots.md @@ -1,6 +1,6 @@ --- title: "Manage Snapshots" -weight: 4 +weight: 5 type: docs aliases: - /maintenance/manage-snapshots.html diff --git a/docs/content/maintenance/metrics.md b/docs/content/maintenance/metrics.md index 3a221f9037c6..139bdfff6bfe 100644 --- a/docs/content/maintenance/metrics.md +++ b/docs/content/maintenance/metrics.md @@ -72,11 +72,6 @@ Below is lists of Paimon built-in metrics. They are summarized into types of sca Gauge Skipped table files by partition filter and value / key stats information in the last scan. - - lastSkippedByBucketAndLevelFilter - Gauge - Skipped table files by bucket, bucket key and level filter in the last scan. - lastSkippedByWholeBucketFilesFilter Gauge @@ -186,6 +181,16 @@ Below is lists of Paimon built-in metrics. They are summarized into types of sca Gauge Number of buckets written in the last commit. + + lastCompactionInputFileSize + Gauge + Total size of the input files for the last compaction. + + + lastCompactionOutputFileSize + Gauge + Total size of the output files for the last compaction. + @@ -237,23 +242,53 @@ Below is lists of Paimon built-in metrics. They are summarized into types of sca maxLevel0FileCount Gauge - The maximum number of level 0 files currently handled by this writer. This value will become larger if asynchronous compaction cannot be done in time. + The maximum number of level 0 files currently handled by this task. This value will become larger if asynchronous compaction cannot be done in time. avgLevel0FileCount Gauge - The average number of level 0 files currently handled by this writer. This value will become larger if asynchronous compaction cannot be done in time. + The average number of level 0 files currently handled by this task. This value will become larger if asynchronous compaction cannot be done in time. compactionThreadBusy Gauge - The maximum business of compaction threads in this parallelism. Currently, there is only one compaction thread in each parallelism, so value of business ranges from 0 (idle) to 100 (compaction running all the time). + The maximum business of compaction threads in this task. Currently, there is only one compaction thread in each parallelism, so value of business ranges from 0 (idle) to 100 (compaction running all the time). avgCompactionTime Gauge The average runtime of compaction threads, calculated based on recorded compaction time data in milliseconds. The value represents the average duration of compaction operations. Higher values indicate longer average compaction times, which may suggest the need for performance optimization. + + compactionCompletedCount + Counter + The total number of compactions that have completed. + + + compactionQueuedCount + Counter + The total number of compactions that are queued/running. + + + maxCompactionInputSize + Gauge + The maximum input file size for this task's compaction. + + + avgCompactionInputSize/td> + Gauge + The average input file size for this task's compaction. + + + maxCompactionOutputSize + Gauge + The maximum output file size for this task's compaction. + + + avgCompactionOutputSize + Gauge + The average output file size for this task's compaction. + diff --git a/docs/content/maintenance/system-tables.md b/docs/content/maintenance/system-tables.md index f19c75756863..462f8c27f887 100644 --- a/docs/content/maintenance/system-tables.md +++ b/docs/content/maintenance/system-tables.md @@ -1,6 +1,6 @@ --- title: "System Tables" -weight: 1 +weight: 2 type: docs aliases: - /maintenance/system-tables.html @@ -216,12 +216,12 @@ You can query the branches of the table. SELECT * FROM my_table$branches; /* -+----------------------+---------------------------+--------------------------+-------------------------+ -| branch_name | created_from_tag | created_from_snapshot | create_time | -+----------------------+---------------------------+--------------------------+-------------------------+ -| branch1 | tag1 | 2 | 2024-07-18 20:31:39.084 | -| branch2 | tag2 | 5 | 2024-07-18 21:11:14.373 | -+----------------------+---------------------------+--------------------------+-------------------------+ ++----------------------+-------------------------+ +| branch_name | create_time | ++----------------------+-------------------------+ +| branch1 | 2024-07-18 20:31:39.084 | +| branch2 | 2024-07-18 21:11:14.373 | ++----------------------+-------------------------+ 2 rows in set */ ``` @@ -272,6 +272,28 @@ SELECT * FROM my_table$manifests /*+ OPTIONS('scan.snapshot-id'='1') */; +--------------------------------+-------------+------------------+-------------------+---------------+ 1 rows in set */ + +- You can also query the manifest with specified tagName +SELECT * FROM my_table$manifests /*+ OPTIONS('scan.tag-name'='tag1') */; +/* ++--------------------------------+-------------+------------------+-------------------+---------------+ +| file_name | file_size | num_added_files | num_deleted_files | schema_id | ++--------------------------------+-------------+------------------+-------------------+---------------+ +| manifest-f4dcab43-ef6b-4713... | 12365| 40 | 0 | 0 | ++--------------------------------+-------------+------------------+-------------------+---------------+ +1 rows in set +*/ + +- You can also query the manifest with specified timestamp in unix milliseconds +SELECT * FROM my_table$manifests /*+ OPTIONS('scan.timestamp-millis'='1678883047356') */; +/* ++--------------------------------+-------------+------------------+-------------------+---------------+ +| file_name | file_size | num_added_files | num_deleted_files | schema_id | ++--------------------------------+-------------+------------------+-------------------+---------------+ +| manifest-f4dcab43-ef6b-4713... | 12365| 40 | 0 | 0 | ++--------------------------------+-------------+------------------+-------------------+---------------+ +1 rows in set +*/ ``` ### Aggregation fields Table @@ -369,3 +391,19 @@ SELECT * FROM sys.catalog_options; */ ``` +### Statistic Table +You can query the statistic information through statistic table. + +```sql +SELECT * FROM T$statistics; + +/* ++--------------+------------+-----------------------+------------------+----------+ +| snapshot_id | schema_id | mergedRecordCount | mergedRecordSize | colstat | ++--------------+------------+-----------------------+------------------+----------+ +| 2 | 0 | 2 | 2 | {} | ++--------------+------------+-----------------------+------------------+----------+ +1 rows in set +*/ +``` + diff --git a/docs/content/maintenance/write-performance.md b/docs/content/maintenance/write-performance.md index 03e734874c05..02362b9096e7 100644 --- a/docs/content/maintenance/write-performance.md +++ b/docs/content/maintenance/write-performance.md @@ -1,6 +1,6 @@ --- title: "Write Performance" -weight: 2 +weight: 3 type: docs aliases: - /maintenance/write-performance.html @@ -160,3 +160,12 @@ You can use fine-grained-resource-management of Flink to increase committer heap 1. Configure Flink Configuration `cluster.fine-grained-resource-management.enabled: true`. (This is default after Flink 1.18) 2. Configure Paimon Table Options: `sink.committer-memory`, for example 300 MB, depends on your `TaskManager`. (`sink.committer-cpu` is also supported) + +## Changelog Compaction + +If Flink's checkpoint interval is short (for example, 30 seconds) and the number of buckets is large, +each snapshot may produce lots of small changelog files. +Too many files may put a burden on the distributed storage cluster. + +In order to compact small changelog files into large ones, you can set the table option `changelog.precommit-compact = true`. +Default value of this option is false, if true, it will add a compact coordinator and worker operator after the writer operator, which copies changelog files into large ones. diff --git a/docs/content/migration/iceberg-compatibility.md b/docs/content/migration/iceberg-compatibility.md new file mode 100644 index 000000000000..17e1c6523398 --- /dev/null +++ b/docs/content/migration/iceberg-compatibility.md @@ -0,0 +1,501 @@ +--- +title: "Iceberg Compatibility" +weight: 4 +type: docs +aliases: +- /migration/iceberg-compatibility.html +--- + + +# Iceberg Compatibility + +Paimon supports generating Iceberg compatible metadata, +so that Paimon tables can be consumed directly by Iceberg readers. + +## Enable Iceberg Compatibility + +Set the following table options, so that Paimon tables can generate Iceberg compatible metadata. + + + + + + + + + + + + + + + + + + +
    OptionDefaultTypeDescription
    metadata.iceberg.storage
    disabledEnum + When set, produce Iceberg metadata after a snapshot is committed, so that Iceberg readers can read Paimon's raw data files. +
      +
    • disabled: Disable Iceberg compatibility support.
    • +
    • table-location: Store Iceberg metadata in each table's directory.
    • +
    • hadoop-catalog: Store Iceberg metadata in a separate directory. This directory can be specified as the warehouse directory of an Iceberg Hadoop catalog.
    • +
    • hive-catalog: Not only store Iceberg metadata like hadoop-catalog, but also create Iceberg external table in Hive.
    • +
    +
    + +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'`, +so you can visit each table with its table path. + +## Example: Query Paimon Append Only Tables on Flink/Spark with Iceberg Connector + +Let's walk through a simple example, where we query Paimon tables with Iceberg connectors in Flink and Spark. +Before trying out this example, make sure that your compute engine already supports Iceberg. +Please refer to Iceberg's document if you haven't set up Iceberg. +* Flink: [Preparation when using Flink SQL Client](https://iceberg.apache.org/docs/latest/flink/#preparation-when-using-flink-sql-client) +* Spark: [Using Iceberg in Spark 3](https://iceberg.apache.org/docs/latest/spark-getting-started/#using-iceberg-in-spark-3) + +Let's now create a Paimon append only table with Iceberg compatibility enabled and insert some data. + +{{< tabs "create-paimon-append-only-table" >}} + +{{< tab "Flink SQL" >}} +```sql +CREATE CATALOG paimon_catalog WITH ( + 'type' = 'paimon', + 'warehouse' = '' +); + +CREATE TABLE paimon_catalog.`default`.cities ( + country STRING, + name STRING +) WITH ( + 'metadata.iceberg.storage' = 'hadoop-catalog' +); + +INSERT INTO paimon_catalog.`default`.cities VALUES ('usa', 'new york'), ('germany', 'berlin'), ('usa', 'chicago'), ('germany', 'hamburg'); +``` +{{< /tab >}} + +{{< tab "Spark SQL" >}} +Start `spark-sql` with the following command line. + +```bash +spark-sql --jars \ + --conf spark.sql.catalog.paimon_catalog=org.apache.paimon.spark.SparkCatalog \ + --conf spark.sql.catalog.paimon_catalog.warehouse= \ + --packages org.apache.iceberg:iceberg-spark-runtime- \ + --conf spark.sql.catalog.iceberg_catalog=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.iceberg_catalog.type=hadoop \ + --conf spark.sql.catalog.iceberg_catalog.warehouse=/iceberg \ + --conf spark.sql.catalog.iceberg_catalog.cache-enabled=false \ # disable iceberg catalog caching to quickly see the result + --conf spark.sql.extensions=org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions,org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions +``` + +Run the following Spark SQL to create Paimon table and insert data. + +```sql +CREATE TABLE paimon_catalog.`default`.cities ( + country STRING, + name STRING +) TBLPROPERTIES ( + 'metadata.iceberg.storage' = 'hadoop-catalog' +); + +INSERT INTO paimon_catalog.`default`.cities VALUES ('usa', 'new york'), ('germany', 'berlin'), ('usa', 'chicago'), ('germany', 'hamburg'); +``` +{{< /tab >}} + +{{< /tabs >}} + +Now let's query this Paimon table with Iceberg connector. + +{{< tabs "query-paimon-append-only-table" >}} + +{{< tab "Flink SQL" >}} +```sql +CREATE CATALOG iceberg_catalog WITH ( + 'type' = 'iceberg', + 'catalog-type' = 'hadoop', + 'warehouse' = '/iceberg', + 'cache-enabled' = 'false' -- disable iceberg catalog caching to quickly see the result +); + +SELECT * FROM iceberg_catalog.`default`.cities WHERE country = 'germany'; +/* ++----+--------------------------------+--------------------------------+ +| op | country | name | ++----+--------------------------------+--------------------------------+ +| +I | germany | berlin | +| +I | germany | hamburg | ++----+--------------------------------+--------------------------------+ +*/ +``` +{{< /tab >}} + +{{< tab "Spark SQL" >}} +```sql +SELECT * FROM iceberg_catalog.`default`.cities WHERE country = 'germany'; +/* +germany berlin +germany hamburg +*/ +``` +{{< /tab >}} + +{{< /tabs >}} + +Let's insert more data and query again. + +{{< tabs "query-paimon-append-only-table-again" >}} + +{{< tab "Flink SQL" >}} +```sql +INSERT INTO paimon_catalog.`default`.cities VALUES ('usa', 'houston'), ('germany', 'munich'); + +SELECT * FROM iceberg_catalog.`default`.cities WHERE country = 'germany'; +/* ++----+--------------------------------+--------------------------------+ +| op | country | name | ++----+--------------------------------+--------------------------------+ +| +I | germany | munich | +| +I | germany | berlin | +| +I | germany | hamburg | ++----+--------------------------------+--------------------------------+ +*/ +``` +{{< /tab >}} + +{{< tab "Spark SQL" >}} +```sql +INSERT INTO paimon_catalog.`default`.cities VALUES ('usa', 'houston'), ('germany', 'munich'); + +SELECT * FROM iceberg_catalog.`default`.cities WHERE country = 'germany'; +/* +germany munich +germany berlin +germany hamburg +*/ +``` +{{< /tab >}} + +{{< /tabs >}} + +## Example: Query Paimon Primary Key Tables on Flink/Spark with Iceberg Connector + +{{< tabs "paimon-primary-key-table" >}} + +{{< tab "Flink SQL" >}} +```sql +CREATE CATALOG paimon_catalog WITH ( + 'type' = 'paimon', + 'warehouse' = '' +); + +CREATE TABLE paimon_catalog.`default`.orders ( + order_id BIGINT, + status STRING, + payment DOUBLE, + PRIMARY KEY (order_id) NOT ENFORCED +) WITH ( + 'metadata.iceberg.storage' = 'hadoop-catalog', + 'compaction.optimization-interval' = '1ms' -- ATTENTION: this option is only for testing, see "timeliness" section below for more information +); + +INSERT INTO paimon_catalog.`default`.orders VALUES (1, 'SUBMITTED', CAST(NULL AS DOUBLE)), (2, 'COMPLETED', 200.0), (3, 'SUBMITTED', CAST(NULL AS DOUBLE)); + +CREATE CATALOG iceberg_catalog WITH ( + 'type' = 'iceberg', + 'catalog-type' = 'hadoop', + 'warehouse' = '/iceberg', + 'cache-enabled' = 'false' -- disable iceberg catalog caching to quickly see the result +); + +SELECT * FROM iceberg_catalog.`default`.orders WHERE status = 'COMPLETED'; +/* ++----+----------------------+--------------------------------+--------------------------------+ +| op | order_id | status | payment | ++----+----------------------+--------------------------------+--------------------------------+ +| +I | 2 | COMPLETED | 200.0 | ++----+----------------------+--------------------------------+--------------------------------+ +*/ + +INSERT INTO paimon_catalog.`default`.orders VALUES (1, 'COMPLETED', 100.0); + +SELECT * FROM iceberg_catalog.`default`.orders WHERE status = 'COMPLETED'; +/* ++----+----------------------+--------------------------------+--------------------------------+ +| op | order_id | status | payment | ++----+----------------------+--------------------------------+--------------------------------+ +| +I | 1 | COMPLETED | 100.0 | +| +I | 2 | COMPLETED | 200.0 | ++----+----------------------+--------------------------------+--------------------------------+ +*/ +``` +{{< /tab >}} + +{{< tab "Spark SQL" >}} +Start `spark-sql` with the following command line. + +```bash +spark-sql --jars \ + --conf spark.sql.catalog.paimon_catalog=org.apache.paimon.spark.SparkCatalog \ + --conf spark.sql.catalog.paimon_catalog.warehouse= \ + --packages org.apache.iceberg:iceberg-spark-runtime- \ + --conf spark.sql.catalog.iceberg_catalog=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.iceberg_catalog.type=hadoop \ + --conf spark.sql.catalog.iceberg_catalog.warehouse=/iceberg \ + --conf spark.sql.catalog.iceberg_catalog.cache-enabled=false \ # disable iceberg catalog caching to quickly see the result + --conf spark.sql.extensions=org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions,org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions +``` + +Run the following Spark SQL to create Paimon table, insert/update data, and query with Iceberg catalog. + +```sql +CREATE TABLE paimon_catalog.`default`.orders ( + order_id BIGINT, + status STRING, + payment DOUBLE +) TBLPROPERTIES ( + 'primary-key' = 'order_id', + 'metadata.iceberg.storage' = 'hadoop-catalog', + 'compaction.optimization-interval' = '1ms' -- ATTENTION: this option is only for testing, see "timeliness" section below for more information +); + +INSERT INTO paimon_catalog.`default`.orders VALUES (1, 'SUBMITTED', CAST(NULL AS DOUBLE)), (2, 'COMPLETED', 200.0), (3, 'SUBMITTED', CAST(NULL AS DOUBLE)); + +SELECT * FROM iceberg_catalog.`default`.orders WHERE status = 'COMPLETED'; +/* +2 COMPLETED 200.0 +*/ + +INSERT INTO paimon_catalog.`default`.orders VALUES (1, 'COMPLETED', 100.0); + +SELECT * FROM iceberg_catalog.`default`.orders WHERE status = 'COMPLETED'; +/* +2 COMPLETED 200.0 +1 COMPLETED 100.0 +*/ +``` +{{< /tab >}} + +{{< /tabs >}} + +### Timeliness + +Paimon primary key tables organize data files as LSM trees, so data files must be merged in memory before querying. +However, Iceberg readers are not able to merge data files, so they can only query data files on the highest level of LSM trees. +Data files on the highest level are produced by the full compaction process. +So **to conclude, for primary key tables, Iceberg readers can only query data after full compaction**. + +By default, there is no guarantee on how frequently Paimon will perform full compaction. +You can configure the following table option, so that Paimon is forced to perform full compaction after several commits. + + + + + + + + + + + + + + + + + + + + + + + + +
    OptionDefaultTypeDescription
    compaction.optimization-interval
    (none)DurationFull compaction will be constantly triggered per time interval. First compaction after the job starts will always be full compaction.
    full-compaction.delta-commits
    (none)IntegerFull compaction will be constantly triggered after delta commits. Only implemented in Flink.
    + +Note that full compaction is a resource-consuming process, so the value of this table option should not be too small. +We recommend full compaction to be performed once or twice per hour. + +## Access Paimon Table from Iceberg Hive Catalog + +When creating Paimon table, set `'metadata.iceberg.storage' = 'hive-catalog'`. +This option value not only store Iceberg metadata like hadoop-catalog, but also create Iceberg external table in Hive. +This Paimon table can be accessed from Iceberg Hive catalog later. + +To provide information about Hive metastore, +you also need to set some (or all) of the following table options when creating Paimon table. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    OptionDefaultTypeDescription
    metadata.iceberg.uri
    StringHive metastore uri for Iceberg Hive catalog.
    metadata.iceberg.hive-conf-dir
    Stringhive-conf-dir for Iceberg Hive catalog.
    metadata.iceberg.hadoop-conf-dir
    Stringhadoop-conf-dir for Iceberg Hive catalog.
    + +## Example: Query Paimon Append Only Tables on Trino with Iceberg Connector + +In this example, we use Trino Iceberg connector to access Paimon table through Iceberg Hive catalog. +Before trying out this example, make sure that you have configured Trino Iceberg connector. +See [Trino's document](https://trino.io/docs/current/connector/iceberg.html#general-configuration) for more information. + +Let's first create a Paimon table with Iceberg compatibility enabled. + +{{< tabs "paimon-append-only-table-trino-1" >}} + +{{< tab "Flink SQL" >}} +```sql +CREATE CATALOG paimon_catalog WITH ( + 'type' = 'paimon', + 'warehouse' = '' +); + +CREATE TABLE paimon_catalog.`default`.animals ( + kind STRING, + name STRING +) WITH ( + 'metadata.iceberg.storage' = 'hive-catalog', + 'metadata.iceberg.uri' = 'thrift://:' +); + +INSERT INTO paimon_catalog.`default`.animals VALUES ('mammal', 'cat'), ('mammal', 'dog'), ('reptile', 'snake'), ('reptile', 'lizard'); +``` +{{< /tab >}} + +{{< tab "Spark SQL" >}} +Start `spark-sql` with the following command line. + +```bash +spark-sql --jars \ + --conf spark.sql.catalog.paimon_catalog=org.apache.paimon.spark.SparkCatalog \ + --conf spark.sql.catalog.paimon_catalog.warehouse= \ + --packages org.apache.iceberg:iceberg-spark-runtime- \ + --conf spark.sql.catalog.iceberg_catalog=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.iceberg_catalog.type=hadoop \ + --conf spark.sql.catalog.iceberg_catalog.warehouse=/iceberg \ + --conf spark.sql.catalog.iceberg_catalog.cache-enabled=false \ # disable iceberg catalog caching to quickly see the result + --conf spark.sql.extensions=org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions,org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions +``` + +Run the following Spark SQL to create Paimon table, insert/update data, and query with Iceberg catalog. + +```sql +CREATE TABLE paimon_catalog.`default`.animals ( + kind STRING, + name STRING +) TBLPROPERTIES ( + 'metadata.iceberg.storage' = 'hive-catalog', + 'metadata.iceberg.uri' = 'thrift://:' +); + +INSERT INTO paimon_catalog.`default`.animals VALUES ('mammal', 'cat'), ('mammal', 'dog'), ('reptile', 'snake'), ('reptile', 'lizard'); +``` +{{< /tab >}} + +{{< /tabs >}} + +Start Trino using Iceberg catalog and query from Paimon table. + +```sql +SELECT * FROM animals WHERE class = 'mammal'; +/* + kind | name +--------+------ + mammal | cat + mammal | dog +*/ +``` + +## Supported Types + +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` | + +*: `TIMESTAMP` and `TIMESTAMP_LTZ` type only support precision from 4 to 6 + +## Other Related Table Options + + + + + + + + + + + + + + + + + + + + + + + + +
    OptionDefaultTypeDescription
    metadata.iceberg.compaction.min.file-num
    10IntegerMinimum number of Iceberg metadata files to trigger metadata compaction.
    metadata.iceberg.compaction.max.file-num
    50IntegerIf number of small Iceberg metadata files exceeds this limit, always trigger metadata compaction regardless of their total size.
    diff --git a/docs/content/migration/upsert-to-partitioned.md b/docs/content/migration/upsert-to-partitioned.md index b82bc942a626..d4fd00ae630b 100644 --- a/docs/content/migration/upsert-to-partitioned.md +++ b/docs/content/migration/upsert-to-partitioned.md @@ -26,6 +26,10 @@ under the License. # Upsert To Partitioned +{{< hint warning >}} +__Note:__ Only Hive Engine can be used to query these upsert-to-partitioned tables. +{{< /hint >}} + The [Tag Management]({{< ref "maintenance/manage-tags" >}}) will maintain the manifests and data files of the snapshot. A typical usage is creating tags daily, then you can maintain the historical data of each day for batch reading. diff --git a/docs/content/primary-key-table/merge-engine/aggregation.md b/docs/content/primary-key-table/merge-engine/aggregation.md index a9b9b5a38cb0..fa667ed35687 100644 --- a/docs/content/primary-key-table/merge-engine/aggregation.md +++ b/docs/content/primary-key-table/merge-engine/aggregation.md @@ -3,7 +3,7 @@ title: "Aggregation" weight: 3 type: docs aliases: -- /cdc-ingestion/merge-engin/aggregation.html +- /primary-key-table/merge-engin/aggregation.html --- + +# Query Performance + +## Table Mode + +The table schema has the greatest impact on query performance. See [Table Mode]({{< ref "primary-key-table/table-mode" >}}). + +For Merge On Read table, the most important thing you should pay attention to is the number of buckets, which will limit +the concurrency of reading data. + +For MOW (Deletion Vectors) or COW table or [Read Optimized]({{< ref "maintenance/system-tables#read-optimized-table" >}}) table, +There is no limit to the concurrency of reading data, and they can also utilize some filtering conditions for non-primary-key columns. + +## Data Skipping By Primary Key Filter + +For a regular bucketed table (For example, bucket = 5), the filtering conditions of the primary key will greatly +accelerate queries and reduce the reading of a large number of files. + +## Data Skipping By File Index + +You can use file index to table with Deletion Vectors enabled, it filters files by index on the read side. + +```sql +CREATE TABLE WITH ( + 'deletion-vectors' = 'true', + 'file-index.bloom-filter.columns' = 'c1,c2', + 'file-index.bloom-filter.c1.items' = '200' +); +``` + +Supported filter types: + +`Bloom Filter`: +* `file-index.bloom-filter.columns`: specify the columns that need bloom filter index. +* `file-index.bloom-filter..fpp` to config false positive probability. +* `file-index.bloom-filter..items` to config the expected distinct items in one data file. + +`Bitmap`: +* `file-index.bitmap.columns`: specify the columns that need bitmap index. + +`Bit-Slice Index Bitmap` +* `file-index.bsi.columns`: specify the columns that need bsi index. + +More filter types will be supported... + +If you want to add file index to existing table, without any rewrite, you can use `rewrite_file_index` procedure. Before +we use the procedure, you should config appropriate configurations in target table. You can use ALTER clause to config +`file-index..columns` to the table. + +How to invoke: see [flink procedures]({{< ref "flink/procedures#procedures" >}}) diff --git a/docs/content/program-api/catalog-api.md b/docs/content/program-api/catalog-api.md index 570577437d86..d016cfa7b204 100644 --- a/docs/content/program-api/catalog-api.md +++ b/docs/content/program-api/catalog-api.md @@ -1,6 +1,6 @@ --- title: "Catalog API" -weight: 3 +weight: 4 type: docs aliases: - /api/catalog-api.html diff --git a/docs/content/program-api/flink-api.md b/docs/content/program-api/flink-api.md index 7cf9d1932ce5..6ecac3909ced 100644 --- a/docs/content/program-api/flink-api.md +++ b/docs/content/program-api/flink-api.md @@ -26,12 +26,8 @@ under the License. # Flink API -{{< hint warning >}} -We do not recommend using programming API. Paimon is designed for SQL first, unless you are a professional Flink developer, even if you do, it can be very difficult. - -We strongly recommend that you use Flink SQL or Spark SQL, or simply use SQL APIs in programs. - -The following documents are not detailed and are for reference only. +{{< hint info >}} +If possible, recommend using Flink SQL or Spark SQL, or simply use SQL APIs in programs. {{< /hint >}} ## Dependency diff --git a/docs/content/program-api/java-api.md b/docs/content/program-api/java-api.md index 97f4ac5ae3dd..1bf0ce82f6b0 100644 --- a/docs/content/program-api/java-api.md +++ b/docs/content/program-api/java-api.md @@ -26,12 +26,8 @@ under the License. # Java API -{{< hint warning >}} -We do not recommend using the Paimon API naked, unless you are a professional downstream ecosystem developer, and even if you do, there will be significant difficulties. - -If you are only using Paimon, we strongly recommend using computing engines such as Flink SQL or Spark SQL. - -The following documents are not detailed and are for reference only. +{{< hint info >}} +If possible, recommend using computing engines such as Flink SQL or Spark SQL. {{< /hint >}} ## Dependency diff --git a/docs/content/program-api/python-api.md b/docs/content/program-api/python-api.md new file mode 100644 index 000000000000..940e8cb52b4d --- /dev/null +++ b/docs/content/program-api/python-api.md @@ -0,0 +1,338 @@ +--- +title: "Python API" +weight: 3 +type: docs +aliases: +- /api/python-api.html +--- + + +# Java-based Implementation For Python API + +[Python SDK ](https://github.com/apache/paimon-python) has defined Python API for Paimon. Currently, there is only a Java-based implementation. + +Java-based implementation will launch a JVM and use `py4j` to execute Java code to read and write Paimon table. + +## Environment Settings + +### SDK Installing + +SDK is published at [paimon-python](https://pypi.org/project/paimon-python/). You can install by +```shell +pip install paimon-python +``` + +### Java Runtime Environment + +This SDK needs JRE 1.8. After installing JRE, make sure that at least one of the following conditions is met: +1. `java` command is available. You can verify it by `java -version`. +2. `JAVA_HOME` and `PATH` variables are set correctly. For example, you can set: +```shell +export JAVA_HOME=/path/to/java-directory +export PATH=$JAVA_HOME/bin:$PATH +``` + +### Set Environment Variables + +Because we need to launch a JVM to access Java code, JVM environment need to be set. Besides, the java code need Hadoop +dependencies, so hadoop environment should be set. + +#### Java classpath + +The package has set necessary paimon core dependencies (Local/Hadoop FileIO, Avro/Orc/Parquet format support and +FileSystem/Jdbc/Hive catalog), so If you just test codes in local or in hadoop environment, you don't need to set classpath. + +If you need other dependencies such as OSS/S3 filesystem jars, or special format and catalog ,please prepare jars and set +classpath via one of the following ways: + +1. Set system environment variable: ```export _PYPAIMON_JAVA_CLASSPATH=/path/to/jars/*``` +2. Set environment variable in Python code: + +```python +import os +from paimon_python_java import constants + +os.environ[constants.PYPAIMON_JAVA_CLASSPATH] = '/path/to/jars/*' +``` + +#### JVM args (optional) + +You can set JVM args via one of the following ways: + +1. Set system environment variable: ```export _PYPAIMON_JVM_ARGS='arg1 arg2 ...'``` +2. Set environment variable in Python code: + +```python +import os +from paimon_python_java import constants + +os.environ[constants.PYPAIMON_JVM_ARGS] = 'arg1 arg2 ...' +``` + +#### Hadoop classpath + +If the machine is in a hadoop environment, please ensure the value of the environment variable HADOOP_CLASSPATH include +path to the common Hadoop libraries, then you don't need to set hadoop. + +Otherwise, you should set hadoop classpath via one of the following ways: + +1. Set system environment variable: ```export _PYPAIMON_HADOOP_CLASSPATH=/path/to/jars/*``` +2. Set environment variable in Python code: + +```python +import os +from paimon_python_java import constants + +os.environ[constants.PYPAIMON_HADOOP_CLASSPATH] = '/path/to/jars/*' +``` + +If you just want to test codes in local, we recommend to use [Flink Pre-bundled hadoop jar](https://flink.apache.org/downloads/#additional-components). + + +## Create Catalog + +Before coming into contact with the Table, you need to create a Catalog. + +```python +from paimon_python_java import Catalog + +# Note that keys and values are all string +catalog_options = { + 'metastore': 'filesystem', + 'warehouse': 'file:///path/to/warehouse' +} +catalog = Catalog.create(catalog_options) +``` + +## Create Database & Table + +You can use the catalog to create table for writing data. + +### Create Database (optional) +Table is located in a database. If you want to create table in a new database, you should create it. + +```python +catalog.create_database( + name='database_name', + ignore_if_exists=True, # If you want to raise error if the database exists, set False + properties={'key': 'value'} # optional database properties +) +``` + +### Create Schema + +Table schema contains fields definition, partition keys, primary keys, table options and comment. For example: + +```python +import pyarrow as pa + +from paimon_python_api import Schema + +pa_schema = pa.schema([ + ('dt', pa.string()), + ('hh', pa.string()), + ('pk', pa.int64()), + ('value', pa.string()) +]) + +schema = Schema( + pa_schema=pa_schema, + partition_keys=['dt', 'hh'], + primary_keys=['dt', 'hh', 'pk'], + options={'bucket': '2'}, + comment='my test table' +) +``` + +All arguments except `pa_schema` is optional. If you have some Pandas data, the `pa_schema` can be extracted from `DataFrame`: + +```python +import pandas as pd +import pyarrow as pa + +from paimon_python_api import Schema + +# Example DataFrame data +data = { + 'dt': ['2024-01-01', '2024-01-01', '2024-01-02'], + 'hh': ['12', '15', '20'], + 'pk': [1, 2, 3], + 'value': ['a', 'b', 'c'], +} +dataframe = pd.DataFrame(data) + +# Get Paimon Schema +record_batch = pa.RecordBatch.from_pandas(dataframe) +schema = Schema( + pa_schema=record_batch.schema, + partition_keys=['dt', 'hh'], + primary_keys=['dt', 'hh', 'pk'], + options={'bucket': '2'}) +``` + +### Create Table +```python +schema = ... +catalog.create_table( + identifier='database_name.table_name', + schema=schema, + ignore_if_exists=True # If you want to raise error if the table exists, set False +) +``` + +## Get Table + +The Table interface provides tools to read and write table. + +```python +table = catalog.get_table('database_name.table_name') +``` + +## Batch Read + +TableRead interface provides parallelly reading for multiple splits. You can set `'max-workers': 'N'` in `catalog_options` +to set thread numbers when reading splits. `max-workers` is 1 by default, that means TableRead will read splits sequentially +if you doesn't set `max-workers`. + +```python +table = catalog.get_table('database_name.table_name') + +# 1. Create table scan and read +read_builder = table.new_read_builder() +table_scan = read_builder.new_scan() +table_read = read_builder.new_read() + +# 2. Get splits +splits = table_scan.plan().splits() + +# 3. Read splits. Support 3 methods: +# 3.1 Read as pandas.DataFrame +dataframe = table_read.to_pandas(splits) + +# 3.2 Read as pyarrow.Table +pa_table = table_read.to_arrow(splits) + +# 3.3 Read as pyarrow.RecordBatchReader +record_batch_reader = table_read.to_arrow_batch_reader(splits) +``` + +## Batch Write + +Paimon table write is Two-Phase Commit, you can write many times, but once committed, no more data can be write. + +{{< hint warning >}} +Currently, Python SDK doesn't support writing primary key table with `bucket=-1`. +{{< /hint >}} + +```python +table = catalog.get_table('database_name.table_name') + +# 1. Create table write and commit +write_builder = table.new_batch_write_builder() +# By default, write data will be appended to table. +# If you want to overwrite table: +# write_builder.overwrite() +# If you want to overwrite partition 'dt=2024-01-01': +# write_builder.overwrite({'dt': '2024-01-01'}) + +table_write = write_builder.new_write() +table_commit = write_builder.new_commit() + +# 2. Write data. Support 3 methods: +# 2.1 Write pandas.DataFrame +dataframe = ... +table_write.write_pandas(dataframe) + +# 2.2 Write pyarrow.Table +pa_table = ... +table_write.write_arrow(pa_table) + +# 2.3 Write pyarrow.RecordBatch +record_batch = ... +table_write.write_arrow_batch(record_batch) + +# 3. Commit data +commit_messages = table_write.prepare_commit() +table_commit.commit(commit_messages) + +# 4. Close resources +table_write.close() +table_commit.close() + +``` + +## Data Types + +| pyarrow | Paimon | +|:-----------------------------------------|:---------| +| pyarrow.int8() | TINYINT | +| pyarrow.int16() | SMALLINT | +| pyarrow.int32() | INT | +| pyarrow.int64() | BIGINT | +| pyarrow.float16()
    pyarrow.float32() | FLOAT | +| pyarrow.float64() | DOUBLE | +| pyarrow.string() | STRING | +| pyarrow.boolean() | BOOLEAN | + +## Predicate + +You can use predicate to filter data when reading. Example: + +```python +# table data: +# f0: 0 1 2 3 4 +# f1: 5 6 7 8 9 +read_builder = table.new_read_builder() +predicate_builder = read_builder.new_predicate_builder() + +# build predicate: f0 < 3 && f1 > 5 +predicate1 = predicate_builder.less_than('f0', 1); +predicate2 = predicate_builder.greater_than('f1', 5); +predicate = predicate_builder.and_predicates([predicate1, predicate2]) + +read_builder = read_builder.with_filter(predicate) +table_scan = read_builder.new_scan() +table_read = read_builder.new_read() +splits = table_scan.plan().splits() +dataframe = table_read.to_pandas(splits) + +# result: +# f0: 1 2 +# f1: 6 7 +``` + +| Predicate kind | Predicate method | +|:----------------------|:----------------------------------------------| +| p1 and p2 | PredicateBuilder.and_predicates([p1, p2]) | +| p1 or p2 | PredicateBuilder.or_predicates([p1, p2]) | +| f = literal | PredicateBuilder.equal(f, literal) | +| f != literal | PredicateBuilder.not_equal(f, literal) | +| f < literal | PredicateBuilder.less_than(f, literal) | +| f <= literal | PredicateBuilder.less_or_equal(f, literal) | +| f > literal | PredicateBuilder.greater_than(f, literal) | +| f >= literal | PredicateBuilder.greater_or_equal(f, literal) | +| f is null | PredicateBuilder.is_null(f) | +| f is not null | PredicateBuilder.is_not_null(f) | +| f.startswith(literal) | PredicateBuilder.startswith(f, literal) | +| f.endswith(literal) | PredicateBuilder.endswith(f, literal) | +| f.contains(literal) | PredicateBuilder.contains(f, literal) | +| f is in [l1, l2] | PredicateBuilder.is_in(f, [l1, l2]) | +| f is not in [l1, l2] | PredicateBuilder.is_not_in(f, [l1, l2]) | +| lower <= f <= upper | PredicateBuilder.between(f, lower, upper) | diff --git a/docs/content/project/download.md b/docs/content/project/download.md index f9189132a53b..5e49811076a6 100644 --- a/docs/content/project/download.md +++ b/docs/content/project/download.md @@ -45,7 +45,6 @@ This documentation is a guide for downloading Paimon Jars. | Spark 3.4 | [paimon-spark-3.4-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-spark-3.4/{{< version >}}/) | | Spark 3.3 | [paimon-spark-3.3-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-spark-3.3/{{< version >}}/) | | Spark 3.2 | [paimon-spark-3.2-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-spark-3.2/{{< version >}}/) | -| Spark 3.1 | [paimon-spark-3.1-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-spark-3.1/{{< version >}}/) | | Hive 3.1 | [paimon-hive-connector-3.1-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-hive-connector-3.1/{{< version >}}/) | | Hive 2.3 | [paimon-hive-connector-2.3-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-hive-connector-2.3/{{< version >}}/) | | Hive 2.2 | [paimon-hive-connector-2.2-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-hive-connector-2.2/{{< version >}}/) | @@ -75,7 +74,6 @@ This documentation is a guide for downloading Paimon Jars. | Spark 3.4 | [paimon-spark-3.4-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-spark-3.4/{{< version >}}/paimon-spark-3.4-{{< version >}}.jar) | | Spark 3.3 | [paimon-spark-3.3-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-spark-3.3/{{< version >}}/paimon-spark-3.3-{{< version >}}.jar) | | Spark 3.2 | [paimon-spark-3.2-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-spark-3.2/{{< version >}}/paimon-spark-3.2-{{< version >}}.jar) | -| Spark 3.1 | [paimon-spark-3.1-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-spark-3.1/{{< version >}}/paimon-spark-3.1-{{< version >}}.jar) | | Hive 3.1 | [paimon-hive-connector-3.1-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-hive-connector-3.1/{{< version >}}/paimon-hive-connector-3.1-{{< version >}}.jar) | | Hive 2.3 | [paimon-hive-connector-2.3-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-hive-connector-2.3/{{< version >}}/paimon-hive-connector-2.3-{{< version >}}.jar) | | Hive 2.2 | [paimon-hive-connector-2.2-{{< version >}}.jar](https://repo.maven.apache.org/maven2/org/apache/paimon/paimon-hive-connector-2.2/{{< version >}}/paimon-hive-connector-2.2-{{< version >}}.jar) | diff --git a/docs/content/spark/_index.md b/docs/content/spark/_index.md index 24661e56f25a..07128574b0e7 100644 --- a/docs/content/spark/_index.md +++ b/docs/content/spark/_index.md @@ -3,7 +3,7 @@ title: Engine Spark icon: bold: true bookCollapseSection: true -weight: 5 +weight: 6 --- @@ -88,7 +96,7 @@ under the License. org.apache.flink - flink-table-planner_${scala.binary.version} + flink-table-planner_${flink.scala.binary.version} ${flink.version} test @@ -106,6 +114,13 @@ under the License. 1.21 test + + + org.apache.iceberg + iceberg-flink-${iceberg.flink.version} + ${iceberg.version} + test + diff --git a/paimon-flink/paimon-flink-1.19/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java b/paimon-flink/paimon-flink-1.19/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java new file mode 100644 index 000000000000..6eabd1db7f38 --- /dev/null +++ b/paimon-flink/paimon-flink-1.19/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog; + +/** + * Dummy placeholder to resolve compatibility issue of CatalogMaterializedTable(introduced in flink + * 1.20). + */ +public interface CatalogMaterializedTable extends CatalogBaseTable { + /** Dummy LogicalRefreshMode placeholder. */ + enum LogicalRefreshMode {} + + /** Dummy RefreshMode placeholder. */ + enum RefreshMode {} + + /** Dummy RefreshStatus placeholder. */ + enum RefreshStatus {} +} diff --git a/paimon-flink/paimon-flink-1.19/src/test/java/org/apache/paimon/flink/iceberg/Flink119IcebergITCase.java b/paimon-flink/paimon-flink-1.19/src/test/java/org/apache/paimon/flink/iceberg/Flink119IcebergITCase.java new file mode 100644 index 000000000000..eb64a6c9de37 --- /dev/null +++ b/paimon-flink/paimon-flink-1.19/src/test/java/org/apache/paimon/flink/iceberg/Flink119IcebergITCase.java @@ -0,0 +1,22 @@ +/* + * 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.iceberg; + +/** IT cases for Paimon Iceberg compatibility in Flink 1.19. */ +public class Flink119IcebergITCase extends FlinkIcebergITCaseBase {} diff --git a/paimon-flink/paimon-flink-cdc/pom.xml b/paimon-flink/paimon-flink-cdc/pom.xml index 82dd412bf1d2..7f5bb3e786da 100644 --- a/paimon-flink/paimon-flink-cdc/pom.xml +++ b/paimon-flink/paimon-flink-cdc/pom.xml @@ -37,7 +37,7 @@ under the License. 1.18.1 3.1.1 3.1.1 - 1.11.1 + 1.11.4 2.2.0 2.9.0 1.19.1 @@ -185,14 +185,14 @@ under the License. org.apache.flink - flink-table-planner_${scala.binary.version} + flink-table-planner_${flink.scala.binary.version} ${flink.version} test org.apache.flink - flink-table-planner_${scala.binary.version} + flink-table-planner_${flink.scala.binary.version} ${flink.version} test test-jar @@ -301,6 +301,11 @@ under the License. + + + org.apache.flink.streaming.runtime.streamrecord.RecordAttributes + org.apache.flink.shaded.org.apache.flink.streaming.runtime.streamrecord.RecordAttributes + org.apache.kafka.connect diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java new file mode 100644 index 000000000000..723c71dc565d --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/streaming/runtime/streamrecord/RecordAttributes.java @@ -0,0 +1,22 @@ +/* + * 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.flink.streaming.runtime.streamrecord; + +/** Placeholder class for new feature introduced since flink 1.19. Should never be used. */ +public class RecordAttributes extends StreamElement {} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java new file mode 100644 index 000000000000..6eabd1db7f38 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.catalog; + +/** + * Dummy placeholder to resolve compatibility issue of CatalogMaterializedTable(introduced in flink + * 1.20). + */ +public interface CatalogMaterializedTable extends CatalogBaseTable { + /** Dummy LogicalRefreshMode placeholder. */ + enum LogicalRefreshMode {} + + /** Dummy RefreshMode placeholder. */ + enum RefreshMode {} + + /** Dummy RefreshStatus placeholder. */ + enum RefreshStatus {} +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/ProcessRecordAttributesUtil.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/ProcessRecordAttributesUtil.java new file mode 100644 index 000000000000..efe5e12b12d7 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/ProcessRecordAttributesUtil.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink; + +import org.apache.paimon.flink.sink.StoreSinkWrite; + +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; + +/** Placeholder class for new feature introduced since flink 1.19. Should never be used. */ +public class ProcessRecordAttributesUtil { + public static void processWithWrite(RecordAttributes recordAttributes, StoreSinkWrite write) {} + + public static void processWithOutput(RecordAttributes recordAttributes, Output output) {} +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java index 4c7db6d28b62..87efeb2a19cf 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java @@ -122,7 +122,7 @@ protected void validateCaseSensitivity() { protected void beforeBuildingSourceSink() throws Exception { Identifier identifier = new Identifier(database, table); // Check if table exists before trying to get or create it - if (catalog.tableExists(identifier)) { + try { fileStoreTable = (FileStoreTable) catalog.getTable(identifier); fileStoreTable = alterTableOptions(identifier, fileStoreTable); try { @@ -146,7 +146,7 @@ protected void beforeBuildingSourceSink() throws Exception { // check partition keys and primary keys in case that user specified them checkConstraints(); } - } else { + } catch (Catalog.TableNotExistException e) { Schema retrievedSchema = retrieveSchema(); computedColumns = buildComputedColumns(computedColumnArgs, retrievedSchema.fields()); Schema paimonSchema = buildPaimonSchema(retrievedSchema); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java index 2b9b08917700..f103396389e5 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SynchronizationActionBase.java @@ -32,10 +32,12 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.table.FileStoreTable; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.connector.source.Source; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.functions.source.SourceFunction; @@ -53,6 +55,7 @@ import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT; import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL; import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_IDLE_TIMEOUT; +import static org.apache.paimon.utils.Preconditions.checkArgument; /** Base {@link Action} for table/database synchronizing job. */ public abstract class SynchronizationActionBase extends ActionBase { @@ -137,6 +140,13 @@ protected CdcTimestampExtractor createCdcTimestampExtractor() { "Unsupported timestamp extractor for current cdc source."); } + protected void validateRuntimeExecutionMode() { + checkArgument( + env.getConfiguration().get(ExecutionOptions.RUNTIME_MODE) + == RuntimeExecutionMode.STREAMING, + "It's only support STREAMING mode for flink-cdc sync table action."); + } + private DataStreamSource buildDataStreamSource(Object source) { if (source instanceof Source) { boolean isAutomaticWatermarkCreationEnabled = diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java index b705bc9e1d88..114344badeed 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/debezium/DebeziumSchemaUtils.java @@ -20,12 +20,14 @@ import org.apache.paimon.flink.action.cdc.TypeMapping; import org.apache.paimon.flink.action.cdc.mysql.MySqlTypeUtils; +import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.DecimalType; import org.apache.paimon.utils.DateTimeUtils; import org.apache.paimon.utils.StringUtils; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; import io.debezium.data.Bits; @@ -36,8 +38,12 @@ import io.debezium.time.MicroTimestamp; import io.debezium.time.Timestamp; import io.debezium.time.ZonedTimestamp; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.util.Utf8; import org.apache.kafka.connect.json.JsonConverterConfig; import javax.annotation.Nullable; @@ -49,13 +55,16 @@ import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZoneOffset; +import java.util.ArrayList; import java.util.Base64; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.function.Supplier; import static org.apache.paimon.flink.action.cdc.TypeMapping.TypeMappingMode.TO_STRING; +import static org.apache.paimon.utils.TypeUtils.OBJECT_MAPPER; /** * Utils to handle 'schema' field in debezium Json. TODO: The methods have many duplicate codes with @@ -101,6 +110,7 @@ public static String transformRawValue( e); } }, + origin, serverTimeZone); } @@ -122,6 +132,7 @@ public static String transformAvroRawValue( className, typeMapping, () -> (ByteBuffer) ((GenericRecord) origin).get(Geometry.WKB_FIELD), + origin, serverTimeZone); } @@ -132,6 +143,7 @@ public static String transformRawValue( @Nullable String className, TypeMapping typeMapping, Supplier geometryGetter, + Object origin, ZoneId serverTimeZone) { if (rawValue == null) { return null; @@ -232,11 +244,73 @@ else if (Date.SCHEMA_NAME.equals(className)) { throw new IllegalArgumentException( String.format("Failed to convert %s to geometry JSON.", rawValue), e); } + } else if ((origin instanceof GenericData.Record) + || (origin instanceof GenericData.Array) + || (origin instanceof Map) + || (origin instanceof List)) { + Object convertedObject = convertAvroObjectToJsonCompatible(origin); + try { + transformed = OBJECT_MAPPER.writer().writeValueAsString(convertedObject); + } catch (JsonProcessingException e) { + throw new RuntimeException( + String.format("Failed to convert %s to JSON.", origin), e); + } } return transformed; } + public static Object convertAvroObjectToJsonCompatible(Object avroObject) { + if (avroObject instanceof GenericData.Record) { + return convertRecord((GenericData.Record) avroObject); + } else if (avroObject instanceof GenericData.Array) { + return convertArray((GenericData.Array) avroObject); + } else if (avroObject instanceof Utf8) { + return avroObject.toString(); + } else if (avroObject instanceof Map) { + return convertMap((Map) avroObject); + } else if (avroObject instanceof List) { + return convertList((List) avroObject); + } else { + return avroObject; + } + } + + private static Map convertMap(Map map) { + Map newMap = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + Object key = convertAvroObjectToJsonCompatible(entry.getKey()); + Object value = convertAvroObjectToJsonCompatible(entry.getValue()); + newMap.put(key, value); + } + return newMap; + } + + private static List convertList(List list) { + List newList = new ArrayList<>(); + for (Object element : list) { + newList.add(convertAvroObjectToJsonCompatible(element)); + } + return newList; + } + + private static Map convertRecord(GenericData.Record record) { + Map map = new HashMap<>(); + for (Schema.Field field : record.getSchema().getFields()) { + Object value = record.get(field.pos()); + map.put(field.name(), convertAvroObjectToJsonCompatible(value)); + } + return map; + } + + private static List convertArray(GenericData.Array array) { + List list = new ArrayList<>(); + for (Object element : array) { + list.add(convertAvroObjectToJsonCompatible(element)); + } + return list; + } + public static DataType toDataType( String debeziumType, @Nullable String className, Map parameters) { if (className == null) { @@ -362,6 +436,30 @@ public static DataType avroToPaimonDataType(Schema schema) { } private static DataType fromDebeziumAvroType(Schema schema) { + LogicalType logicalType = schema.getLogicalType(); + if (logicalType != null) { + if (logicalType instanceof LogicalTypes.Date) { + return DataTypes.DATE(); + } else if (logicalType instanceof LogicalTypes.TimestampMillis) { + return DataTypes.TIMESTAMP_MILLIS(); + } else if (logicalType instanceof LogicalTypes.TimestampMicros) { + return DataTypes.TIMESTAMP(); + } else if (logicalType instanceof LogicalTypes.Decimal) { + LogicalTypes.Decimal decimalType = (LogicalTypes.Decimal) logicalType; + return DataTypes.DECIMAL(decimalType.getPrecision(), decimalType.getScale()); + } else if (logicalType instanceof LogicalTypes.TimeMillis) { + return DataTypes.TIME(3); + } else if (logicalType instanceof LogicalTypes.TimeMicros) { + return DataTypes.TIME(6); + } else if (logicalType instanceof LogicalTypes.LocalTimestampMicros) { + return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(); + } else if (logicalType instanceof LogicalTypes.LocalTimestampMillis) { + return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3); + } else { + throw new UnsupportedOperationException( + String.format("Don't support logical avro type '%s' yet.", logicalType)); + } + } Schema.Type avroType = schema.getType(); switch (avroType) { case BOOLEAN: @@ -378,8 +476,39 @@ private static DataType fromDebeziumAvroType(Schema schema) { case LONG: return DataTypes.BIGINT(); case STRING: - case RECORD: return DataTypes.STRING(); + case RECORD: + List fields = new ArrayList<>(); + for (Schema.Field field : schema.getFields()) { + DataType fieldType = fromDebeziumAvroType(field.schema()); + fields.add(DataTypes.FIELD(field.pos(), field.name(), fieldType, field.doc())); + } + return DataTypes.ROW(fields.toArray(new DataField[0])); + case ARRAY: + Schema elementSchema = schema.getElementType(); + DataType elementType = fromDebeziumAvroType(elementSchema); + return DataTypes.ARRAY(elementType); + case MAP: + DataType valueType = fromDebeziumAvroType(schema.getValueType()); + return DataTypes.MAP(DataTypes.STRING(), valueType); + case UNION: + List unionTypes = schema.getTypes(); + // Check if it's a nullable type union + if (unionTypes.size() == 2 + && unionTypes.contains(Schema.create(Schema.Type.NULL))) { + Schema actualSchema = + unionTypes.stream() + .filter(s -> s.getType() != Schema.Type.NULL) + .findFirst() + .orElseThrow( + () -> + new IllegalStateException( + "Union type does not contain a non-null type")); + return fromDebeziumAvroType(actualSchema) + .copy(true); // Return nullable version of the non-null type + } + // Handle generic unions or throw an exception + throw new UnsupportedOperationException("Generic unions are not supported"); default: throw new UnsupportedOperationException( String.format("Don't support avro type '%s' yet.", avroType)); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/dms/DMSDataFormat.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/dms/DMSDataFormat.java new file mode 100644 index 000000000000..43228fca4554 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/dms/DMSDataFormat.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.format.dms; + +import org.apache.paimon.flink.action.cdc.format.AbstractJsonDataFormat; +import org.apache.paimon.flink.action.cdc.format.RecordParserFactory; + +/** + * Supports the message queue's AWS DMS json data format and provides definitions for the message + * queue's record json deserialization class and parsing class {@link DMSRecordParser}. + */ +public class DMSDataFormat extends AbstractJsonDataFormat { + + @Override + protected RecordParserFactory parser() { + return DMSRecordParser::new; + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/dms/DMSDataFormatFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/dms/DMSDataFormatFactory.java new file mode 100644 index 000000000000..0be1270e8341 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/dms/DMSDataFormatFactory.java @@ -0,0 +1,38 @@ +/* + * 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.cdc.format.dms; + +import org.apache.paimon.flink.action.cdc.format.DataFormat; +import org.apache.paimon.flink.action.cdc.format.DataFormatFactory; + +/** Factory to create {@link DMSDataFormat}. */ +public class DMSDataFormatFactory implements DataFormatFactory { + + public static final String IDENTIFIER = "aws-dms-json"; + + @Override + public DataFormat create() { + return new DMSDataFormat(); + } + + @Override + public String identifier() { + return IDENTIFIER; + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/dms/DMSRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/dms/DMSRecordParser.java new file mode 100644 index 000000000000..8fc4808dd2d6 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/dms/DMSRecordParser.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.flink.action.cdc.format.dms; + +import org.apache.paimon.flink.action.cdc.ComputedColumn; +import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.action.cdc.format.AbstractJsonRecordParser; +import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.utils.Pair; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.node.ObjectNode; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * The {@code DMSRecordParser} class extends the abstract {@link AbstractJsonRecordParser} and is + * designed to parse records from AWS DMS's JSON change data capture (CDC) format. AWS DMS is a CDC + * solution for RDMS that captures row-level changes to database tables and outputs them in JSON + * format. This parser extracts relevant information from the DMS-JSON format and converts it into a + * list of {@link RichCdcMultiplexRecord} objects. + * + *

    The class supports various database operations such as INSERT, UPDATE, and DELETE, and creates + * corresponding {@link RichCdcMultiplexRecord} objects to represent these changes. + * + *

    Validation is performed to ensure that the JSON records contain all necessary fields, and the + * class also supports schema extraction for the Kafka topic. + */ +public class DMSRecordParser extends AbstractJsonRecordParser { + + private static final String FIELD_DATA = "data"; + private static final String FIELD_METADATA = "metadata"; + private static final String FIELD_TYPE = "record-type"; + private static final String FIELD_OP = "operation"; + private static final String FIELD_DATABASE = "schema-name"; + private static final String FIELD_TABLE = "table-name"; + + private static final String OP_LOAD = "load"; + private static final String OP_INSERT = "insert"; + private static final String OP_UPDATE = "update"; + private static final String OP_DELETE = "delete"; + + private static final String BEFORE_PREFIX = "BI_"; + + public DMSRecordParser(TypeMapping typeMapping, List computedColumns) { + super(typeMapping, computedColumns); + } + + @Override + protected @Nullable String getTableName() { + JsonNode metaNode = getAndCheck(FIELD_METADATA); + return metaNode.get(FIELD_TABLE).asText(); + } + + @Override + protected List extractRecords() { + if (isDDL()) { + return Collections.emptyList(); + } + + JsonNode dataNode = getAndCheck(dataField()); + String operation = getAndCheck(FIELD_METADATA).get(FIELD_OP).asText(); + List records = new ArrayList<>(); + + switch (operation) { + case OP_LOAD: + case OP_INSERT: + processRecord(dataNode, RowKind.INSERT, records); + break; + case OP_UPDATE: + Pair dataAndBeforeNodes = splitBeforeAndData(dataNode); + processRecord(dataAndBeforeNodes.getRight(), RowKind.DELETE, records); + processRecord(dataAndBeforeNodes.getLeft(), RowKind.INSERT, records); + break; + case OP_DELETE: + processRecord(dataNode, RowKind.DELETE, records); + break; + default: + throw new UnsupportedOperationException("Unknown record operation: " + operation); + } + + return records; + } + + @Override + protected @Nullable String getDatabaseName() { + JsonNode metaNode = getAndCheck(FIELD_METADATA); + return metaNode.get(FIELD_DATABASE).asText(); + } + + @Override + protected String primaryField() { + return null; + } + + @Override + protected String dataField() { + return FIELD_DATA; + } + + @Override + protected String format() { + return "aws-dms-json"; + } + + @Override + protected boolean isDDL() { + String recordType = getAndCheck(FIELD_METADATA).get(FIELD_TYPE).asText(); + return !"data".equals(recordType); + } + + private Pair splitBeforeAndData(JsonNode dataNode) { + JsonNode newDataNode = dataNode.deepCopy(); + JsonNode beforeDataNode = dataNode.deepCopy(); + + Iterator> newDataFields = newDataNode.fields(); + while (newDataFields.hasNext()) { + Map.Entry next = newDataFields.next(); + if (next.getKey().startsWith(BEFORE_PREFIX)) { + newDataFields.remove(); + } + } + + Iterator> beforeDataFields = beforeDataNode.fields(); + while (beforeDataFields.hasNext()) { + Map.Entry next = beforeDataFields.next(); + if (next.getKey().startsWith(BEFORE_PREFIX)) { + String key = next.getKey().replaceFirst(BEFORE_PREFIX, ""); + ((ObjectNode) beforeDataNode).set(key, next.getValue()); + beforeDataFields.remove(); + } + } + + return Pair.of(newDataNode, beforeDataNode); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseAction.java index 9f3ed1085600..3166a3c82ae5 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncDatabaseAction.java @@ -66,6 +66,7 @@ protected CdcTimestampExtractor createCdcTimestampExtractor() { @Override protected MongoDBSource buildSource() { + validateRuntimeExecutionMode(); return MongoDBActionUtils.buildMongodbSource( cdcSourceConfig, CdcActionCommonUtils.combinedModeTableList( diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableAction.java index 16dbbadfd776..34128a62fcde 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableAction.java @@ -77,6 +77,7 @@ protected CdcTimestampExtractor createCdcTimestampExtractor() { @Override protected MongoDBSource buildSource() { + validateRuntimeExecutionMode(); String tableList = cdcSourceConfig.get(MongoDBSourceOptions.DATABASE) + "\\." diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java index 01be020f7405..f8ea8cdc4438 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseAction.java @@ -188,6 +188,7 @@ protected CdcTimestampExtractor createCdcTimestampExtractor() { @Override protected MySqlSource buildSource() { + validateRuntimeExecutionMode(); return MySqlActionUtils.buildMySqlSource( cdcSourceConfig, tableList( diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java index a05832b1d033..d73d9702f1e1 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableAction.java @@ -102,6 +102,7 @@ protected Schema retrieveSchema() throws Exception { @Override protected MySqlSource buildSource() { + validateRuntimeExecutionMode(); String tableList = String.format( "(%s)\\.(%s)", diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableAction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableAction.java index f66c20dfa02a..7dc1b019cd02 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableAction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableAction.java @@ -103,6 +103,7 @@ protected Schema retrieveSchema() throws Exception { @Override protected JdbcIncrementalSource buildSource() { + validateRuntimeExecutionMode(); List pkTables = postgresSchemasInfo.pkTables(); Set schemaList = new HashSet<>(); String[] tableList = new String[pkTables.size()]; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecord.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecord.java index 9adca753dc55..b23d0d6f06de 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecord.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecord.java @@ -35,11 +35,12 @@ public class CdcRecord implements Serializable { private RowKind kind; - private final Map fields; + // field name -> value + private final Map data; - public CdcRecord(RowKind kind, Map fields) { + public CdcRecord(RowKind kind, Map data) { this.kind = kind; - this.fields = fields; + this.data = data; } public static CdcRecord emptyRecord() { @@ -50,16 +51,16 @@ public RowKind kind() { return kind; } - public Map fields() { - return fields; + public Map data() { + return data; } public CdcRecord fieldNameLowerCase() { - Map newFields = new HashMap<>(); - for (Map.Entry entry : fields.entrySet()) { - newFields.put(entry.getKey().toLowerCase(), entry.getValue()); + Map newData = new HashMap<>(); + for (Map.Entry entry : data.entrySet()) { + newData.put(entry.getKey().toLowerCase(), entry.getValue()); } - return new CdcRecord(kind, newFields); + return new CdcRecord(kind, newData); } @Override @@ -69,16 +70,16 @@ public boolean equals(Object o) { } CdcRecord that = (CdcRecord) o; - return Objects.equals(kind, that.kind) && Objects.equals(fields, that.fields); + return Objects.equals(kind, that.kind) && Objects.equals(data, that.data); } @Override public int hashCode() { - return Objects.hash(kind, fields); + return Objects.hash(kind, data); } @Override public String toString() { - return kind.shortString() + " " + fields; + return kind.shortString() + " " + data; } } 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 a604b304524a..7d72fe3e801f 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 @@ -28,6 +28,7 @@ import org.apache.paimon.flink.sink.StoreSinkWrite; import org.apache.paimon.flink.sink.StoreSinkWriteImpl; import org.apache.paimon.flink.sink.StoreSinkWriteState; +import org.apache.paimon.flink.sink.StoreSinkWriteStateImpl; import org.apache.paimon.memory.HeapMemorySegmentPool; import org.apache.paimon.memory.MemoryPoolFactory; import org.apache.paimon.options.Options; @@ -98,7 +99,7 @@ public void initializeState(StateInitializationContext context) throws Exception context, "commit_user_state", String.class, initialCommitUser); // TODO: should use CdcRecordMultiChannelComputer to filter - state = new StoreSinkWriteState(context, (tableName, partition, bucket) -> true); + state = new StoreSinkWriteStateImpl(context, (tableName, partition, bucket) -> true); tables = new HashMap<>(); writes = new HashMap<>(); compactExecutor = diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordUtils.java index 0d192dd538d6..91979a2c99b8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordUtils.java @@ -54,7 +54,7 @@ public static GenericRow projectAsInsert(CdcRecord record, List dataF GenericRow genericRow = new GenericRow(dataFields.size()); for (int i = 0; i < dataFields.size(); i++) { DataField dataField = dataFields.get(i); - String fieldValue = record.fields().get(dataField.name()); + String fieldValue = record.data().get(dataField.name()); if (fieldValue != null) { genericRow.setField( i, TypeUtils.castFromCdcValueString(fieldValue, dataField.type())); @@ -83,7 +83,7 @@ public static Optional toGenericRow(CdcRecord record, List fieldNames = dataFields.stream().map(DataField::name).collect(Collectors.toList()); - for (Map.Entry field : record.fields().entrySet()) { + for (Map.Entry field : record.data().entrySet()) { String key = field.getKey(); String value = field.getValue(); @@ -117,14 +117,14 @@ public static Optional toGenericRow(CdcRecord record, List fieldNames) { - Map fields = new HashMap<>(); + Map data = new HashMap<>(); for (int i = 0; i < row.getFieldCount(); i++) { Object field = row.getField(i); if (field != null) { - fields.put(fieldNames.get(i), field.toString()); + data.put(fieldNames.get(i), field.toString()); } } - return new CdcRecord(row.getRowKind(), fields); + return new CdcRecord(row.getRowKind(), data); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcRecord.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcRecord.java index 7fc0c3ff7b09..04b86fea568f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcRecord.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichCdcRecord.java @@ -48,7 +48,7 @@ public RichCdcRecord(CdcRecord cdcRecord, List fields) { } public boolean hasPayload() { - return !cdcRecord.fields().isEmpty(); + return !cdcRecord.data().isEmpty(); } public RowKind kind() { @@ -95,7 +95,7 @@ public static class Builder { private final RowKind kind; private final AtomicInteger fieldId; private final List fields = new ArrayList<>(); - private final Map fieldValues = new HashMap<>(); + private final Map data = new HashMap<>(); public Builder(RowKind kind, AtomicInteger fieldId) { this.kind = kind; @@ -109,12 +109,12 @@ public Builder field(String name, DataType type, String value) { public Builder field( String name, DataType type, String value, @Nullable String description) { fields.add(new DataField(fieldId.incrementAndGet(), name, type, description)); - fieldValues.put(name, value); + data.put(name, value); return this; } public RichCdcRecord build() { - return new RichCdcRecord(new CdcRecord(kind, fieldValues), fields); + return new RichCdcRecord(new CdcRecord(kind, data), fields); } } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichEventParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichEventParser.java index 10dbdcc8dc8c..01a002214e80 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichEventParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/RichEventParser.java @@ -24,7 +24,6 @@ import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; -import java.util.Objects; /** A {@link EventParser} for {@link RichCdcRecord}. */ public class RichEventParser implements EventParser { @@ -48,7 +47,7 @@ public List parseSchemaChange() { // When the order of the same field is different, its ID may also be // different, // so the comparison should not include the ID. - if (!dataFieldEqualsIgnoreId(previous, dataField)) { + if (!DataField.dataFieldEqualsIgnoreId(previous, dataField)) { previousDataFields.put(dataField.name(), dataField); change.add(dataField); } @@ -56,18 +55,6 @@ public List parseSchemaChange() { return change; } - private boolean dataFieldEqualsIgnoreId(DataField dataField1, DataField dataField2) { - if (dataField1 == dataField2) { - return true; - } else if (dataField1 != null && dataField2 != null) { - return Objects.equals(dataField1.name(), dataField2.name()) - && Objects.equals(dataField1.type(), dataField2.type()) - && Objects.equals(dataField1.description(), dataField2.description()); - } else { - return false; - } - } - @Override public List parseRecords() { if (record.hasPayload()) { diff --git a/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index 9c4c4d0ac3a2..17b8b29a2009 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-flink/paimon-flink-cdc/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -33,3 +33,4 @@ org.apache.paimon.flink.action.cdc.format.debezium.DebeziumJsonDataFormatFactory org.apache.paimon.flink.action.cdc.format.json.JsonDataFormatFactory org.apache.paimon.flink.action.cdc.format.maxwell.MaxwellDataFormatFactory org.apache.paimon.flink.action.cdc.format.ogg.OggDataFormatFactory +org.apache.paimon.flink.action.cdc.format.dms.DMSDataFormatFactory diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java index 08289569086a..00a8b236173b 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java @@ -38,6 +38,7 @@ import org.apache.paimon.types.RowType; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.core.execution.JobClient; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.AfterEach; @@ -214,6 +215,15 @@ protected List nullableToArgs(String argKey, @Nullable T nullable) { } public JobClient runActionWithDefaultEnv(ActionBase action) throws Exception { + env.setRuntimeMode(RuntimeExecutionMode.STREAMING); + action.withStreamExecutionEnvironment(env).build(); + JobClient client = env.executeAsync(); + waitJobRunning(client); + return client; + } + + public JobClient runActionWithBatchEnv(ActionBase action) throws Exception { + env.setRuntimeMode(RuntimeExecutionMode.BATCH); action.withStreamExecutionEnvironment(env).build(); JobClient client = env.executeAsync(); waitJobRunning(client); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaAWSDMSSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaAWSDMSSyncDatabaseActionITCase.java new file mode 100644 index 000000000000..da9f863dc07b --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaAWSDMSSyncDatabaseActionITCase.java @@ -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. + */ + +package org.apache.paimon.flink.action.cdc.kafka; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.util.Map; + +/** IT cases for {@link KafkaSyncDatabaseAction}. */ +public class KafkaAWSDMSSyncDatabaseActionITCase extends KafkaSyncDatabaseActionITCase { + + private static final String AWSDMS = "aws-dms"; + + @Override + protected KafkaSyncDatabaseActionBuilder syncDatabaseActionBuilder( + Map kafkaConfig) { + KafkaSyncDatabaseActionBuilder builder = new KafkaSyncDatabaseActionBuilder(kafkaConfig); + builder.withPrimaryKeys("id"); + return builder; + } + + @Test + @Timeout(60) + public void testSchemaEvolutionMultiTopic() throws Exception { + testSchemaEvolutionMultiTopic(AWSDMS); + } + + @Test + @Timeout(60) + public void testSchemaEvolutionOneTopic() throws Exception { + testSchemaEvolutionOneTopic(AWSDMS); + } + + @Test + public void testTopicIsEmpty() { + testTopicIsEmpty(AWSDMS); + } + + @Test + @Timeout(60) + public void testTableAffixMultiTopic() throws Exception { + testTableAffixMultiTopic(AWSDMS); + } + + @Test + @Timeout(60) + public void testTableAffixOneTopic() throws Exception { + testTableAffixOneTopic(AWSDMS); + } + + @Test + @Timeout(60) + public void testIncludingTables() throws Exception { + testIncludingTables(AWSDMS); + } + + @Test + @Timeout(60) + public void testExcludingTables() throws Exception { + testExcludingTables(AWSDMS); + } + + @Test + @Timeout(60) + public void testIncludingAndExcludingTables() throws Exception { + testIncludingAndExcludingTables(AWSDMS); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaAWSDMSSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaAWSDMSSyncTableActionITCase.java new file mode 100644 index 000000000000..02ac86cdab69 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaAWSDMSSyncTableActionITCase.java @@ -0,0 +1,82 @@ +/* + * 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.cdc.kafka; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +/** IT cases for {@link KafkaSyncTableAction}. */ +public class KafkaAWSDMSSyncTableActionITCase extends KafkaSyncTableActionITCase { + + private static final String AWSDMS = "aws-dms"; + + @Test + @Timeout(60) + public void testSchemaEvolution() throws Exception { + runSingleTableSchemaEvolution("schemaevolution", AWSDMS); + } + + @Test + @Timeout(60) + public void testAssertSchemaCompatible() throws Exception { + testAssertSchemaCompatible(AWSDMS); + } + + @Test + @Timeout(60) + public void testStarUpOptionSpecific() throws Exception { + testStarUpOptionSpecific(AWSDMS); + } + + @Test + @Timeout(60) + public void testStarUpOptionLatest() throws Exception { + testStarUpOptionLatest(AWSDMS); + } + + @Test + @Timeout(60) + public void testStarUpOptionTimestamp() throws Exception { + testStarUpOptionTimestamp(AWSDMS); + } + + @Test + @Timeout(60) + public void testStarUpOptionEarliest() throws Exception { + testStarUpOptionEarliest(AWSDMS); + } + + @Test + @Timeout(60) + public void testStarUpOptionGroup() throws Exception { + testStarUpOptionGroup(AWSDMS); + } + + @Test + @Timeout(60) + public void testComputedColumn() throws Exception { + testComputedColumn(AWSDMS); + } + + @Test + @Timeout(60) + public void testFieldValNullSyncTable() throws Exception { + testTableFiledValNull(AWSDMS); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java index 8f41a370e9fd..8db2e7ffe92a 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaActionITCaseBase.java @@ -227,7 +227,8 @@ private Map describeExternalTopics() { } } - private void logTopicPartitionStatus(Map topicDescriptions) { + private synchronized void logTopicPartitionStatus( + Map topicDescriptions) { List partitions = new ArrayList<>(); topicDescriptions.forEach( (topic, description) -> diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java index 04dfb3769a61..430598753629 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableActionITCase.java @@ -106,10 +106,16 @@ public void testKafkaBuildSchemaWithDelete() throws Exception { @Test @Timeout(60) - public void testSchemaIncludeRecord1() throws Exception { + public void testSchemaIncludeRecord() throws Exception { testSchemaIncludeRecord(DEBEZIUM); } + @Test + @Timeout(60) + public void testSchemaEvolutionWithSchemaIncludeRecord() throws Exception { + runSingleTableSchemaEvolutionWithSchemaIncludeRecord("schemaevolution", DEBEZIUM); + } + @Test @Timeout(60) public void testAllTypesWithSchema() throws Exception { diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java index 117994835600..f5b6bb5923e9 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java @@ -28,7 +28,9 @@ import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.flink.api.common.JobStatus; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; import java.util.ArrayList; import java.util.Arrays; @@ -136,6 +138,120 @@ private void testSchemaEvolutionImpl(String topic, String sourceDir, String form waitForResult(expected, table, rowType, primaryKeys); } + protected void runSingleTableSchemaEvolutionWithSchemaIncludeRecord( + String sourceDir, String format) throws Exception { + final String topic = "schema_evolution"; + createTestTopic(topic, 1, 1); + writeRecordsToKafka( + topic, "kafka/%s/table/schema/%s/%s-data-1.txt", format, sourceDir, format); + + Map kafkaConfig = getBasicKafkaConfig(); + kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); + kafkaConfig.put(TOPIC.key(), topic); + KafkaSyncTableAction action = + syncTableActionBuilder(kafkaConfig) + .withPrimaryKeys("id") + .withTableConfig(getBasicTableConfig()) + .build(); + JobClient jobClient = runActionWithDefaultEnv(action); + + FileStoreTable table = getFileStoreTable(tableName); + + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.DOUBLE() + }, + new String[] {"id", "name", "description", "weight"}); + List primaryKeys = Collections.singletonList("id"); + List expected = + Collections.singletonList("+I[101, scooter, Small 2-wheel scooter, 3.14]"); + waitForResult(expected, table, rowType, primaryKeys); + + // add column + writeRecordsToKafka( + topic, "kafka/%s/table/schema/%s/%s-data-2.txt", format, sourceDir, format); + + rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.DOUBLE(), + DataTypes.INT() + }, + new String[] {"id", "name", "description", "weight", "age"}); + expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14, NULL]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18]"); + waitForResult(expected, table, rowType, primaryKeys); + + // column type promotion (int32 -> int64) + writeRecordsToKafka( + topic, "kafka/%s/table/schema/%s/%s-data-3.txt", format, sourceDir, format); + + rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.DOUBLE(), + DataTypes.BIGINT() + }, + new String[] {"id", "name", "description", "weight", "age"}); + expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14, NULL]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24]"); + waitForResult(expected, table, rowType, primaryKeys); + + // column type changed ignore (int64 -> int32) + writeRecordsToKafka( + topic, "kafka/%s/table/schema/%s/%s-data-4.txt", format, sourceDir, format); + + rowType = + RowType.of( + new DataType[] { + DataTypes.INT().notNull(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.DOUBLE(), + DataTypes.BIGINT() + }, + new String[] {"id", "name", "description", "weight", "age"}); + expected = + Arrays.asList( + "+I[101, scooter, Small 2-wheel scooter, 3.14, NULL]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8, 18]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75, 24]", + "+I[105, hammer, 14oz carpenter's hammer, 0.875, 24]"); + waitForResult(expected, table, rowType, primaryKeys); + + // column type covert exception (int64 -> string) + writeRecordsToKafka( + topic, "kafka/%s/table/schema/%s/%s-data-5.txt", format, sourceDir, format); + + while (true) { + JobStatus status = jobClient.getJobStatus().get(); + if (status != JobStatus.RUNNING) { + assertThatThrownBy(() -> jobClient.getJobExecutionResult().get()) + .satisfies( + anyCauseMatches( + UnsupportedOperationException.class, + "Cannot convert field age from type BIGINT to STRING of Paimon table")); + break; + } + Thread.sleep(1000); + } + } + public void testNotSupportFormat(String format) throws Exception { final String topic = "not_support"; createTestTopic(topic, 1, 1); @@ -478,7 +594,10 @@ public void testKafkaBuildSchemaWithDelete(String format) throws Exception { final String topic = "test_kafka_schema"; createTestTopic(topic, 1, 1); writeRecordsToKafka( - topic, "kafka/%s/table/schema/schemaevolution/%s-data-4.txt", format, format); + topic, + "kafka/%s/table/schema/schemaevolution/%s-data-with-delete.txt", + format, + format); Configuration kafkaConfig = Configuration.fromMap(getBasicKafkaConfig()); kafkaConfig.setString(VALUE_FORMAT.key(), format + "-json"); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java index 48c6cd481103..b4f31f2d6d3d 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongoDBSyncTableActionITCase.java @@ -34,7 +34,9 @@ import java.util.List; import java.util.Map; +import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** IT cases for {@link MongoDBSyncTableAction}. */ public class MongoDBSyncTableActionITCase extends MongoDBActionITCaseBase { @@ -398,4 +400,26 @@ public void testComputedColumnWithCaseInsensitive() throws Exception { rowType, Collections.singletonList("_id")); } + + @Test + @Timeout(60) + public void testRuntimeExecutionModeCheckForCdcSync() { + Map mongodbConfig = getBasicMongoDBConfig(); + mongodbConfig.put("database", database); + mongodbConfig.put("collection", "products"); + mongodbConfig.put("field.name", "_id,name,description"); + mongodbConfig.put("parser.path", "$._id,$.name,$.description"); + mongodbConfig.put("schema.start.mode", "specified"); + + MongoDBSyncTableAction action = + syncTableActionBuilder(mongodbConfig) + .withTableConfig(getBasicTableConfig()) + .build(); + + assertThatThrownBy(() -> runActionWithBatchEnv(action)) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "It's only support STREAMING mode for flink-cdc sync table action")); + } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseTableListITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseTableListITCase.java index c15cbf89884e..21866a0a2fbc 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseTableListITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseTableListITCase.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.action.cdc.mysql; import org.apache.paimon.flink.action.MultiTablesSinkMode; +import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; @@ -28,7 +29,9 @@ import org.junit.jupiter.api.Timeout; import java.sql.Statement; +import java.util.Arrays; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; @@ -45,7 +48,7 @@ public static void startContainers() { } @Test - @Timeout(180) + @Timeout(60) public void testActionRunResult() throws Exception { Map mySqlConfig = getBasicMySqlConfig(); mySqlConfig.put( @@ -80,14 +83,18 @@ public void testActionRunResult() throws Exception { // test newly created tables if (mode == COMBINED) { try (Statement statement = getStatement()) { - // ensure the job steps into incremental phase - waitForResult( - Collections.singletonList("+I[1, A]"), - getFileStoreTable("shard_1_t2"), + FileStoreTable t2 = getFileStoreTable("shard_1_t2"); + RowType rowTypeT2 = RowType.of( new DataType[] {DataTypes.INT().notNull(), DataTypes.VARCHAR(100)}, - new String[] {"k", "name"}), - Collections.singletonList("k")); + new String[] {"k", "name"}); + List primaryKeysT2 = Collections.singletonList("k"); + + // ensure the job steps into incremental phase + waitForResult(Collections.singletonList("+I[1, A]"), t2, rowTypeT2, primaryKeysT2); + statement.executeUpdate("USE shard_1"); + statement.executeUpdate("INSERT INTO t2 VALUES (2, 'B')"); + waitForResult(Arrays.asList("+I[1, A]", "+I[2, B]"), t2, rowTypeT2, primaryKeysT2); // case 1: new tables in existed database statement.executeUpdate("USE shard_2"); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index ce18d0b1f0e8..bdeab07a746c 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -1497,4 +1497,39 @@ public void testUnknowMysqlScanStartupMode() { + scanStartupMode + "'. Valid scan.startup.mode for MySQL CDC are [initial, earliest-offset, latest-offset, specific-offset, timestamp, snapshot]")); } + + @Test + @Timeout(1000) + public void testRuntimeExecutionModeCheckForCdcSync() throws Exception { + Map mySqlConfig = getBasicMySqlConfig(); + mySqlConfig.put("database-name", "check_cdc_sync_runtime_execution_mode"); + mySqlConfig.put("table-name", "t"); + + Map tableConfig = getBasicTableConfig(); + tableConfig.put(CoreOptions.WRITE_ONLY.key(), "true"); + + MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build(); + + assertThatThrownBy(() -> runActionWithBatchEnv(action)) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "It's only support STREAMING mode for flink-cdc sync table action")); + + runActionWithDefaultEnv(action); + + FileStoreTable table = getFileStoreTable(); + + try (Statement statement = getStatement()) { + statement.executeUpdate("USE check_cdc_sync_runtime_execution_mode"); + statement.executeUpdate("INSERT INTO t VALUES (1, 'one'), (2, 'two')"); + RowType rowType = + RowType.of( + new DataType[] {DataTypes.INT().notNull(), DataTypes.VARCHAR(10)}, + new String[] {"k", "v1"}); + List primaryKeys = Collections.singletonList("k"); + List expected = Arrays.asList("+I[1, one]", "+I[2, two]"); + waitForResult(expected, table, rowType, primaryKeys); + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java index 10f14ca732d5..58d122b3c1ab 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/postgres/PostgresSyncTableActionITCase.java @@ -789,4 +789,29 @@ public void testCatalogAndTableConfig() { private FileStoreTable getFileStoreTable() throws Exception { return getFileStoreTable(tableName); } + + @Test + @Timeout(60) + public void testRuntimeExecutionModeCheckForCdcSync() { + Map postgresConfig = getBasicPostgresConfig(); + postgresConfig.put(PostgresSourceOptions.DATABASE_NAME.key(), DATABASE_NAME); + postgresConfig.put(PostgresSourceOptions.SCHEMA_NAME.key(), SCHEMA_NAME); + postgresConfig.put(PostgresSourceOptions.TABLE_NAME.key(), "schema_evolution_\\d+"); + + PostgresSyncTableAction action = + syncTableActionBuilder(postgresConfig) + .withCatalogConfig( + Collections.singletonMap( + CatalogOptions.METASTORE.key(), "test-alter-table")) + .withTableConfig(getBasicTableConfig()) + .withPartitionKeys("pt") + .withPrimaryKeys("pt", "_id") + .build(); + + assertThatThrownBy(() -> runActionWithBatchEnv(action)) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "It's only support STREAMING mode for flink-cdc sync table action")); + } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java index ce0d484f4ce4..867cbdbae002 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcMultiplexRecordChannelComputerTest.java @@ -163,9 +163,9 @@ private void testImpl(Identifier tableId, List> input) { // assert that insert and delete records are routed into same channel - for (Map fields : input) { - CdcRecord insertRecord = new CdcRecord(RowKind.INSERT, fields); - CdcRecord deleteRecord = new CdcRecord(RowKind.DELETE, fields); + for (Map data : input) { + CdcRecord insertRecord = new CdcRecord(RowKind.INSERT, data); + CdcRecord deleteRecord = new CdcRecord(RowKind.DELETE, data); assertThat( channelComputer.channel( @@ -184,8 +184,8 @@ private void testImpl(Identifier tableId, List> input) { // assert that channel >= 0 int numTests = random.nextInt(10) + 1; for (int test = 0; test < numTests; test++) { - Map fields = input.get(random.nextInt(input.size())); - CdcRecord record = new CdcRecord(RowKind.INSERT, fields); + Map data = input.get(random.nextInt(input.size())); + CdcRecord record = new CdcRecord(RowKind.INSERT, data); int numBuckets = random.nextInt(numChannels * 4) + 1; for (int i = 0; i < numBuckets; i++) { diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordChannelComputerTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordChannelComputerTest.java index 9a19013e2983..8271ad18751c 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordChannelComputerTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordChannelComputerTest.java @@ -128,9 +128,9 @@ private void testImpl(TableSchema schema, List> input) { // assert that channel(record) and channel(partition, bucket) gives the same result - for (Map fields : input) { - CdcRecord insertRecord = new CdcRecord(RowKind.INSERT, fields); - CdcRecord deleteRecord = new CdcRecord(RowKind.DELETE, fields); + for (Map data : input) { + CdcRecord insertRecord = new CdcRecord(RowKind.INSERT, data); + CdcRecord deleteRecord = new CdcRecord(RowKind.DELETE, data); extractor.setRecord(random.nextBoolean() ? insertRecord : deleteRecord); BinaryRow partition = extractor.partition(); @@ -151,8 +151,8 @@ private void testImpl(TableSchema schema, List> input) { bucketsPerChannel.put(i, 0); } - Map fields = input.get(random.nextInt(input.size())); - extractor.setRecord(new CdcRecord(RowKind.INSERT, fields)); + Map data = input.get(random.nextInt(input.size())); + extractor.setRecord(new CdcRecord(RowKind.INSERT, data)); BinaryRow partition = extractor.partition(); int numBuckets = random.nextInt(numChannels * 4) + 1; diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordKeyAndBucketExtractorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordKeyAndBucketExtractorTest.java index 8384b7155a0e..802a3ea9d4cf 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordKeyAndBucketExtractorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordKeyAndBucketExtractorTest.java @@ -87,19 +87,19 @@ public void testExtract() throws Exception { StringData.fromString(v2)); expected.setRecord(rowData); - Map fields = new HashMap<>(); - fields.put("pt1", pt1); - fields.put("pt2", String.valueOf(pt2)); - fields.put("k1", String.valueOf(k1)); - fields.put("v1", String.valueOf(v1)); - fields.put("k2", k2); - fields.put("v2", v2); - - actual.setRecord(new CdcRecord(RowKind.INSERT, fields)); + Map data = new HashMap<>(); + data.put("pt1", pt1); + data.put("pt2", String.valueOf(pt2)); + data.put("k1", String.valueOf(k1)); + data.put("v1", String.valueOf(v1)); + data.put("k2", k2); + data.put("v2", v2); + + actual.setRecord(new CdcRecord(RowKind.INSERT, data)); assertThat(actual.partition()).isEqualTo(expected.partition()); assertThat(actual.bucket()).isEqualTo(expected.bucket()); - actual.setRecord(new CdcRecord(RowKind.DELETE, fields)); + actual.setRecord(new CdcRecord(RowKind.DELETE, data)); assertThat(actual.partition()).isEqualTo(expected.partition()); assertThat(actual.bucket()).isEqualTo(expected.bucket()); } @@ -122,19 +122,19 @@ public void testNullPartition() throws Exception { null, null, k1, v1, StringData.fromString(k2), StringData.fromString(v2)); expected.setRecord(rowData); - Map fields = new HashMap<>(); - fields.put("pt1", null); - fields.put("pt2", null); - fields.put("k1", String.valueOf(k1)); - fields.put("v1", String.valueOf(v1)); - fields.put("k2", k2); - fields.put("v2", v2); + Map data = new HashMap<>(); + data.put("pt1", null); + data.put("pt2", null); + data.put("k1", String.valueOf(k1)); + data.put("v1", String.valueOf(v1)); + data.put("k2", k2); + data.put("v2", v2); - actual.setRecord(new CdcRecord(RowKind.INSERT, fields)); + actual.setRecord(new CdcRecord(RowKind.INSERT, data)); assertThat(actual.partition()).isEqualTo(expected.partition()); assertThat(actual.bucket()).isEqualTo(expected.bucket()); - actual.setRecord(new CdcRecord(RowKind.DELETE, fields)); + actual.setRecord(new CdcRecord(RowKind.DELETE, data)); assertThat(actual.partition()).isEqualTo(expected.partition()); assertThat(actual.bucket()).isEqualTo(expected.bucket()); } @@ -161,19 +161,19 @@ public void testEmptyPartition() throws Exception { StringData.fromString(v2)); expected.setRecord(rowData); - Map fields = new HashMap<>(); - fields.put("pt1", ""); - fields.put("pt2", null); - fields.put("k1", String.valueOf(k1)); - fields.put("v1", String.valueOf(v1)); - fields.put("k2", k2); - fields.put("v2", v2); + Map data = new HashMap<>(); + data.put("pt1", ""); + data.put("pt2", null); + data.put("k1", String.valueOf(k1)); + data.put("v1", String.valueOf(v1)); + data.put("k2", k2); + data.put("v2", v2); - actual.setRecord(new CdcRecord(RowKind.INSERT, fields)); + actual.setRecord(new CdcRecord(RowKind.INSERT, data)); assertThat(actual.partition()).isEqualTo(expected.partition()); assertThat(actual.bucket()).isEqualTo(expected.bucket()); - actual.setRecord(new CdcRecord(RowKind.DELETE, fields)); + actual.setRecord(new CdcRecord(RowKind.DELETE, data)); assertThat(actual.partition()).isEqualTo(expected.partition()); assertThat(actual.bucket()).isEqualTo(expected.bucket()); } 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 2a1bb4004306..8c78ab853a60 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 @@ -172,16 +172,14 @@ public void testAsyncTableCreate() throws Exception { t.start(); // check that records should be processed after table is created - Map fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "1"); - fields.put("v", "10"); + Map data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "1"); + data.put("v", "10"); CdcMultiplexRecord expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); CdcMultiplexRecord actual = runner.poll(1); @@ -192,15 +190,13 @@ public void testAsyncTableCreate() throws Exception { assertThat(actual).isEqualTo(expected); // after table is created, record should be processed immediately - fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "3"); - fields.put("v", "30"); + data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "3"); + data.put("v", "30"); expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.take(); assertThat(actual).isEqualTo(expected); @@ -227,16 +223,14 @@ public void testInitializeState() throws Exception { t.start(); // check that records should be processed after table is created - Map fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "1"); - fields.put("v", "10"); + Map data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "1"); + data.put("v", "10"); CdcMultiplexRecord expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); CdcMultiplexRecord actual = runner.poll(1); @@ -254,15 +248,13 @@ public void testInitializeState() throws Exception { assertThat(operator.writes().size()).isEqualTo(1); // after table is created, record should be processed immediately - fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "3"); - fields.put("v", "30"); + data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "3"); + data.put("v", "30"); expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.take(); assertThat(actual).isEqualTo(expected); @@ -302,44 +294,38 @@ public void testSingleTableAddColumn() throws Exception { // check that records with compatible schema can be processed immediately - Map fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "1"); - fields.put("v", "10"); + Map data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "1"); + data.put("v", "10"); CdcMultiplexRecord expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); CdcMultiplexRecord actual = runner.take(); assertThat(actual).isEqualTo(expected); - fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "2"); + data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "2"); expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.take(); assertThat(actual).isEqualTo(expected); - // check that records with new fields should be processed after schema is updated + // check that records with new data should be processed after schema is updated - fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "3"); - fields.put("v", "30"); - fields.put("v2", "300"); + data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "3"); + data.put("v", "30"); + data.put("v2", "300"); expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -383,34 +369,30 @@ public void testSingleTableUpdateColumnType() throws Exception { // check that records with compatible schema can be processed immediately - Map fields = new HashMap<>(); - fields.put("k", "1"); - fields.put("v1", "10"); - fields.put("v2", "0.625"); - fields.put("v3", "one"); - fields.put("v4", "b_one"); + Map data = new HashMap<>(); + data.put("k", "1"); + data.put("v1", "10"); + data.put("v2", "0.625"); + data.put("v3", "one"); + data.put("v4", "b_one"); CdcMultiplexRecord expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); CdcMultiplexRecord actual = runner.take(); assertThat(actual).isEqualTo(expected); - // check that records with new fields should be processed after schema is updated + // check that records with new data should be processed after schema is updated // int -> bigint - fields = new HashMap<>(); - fields.put("k", "2"); - fields.put("v1", "12345678987654321"); - fields.put("v2", "0.25"); + data = new HashMap<>(); + data.put("k", "2"); + data.put("v1", "12345678987654321"); + data.put("v2", "0.25"); expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -422,15 +404,13 @@ public void testSingleTableUpdateColumnType() throws Exception { // float -> double - fields = new HashMap<>(); - fields.put("k", "3"); - fields.put("v1", "100"); - fields.put("v2", "1.0000000000009095"); + data = new HashMap<>(); + data.put("k", "3"); + data.put("v1", "100"); + data.put("v2", "1.0000000000009095"); expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -441,15 +421,13 @@ public void testSingleTableUpdateColumnType() throws Exception { // varchar(5) -> varchar(10) - fields = new HashMap<>(); - fields.put("k", "4"); - fields.put("v1", "40"); - fields.put("v3", "long four"); + data = new HashMap<>(); + data.put("k", "4"); + data.put("v1", "40"); + data.put("v3", "long four"); expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -460,15 +438,13 @@ public void testSingleTableUpdateColumnType() throws Exception { // varbinary(5) -> varbinary(10) - fields = new HashMap<>(); - fields.put("k", "5"); - fields.put("v1", "50"); - fields.put("v4", "long five~"); + data = new HashMap<>(); + data.put("k", "5"); + data.put("v1", "50"); + data.put("v4", "long five~"); expected = CdcMultiplexRecord.fromCdcRecord( - databaseName, - tableId.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + databaseName, tableId.getObjectName(), new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -499,53 +475,53 @@ public void testMultiTableUpdateColumnType() throws Exception { // check that records with compatible schema from different tables // can be processed immediately - Map fields; + Map data; // first table record - fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "1"); - fields.put("v", "10"); + data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "1"); + data.put("v", "10"); CdcMultiplexRecord expected = CdcMultiplexRecord.fromCdcRecord( databaseName, firstTable.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); CdcMultiplexRecord actual = runner.take(); assertThat(actual).isEqualTo(expected); // second table record - fields = new HashMap<>(); - fields.put("k", "1"); - fields.put("v1", "10"); - fields.put("v2", "0.625"); - fields.put("v3", "one"); - fields.put("v4", "b_one"); + data = new HashMap<>(); + data.put("k", "1"); + data.put("v1", "10"); + data.put("v2", "0.625"); + data.put("v3", "one"); + data.put("v4", "b_one"); expected = CdcMultiplexRecord.fromCdcRecord( databaseName, secondTable.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.take(); assertThat(actual).isEqualTo(expected); - // check that records with new fields should be processed after schema is updated + // check that records with new data should be processed after schema is updated // int -> bigint SchemaManager schemaManager; // first table - fields = new HashMap<>(); - fields.put("pt", "1"); - fields.put("k", "123456789876543211"); - fields.put("v", "varchar"); + data = new HashMap<>(); + data.put("pt", "1"); + data.put("k", "123456789876543211"); + data.put("v", "varchar"); expected = CdcMultiplexRecord.fromCdcRecord( databaseName, firstTable.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -556,15 +532,15 @@ public void testMultiTableUpdateColumnType() throws Exception { assertThat(actual).isEqualTo(expected); // second table - fields = new HashMap<>(); - fields.put("k", "2"); - fields.put("v1", "12345678987654321"); - fields.put("v2", "0.25"); + data = new HashMap<>(); + data.put("k", "2"); + data.put("v1", "12345678987654321"); + data.put("v2", "0.25"); expected = CdcMultiplexRecord.fromCdcRecord( databaseName, secondTable.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -577,15 +553,15 @@ public void testMultiTableUpdateColumnType() throws Exception { // below are schema changes only from the second table // float -> double - fields = new HashMap<>(); - fields.put("k", "3"); - fields.put("v1", "100"); - fields.put("v2", "1.0000000000009095"); + data = new HashMap<>(); + data.put("k", "3"); + data.put("v1", "100"); + data.put("v2", "1.0000000000009095"); expected = CdcMultiplexRecord.fromCdcRecord( databaseName, secondTable.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -597,15 +573,15 @@ public void testMultiTableUpdateColumnType() throws Exception { // varchar(5) -> varchar(10) - fields = new HashMap<>(); - fields.put("k", "4"); - fields.put("v1", "40"); - fields.put("v3", "long four"); + data = new HashMap<>(); + data.put("k", "4"); + data.put("v1", "40"); + data.put("v3", "long four"); expected = CdcMultiplexRecord.fromCdcRecord( databaseName, secondTable.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -617,15 +593,15 @@ public void testMultiTableUpdateColumnType() throws Exception { // varbinary(5) -> varbinary(10) - fields = new HashMap<>(); - fields.put("k", "5"); - fields.put("v1", "50"); - fields.put("v4", "long five~"); + data = new HashMap<>(); + data.put("k", "5"); + data.put("v1", "50"); + data.put("v4", "long five~"); expected = CdcMultiplexRecord.fromCdcRecord( databaseName, secondTable.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -651,33 +627,33 @@ public void testUsingTheSameCompactExecutor() throws Exception { t.start(); // write records to two tables thus two FileStoreWrite will be created - Map fields; + Map data; // first table record - fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "1"); - fields.put("v", "10"); + data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "1"); + data.put("v", "10"); CdcMultiplexRecord expected = CdcMultiplexRecord.fromCdcRecord( databaseName, firstTable.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); // second table record - fields = new HashMap<>(); - fields.put("k", "1"); - fields.put("v1", "10"); - fields.put("v2", "0.625"); - fields.put("v3", "one"); - fields.put("v4", "b_one"); + data = new HashMap<>(); + data.put("k", "1"); + data.put("v1", "10"); + data.put("v2", "0.625"); + data.put("v3", "one"); + data.put("v4", "b_one"); expected = CdcMultiplexRecord.fromCdcRecord( databaseName, secondTable.getObjectName(), - new CdcRecord(RowKind.INSERT, fields)); + new CdcRecord(RowKind.INSERT, data)); runner.offer(expected); // get and check compactExecutor from two FileStoreWrite 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 9af7eabdaaad..f3693fe405de 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 @@ -106,31 +106,31 @@ public void testAddColumn() throws Exception { // check that records with compatible schema can be processed immediately - Map fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "1"); - fields.put("v", "10"); - CdcRecord expected = new CdcRecord(RowKind.INSERT, fields); + Map data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "1"); + data.put("v", "10"); + CdcRecord expected = new CdcRecord(RowKind.INSERT, data); runner.offer(expected); CdcRecord actual = runner.take(); assertThat(actual).isEqualTo(expected); - fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "2"); - expected = new CdcRecord(RowKind.INSERT, fields); + data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "2"); + expected = new CdcRecord(RowKind.INSERT, data); runner.offer(expected); actual = runner.take(); assertThat(actual).isEqualTo(expected); - // check that records with new fields should be processed after schema is updated + // check that records with new data should be processed after schema is updated - fields = new HashMap<>(); - fields.put("pt", "0"); - fields.put("k", "3"); - fields.put("v", "30"); - fields.put("v2", "300"); - expected = new CdcRecord(RowKind.INSERT, fields); + data = new HashMap<>(); + data.put("pt", "0"); + data.put("k", "3"); + data.put("v", "30"); + data.put("v2", "300"); + expected = new CdcRecord(RowKind.INSERT, data); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -172,26 +172,26 @@ public void testUpdateColumnType() throws Exception { // check that records with compatible schema can be processed immediately - Map fields = new HashMap<>(); - fields.put("k", "1"); - fields.put("v1", "10"); - fields.put("v2", "0.625"); - fields.put("v3", "one"); - fields.put("v4", "b_one"); - CdcRecord expected = new CdcRecord(RowKind.INSERT, fields); + Map data = new HashMap<>(); + data.put("k", "1"); + data.put("v1", "10"); + data.put("v2", "0.625"); + data.put("v3", "one"); + data.put("v4", "b_one"); + CdcRecord expected = new CdcRecord(RowKind.INSERT, data); runner.offer(expected); CdcRecord actual = runner.take(); assertThat(actual).isEqualTo(expected); - // check that records with new fields should be processed after schema is updated + // check that records with new data should be processed after schema is updated // int -> bigint - fields = new HashMap<>(); - fields.put("k", "2"); - fields.put("v1", "12345678987654321"); - fields.put("v2", "0.25"); - expected = new CdcRecord(RowKind.INSERT, fields); + data = new HashMap<>(); + data.put("k", "2"); + data.put("v1", "12345678987654321"); + data.put("v2", "0.25"); + expected = new CdcRecord(RowKind.INSERT, data); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -203,11 +203,11 @@ public void testUpdateColumnType() throws Exception { // float -> double - fields = new HashMap<>(); - fields.put("k", "3"); - fields.put("v1", "100"); - fields.put("v2", "1.0000000000009095"); - expected = new CdcRecord(RowKind.INSERT, fields); + data = new HashMap<>(); + data.put("k", "3"); + data.put("v1", "100"); + data.put("v2", "1.0000000000009095"); + expected = new CdcRecord(RowKind.INSERT, data); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -218,11 +218,11 @@ public void testUpdateColumnType() throws Exception { // varchar(5) -> varchar(10) - fields = new HashMap<>(); - fields.put("k", "4"); - fields.put("v1", "40"); - fields.put("v3", "long four"); - expected = new CdcRecord(RowKind.INSERT, fields); + data = new HashMap<>(); + data.put("k", "4"); + data.put("v1", "40"); + data.put("v3", "long four"); + expected = new CdcRecord(RowKind.INSERT, data); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); @@ -233,11 +233,11 @@ public void testUpdateColumnType() throws Exception { // varbinary(5) -> varbinary(10) - fields = new HashMap<>(); - fields.put("k", "5"); - fields.put("v1", "50"); - fields.put("v4", "long five~"); - expected = new CdcRecord(RowKind.INSERT, fields); + data = new HashMap<>(); + data.put("k", "5"); + data.put("v1", "50"); + data.put("v4", "long five~"); + expected = new CdcRecord(RowKind.INSERT, data); runner.offer(expected); actual = runner.poll(1); assertThat(actual).isNull(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestTable.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestTable.java index 525a05096942..6a38c1c2659d 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestTable.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestTable.java @@ -114,18 +114,18 @@ public TestTable( } events.add(new TestCdcEvent(tableName, currentDataFieldList(fieldNames, isBigInt))); } else { - Map fields = new HashMap<>(); + Map data = new HashMap<>(); int key = random.nextInt(numKeys); - fields.put("k", String.valueOf(key)); + data.put("k", String.valueOf(key)); int pt = key % numPartitions; - fields.put("pt", String.valueOf(pt)); + data.put("pt", String.valueOf(pt)); for (int j = 0; j < fieldNames.size(); j++) { String fieldName = fieldNames.get(j); if (isBigInt.get(j)) { - fields.put(fieldName, String.valueOf(random.nextLong())); + data.put(fieldName, String.valueOf(random.nextLong())); } else { - fields.put(fieldName, String.valueOf(random.nextInt())); + data.put(fieldName, String.valueOf(random.nextInt())); } } @@ -140,8 +140,8 @@ public TestTable( shouldInsert = random.nextInt(5) > 0; } if (shouldInsert) { - records.add(new CdcRecord(RowKind.INSERT, fields)); - expected.put(key, fields); + records.add(new CdcRecord(RowKind.INSERT, data)); + expected.put(key, data); } } // Generate test data for append table @@ -149,8 +149,8 @@ public TestTable( if (expected.containsKey(key)) { records.add(new CdcRecord(RowKind.DELETE, expected.get(key))); } else { - records.add(new CdcRecord(RowKind.INSERT, fields)); - expected.put(key, fields); + records.add(new CdcRecord(RowKind.INSERT, data)); + expected.put(key, data); } } events.add(new TestCdcEvent(tableName, records, Objects.hash(tableName, key))); diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/include/topic0/aws-dms-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/include/topic0/aws-dms-data-1.txt new file mode 100644 index 000000000000..d779b9fb4ad5 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/include/topic0/aws-dms-data-1.txt @@ -0,0 +1,22 @@ +/* + * 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. + */ + +{"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database_affix","table-name":"paimon_1","transaction-id":670014899490}} +{"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database_affix","table-name":"paimon_2","transaction-id":670014899490}} +{"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database_affix","table-name":"ignore","transaction-id":670014899490}} +{"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database_affix","table-name":"flink","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic0/aws-dms-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic0/aws-dms-data-1.txt new file mode 100644 index 000000000000..5ac7c5dbecef --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic0/aws-dms-data-1.txt @@ -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. + */ + +{"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t1","transaction-id":670014899490}} +{"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t1","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic0/aws-dms-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic0/aws-dms-data-2.txt new file mode 100644 index 000000000000..56e1b53c1017 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic0/aws-dms-data-2.txt @@ -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. + */ + +{"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14,"address":"Beijing"},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t1","transaction-id":670014899490}} +{"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1,"address":"Shanghai"},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t1","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic1/aws-dms-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic1/aws-dms-data-1.txt new file mode 100644 index 000000000000..a0351adb7fd6 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic1/aws-dms-data-1.txt @@ -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. + */ + +{"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t2","transaction-id":670014899490}} +{"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t2","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic1/aws-dms-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic1/aws-dms-data-2.txt new file mode 100644 index 000000000000..e59ef1c9a479 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/prefixsuffix/topic1/aws-dms-data-2.txt @@ -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. + */ + +{"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8,"age":19},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t2","transaction-id":670014899490}} +{"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"age":25},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t2","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic0/aws-dms-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic0/aws-dms-data-1.txt new file mode 100644 index 000000000000..5ac7c5dbecef --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic0/aws-dms-data-1.txt @@ -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. + */ + +{"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t1","transaction-id":670014899490}} +{"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t1","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic0/aws-dms-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic0/aws-dms-data-2.txt new file mode 100644 index 000000000000..eeb254d71c4e --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic0/aws-dms-data-2.txt @@ -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. + */ + +{"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8,"age":19},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t1","transaction-id":670014899490}} +{"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"age":25},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t1","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic1/aws-dms-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic1/aws-dms-data-1.txt new file mode 100644 index 000000000000..a0351adb7fd6 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic1/aws-dms-data-1.txt @@ -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. + */ + +{"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t2","transaction-id":670014899490}} +{"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t2","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic1/aws-dms-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic1/aws-dms-data-2.txt new file mode 100644 index 000000000000..a189a9d85df7 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/database/schemaevolution/topic1/aws-dms-data-2.txt @@ -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. + */ + +{"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8,"address":"Beijing"},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t2","transaction-id":670014899490}} +{"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"address":"Shanghai"},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"paimon_sync_database","table-name":"t2","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/util/SQLConfUtils.scala b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/computedcolumn/aws-dms-data-1.txt similarity index 76% rename from paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/util/SQLConfUtils.scala rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/computedcolumn/aws-dms-data-1.txt index ca402d794c58..cf9112abc4bf 100644 --- a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/util/SQLConfUtils.scala +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/computedcolumn/aws-dms-data-1.txt @@ -16,13 +16,4 @@ * limitations under the License. */ -package org.apache.paimon.spark.util - -import org.apache.paimon.catalog.Catalog - -import org.apache.spark.sql.internal.SQLConf - -/** SQLConf utils. */ -object SQLConfUtils { - def defaultDatabase(sqlConf: SQLConf): String = Catalog.DEFAULT_DATABASE -} +{"data":{"_id":101,"_date":"2023-03-23"},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-1.txt new file mode 100644 index 000000000000..42a00afe5b5e --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-1.txt @@ -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. + */ + +{"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} +{"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-2.txt new file mode 100644 index 000000000000..3ecfdab8b1a4 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-2.txt @@ -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. + */ + +{"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8,"age":18},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} +{"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75,"age":24},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-3.txt new file mode 100644 index 000000000000..04e18e1db548 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-3.txt @@ -0,0 +1,22 @@ +/* + * 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. + */ + +{"data":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875,"address":"Shanghai"},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} +{"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"delete","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} +{"data":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875,"address":"Beijing","BI_id":105,"BI_name":"hammer","BI_description":"14oz carpenter's hammer","BI_weight":0.875,"BI_address":"Shanghai"},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"update","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} +{"data":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.3},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-4.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-4.txt new file mode 100644 index 000000000000..e93607aed68d --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/schemaevolution/aws-dms-data-4.txt @@ -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. + */ + +{"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":null},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} +{"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/startupmode/aws-dms-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/startupmode/aws-dms-data-1.txt new file mode 100644 index 000000000000..42a00afe5b5e --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/startupmode/aws-dms-data-1.txt @@ -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. + */ + +{"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} +{"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/startupmode/aws-dms-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/startupmode/aws-dms-data-2.txt new file mode 100644 index 000000000000..70c0fb1675ea --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/startupmode/aws-dms-data-2.txt @@ -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. + */ + +{"data":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.8},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} +{"data":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/watermark/aws-dms-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/watermark/aws-dms-data-1.txt new file mode 100644 index 000000000000..42a00afe5b5e --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aws-dms/table/watermark/aws-dms-data-1.txt @@ -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. + */ + +{"data":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.14},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} +{"data":{"id":102,"name":"car battery","description":"12V car battery","weight":8.1},"metadata":{"timestamp":"2024-10-27T12:39:03.210671Z","record-type":"data","operation":"insert","partition-key-type":"schema-table","schema-name":"test","table-name":"product","transaction-id":670014899490}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-1.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-1.txt index 3b34b6f8f581..7dd068d00568 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-1.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-1.txt @@ -16,5 +16,4 @@ * limitations under the License. */ -{"schema":null, "payload":{"before": null, "after": {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} -{"schema":null, "payload":{"before": null, "after": {"id": 102, "name": "car battery", "description": "12V car battery", "weight": 8.1}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"}, "payload":{"before": null, "after": {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-2.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-2.txt index 3b0571d72686..0eec06df3c37 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-2.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-2.txt @@ -16,5 +16,4 @@ * limitations under the License. */ -{"schema":null, "payload":{"before": null, "after": {"id": 103, "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": 0.8, "age": 18}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} -{"schema":null, "payload":{"before": null, "after": {"id": 104, "name": "hammer", "description": "12oz carpenter's hammer", "weight": 0.75, "age": 24}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"},{"type":"int32","optional":true,"field":"age"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"}, "payload":{"before": null, "after": {"id": 103, "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": 0.8, "age": 18}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-3.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-3.txt index ea99eb71f33c..754015d7d066 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-3.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-3.txt @@ -16,7 +16,4 @@ * limitations under the License. */ -{"schema":null, "payload":{"before": null, "after": {"id": 105, "name": "hammer", "description": "14oz carpenter's hammer", "weight": 0.875, "address": "Shanghai"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} -{"schema":null, "payload":{"before": {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14}, "after": null, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "d", "ts_ms": 1596684883000, "transaction": null}} -{"schema":null, "payload":{"before": {"address": "Shanghai"}, "after": {"id": 105, "name": "hammer", "description": "14oz carpenter's hammer", "weight": 0.875, "address": "Beijing"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684906000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "u", "ts_ms": 1596684906000, "transaction": null}} -{"schema":null, "payload":{"before": null, "after": {"id": 107, "name": "rocks", "description": "box of assorted rocks", "weight": 5.3}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"},{"type":"int64","optional":true,"field":"age"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"}, "payload":{"before": null, "after": {"id": 104, "name": "hammer", "description": "12oz carpenter's hammer", "weight": 0.75, "age": 24}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-4.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-4.txt index 911c9ce0a3a2..154f41dd9c45 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-4.txt +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-4.txt @@ -16,4 +16,4 @@ * limitations under the License. */ -{"schema":null, "payload":{"before": {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14}, "after": null, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "d", "ts_ms": 1596684883000, "transaction": null}} \ No newline at end of file +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"},{"type":"int32","optional":true,"field":"age"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"}, "payload":{"before": null, "after": {"id": 105, "name": "hammer", "description": "14oz carpenter's hammer", "weight": 0.875, "age": 24}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-5.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-5.txt new file mode 100644 index 000000000000..fde64c550f41 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-5.txt @@ -0,0 +1,19 @@ +/* + * 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. + */ + +{"schema":{"type":"struct","fields":[{"type":"struct","fields":[{"type":"int32","optional":false,"field":"id"},{"type":"string","optional":false,"field":"name"},{"type":"string","optional":true,"field":"description"},{"type":"double","optional":true,"field":"weight"},{"type":"string","optional":true,"field":"age"}],"optional":true,"name":"dbserver1.inventory.products.Value","field":"after"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"version"},{"type":"string","optional":false,"field":"connector"},{"type":"string","optional":false,"field":"name"},{"type":"int64","optional":false,"field":"ts_ms"},{"type":"string","optional":true,"name":"io.debezium.data.Enum","version":1,"parameters":{"allowed":"true,last,false"},"default":"false","field":"snapshot"},{"type":"string","optional":false,"field":"db"},{"type":"string","optional":true,"field":"table"},{"type":"int64","optional":false,"field":"server_id"},{"type":"string","optional":true,"field":"gtid"},{"type":"string","optional":false,"field":"file"},{"type":"int64","optional":false,"field":"pos"},{"type":"int32","optional":false,"field":"row"},{"type":"int64","optional":true,"field":"thread"},{"type":"string","optional":true,"field":"query"}],"optional":false,"name":"io.debezium.connector.mysql.Source","field":"source"},{"type":"string","optional":false,"field":"op"},{"type":"int64","optional":true,"field":"ts_ms"},{"type":"struct","fields":[{"type":"string","optional":false,"field":"id"},{"type":"int64","optional":false,"field":"total_order"},{"type":"int64","optional":false,"field":"data_collection_order"}],"optional":true,"field":"transaction"}],"optional":false,"name":"dbserver1.inventory.products.Envelope"}, "payload":{"before": null, "after": {"id": 105, "name": "hammer", "description": "12oz carpenter's hammer", "weight": 0.75, "age": "24"}, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "c", "ts_ms": 1596684883000, "transaction": null}} \ No newline at end of file diff --git a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonIncompatiblePHRRules.scala b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-with-delete.txt similarity index 62% rename from paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonIncompatiblePHRRules.scala rename to paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-with-delete.txt index 7a1edf2f8c7e..911c9ce0a3a2 100644 --- a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonIncompatiblePHRRules.scala +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/debezium/table/schema/schemaevolution/debezium-data-with-delete.txt @@ -16,15 +16,4 @@ * limitations under the License. */ -package org.apache.paimon.spark.catalyst.analysis - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.rules.Rule - -/** These post-hoc resolution rules are incompatible between different versions of spark. */ -case class PaimonIncompatiblePHRRules(session: SparkSession) extends Rule[LogicalPlan] { - - override def apply(plan: LogicalPlan): LogicalPlan = plan - -} +{"schema":null, "payload":{"before": {"id": 101, "name": "scooter", "description": "Small 2-wheel scooter", "weight": 3.14}, "after": null, "source": {"version": "1.9.7.Final", "connector": "mysql", "name": "mysql_binlog_source", "ts_ms": 1596684883000, "snapshot": "false", "db": "test", "sequence": null, "table": "product", "server_id": 0, "gtid": null, "file": "", "pos": 0, "row": 0, "thread": null, "query": null}, "op": "d", "ts_ms": 1596684883000, "transaction": null}} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_table_setup.sql b/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_table_setup.sql index 965f884ec680..10a0f20d45aa 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_table_setup.sql +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_table_setup.sql @@ -445,3 +445,14 @@ CREATE TABLE t ( k INT PRIMARY KEY, v1 VARCHAR(10) ); + +-- ################################################################################ +-- testRuntimeExecutionModeCheckForCdcSync +-- ################################################################################ + +CREATE DATABASE check_cdc_sync_runtime_execution_mode; +USE check_cdc_sync_runtime_execution_mode; +CREATE TABLE t ( + k INT PRIMARY KEY, + v1 VARCHAR(10) +); \ No newline at end of file diff --git a/paimon-flink/paimon-flink-common/pom.xml b/paimon-flink/paimon-flink-common/pom.xml index e086d74786ba..4452af266e5e 100644 --- a/paimon-flink/paimon-flink-common/pom.xml +++ b/paimon-flink/paimon-flink-common/pom.xml @@ -59,6 +59,28 @@ under the License. provided + + org.apache.flink + flink-clients + ${flink.version} + test + + + + org.apache.flink + flink-sql-gateway + ${flink.version} + test + + + + org.apache.flink + flink-sql-gateway + ${flink.version} + test-jar + test + + @@ -100,7 +122,7 @@ under the License. org.apache.flink - flink-table-planner_${scala.binary.version} + flink-table-planner_${flink.scala.binary.version} ${flink.version} test test-jar @@ -108,7 +130,7 @@ under the License. org.apache.flink - flink-table-planner_${scala.binary.version} + flink-table-planner_${flink.scala.binary.version} ${flink.version} test @@ -128,6 +150,19 @@ under the License. test + + org.apache.iceberg + iceberg-core + ${iceberg.version} + test + + + + org.apache.iceberg + iceberg-data + ${iceberg.version} + test + diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java index 86a22dc07c9e..9f90a2cd0130 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java @@ -22,20 +22,20 @@ import org.apache.paimon.CoreOptions.LogConsistency; import org.apache.paimon.CoreOptions.StreamingReadMode; import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.Timestamp; import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.flink.sink.FlinkTableSink; import org.apache.paimon.flink.source.DataTableSource; import org.apache.paimon.flink.source.SystemTableSource; -import org.apache.paimon.flink.source.table.PushedRichTableSource; -import org.apache.paimon.flink.source.table.PushedTableSource; -import org.apache.paimon.flink.source.table.RichTableSource; import org.apache.paimon.lineage.LineageMeta; import org.apache.paimon.lineage.LineageMetaFactory; import org.apache.paimon.lineage.TableLineageEntity; import org.apache.paimon.lineage.TableLineageEntityImpl; import org.apache.paimon.options.Options; +import org.apache.paimon.options.OptionsUtils; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.table.FileStoreTable; @@ -72,7 +72,6 @@ import java.util.Optional; import java.util.Set; import java.util.function.BiConsumer; -import java.util.regex.Matcher; import java.util.regex.Pattern; import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; @@ -83,7 +82,6 @@ import static org.apache.paimon.CoreOptions.StartupMode.FROM_SNAPSHOT_FULL; import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM; import static org.apache.paimon.flink.FlinkConnectorOptions.NONE; -import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_PUSH_DOWN; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; import static org.apache.paimon.flink.log.LogStoreTableFactory.discoverLogStoreFactory; @@ -93,6 +91,12 @@ public abstract class AbstractFlinkTableFactory private static final Logger LOG = LoggerFactory.getLogger(AbstractFlinkTableFactory.class); + @Nullable private final FlinkCatalog flinkCatalog; + + public AbstractFlinkTableFactory(@Nullable FlinkCatalog flinkCatalog) { + this.flinkCatalog = flinkCatalog; + } + @Override public DynamicTableSource createDynamicTableSource(Context context) { CatalogTable origin = context.getCatalogTable().getOrigin(); @@ -100,11 +104,10 @@ public DynamicTableSource createDynamicTableSource(Context context) { context.getConfiguration().get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; if (origin instanceof SystemCatalogTable) { - return new PushedTableSource( - new SystemTableSource( - ((SystemCatalogTable) origin).table(), - isStreamingMode, - context.getObjectIdentifier())); + return new SystemTableSource( + ((SystemCatalogTable) origin).table(), + isStreamingMode, + context.getObjectIdentifier()); } else { Table table = buildPaimonTable(context); if (table instanceof FileStoreTable) { @@ -120,16 +123,12 @@ public DynamicTableSource createDynamicTableSource(Context context) { } }); } - DataTableSource source = - new DataTableSource( - context.getObjectIdentifier(), - table, - isStreamingMode, - context, - createOptionalLogStoreFactory(context).orElse(null)); - return new Options(table.options()).get(SCAN_PUSH_DOWN) - ? new PushedRichTableSource(source) - : new RichTableSource(source); + return new DataTableSource( + context.getObjectIdentifier(), + table, + isStreamingMode, + context, + createOptionalLogStoreFactory(context).orElse(null)); } } @@ -236,11 +235,11 @@ static CatalogContext createCatalogContext(DynamicTableFactory.Context context) Options.fromMap(context.getCatalogTable().getOptions()), new FlinkFileIOLoader()); } - static Table buildPaimonTable(DynamicTableFactory.Context context) { + Table buildPaimonTable(DynamicTableFactory.Context context) { CatalogTable origin = context.getCatalogTable().getOrigin(); Table table; - Map dynamicOptions = getDynamicTableConfigOptions(context); + Map dynamicOptions = getDynamicConfigOptions(context); dynamicOptions.forEach( (key, newValue) -> { String oldValue = origin.getOptions().get(key); @@ -250,18 +249,31 @@ static Table buildPaimonTable(DynamicTableFactory.Context context) { }); Map newOptions = new HashMap<>(); newOptions.putAll(origin.getOptions()); + // dynamic options should override origin options newOptions.putAll(dynamicOptions); - // notice that the Paimon table schema must be the same with the Flink's + FileStoreTable fileStoreTable; if (origin instanceof DataCatalogTable) { - FileStoreTable fileStoreTable = (FileStoreTable) ((DataCatalogTable) origin).table(); - table = fileStoreTable.copyWithoutTimeTravel(newOptions); + fileStoreTable = (FileStoreTable) ((DataCatalogTable) origin).table(); + } else if (flinkCatalog == null) { + // In case Paimon is directly used as a Flink connector, instead of through catalog. + fileStoreTable = FileStoreTableFactory.create(createCatalogContext(context)); } else { - table = - FileStoreTableFactory.create(createCatalogContext(context)) - .copyWithoutTimeTravel(newOptions); + // In cases like materialized table, the Paimon table might not be DataCatalogTable, + // but can still be acquired through the catalog. + Identifier identifier = + Identifier.create( + context.getObjectIdentifier().getDatabaseName(), + context.getObjectIdentifier().getObjectName()); + try { + fileStoreTable = (FileStoreTable) flinkCatalog.catalog().getTable(identifier); + } catch (Catalog.TableNotExistException e) { + throw new RuntimeException(e); + } } + table = fileStoreTable.copyWithoutTimeTravel(newOptions); + // notice that the Paimon table schema must be the same with the Flink's Schema schema = FlinkCatalog.fromCatalogTable(context.getCatalogTable()); RowType rowType = toLogicalType(schema.rowType()); @@ -313,16 +325,19 @@ static boolean schemaEquals(RowType rowType1, RowType rowType2) { /** * The dynamic option's format is: * - *

    {@link - * FlinkConnectorOptions#TABLE_DYNAMIC_OPTION_PREFIX}.${catalog}.${database}.${tableName}.key = - * value. These job level configs will be extracted and injected into the target table option. + *

    Global Options: key = value . + * + *

    Table Options: {@link + * FlinkConnectorOptions#TABLE_DYNAMIC_OPTION_PREFIX}${catalog}.${database}.${tableName}.key = + * value. + * + *

    These job level options will be extracted and injected into the target table option. Table + * options will override global options if there are conflicts. * * @param context The table factory context. * @return The dynamic options of this target table. */ - static Map getDynamicTableConfigOptions(DynamicTableFactory.Context context) { - - Map optionsFromTableConfig = new HashMap<>(); + static Map getDynamicConfigOptions(DynamicTableFactory.Context context) { ReadableConfig config = context.getConfiguration(); @@ -338,23 +353,14 @@ static Map getDynamicTableConfigOptions(DynamicTableFactory.Cont String template = String.format( - "(%s)\\.(%s|\\*)\\.(%s|\\*)\\.(%s|\\*)\\.(.+)", + "(%s)(%s|\\*)\\.(%s|\\*)\\.(%s|\\*)\\.(.+)", FlinkConnectorOptions.TABLE_DYNAMIC_OPTION_PREFIX, context.getObjectIdentifier().getCatalogName(), context.getObjectIdentifier().getDatabaseName(), context.getObjectIdentifier().getObjectName()); Pattern pattern = Pattern.compile(template); - - conf.keySet() - .forEach( - (key) -> { - if (key.startsWith(FlinkConnectorOptions.TABLE_DYNAMIC_OPTION_PREFIX)) { - Matcher matcher = pattern.matcher(key); - if (matcher.find()) { - optionsFromTableConfig.put(matcher.group(5), conf.get(key)); - } - } - }); + Map optionsFromTableConfig = + OptionsUtils.convertToDynamicTableProperties(conf, "", pattern, 5); if (!optionsFromTableConfig.isEmpty()) { LOG.info( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index 4ccf11f4ecf6..cae6e6f0e367 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -19,24 +19,30 @@ package org.apache.paimon.flink; import org.apache.paimon.CoreOptions; +import org.apache.paimon.TableType; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.procedure.ProcedureUtil; import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.operation.FileStoreCommit; import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.stats.Statistics; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FormatTable; import org.apache.paimon.table.Table; +import org.apache.paimon.table.sink.BatchWriteBuilder; import org.apache.paimon.table.source.ReadBuilder; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.InternalRowPartitionComputer; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.StringUtils; +import org.apache.paimon.view.View; +import org.apache.paimon.view.ViewImpl; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.catalog.AbstractCatalog; @@ -44,15 +50,17 @@ import org.apache.flink.table.catalog.CatalogDatabase; import org.apache.flink.table.catalog.CatalogDatabaseImpl; import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogMaterializedTable; import org.apache.flink.table.catalog.CatalogPartition; import org.apache.flink.table.catalog.CatalogPartitionImpl; import org.apache.flink.table.catalog.CatalogPartitionSpec; import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.CatalogView; import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.IntervalFreshness; import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.ResolvedCatalogBaseTable; -import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedCatalogView; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.TableChange; import org.apache.flink.table.catalog.TableChange.AddColumn; @@ -62,10 +70,13 @@ import org.apache.flink.table.catalog.TableChange.DropColumn; import org.apache.flink.table.catalog.TableChange.DropWatermark; import org.apache.flink.table.catalog.TableChange.First; +import org.apache.flink.table.catalog.TableChange.MaterializedTableChange; import org.apache.flink.table.catalog.TableChange.ModifyColumnComment; import org.apache.flink.table.catalog.TableChange.ModifyColumnName; import org.apache.flink.table.catalog.TableChange.ModifyColumnPosition; import org.apache.flink.table.catalog.TableChange.ModifyPhysicalColumnType; +import org.apache.flink.table.catalog.TableChange.ModifyRefreshHandler; +import org.apache.flink.table.catalog.TableChange.ModifyRefreshStatus; import org.apache.flink.table.catalog.TableChange.ModifyWatermark; import org.apache.flink.table.catalog.TableChange.ResetOption; import org.apache.flink.table.catalog.TableChange.SetOption; @@ -75,6 +86,7 @@ import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; import org.apache.flink.table.catalog.exceptions.ProcedureNotExistException; import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; @@ -95,6 +107,7 @@ import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; @@ -102,6 +115,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.function.Function; import java.util.stream.Collectors; import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; @@ -113,7 +127,21 @@ import static org.apache.flink.table.descriptors.Schema.SCHEMA; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; +import static org.apache.flink.table.utils.EncodingUtils.decodeBase64ToBytes; +import static org.apache.flink.table.utils.EncodingUtils.encodeBytesToBase64; +import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_DEFINITION_QUERY; +import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_INTERVAL_FRESHNESS; +import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_INTERVAL_FRESHNESS_TIME_UNIT; +import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_LOGICAL_REFRESH_MODE; +import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_HANDLER_BYTES; +import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_HANDLER_DESCRIPTION; +import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_MODE; +import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_STATUS; import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.catalog.Catalog.LAST_UPDATE_TIME_PROP; +import static org.apache.paimon.catalog.Catalog.NUM_FILES_PROP; +import static org.apache.paimon.catalog.Catalog.NUM_ROWS_PROP; +import static org.apache.paimon.catalog.Catalog.TOTAL_SIZE_PROP; import static org.apache.paimon.flink.FlinkCatalogOptions.DISABLE_CREATE_TABLE_IN_DEFAULT_DB; import static org.apache.paimon.flink.FlinkCatalogOptions.LOG_SYSTEM_AUTO_REGISTER; import static org.apache.paimon.flink.FlinkCatalogOptions.REGISTER_TIMEOUT; @@ -126,6 +154,8 @@ import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeWatermarkSpec; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.nonPhysicalColumnsCount; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.serializeNewWatermarkSpec; +import static org.apache.paimon.flink.utils.TableStatsUtil.createTableColumnStats; +import static org.apache.paimon.flink.utils.TableStatsUtil.createTableStats; import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.paimon.utils.Preconditions.checkNotNull; @@ -133,12 +163,8 @@ public class FlinkCatalog extends AbstractCatalog { private static final Logger LOG = LoggerFactory.getLogger(FlinkCatalog.class); - public static final String NUM_ROWS_KEY = "numRows"; - public static final String LAST_UPDATE_TIME_KEY = "lastUpdateTime"; - public static final String TOTAL_SIZE_KEY = "totalSize"; - public static final String NUM_FILES_KEY = "numFiles"; - private final ClassLoader classLoader; + private final ClassLoader classLoader; private final Catalog catalog; private final String name; private final boolean logStoreAutoRegister; @@ -161,7 +187,9 @@ public FlinkCatalog( this.logStoreAutoRegisterTimeout = options.get(REGISTER_TIMEOUT); this.disableCreateTableInDefaultDatabase = options.get(DISABLE_CREATE_TABLE_IN_DEFAULT_DB); if (!disableCreateTableInDefaultDatabase) { - if (!catalog.databaseExists(defaultDatabase)) { + try { + getDatabase(defaultDatabase); + } catch (DatabaseNotExistException e) { try { catalog.createDatabase(defaultDatabase, true); } catch (Catalog.DatabaseAlreadyExistException ignore) { @@ -176,7 +204,7 @@ public Catalog catalog() { @Override public Optional getFactory() { - return Optional.of(new FlinkTableFactory()); + return Optional.of(new FlinkTableFactory(this)); } @Override @@ -186,7 +214,12 @@ public List listDatabases() throws CatalogException { @Override public boolean databaseExists(String databaseName) throws CatalogException { - return catalog.databaseExists(databaseName); + try { + catalog.getDatabase(databaseName); + return true; + } catch (Catalog.DatabaseNotExistException e) { + return false; + } } @Override @@ -242,7 +275,7 @@ public List listTables(String databaseName) } @Override - public CatalogTable getTable(ObjectPath tablePath) + public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException { return getTable(tablePath, null); } @@ -250,17 +283,22 @@ public CatalogTable getTable(ObjectPath tablePath) /** * Do not annotate with @override here to maintain compatibility with Flink 1.17-. */ - public CatalogTable getTable(ObjectPath tablePath, long timestamp) + public CatalogBaseTable getTable(ObjectPath tablePath, long timestamp) throws TableNotExistException, CatalogException { return getTable(tablePath, Long.valueOf(timestamp)); } - private CatalogTable getTable(ObjectPath tablePath, @Nullable Long timestamp) + private CatalogBaseTable getTable(ObjectPath tablePath, @Nullable Long timestamp) throws TableNotExistException { Table table; try { table = catalog.getTable(toIdentifier(tablePath)); } catch (Catalog.TableNotExistException e) { + Optional view = getView(tablePath, timestamp); + if (view.isPresent()) { + return view.get(); + } + throw new TableNotExistException(getName(), tablePath); } @@ -286,19 +324,70 @@ private CatalogTable getTable(ObjectPath tablePath, @Nullable Long timestamp) } } + private Optional getView(ObjectPath tablePath, @Nullable Long timestamp) { + View view; + try { + view = catalog.getView(toIdentifier(tablePath)); + } catch (Catalog.ViewNotExistException e) { + return Optional.empty(); + } + + if (timestamp != null) { + throw new UnsupportedOperationException( + String.format("View %s does not support time travel.", tablePath)); + } + + org.apache.flink.table.api.Schema schema = + org.apache.flink.table.api.Schema.newBuilder() + .fromRowDataType(fromLogicalToDataType(toLogicalType(view.rowType()))) + .build(); + return Optional.of( + CatalogView.of( + schema, + view.comment().orElse(null), + view.query(), + view.query(), + view.options())); + } + @Override public boolean tableExists(ObjectPath tablePath) throws CatalogException { - return catalog.tableExists(toIdentifier(tablePath)); + Identifier identifier = toIdentifier(tablePath); + try { + catalog.getTable(identifier); + return true; + } catch (Catalog.TableNotExistException e) { + try { + catalog.getView(identifier); + return true; + } catch (Catalog.ViewNotExistException ex) { + return false; + } + } } @Override public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) throws TableNotExistException, CatalogException { Identifier identifier = toIdentifier(tablePath); - Table table = null; try { - if (logStoreAutoRegister && catalog.tableExists(identifier)) { - table = catalog.getTable(identifier); + catalog.getView(identifier); + try { + catalog.dropView(identifier, ignoreIfNotExists); + return; + } catch (Catalog.ViewNotExistException e) { + throw new RuntimeException("Unexpected exception.", e); + } + } catch (Catalog.ViewNotExistException ignored) { + } + + try { + Table table = null; + if (logStoreAutoRegister) { + try { + table = catalog.getTable(identifier); + } catch (Catalog.TableNotExistException ignored) { + } } catalog.dropTable(toIdentifier(tablePath), ignoreIfNotExists); if (logStoreAutoRegister && table != null) { @@ -312,22 +401,25 @@ public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) @Override public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { - if (!(table instanceof CatalogTable)) { - throw new UnsupportedOperationException( - "Only support CatalogTable, but is: " + table.getClass()); - } - if (Objects.equals(getDefaultDatabase(), tablePath.getDatabaseName()) && disableCreateTableInDefaultDatabase) { throw new UnsupportedOperationException( "Creating table in default database is disabled, please specify a database name."); } + if (table instanceof CatalogView) { + createView(tablePath, (ResolvedCatalogView) table, ignoreIfExists); + return; + } + Identifier identifier = toIdentifier(tablePath); // the returned value of "table.getOptions" may be unmodifiable (for example from // TableDescriptor) Map options = new HashMap<>(table.getOptions()); - Schema paimonSchema = buildPaimonSchema(identifier, (CatalogTable) table, options); + if (table instanceof CatalogMaterializedTable) { + fillOptionsForMaterializedTable((CatalogMaterializedTable) table, options); + } + Schema paimonSchema = buildPaimonSchema(identifier, table, options); boolean unRegisterLogSystem = false; try { @@ -345,8 +437,71 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ig } } + private void createView(ObjectPath tablePath, ResolvedCatalogView table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException { + Identifier identifier = toIdentifier(tablePath); + org.apache.paimon.types.RowType.Builder builder = org.apache.paimon.types.RowType.builder(); + table.getResolvedSchema() + .getColumns() + .forEach( + column -> + builder.field( + column.getName(), + toDataType(column.getDataType().getLogicalType()), + column.getComment().orElse(null))); + View view = + new ViewImpl( + identifier, + builder.build(), + table.getOriginalQuery(), + table.getComment(), + table.getOptions()); + try { + catalog.createView(identifier, view, ignoreIfExists); + } catch (Catalog.ViewAlreadyExistException e) { + throw new TableAlreadyExistException(getName(), tablePath); + } catch (Catalog.DatabaseNotExistException e) { + throw new DatabaseNotExistException(getName(), tablePath.getDatabaseName()); + } + } + + private static void fillOptionsForMaterializedTable( + CatalogMaterializedTable mt, Map options) { + Options mtOptions = new Options(); + mtOptions.set(CoreOptions.TYPE, TableType.MATERIALIZED_TABLE); + mtOptions.set(MATERIALIZED_TABLE_DEFINITION_QUERY, mt.getDefinitionQuery()); + mtOptions.set( + MATERIALIZED_TABLE_INTERVAL_FRESHNESS, mt.getDefinitionFreshness().getInterval()); + mtOptions.set( + MATERIALIZED_TABLE_INTERVAL_FRESHNESS_TIME_UNIT, + CoreOptions.MaterializedTableIntervalFreshnessTimeUnit.valueOf( + mt.getDefinitionFreshness().getTimeUnit().name())); + mtOptions.set( + MATERIALIZED_TABLE_LOGICAL_REFRESH_MODE, + CoreOptions.MaterializedTableRefreshMode.valueOf( + mt.getLogicalRefreshMode().name())); + mtOptions.set( + MATERIALIZED_TABLE_REFRESH_MODE, + CoreOptions.MaterializedTableRefreshMode.valueOf(mt.getRefreshMode().name())); + mtOptions.set( + MATERIALIZED_TABLE_REFRESH_STATUS, + CoreOptions.MaterializedTableRefreshStatus.valueOf(mt.getRefreshStatus().name())); + mt.getRefreshHandlerDescription() + .ifPresent( + desc -> + mtOptions.set( + MATERIALIZED_TABLE_REFRESH_HANDLER_DESCRIPTION, desc)); + byte[] serializedRefreshHandler = mt.getSerializedRefreshHandler(); + if (serializedRefreshHandler != null) { + mtOptions.set( + MATERIALIZED_TABLE_REFRESH_HANDLER_BYTES, + encodeBytesToBase64(serializedRefreshHandler)); + } + options.putAll(mtOptions.toMap()); + } + protected Schema buildPaimonSchema( - Identifier identifier, CatalogTable catalogTable, Map options) { + Identifier identifier, CatalogBaseTable catalogTable, Map options) { String connector = options.get(CONNECTOR.key()); options.remove(CONNECTOR.key()); if (!StringUtils.isNullOrWhitespaceOnly(connector) @@ -382,7 +537,10 @@ protected Schema buildPaimonSchema( } } - return fromCatalogTable(catalogTable.copy(options)); + if (catalogTable instanceof CatalogTable) { + return fromCatalogTable(((CatalogTable) catalogTable).copy(options)); + } + return fromCatalogTable(((CatalogMaterializedTable) catalogTable).copy(options)); } private List toSchemaChange( @@ -505,10 +663,45 @@ private List toSchemaChange( throw new UnsupportedOperationException( "Change is not supported: " + change.getClass()); } + } else if (change instanceof MaterializedTableChange + && handleMaterializedTableChange(change, schemaChanges)) { + return schemaChanges; } throw new UnsupportedOperationException("Change is not supported: " + change.getClass()); } + /** + * Try handle change related to materialized table. + * + * @return true, if change can be identified as {@link MaterializedTableChange} and is handled + * properly. Otherwise, false. + */ + protected boolean handleMaterializedTableChange( + TableChange change, List schemaChanges) { + if (change instanceof ModifyRefreshStatus) { + ModifyRefreshStatus modifyRefreshStatus = (ModifyRefreshStatus) change; + CatalogMaterializedTable.RefreshStatus newRefreshStatus = + modifyRefreshStatus.getRefreshStatus(); + schemaChanges.add( + SchemaChange.setOption( + MATERIALIZED_TABLE_REFRESH_STATUS.key(), newRefreshStatus.name())); + return true; + } else if (change instanceof ModifyRefreshHandler) { + ModifyRefreshHandler modifyRefreshHandler = (ModifyRefreshHandler) change; + String newHandlerDesc = modifyRefreshHandler.getRefreshHandlerDesc(); + byte[] newHandlerBytes = modifyRefreshHandler.getRefreshHandlerBytes(); + schemaChanges.add( + SchemaChange.setOption( + MATERIALIZED_TABLE_REFRESH_HANDLER_DESCRIPTION.key(), newHandlerDesc)); + schemaChanges.add( + SchemaChange.setOption( + MATERIALIZED_TABLE_REFRESH_HANDLER_BYTES.key(), + encodeBytesToBase64(newHandlerBytes))); + return true; + } + return false; + } + @Override public void alterTable( ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) @@ -517,10 +710,10 @@ public void alterTable( return; } - CatalogTable table = getTable(tablePath); + CatalogBaseTable table = getTable(tablePath); // Currently, Flink SQL only support altering table properties. - validateAlterTable(table, (CatalogTable) newTable); + validateAlterTable(table, newTable); List changes = new ArrayList<>(); Map oldProperties = table.getOptions(); @@ -575,8 +768,10 @@ public void alterTable( throw new TableNotExistException(getName(), tablePath); } - Preconditions.checkArgument(table instanceof FileStoreTable, "Can't alter system table."); - validateAlterTable(toCatalogTable(table), (CatalogTable) newTable); + checkArgument( + table instanceof FileStoreTable, + "Only support alter data table, but is: " + table.getClass()); + validateAlterTable(toCatalogTable(table), newTable); Map oldTableNonPhysicalColumnIndex = FlinkCatalogPropertiesUtil.nonPhysicalColumns( table.options(), table.rowType().getFieldNames()); @@ -669,36 +864,59 @@ private void setWatermarkOptions( .asSerializableString())); } - private static void validateAlterTable(CatalogTable ct1, CatalogTable ct2) { + private static void validateAlterTable(CatalogBaseTable ct1, CatalogBaseTable ct2) { if (ct1 instanceof SystemCatalogTable) { throw new UnsupportedOperationException("Can't alter system table."); } - org.apache.flink.table.api.TableSchema ts1 = ct1.getSchema(); - org.apache.flink.table.api.TableSchema ts2 = ct2.getSchema(); - boolean pkEquality = false; - - if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { - pkEquality = - Objects.equals( - ts1.getPrimaryKey().get().getType(), - ts2.getPrimaryKey().get().getType()) - && Objects.equals( - ts1.getPrimaryKey().get().getColumns(), - ts2.getPrimaryKey().get().getColumns()); - } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { - pkEquality = true; + boolean table1IsMaterialized = ct1 instanceof CatalogMaterializedTable; + boolean table2IsMaterialized = ct2 instanceof CatalogMaterializedTable; + if ((table1IsMaterialized || table2IsMaterialized) + && !(table1IsMaterialized && table2IsMaterialized)) { + throw new UnsupportedOperationException( + "Convert a non-materialized table to materialized table or vice versa is not allowed."); } + // materialized table is not resolved at this time. + if (!table1IsMaterialized) { + org.apache.flink.table.api.TableSchema ts1 = ct1.getSchema(); + org.apache.flink.table.api.TableSchema ts2 = ct2.getSchema(); + boolean pkEquality = false; + + if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { + pkEquality = + Objects.equals( + ts1.getPrimaryKey().get().getType(), + ts2.getPrimaryKey().get().getType()) + && Objects.equals( + ts1.getPrimaryKey().get().getColumns(), + ts2.getPrimaryKey().get().getColumns()); + } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { + pkEquality = true; + } - if (!pkEquality) { - throw new UnsupportedOperationException("Altering primary key is not supported yet."); + if (!pkEquality) { + throw new UnsupportedOperationException( + "Altering primary key is not supported yet."); + } } - if (!ct1.getPartitionKeys().equals(ct2.getPartitionKeys())) { + if (!getPartitionKeys(ct1).equals(getPartitionKeys(ct2))) { throw new UnsupportedOperationException( "Altering partition keys is not supported yet."); } } + private static List getPartitionKeys(CatalogBaseTable table) { + if (table instanceof CatalogTable) { + return ((CatalogTable) table).getPartitionKeys(); + } else if (table instanceof CatalogMaterializedTable) { + return ((CatalogMaterializedTable) table).getPartitionKeys(); + } else { + throw new UnsupportedOperationException( + "Only support CatalogTable and CatalogMaterializedTable, but is: " + + table.getClass()); + } + } + @Override public final void open() throws CatalogException {} @@ -711,7 +929,7 @@ public final void close() throws CatalogException { } } - private CatalogTableImpl toCatalogTable(Table table) { + private CatalogBaseTable toCatalogTable(Table table) { Map newOptions = new HashMap<>(table.options()); TableSchema.Builder builder = TableSchema.builder(); @@ -761,6 +979,10 @@ private CatalogTableImpl toCatalogTable(Table table) { removeProperties.putTableSchema(SCHEMA, schema); removeProperties.asMap().keySet().forEach(newOptions::remove); + Options options = Options.fromMap(newOptions); + if (TableType.MATERIALIZED_TABLE == options.get(CoreOptions.TYPE)) { + return buildMaterializedTable(table, newOptions, schema, options); + } return new DataCatalogTable( table, schema, @@ -770,9 +992,54 @@ private CatalogTableImpl toCatalogTable(Table table) { nonPhysicalColumnComments); } - public static Schema fromCatalogTable(CatalogTable table) { - ResolvedCatalogTable catalogTable = (ResolvedCatalogTable) table; - ResolvedSchema schema = catalogTable.getResolvedSchema(); + private CatalogMaterializedTable buildMaterializedTable( + Table table, Map newOptions, TableSchema schema, Options options) { + String definitionQuery = options.get(MATERIALIZED_TABLE_DEFINITION_QUERY); + IntervalFreshness freshness = + IntervalFreshness.of( + options.get(MATERIALIZED_TABLE_INTERVAL_FRESHNESS), + IntervalFreshness.TimeUnit.valueOf( + options.get(MATERIALIZED_TABLE_INTERVAL_FRESHNESS_TIME_UNIT) + .name())); + CatalogMaterializedTable.LogicalRefreshMode logicalRefreshMode = + CatalogMaterializedTable.LogicalRefreshMode.valueOf( + options.get(MATERIALIZED_TABLE_LOGICAL_REFRESH_MODE).name()); + CatalogMaterializedTable.RefreshMode refreshMode = + CatalogMaterializedTable.RefreshMode.valueOf( + options.get(MATERIALIZED_TABLE_REFRESH_MODE).name()); + CatalogMaterializedTable.RefreshStatus refreshStatus = + CatalogMaterializedTable.RefreshStatus.valueOf( + options.get(MATERIALIZED_TABLE_REFRESH_STATUS).name()); + String refreshHandlerDescription = + options.get(MATERIALIZED_TABLE_REFRESH_HANDLER_DESCRIPTION); + byte[] serializedRefreshHandler = + decodeRefreshHandlerBytes(options.get(MATERIALIZED_TABLE_REFRESH_HANDLER_BYTES)); + // remove materialized table related options + allMaterializedTableAttributes().forEach(newOptions::remove); + return CatalogMaterializedTable.newBuilder() + .schema(schema.toSchema()) + .comment(table.comment().orElse("")) + .partitionKeys(table.partitionKeys()) + .options(newOptions) + .definitionQuery(definitionQuery) + .freshness(freshness) + .logicalRefreshMode(logicalRefreshMode) + .refreshMode(refreshMode) + .refreshStatus(refreshStatus) + .refreshHandlerDescription(refreshHandlerDescription) + .serializedRefreshHandler(serializedRefreshHandler) + .build(); + } + + private byte[] decodeRefreshHandlerBytes(String refreshHandlerBytes) { + return org.apache.flink.util.StringUtils.isNullOrWhitespaceOnly(refreshHandlerBytes) + ? null + : decodeBase64ToBytes(refreshHandlerBytes); + } + + public static Schema fromCatalogTable(CatalogBaseTable catalogTable) { + ResolvedSchema schema = + ((ResolvedCatalogBaseTable) catalogTable).getResolvedSchema(); RowType rowType = (RowType) schema.toPhysicalRowDataType().getLogicalType(); Map options = new HashMap<>(catalogTable.getOptions()); @@ -788,7 +1055,7 @@ public static Schema fromCatalogTable(CatalogTable table) { schema.getPrimaryKey() .map(pk -> pk.getColumns()) .orElse(Collections.emptyList())) - .partitionKeys(catalogTable.getPartitionKeys()); + .partitionKeys(getPartitionKeys(catalogTable)); Map columnComments = getColumnComments(catalogTable); rowType.getFields() .forEach( @@ -801,7 +1068,7 @@ public static Schema fromCatalogTable(CatalogTable table) { return schemaBuilder.build(); } - private static Map getColumnComments(CatalogTable catalogTable) { + private static Map getColumnComments(CatalogBaseTable catalogTable) { return catalogTable.getUnresolvedSchema().getColumns().stream() .filter(c -> c.getComment().isPresent()) .collect( @@ -843,15 +1110,27 @@ public final void renameTable( try { catalog.renameTable(toIdentifier(tablePath), toIdentifier(toTable), ignoreIfNotExists); } catch (Catalog.TableNotExistException e) { - throw new TableNotExistException(getName(), tablePath); + try { + catalog.renameView( + toIdentifier(tablePath), toIdentifier(toTable), ignoreIfNotExists); + } catch (Catalog.ViewNotExistException ex) { + throw new TableNotExistException(getName(), tablePath); + } catch (Catalog.ViewAlreadyExistException ex) { + throw new TableAlreadyExistException(getName(), toTable); + } } catch (Catalog.TableAlreadyExistException e) { throw new TableAlreadyExistException(getName(), toTable); } } @Override - public final List listViews(String databaseName) throws CatalogException { - return Collections.emptyList(); + public final List listViews(String databaseName) + throws DatabaseNotExistException, CatalogException { + try { + return catalog.listViews(databaseName); + } catch (Catalog.DatabaseNotExistException e) { + throw new DatabaseNotExistException(getName(), databaseName); + } } @Override @@ -900,9 +1179,12 @@ private List getPartitionSpecs( getPartitionEntries(table, tablePath, partitionSpec); org.apache.paimon.types.RowType partitionRowType = table.schema().logicalPartitionType(); + CoreOptions options = new CoreOptions(table.options()); InternalRowPartitionComputer partitionComputer = FileStorePathFactory.getPartitionComputer( - partitionRowType, new CoreOptions(table.options()).partitionDefaultName()); + partitionRowType, + options.partitionDefaultName(), + options.legacyPartitionName()); return partitionEntries.stream() .map( @@ -943,11 +1225,11 @@ public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec // This was already filtered by the expected partition. PartitionEntry partitionEntry = partitionEntries.get(0); Map properties = new HashMap<>(); - properties.put(NUM_ROWS_KEY, String.valueOf(partitionEntry.recordCount())); + properties.put(NUM_ROWS_PROP, String.valueOf(partitionEntry.recordCount())); properties.put( - LAST_UPDATE_TIME_KEY, String.valueOf(partitionEntry.lastFileCreationTime())); - properties.put(NUM_FILES_KEY, String.valueOf(partitionEntry.fileCount())); - properties.put(TOTAL_SIZE_KEY, String.valueOf(partitionEntry.fileSizeInBytes())); + LAST_UPDATE_TIME_PROP, String.valueOf(partitionEntry.lastFileCreationTime())); + properties.put(NUM_FILES_PROP, String.valueOf(partitionEntry.fileCount())); + properties.put(TOTAL_SIZE_PROP, String.valueOf(partitionEntry.fileSizeInBytes())); return new CatalogPartitionImpl(properties, ""); } catch (TableNotPartitionedException | TableNotExistException e) { throw new PartitionNotExistException(getName(), tablePath, partitionSpec); @@ -971,8 +1253,19 @@ public final void createPartition( CatalogPartitionSpec partitionSpec, CatalogPartition partition, boolean ignoreIfExists) - throws CatalogException { - throw new UnsupportedOperationException(); + throws CatalogException, PartitionAlreadyExistsException { + if (partitionExists(tablePath, partitionSpec)) { + if (!ignoreIfExists) { + throw new PartitionAlreadyExistsException(getName(), tablePath, partitionSpec); + } + } + + try { + Identifier identifier = toIdentifier(tablePath); + catalog.createPartition(identifier, partitionSpec.getPartitionSpec()); + } catch (Catalog.TableNotExistException e) { + throw new CatalogException(e); + } } @Override @@ -1071,8 +1364,9 @@ public final CatalogColumnStatistics getPartitionColumnStatistics( @Override public final void alterTableStatistics( ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); + throws CatalogException, TableNotExistException { + alterTableStatisticsInternal( + tablePath, t -> createTableStats(t, tableStatistics), ignoreIfNotExists); } @Override @@ -1080,8 +1374,38 @@ public final void alterTableColumnStatistics( ObjectPath tablePath, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); + throws CatalogException, TableNotExistException { + alterTableStatisticsInternal( + tablePath, t -> createTableColumnStats(t, columnStatistics), ignoreIfNotExists); + } + + private void alterTableStatisticsInternal( + ObjectPath tablePath, + Function statistics, + boolean ignoreIfNotExists) + throws TableNotExistException { + try { + Table table = catalog.getTable(toIdentifier(tablePath)); + checkArgument( + table instanceof FileStoreTable, "Now only support analyze FileStoreTable."); + if (!table.latestSnapshotId().isPresent()) { + LOG.info("Skipping analyze table because the snapshot is null."); + return; + } + + FileStoreTable storeTable = (FileStoreTable) table; + Statistics tableStats = statistics.apply(storeTable); + if (tableStats != null) { + String commitUser = storeTable.coreOptions().createCommitUser(); + try (FileStoreCommit commit = storeTable.store().newCommit(commitUser)) { + commit.commitStatistics(tableStats, BatchWriteBuilder.COMMIT_IDENTIFIER); + } + } + } catch (Catalog.TableNotExistException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath); + } + } } @Override @@ -1134,4 +1458,16 @@ private boolean isCalledFromFlinkRecomputeStatisticsProgram() { } return false; } + + private List allMaterializedTableAttributes() { + return Arrays.asList( + MATERIALIZED_TABLE_DEFINITION_QUERY.key(), + MATERIALIZED_TABLE_INTERVAL_FRESHNESS.key(), + MATERIALIZED_TABLE_INTERVAL_FRESHNESS_TIME_UNIT.key(), + MATERIALIZED_TABLE_REFRESH_MODE.key(), + MATERIALIZED_TABLE_LOGICAL_REFRESH_MODE.key(), + MATERIALIZED_TABLE_REFRESH_STATUS.key(), + MATERIALIZED_TABLE_REFRESH_HANDLER_DESCRIPTION.key(), + MATERIALIZED_TABLE_REFRESH_HANDLER_BYTES.key()); + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java index 2a8f5f4cd7cd..5716cfca1baa 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java @@ -44,7 +44,7 @@ public class FlinkConnectorOptions { public static final String NONE = "none"; - public static final String TABLE_DYNAMIC_OPTION_PREFIX = "paimon"; + public static final String TABLE_DYNAMIC_OPTION_PREFIX = "paimon."; public static final int MIN_CLUSTERING_SAMPLE_FACTOR = 20; @@ -244,15 +244,6 @@ public class FlinkConnectorOptions { "Weight of managed memory for RocksDB in cross-partition update, Flink will compute the memory size " + "according to the weight, the actual memory used depends on the running environment."); - public static final ConfigOption SCAN_PUSH_DOWN = - ConfigOptions.key("scan.push-down") - .booleanType() - .defaultValue(true) - .withDescription( - "If true, flink will push down projection, filters, limit to the source. The cost is that it " - + "is difficult to reuse the source in a job. With flink 1.18 or higher version, it " - + "is possible to reuse the source even with projection push down."); - public static final ConfigOption SOURCE_CHECKPOINT_ALIGN_ENABLED = ConfigOptions.key("source.checkpoint-align.enabled") .booleanType() @@ -320,6 +311,15 @@ public class FlinkConnectorOptions { .withDescription( "If the pending snapshot count exceeds the threshold, lookup operator will refresh the table in sync."); + public static final ConfigOption LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST = + ConfigOptions.key("lookup.refresh.time-periods-blacklist") + .stringType() + .noDefaultValue() + .withDescription( + "The blacklist contains several time periods. During these time periods, the lookup table's " + + "cache refreshing is forbidden. Blacklist format is start1->end1,start2->end2,... , " + + "and the time format is yyyy-MM-dd HH:mm. Only used when lookup table is FULL cache mode."); + public static final ConfigOption SINK_AUTO_TAG_FOR_SAVEPOINT = ConfigOptions.key("sink.savepoint.auto-tag") .booleanType() @@ -364,12 +364,13 @@ public class FlinkConnectorOptions { "You can specify time interval for partition, for example, " + "daily partition is '1 d', hourly partition is '1 h'."); - public static final ConfigOption PARTITION_MARK_DONE_WHEN_END_INPUT = - ConfigOptions.key("partition.end-input-to-done") - .booleanType() - .defaultValue(false) + public static final ConfigOption PARTITION_IDLE_TIME_TO_REPORT_STATISTIC = + key("partition.idle-time-to-report-statistic") + .durationType() + .defaultValue(Duration.ofHours(1)) .withDescription( - "Whether mark the done status to indicate that the data is ready when end input."); + "Set a time duration when a partition has no new data after this time duration, " + + "start to report the partition statistics to hms."); public static final ConfigOption CLUSTERING_COLUMNS = key("sink.clustering.by-columns") @@ -413,6 +414,33 @@ public class FlinkConnectorOptions { .withDescription( "Optional endInput watermark used in case of batch mode or bounded stream."); + public static final ConfigOption CHANGELOG_PRECOMMIT_COMPACT = + key("changelog.precommit-compact") + .booleanType() + .defaultValue(false) + .withDescription( + "If true, it will add a changelog compact coordinator and worker operator after the writer operator," + + "in order to compact several changelog files from the same partition into large ones, " + + "which can decrease the number of small files. "); + + public static final ConfigOption SOURCE_OPERATOR_UID_SUFFIX = + key("source.operator-uid.suffix") + .stringType() + .noDefaultValue() + .withDescription( + "Set the uid suffix for the source operators. After setting, the uid format is " + + "${UID_PREFIX}_${TABLE_NAME}_${USER_UID_SUFFIX}. If the uid suffix is not set, flink will " + + "automatically generate the operator uid, which may be incompatible when the topology changes."); + + public static final ConfigOption SINK_OPERATOR_UID_SUFFIX = + key("sink.operator-uid.suffix") + .stringType() + .noDefaultValue() + .withDescription( + "Set the uid suffix for the writer, dynamic bucket assigner and committer operators. The uid format is " + + "${UID_PREFIX}_${TABLE_NAME}_${USER_UID_SUFFIX}. If the uid suffix is not set, flink will " + + "automatically generate the operator uid, which may be incompatible when the topology changes."); + public static List> getOptions() { final Field[] fields = FlinkConnectorOptions.class.getFields(); final List> list = new ArrayList<>(fields.length); @@ -428,6 +456,11 @@ public static List> getOptions() { return list; } + public static String generateCustomUid( + String uidPrefix, String tableName, String userDefinedSuffix) { + return String.format("%s_%s_%s", uidPrefix, tableName, userDefinedSuffix); + } + /** The mode of lookup cache. */ public enum LookupCacheMode { /** Auto mode, try to use partial mode. */ diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java index f6c206a7f422..37bed2d0480f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalog.java @@ -498,7 +498,11 @@ public List listProcedures(String dbName) */ public Procedure getProcedure(ObjectPath procedurePath) throws ProcedureNotExistException, CatalogException { - return ProcedureUtil.getProcedure(paimon.catalog(), procedurePath) - .orElse(flink.getProcedure(procedurePath)); + Optional procedure = ProcedureUtil.getProcedure(paimon.catalog(), procedurePath); + if (procedure.isPresent()) { + return procedure.get(); + } else { + return flink.getProcedure(procedurePath); + } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalogFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalogFactory.java index dc2a0f06b648..7c3a13c6f377 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalogFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkGenericCatalogFactory.java @@ -89,6 +89,7 @@ public static FlinkGenericCatalog createCatalog( ClassLoader cl, Map optionMap, String name, Catalog flinkCatalog) { Options options = Options.fromMap(optionMap); options.set(CatalogOptions.METASTORE, "hive"); + options.set(CatalogOptions.FORMAT_TABLE_ENABLED, false); FlinkCatalog paimon = new FlinkCatalog( org.apache.paimon.catalog.CatalogFactory.createCatalog( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java index 96c81fdb720c..d5c1ed043b56 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java @@ -30,11 +30,20 @@ import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.DynamicTableFactory; +import javax.annotation.Nullable; + import static org.apache.paimon.CoreOptions.AUTO_CREATE; import static org.apache.paimon.flink.FlinkCatalogFactory.IDENTIFIER; /** A paimon {@link DynamicTableFactory} to create source and sink. */ public class FlinkTableFactory extends AbstractFlinkTableFactory { + public FlinkTableFactory() { + this(null); + } + + public FlinkTableFactory(@Nullable FlinkCatalog flinkCatalog) { + super(flinkCatalog); + } @Override public String factoryIdentifier() { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FormatCatalogTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FormatCatalogTable.java index 95aff5d84796..2e944f930cbb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FormatCatalogTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FormatCatalogTable.java @@ -20,7 +20,6 @@ import org.apache.paimon.table.FormatTable; -import org.apache.flink.connector.file.table.FileSystemTableFactory; import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -30,17 +29,16 @@ import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.PATH; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; import static org.apache.flink.table.factories.FactoryUtil.FORMAT; import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; +import static org.apache.paimon.table.FormatTableOptions.FIELD_DELIMITER; /** A {@link CatalogTable} to represent format table. */ public class FormatCatalogTable implements CatalogTable { @@ -83,18 +81,17 @@ public CatalogTable copy(Map map) { public Map getOptions() { if (cachedOptions == null) { cachedOptions = new HashMap<>(); - FileSystemTableFactory fileSystemFactory = new FileSystemTableFactory(); - Set validOptions = new HashSet<>(); - fileSystemFactory.requiredOptions().forEach(o -> validOptions.add(o.key())); - fileSystemFactory.optionalOptions().forEach(o -> validOptions.add(o.key())); String format = table.format().name().toLowerCase(); - table.options() - .forEach( - (k, v) -> { - if (validOptions.contains(k) || k.startsWith(format + ".")) { - cachedOptions.put(k, v); - } - }); + Map options = table.options(); + options.forEach( + (k, v) -> { + if (k.startsWith(format + ".")) { + cachedOptions.put(k, v); + } + }); + if (options.containsKey(FIELD_DELIMITER.key())) { + cachedOptions.put("csv.field-delimiter", options.get(FIELD_DELIMITER.key())); + } cachedOptions.put(CONNECTOR.key(), "filesystem"); cachedOptions.put(PATH.key(), table.location()); cachedOptions.put(FORMAT.key(), format); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/ProcessRecordAttributesUtil.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/ProcessRecordAttributesUtil.java new file mode 100644 index 000000000000..fae52e3faff0 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/ProcessRecordAttributesUtil.java @@ -0,0 +1,33 @@ +/* + * 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; + +import org.apache.paimon.flink.sink.StoreSinkWrite; + +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; + +/** Utils for {@link RecordAttributes}. */ +public class ProcessRecordAttributesUtil { + public static void processWithWrite(RecordAttributes recordAttributes, StoreSinkWrite write) {} + + public static void processWithOutput(RecordAttributes recordAttributes, Output output) { + output.emitRecordAttributes(recordAttributes); + } +} 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 19c209709b5c..8ea120015609 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 @@ -186,14 +186,14 @@ protected Predicate getPredicate() throws Exception { predicate = simpleSqlPredicateConvertor.convertSqlToPredicate(whereSql); } - // Check whether predicate contain non parition key. + // Check whether predicate contain non partition key. if (predicate != null) { LOGGER.info("the partition predicate of compaction is {}", predicate); PartitionPredicateVisitor partitionPredicateVisitor = new PartitionPredicateVisitor(table.partitionKeys()); Preconditions.checkArgument( predicate.visit(partitionPredicateVisitor), - "Only parition key can be specialized in compaction action."); + "Only partition key can be specialized in compaction action."); } return predicate; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java index ef6772e36eed..fda9ff695e1e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseAction.java @@ -54,6 +54,7 @@ import java.util.regex.Pattern; import static org.apache.paimon.flink.sink.FlinkStreamPartitioner.partition; +import static org.apache.paimon.flink.sink.FlinkStreamPartitioner.rebalance; /** Database compact action for Flink. */ public class CompactDatabaseAction extends ActionBase { @@ -208,6 +209,11 @@ private void buildForCombinedMode() { .toMillis()) .withPartitionIdleTime(partitionIdleTime); + Integer parallelism = + tableOptions.get(FlinkConnectorOptions.SINK_PARALLELISM) == null + ? env.getParallelism() + : tableOptions.get(FlinkConnectorOptions.SINK_PARALLELISM); + // multi bucket table which has multi bucket in a partition like fix bucket and dynamic // bucket DataStream awareBucketTableSource = @@ -217,14 +223,16 @@ private void buildForCombinedMode() { .withContinuousMode(isStreaming) .buildAwareBucketTableSource(), new BucketsRowChannelComputer(), - tableOptions.get(FlinkConnectorOptions.SINK_PARALLELISM)); + parallelism); // unaware bucket table DataStream unawareBucketTableSource = - sourceBuilder - .withEnv(env) - .withContinuousMode(isStreaming) - .buildForUnawareBucketsTableSource(); + rebalance( + sourceBuilder + .withEnv(env) + .withContinuousMode(isStreaming) + .buildForUnawareBucketsTableSource(), + parallelism); new CombinedTableCompactorSink(catalogLoader(), tableOptions) .sinkFrom(awareBucketTableSource, unawareBucketTableSource); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateOrReplaceTagActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateOrReplaceTagActionFactory.java new file mode 100644 index 000000000000..fecb6895b682 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateOrReplaceTagActionFactory.java @@ -0,0 +1,65 @@ +/* + * 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.utils.TimeUtils; + +import org.apache.flink.api.java.tuple.Tuple3; + +import java.time.Duration; +import java.util.Map; +import java.util.Optional; + +/** Factory to create {@link ReplaceTagAction} or {@link ReplaceTagAction}. */ +public abstract class CreateOrReplaceTagActionFactory implements ActionFactory { + + private static final String TAG_NAME = "tag_name"; + private static final String SNAPSHOT = "snapshot"; + private static final String TIME_RETAINED = "time_retained"; + + @Override + public Optional create(MultipleParameterToolAdapter params) { + checkRequiredArgument(params, TAG_NAME); + + Tuple3 tablePath = getTablePath(params); + Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + String tagName = params.get(TAG_NAME); + + Long snapshot = null; + if (params.has(SNAPSHOT)) { + snapshot = Long.parseLong(params.get(SNAPSHOT)); + } + + Duration timeRetained = null; + if (params.has(TIME_RETAINED)) { + timeRetained = TimeUtils.parseDuration(params.get(TIME_RETAINED)); + } + + return Optional.of( + createOrReplaceTagAction( + tablePath, catalogConfig, tagName, snapshot, timeRetained)); + } + + abstract Action createOrReplaceTagAction( + Tuple3 tablePath, + Map catalogConfig, + String tagName, + Long snapshot, + Duration timeRetained); +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagActionFactory.java index 7769fa1d792f..c525943122bc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CreateTagActionFactory.java @@ -18,56 +18,36 @@ package org.apache.paimon.flink.action; -import org.apache.paimon.utils.TimeUtils; - import org.apache.flink.api.java.tuple.Tuple3; import java.time.Duration; import java.util.Map; -import java.util.Optional; /** Factory to create {@link CreateTagAction}. */ -public class CreateTagActionFactory implements ActionFactory { +public class CreateTagActionFactory extends CreateOrReplaceTagActionFactory { public static final String IDENTIFIER = "create_tag"; - private static final String TAG_NAME = "tag_name"; - private static final String SNAPSHOT = "snapshot"; - private static final String TIME_RETAINED = "time_retained"; - @Override public String identifier() { return IDENTIFIER; } @Override - public Optional create(MultipleParameterToolAdapter params) { - checkRequiredArgument(params, TAG_NAME); - - Tuple3 tablePath = getTablePath(params); - Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); - String tagName = params.get(TAG_NAME); - - Long snapshot = null; - if (params.has(SNAPSHOT)) { - snapshot = Long.parseLong(params.get(SNAPSHOT)); - } - - Duration timeRetained = null; - if (params.has(TIME_RETAINED)) { - timeRetained = TimeUtils.parseDuration(params.get(TIME_RETAINED)); - } - - CreateTagAction action = - new CreateTagAction( - tablePath.f0, - tablePath.f1, - tablePath.f2, - catalogConfig, - tagName, - snapshot, - timeRetained); - return Optional.of(action); + Action createOrReplaceTagAction( + Tuple3 tablePath, + Map catalogConfig, + String tagName, + Long snapshot, + Duration timeRetained) { + return new CreateTagAction( + tablePath.f0, + tablePath.f1, + tablePath.f2, + catalogConfig, + tagName, + snapshot, + timeRetained); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsAction.java new file mode 100644 index 000000000000..c1231ed3ad54 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsAction.java @@ -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. + */ + +package org.apache.paimon.flink.action; + +import org.apache.paimon.flink.procedure.ExpireTagsProcedure; + +import org.apache.flink.table.procedure.DefaultProcedureContext; + +import java.util.Map; + +/** Expire tags action for Flink. */ +public class ExpireTagsAction extends ActionBase { + + private final String table; + private final String olderThan; + + public ExpireTagsAction( + String warehouse, String table, String olderThan, Map catalogConfig) { + super(warehouse, catalogConfig); + this.table = table; + this.olderThan = olderThan; + } + + @Override + public void run() throws Exception { + ExpireTagsProcedure expireTagsProcedure = new ExpireTagsProcedure(); + expireTagsProcedure.withCatalog(catalog); + expireTagsProcedure.call(new DefaultProcedureContext(env), table, olderThan); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsActionFactory.java new file mode 100644 index 000000000000..e9bbb0a3bdc7 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpireTagsActionFactory.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.flink.action; + +import java.util.Map; +import java.util.Optional; + +/** Factory to create {@link ExpireTagsAction}. */ +public class ExpireTagsActionFactory implements ActionFactory { + + private static final String IDENTIFIER = "expire_tags"; + + private static final String OLDER_THAN = "older_than"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public Optional create(MultipleParameterToolAdapter params) { + String warehouse = params.get(WAREHOUSE); + String table = params.get(TABLE); + String olderThan = params.get(OLDER_THAN); + Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + + ExpireTagsAction expireTagsAction = + new ExpireTagsAction(warehouse, table, olderThan, catalogConfig); + return Optional.of(expireTagsAction); + } + + @Override + public void printHelp() { + System.out.println("Action \"expire_tags\" expire tags by time."); + System.out.println(); + + System.out.println("Syntax:"); + System.out.println( + " expire_tags --warehouse " + + "--table " + + "[--older_than ]"); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FlinkActions.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FlinkActions.java index 3ed70ddcc42b..03d751d5f167 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FlinkActions.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/FlinkActions.java @@ -22,12 +22,7 @@ import static org.apache.paimon.flink.action.ActionFactory.printDefaultHelp; -/** - * Table maintenance actions for Flink. - * - * @deprecated Compatible with older versions of usage - */ -@Deprecated +/** Table maintenance actions for Flink. */ public class FlinkActions { // ------------------------------------------------------------------------ diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagAction.java new file mode 100644 index 000000000000..3b12b4c119a1 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagAction.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action; + +import org.apache.paimon.utils.StringUtils; + +import java.util.Map; + +/** Rename Tag action for Flink. */ +public class RenameTagAction extends TableActionBase { + private final String tagName; + private final String targetTagName; + + public RenameTagAction( + String warehouse, + String databaseName, + String tableName, + Map catalogConfig, + String tagName, + String targetTagName) { + super(warehouse, databaseName, tableName, catalogConfig); + this.tagName = tagName; + this.targetTagName = targetTagName; + } + + @Override + public void run() throws Exception { + if (StringUtils.isEmpty(tagName) || StringUtils.isEmpty(targetTagName)) { + throw new RuntimeException( + String.format( + "The specified tag name [%s] or target tag name [%s] cannot be empty.", + tagName, targetTagName)); + } + table.renameTag(tagName, targetTagName); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagActionFactory.java new file mode 100644 index 000000000000..84f174d39bbb --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RenameTagActionFactory.java @@ -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. + */ + +package org.apache.paimon.flink.action; + +import org.apache.flink.api.java.tuple.Tuple3; + +import java.util.Map; +import java.util.Optional; + +/** Factory to create {@link RenameTagActionFactory}. */ +public class RenameTagActionFactory implements ActionFactory { + + private static final String IDENTIFIER = "rename_tag"; + + private static final String TAG_NAME = "tag_name"; + private static final String TARGET_TAG_NAME = "target_tag_name"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public Optional create(MultipleParameterToolAdapter params) { + checkRequiredArgument(params, TAG_NAME); + checkRequiredArgument(params, TARGET_TAG_NAME); + + Tuple3 tablePath = getTablePath(params); + Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + String tagName = params.get(TAG_NAME); + String targetTagName = params.get(TARGET_TAG_NAME); + + RenameTagAction action = + new RenameTagAction( + tablePath.f0, + tablePath.f1, + tablePath.f2, + catalogConfig, + tagName, + targetTagName); + return Optional.of(action); + } + + @Override + public void printHelp() { + System.out.println("Action \"rename_tag\" rename a tag with a new tag name."); + System.out.println(); + + System.out.println("Syntax:"); + System.out.println( + " rename_tag --warehouse --tag_name " + + "--target_tag_name "); + System.out.println(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagAction.java new file mode 100644 index 000000000000..09a85fe8a25a --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagAction.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Map; + +/** Replace tag action for Flink. */ +public class ReplaceTagAction extends TableActionBase { + + private final String tagName; + private final @Nullable Long snapshotId; + private final @Nullable Duration timeRetained; + + public ReplaceTagAction( + String warehouse, + String databaseName, + String tableName, + Map catalogConfig, + String tagName, + @Nullable Long snapshotId, + @Nullable Duration timeRetained) { + super(warehouse, databaseName, tableName, catalogConfig); + this.tagName = tagName; + this.timeRetained = timeRetained; + this.snapshotId = snapshotId; + } + + @Override + public void run() throws Exception { + table.replaceTag(tagName, snapshotId, timeRetained); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagActionFactory.java new file mode 100644 index 000000000000..a734e9cfbdc5 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ReplaceTagActionFactory.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action; + +import org.apache.flink.api.java.tuple.Tuple3; + +import java.time.Duration; +import java.util.Map; + +/** Factory to create {@link ReplaceTagAction}. */ +public class ReplaceTagActionFactory extends CreateOrReplaceTagActionFactory { + + public static final String IDENTIFIER = "replace_tag"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + Action createOrReplaceTagAction( + Tuple3 tablePath, + Map catalogConfig, + String tagName, + Long snapshot, + Duration timeRetained) { + return new ReplaceTagAction( + tablePath.f0, + tablePath.f1, + tablePath.f2, + catalogConfig, + tagName, + snapshot, + timeRetained); + } + + @Override + public void printHelp() { + System.out.println("Action \"replace_tag\" to replace an existing tag with new tag info."); + System.out.println(); + + System.out.println("Syntax:"); + System.out.println( + " replace_tag --warehouse --database " + + "--table --tag_name [--snapshot ] [--time_retained ]"); + System.out.println(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java index 615b448ec9bb..6db8ab4fef75 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java @@ -57,6 +57,7 @@ public void run() throws Exception { if (Objects.isNull(nextSnapshotId)) { consumerManager.deleteConsumer(consumerId); } else { + dataTable.snapshotManager().snapshot(nextSnapshotId); consumerManager.resetConsumer(consumerId, new Consumer(nextSnapshotId)); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampAction.java new file mode 100644 index 000000000000..ca706101cc1d --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampAction.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action; + +import org.apache.paimon.Snapshot; +import org.apache.paimon.table.DataTable; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** Rollback to specific timestamp action for Flink. */ +public class RollbackToTimestampAction extends TableActionBase { + + private static final Logger LOG = LoggerFactory.getLogger(RollbackToTimestampAction.class); + + private final Long timestamp; + + public RollbackToTimestampAction( + String warehouse, + String databaseName, + String tableName, + Long timestamp, + Map catalogConfig) { + super(warehouse, databaseName, tableName, catalogConfig); + this.timestamp = timestamp; + } + + @Override + public void run() throws Exception { + LOG.debug("Run rollback-to-timestamp action with timestamp '{}'.", timestamp); + + if (!(table instanceof DataTable)) { + throw new IllegalArgumentException("Unknown table: " + identifier); + } + + FileStoreTable fileStoreTable = (FileStoreTable) table; + Snapshot snapshot = fileStoreTable.snapshotManager().earlierOrEqualTimeMills(timestamp); + Preconditions.checkNotNull( + snapshot, String.format("count not find snapshot earlier than %s", timestamp)); + fileStoreTable.rollbackTo(snapshot.id()); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampActionFactory.java new file mode 100644 index 000000000000..c694ac0041b5 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/RollbackToTimestampActionFactory.java @@ -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. + */ + +package org.apache.paimon.flink.action; + +import org.apache.flink.api.java.tuple.Tuple3; + +import java.util.Map; +import java.util.Optional; + +/** Factory to create {@link RollbackToTimestampAction}. */ +public class RollbackToTimestampActionFactory implements ActionFactory { + + public static final String IDENTIFIER = "rollback_to_timestamp"; + + private static final String TIMESTAMP = "timestamp"; + + @Override + public String identifier() { + return IDENTIFIER; + } + + @Override + public Optional create(MultipleParameterToolAdapter params) { + Tuple3 tablePath = getTablePath(params); + + checkRequiredArgument(params, TIMESTAMP); + String timestamp = params.get(TIMESTAMP); + + Map catalogConfig = optionalConfigMap(params, CATALOG_CONF); + + RollbackToTimestampAction action = + new RollbackToTimestampAction( + tablePath.f0, + tablePath.f1, + tablePath.f2, + Long.parseLong(timestamp), + catalogConfig); + + return Optional.of(action); + } + + @Override + public void printHelp() { + System.out.println( + "Action \"rollback_to_timestamp\" roll back a table to a specific timestamp."); + System.out.println(); + + System.out.println("Syntax:"); + System.out.println( + " rollback_to --warehouse --database " + + "--table --timestamp "); + System.out.println(" can be a long value representing a timestamp."); + System.out.println(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactor.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactor.java index a34f009072be..e8021329c9ce 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactor.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactor.java @@ -100,14 +100,31 @@ public void processElement(UnawareAppendCompactionTask task) throws Exception { metricsReporter.reportCompactionTime( System.currentTimeMillis() - startMillis); + metricsReporter + .increaseCompactionsCompletedCount(); } }, LOG); return commitMessage; } finally { MetricUtils.safeCall(this::stopTimer, LOG); + MetricUtils.safeCall( + this::decreaseCompactionsQueuedCount, LOG); } })); + recordCompactionsQueuedRequest(); + } + + private void recordCompactionsQueuedRequest() { + if (metricsReporter != null) { + metricsReporter.increaseCompactionsQueuedCount(); + } + } + + private void decreaseCompactionsQueuedCount() { + if (metricsReporter != null) { + metricsReporter.decreaseCompactionsQueuedCount(); + } } private void startTimer() { 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 new file mode 100644 index 000000000000..cd0b8716a7d1 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactCoordinateOperator.java @@ -0,0 +1,173 @@ +/* + * 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.compact.changelog; + +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.flink.sink.Committable; +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.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.types.Either; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Coordinator operator for compacting changelog files. + * + *

    {@link ChangelogCompactCoordinateOperator} calculates the file size of changelog files + * contained in all buckets within each partition from {@link Committable} message emitted from + * writer operator. And emit {@link ChangelogCompactTask} to {@link ChangelogCompactWorkerOperator}. + */ +public class ChangelogCompactCoordinateOperator + extends AbstractStreamOperator> + implements OneInputStreamOperator>, + BoundedOneInput { + private final FileStoreTable table; + + private transient long checkpointId; + private transient Map partitionChangelogs; + + public ChangelogCompactCoordinateOperator(FileStoreTable table) { + this.table = table; + } + + @Override + public void open() throws Exception { + super.open(); + + checkpointId = Long.MIN_VALUE; + partitionChangelogs = new HashMap<>(); + } + + public void processElement(StreamRecord record) { + Committable committable = record.getValue(); + checkpointId = Math.max(checkpointId, committable.checkpointId()); + if (committable.kind() != Committable.Kind.FILE) { + output.collect(new StreamRecord<>(Either.Left(record.getValue()))); + return; + } + + CommitMessageImpl message = (CommitMessageImpl) committable.wrappedCommittable(); + if (message.newFilesIncrement().changelogFiles().isEmpty() + && message.compactIncrement().changelogFiles().isEmpty()) { + output.collect(new StreamRecord<>(Either.Left(record.getValue()))); + return; + } + + BinaryRow partition = message.partition(); + Integer bucket = message.bucket(); + long targetFileSize = table.coreOptions().targetFileSize(false); + for (DataFileMeta meta : message.newFilesIncrement().changelogFiles()) { + partitionChangelogs + .computeIfAbsent(partition, k -> new PartitionChangelog()) + .addNewChangelogFile(bucket, meta); + PartitionChangelog partitionChangelog = partitionChangelogs.get(partition); + if (partitionChangelog.totalFileSize >= targetFileSize) { + emitPartitionChanglogCompactTask(partition); + } + } + for (DataFileMeta meta : message.compactIncrement().changelogFiles()) { + partitionChangelogs + .computeIfAbsent(partition, k -> new PartitionChangelog()) + .addCompactChangelogFile(bucket, meta); + PartitionChangelog partitionChangelog = partitionChangelogs.get(partition); + if (partitionChangelog.totalFileSize >= targetFileSize) { + emitPartitionChanglogCompactTask(partition); + } + } + + CommitMessageImpl newMessage = + new CommitMessageImpl( + message.partition(), + message.bucket(), + new DataIncrement( + message.newFilesIncrement().newFiles(), + message.newFilesIncrement().deletedFiles(), + Collections.emptyList()), + new CompactIncrement( + message.compactIncrement().compactBefore(), + message.compactIncrement().compactAfter(), + Collections.emptyList()), + message.indexIncrement()); + Committable newCommittable = + new Committable(committable.checkpointId(), Committable.Kind.FILE, newMessage); + output.collect(new StreamRecord<>(Either.Left(newCommittable))); + } + + public void prepareSnapshotPreBarrier(long checkpointId) { + emitAllPartitionsChanglogCompactTask(); + } + + public void endInput() { + emitAllPartitionsChanglogCompactTask(); + } + + private void emitPartitionChanglogCompactTask(BinaryRow partition) { + PartitionChangelog partitionChangelog = partitionChangelogs.get(partition); + output.collect( + new StreamRecord<>( + Either.Right( + new ChangelogCompactTask( + checkpointId, + partition, + partitionChangelog.newFileChangelogFiles, + partitionChangelog.compactChangelogFiles)))); + partitionChangelogs.remove(partition); + } + + private void emitAllPartitionsChanglogCompactTask() { + List partitions = new ArrayList<>(partitionChangelogs.keySet()); + for (BinaryRow partition : partitions) { + emitPartitionChanglogCompactTask(partition); + } + } + + private static class PartitionChangelog { + private long totalFileSize; + private final Map> newFileChangelogFiles; + private final Map> compactChangelogFiles; + + public PartitionChangelog() { + totalFileSize = 0; + newFileChangelogFiles = new HashMap<>(); + compactChangelogFiles = new HashMap<>(); + } + + public void addNewChangelogFile(Integer bucket, DataFileMeta file) { + totalFileSize += file.fileSize(); + newFileChangelogFiles.computeIfAbsent(bucket, k -> new ArrayList<>()).add(file); + } + + public void addCompactChangelogFile(Integer bucket, DataFileMeta file) { + totalFileSize += file.fileSize(); + compactChangelogFiles.computeIfAbsent(bucket, k -> new ArrayList<>()).add(file); + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.java new file mode 100644 index 000000000000..6b95e369074b --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.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.flink.compact.changelog; + +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.flink.compact.changelog.format.CompactedChangelogReadOnlyFormat; +import org.apache.paimon.flink.sink.Committable; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.PositionOutputStream; +import org.apache.paimon.fs.SeekableInputStream; +import org.apache.paimon.io.CompactIncrement; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.io.DataFilePathFactory; +import org.apache.paimon.io.DataIncrement; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.paimon.utils.FileStorePathFactory; +import org.apache.paimon.utils.IOUtils; +import org.apache.paimon.utils.Preconditions; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.UUID; + +/** + * {@link ChangelogCompactTask} to compact several changelog files from the same partition into one + * file, in order to reduce the number of small files. + */ +public class ChangelogCompactTask implements Serializable { + private final long checkpointId; + private final BinaryRow partition; + private final Map> newFileChangelogFiles; + private final Map> compactChangelogFiles; + + public ChangelogCompactTask( + long checkpointId, + BinaryRow partition, + Map> newFileChangelogFiles, + Map> compactChangelogFiles) { + this.checkpointId = checkpointId; + this.partition = partition; + this.newFileChangelogFiles = newFileChangelogFiles; + this.compactChangelogFiles = compactChangelogFiles; + } + + public long checkpointId() { + return checkpointId; + } + + public BinaryRow partition() { + return partition; + } + + public Map> newFileChangelogFiles() { + return newFileChangelogFiles; + } + + public Map> compactChangelogFiles() { + return compactChangelogFiles; + } + + public List doCompact(FileStoreTable table) throws Exception { + FileStorePathFactory pathFactory = table.store().pathFactory(); + OutputStream outputStream = new OutputStream(); + List results = new ArrayList<>(); + + // copy all changelog files to a new big file + for (Map.Entry> entry : newFileChangelogFiles.entrySet()) { + int bucket = entry.getKey(); + DataFilePathFactory dataFilePathFactory = + pathFactory.createDataFilePathFactory(partition, bucket); + for (DataFileMeta meta : entry.getValue()) { + copyFile( + outputStream, + results, + table, + dataFilePathFactory.toPath(meta.fileName()), + bucket, + false, + meta); + } + } + for (Map.Entry> entry : compactChangelogFiles.entrySet()) { + Integer bucket = entry.getKey(); + DataFilePathFactory dataFilePathFactory = + pathFactory.createDataFilePathFactory(partition, bucket); + for (DataFileMeta meta : entry.getValue()) { + copyFile( + outputStream, + results, + table, + dataFilePathFactory.toPath(meta.fileName()), + bucket, + true, + meta); + } + } + outputStream.out.close(); + + return produceNewCommittables(results, table, pathFactory, outputStream.path); + } + + private void copyFile( + OutputStream outputStream, + List results, + FileStoreTable table, + Path path, + int bucket, + boolean isCompactResult, + DataFileMeta meta) + throws Exception { + if (!outputStream.isInitialized) { + Path outputPath = + new Path(path.getParent(), "tmp-compacted-changelog-" + UUID.randomUUID()); + outputStream.init(outputPath, table.fileIO().newOutputStream(outputPath, false)); + } + long offset = outputStream.out.getPos(); + try (SeekableInputStream in = table.fileIO().newInputStream(path)) { + IOUtils.copyBytes(in, outputStream.out, IOUtils.BLOCKSIZE, false); + } + table.fileIO().deleteQuietly(path); + results.add( + new Result( + bucket, isCompactResult, meta, offset, outputStream.out.getPos() - offset)); + } + + private List produceNewCommittables( + List results, + FileStoreTable table, + FileStorePathFactory pathFactory, + Path changelogTempPath) + throws IOException { + Result baseResult = results.get(0); + Preconditions.checkArgument(baseResult.offset == 0); + DataFilePathFactory dataFilePathFactory = + pathFactory.createDataFilePathFactory(partition, baseResult.bucket); + // see Java docs of `CompactedChangelogFormatReaderFactory` + String realName = + "compacted-changelog-" + + UUID.randomUUID() + + "$" + + baseResult.bucket + + "-" + + baseResult.length; + table.fileIO() + .rename( + changelogTempPath, + dataFilePathFactory.toPath( + realName + + "." + + CompactedChangelogReadOnlyFormat.getIdentifier( + baseResult.meta.fileFormat()))); + + List newCommittables = new ArrayList<>(); + + Map> bucketedResults = new HashMap<>(); + for (Result result : results) { + bucketedResults.computeIfAbsent(result.bucket, b -> new ArrayList<>()).add(result); + } + + for (Map.Entry> entry : bucketedResults.entrySet()) { + List newFilesChangelog = new ArrayList<>(); + List compactChangelog = new ArrayList<>(); + for (Result result : entry.getValue()) { + // see Java docs of `CompactedChangelogFormatReaderFactory` + String name = + (result.offset == 0 + ? realName + : realName + "-" + result.offset + "-" + result.length) + + "." + + CompactedChangelogReadOnlyFormat.getIdentifier( + result.meta.fileFormat()); + if (result.isCompactResult) { + compactChangelog.add(result.meta.rename(name)); + } else { + newFilesChangelog.add(result.meta.rename(name)); + } + } + + CommitMessageImpl newMessage = + new CommitMessageImpl( + partition, + entry.getKey(), + new DataIncrement( + Collections.emptyList(), + Collections.emptyList(), + newFilesChangelog), + new CompactIncrement( + Collections.emptyList(), + Collections.emptyList(), + compactChangelog)); + newCommittables.add(new Committable(checkpointId, Committable.Kind.FILE, newMessage)); + } + return newCommittables; + } + + public int hashCode() { + return Objects.hash(checkpointId, partition, newFileChangelogFiles, compactChangelogFiles); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + ChangelogCompactTask that = (ChangelogCompactTask) o; + return checkpointId == that.checkpointId + && Objects.equals(partition, that.partition) + && Objects.equals(newFileChangelogFiles, that.newFileChangelogFiles) + && Objects.equals(compactChangelogFiles, that.compactChangelogFiles); + } + + @Override + public String toString() { + return String.format( + "ChangelogCompactionTask {" + + "partition = %s, " + + "newFileChangelogFiles = %s, " + + "compactChangelogFiles = %s}", + partition, newFileChangelogFiles, compactChangelogFiles); + } + + private static class OutputStream { + + private Path path; + private PositionOutputStream out; + private boolean isInitialized; + + private OutputStream() { + this.isInitialized = false; + } + + private void init(Path path, PositionOutputStream out) { + this.path = path; + this.out = out; + this.isInitialized = true; + } + } + + private static class Result { + + private final int bucket; + private final boolean isCompactResult; + private final DataFileMeta meta; + private final long offset; + private final long length; + + private Result( + int bucket, boolean isCompactResult, DataFileMeta meta, long offset, long length) { + this.bucket = bucket; + this.isCompactResult = isCompactResult; + this.meta = meta; + this.offset = offset; + this.length = length; + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializer.java new file mode 100644 index 000000000000..34f9d035d8d2 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializer.java @@ -0,0 +1,113 @@ +/* + * 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.compact.changelog; + +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.io.DataFileMetaSerializer; +import org.apache.paimon.io.DataInputDeserializer; +import org.apache.paimon.io.DataInputView; +import org.apache.paimon.io.DataOutputView; +import org.apache.paimon.io.DataOutputViewStreamWrapper; + +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.util.CollectionUtil; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.utils.SerializationUtils.deserializeBinaryRow; +import static org.apache.paimon.utils.SerializationUtils.serializeBinaryRow; + +/** Serializer for {@link ChangelogCompactTask}. */ +public class ChangelogCompactTaskSerializer + implements SimpleVersionedSerializer { + private static final int CURRENT_VERSION = 1; + + private final DataFileMetaSerializer dataFileSerializer; + + public ChangelogCompactTaskSerializer() { + this.dataFileSerializer = new DataFileMetaSerializer(); + } + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(ChangelogCompactTask obj) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + serialize(obj, view); + return out.toByteArray(); + } + + @Override + public ChangelogCompactTask deserialize(int version, byte[] serialized) throws IOException { + DataInputDeserializer view = new DataInputDeserializer(serialized); + return deserialize(version, view); + } + + private void serialize(ChangelogCompactTask task, DataOutputView view) throws IOException { + view.writeLong(task.checkpointId()); + serializeBinaryRow(task.partition(), view); + // serialize newFileChangelogFiles map + serializeMap(task.newFileChangelogFiles(), view); + serializeMap(task.compactChangelogFiles(), view); + } + + private ChangelogCompactTask deserialize(int version, DataInputView view) throws IOException { + if (version != getVersion()) { + throw new RuntimeException("Can not deserialize version: " + version); + } + + return new ChangelogCompactTask( + view.readLong(), + deserializeBinaryRow(view), + deserializeMap(view), + deserializeMap(view)); + } + + private void serializeMap(Map> map, DataOutputView view) + throws IOException { + view.writeInt(map.size()); + for (Map.Entry> entry : map.entrySet()) { + view.writeInt(entry.getKey()); + if (entry.getValue() == null) { + throw new IllegalArgumentException( + "serialize error. no value for bucket-" + entry.getKey()); + } + dataFileSerializer.serializeList(entry.getValue(), view); + } + } + + private Map> deserializeMap(DataInputView view) throws IOException { + final int size = view.readInt(); + + final Map> map = + CollectionUtil.newHashMapWithExpectedSize(size); + for (int i = 0; i < size; i++) { + map.put(view.readInt(), dataFileSerializer.deserializeList(view)); + } + + return map; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactWorkerOperator.java new file mode 100644 index 000000000000..260c25a31561 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactWorkerOperator.java @@ -0,0 +1,54 @@ +/* + * 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.compact.changelog; + +import org.apache.paimon.flink.sink.Committable; +import org.apache.paimon.table.FileStoreTable; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.types.Either; + +import java.util.List; + +/** + * Receive and process the {@link ChangelogCompactTask}s emitted by {@link + * ChangelogCompactCoordinateOperator}. + */ +public class ChangelogCompactWorkerOperator extends AbstractStreamOperator + implements OneInputStreamOperator, Committable> { + private final FileStoreTable table; + + public ChangelogCompactWorkerOperator(FileStoreTable table) { + this.table = table; + } + + public void processElement(StreamRecord> record) + throws Exception { + + if (record.getValue().isLeft()) { + output.collect(new StreamRecord<>(record.getValue().left())); + } else { + ChangelogCompactTask task = record.getValue().right(); + List committables = task.doCompact(table); + committables.forEach(committable -> output.collect(new StreamRecord<>(committable))); + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java new file mode 100644 index 000000000000..5cae899a0704 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogTaskTypeInfo.java @@ -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. + */ + +package org.apache.paimon.flink.compact.changelog; + +import org.apache.paimon.flink.sink.NoneCopyVersionedSerializerTypeSerializerProxy; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +/** Type information for {@link ChangelogCompactTask}. */ +public class ChangelogTaskTypeInfo extends TypeInformation { + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 1; + } + + @Override + public int getTotalFields() { + return 1; + } + + @Override + public Class getTypeClass() { + return ChangelogCompactTask.class; + } + + @Override + public boolean isKeyType() { + return false; + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig config) { + // we don't need copy for task + return new NoneCopyVersionedSerializerTypeSerializerProxy( + ChangelogCompactTaskSerializer::new) {}; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof ChangelogTaskTypeInfo; + } + + @Override + public boolean equals(Object obj) { + return obj instanceof ChangelogTaskTypeInfo; + } + + @Override + public String toString() { + return "ChangelogCompactionTask"; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/format/CompactedChangelogFormatReaderFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/format/CompactedChangelogFormatReaderFactory.java new file mode 100644 index 000000000000..e17566f302cd --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/format/CompactedChangelogFormatReaderFactory.java @@ -0,0 +1,258 @@ +/* + * 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.compact.changelog.format; + +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fileindex.FileIndexResult; +import org.apache.paimon.format.FormatReaderFactory; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.FileStatus; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.PositionOutputStream; +import org.apache.paimon.fs.SeekableInputStream; +import org.apache.paimon.reader.RecordReader; + +import java.io.EOFException; +import java.io.IOException; + +/** + * {@link FormatReaderFactory} for compacted changelog. + * + *

    File Name Protocol + * + *

    There are two kinds of file name. In the following description, bid1 and + * bid2 are bucket id, off is offset, len1 and len2 + * are lengths. + * + *

      + *
    • bucket-bid1/compacted-changelog-xxx$bid1-len1: This is the real file name. If + * this file name is recorded in manifest file meta, reader should read the bytes of this file + * starting from offset 0 with length len1. + *
    • bucket-bid2/compacted-changelog-xxx$bid1-len1-off-len2: This is the fake file + * name. Reader should read the bytes of file + * bucket-bid1/compacted-changelog-xxx$bid1-len1 starting from offset off + * with length len2. + *
    + */ +public class CompactedChangelogFormatReaderFactory implements FormatReaderFactory { + + private final FormatReaderFactory wrapped; + + public CompactedChangelogFormatReaderFactory(FormatReaderFactory wrapped) { + this.wrapped = wrapped; + } + + @Override + public RecordReader createReader(Context context) throws IOException { + OffsetReadOnlyFileIO fileIO = new OffsetReadOnlyFileIO(context.fileIO()); + long length = decodePath(context.filePath()).length; + + return wrapped.createReader( + new Context() { + + @Override + public FileIO fileIO() { + return fileIO; + } + + @Override + public Path filePath() { + return context.filePath(); + } + + @Override + public long fileSize() { + return length; + } + + @Override + public FileIndexResult fileIndex() { + return context.fileIndex(); + } + }); + } + + private static DecodeResult decodePath(Path path) { + String[] nameAndFormat = path.getName().split("\\."); + String[] names = nameAndFormat[0].split("\\$"); + String[] split = names[1].split("-"); + if (split.length == 2) { + return new DecodeResult(path, 0, Long.parseLong(split[1])); + } else { + Path realPath = + new Path( + path.getParent().getParent(), + "bucket-" + + split[0] + + "/" + + names[0] + + "$" + + split[0] + + "-" + + split[1] + + "." + + nameAndFormat[1]); + return new DecodeResult(realPath, Long.parseLong(split[2]), Long.parseLong(split[3])); + } + } + + private static class DecodeResult { + + private final Path path; + private final long offset; + private final long length; + + private DecodeResult(Path path, long offset, long length) { + this.path = path; + this.offset = offset; + this.length = length; + } + } + + private static class OffsetReadOnlyFileIO implements FileIO { + + private final FileIO wrapped; + + private OffsetReadOnlyFileIO(FileIO wrapped) { + this.wrapped = wrapped; + } + + @Override + public boolean isObjectStore() { + return wrapped.isObjectStore(); + } + + @Override + public void configure(CatalogContext context) { + wrapped.configure(context); + } + + @Override + public SeekableInputStream newInputStream(Path path) throws IOException { + DecodeResult result = decodePath(path); + return new OffsetSeekableInputStream( + wrapped.newInputStream(result.path), result.offset, result.length); + } + + @Override + public PositionOutputStream newOutputStream(Path path, boolean overwrite) + throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public FileStatus getFileStatus(Path path) throws IOException { + DecodeResult result = decodePath(path); + FileStatus status = wrapped.getFileStatus(result.path); + + return new FileStatus() { + + @Override + public long getLen() { + return result.length; + } + + @Override + public boolean isDir() { + return status.isDir(); + } + + @Override + public Path getPath() { + return path; + } + + @Override + public long getModificationTime() { + return status.getModificationTime(); + } + }; + } + + @Override + public FileStatus[] listStatus(Path path) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean exists(Path path) throws IOException { + return wrapped.exists(decodePath(path).path); + } + + @Override + public boolean delete(Path path, boolean recursive) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean mkdirs(Path path) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean rename(Path src, Path dst) throws IOException { + throw new UnsupportedOperationException(); + } + } + + private static class OffsetSeekableInputStream extends SeekableInputStream { + + private final SeekableInputStream wrapped; + private final long offset; + private final long length; + + private OffsetSeekableInputStream(SeekableInputStream wrapped, long offset, long length) + throws IOException { + this.wrapped = wrapped; + this.offset = offset; + this.length = length; + wrapped.seek(offset); + } + + @Override + public void seek(long desired) throws IOException { + wrapped.seek(offset + desired); + } + + @Override + public long getPos() throws IOException { + return wrapped.getPos() - offset; + } + + @Override + public int read() throws IOException { + if (getPos() >= length) { + throw new EOFException(); + } + return wrapped.read(); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + long realLen = Math.min(len, length - getPos()); + return wrapped.read(b, off, (int) realLen); + } + + @Override + public void close() throws IOException { + wrapped.close(); + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/format/CompactedChangelogReadOnlyFormat.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/format/CompactedChangelogReadOnlyFormat.java new file mode 100644 index 000000000000..39bed81505c6 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/format/CompactedChangelogReadOnlyFormat.java @@ -0,0 +1,106 @@ +/* + * 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.compact.changelog.format; + +import org.apache.paimon.format.FileFormat; +import org.apache.paimon.format.FileFormatFactory; +import org.apache.paimon.format.FormatReaderFactory; +import org.apache.paimon.format.FormatWriterFactory; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.types.RowType; + +import javax.annotation.Nullable; + +import java.util.List; + +/** {@link FileFormat} for compacted changelog. */ +public class CompactedChangelogReadOnlyFormat extends FileFormat { + + private final FileFormat wrapped; + + protected CompactedChangelogReadOnlyFormat(String formatIdentifier, FileFormat wrapped) { + super(formatIdentifier); + this.wrapped = wrapped; + } + + @Override + public FormatReaderFactory createReaderFactory( + RowType projectedRowType, @Nullable List filters) { + return new CompactedChangelogFormatReaderFactory( + wrapped.createReaderFactory(projectedRowType, filters)); + } + + @Override + public FormatWriterFactory createWriterFactory(RowType type) { + throw new UnsupportedOperationException(); + } + + @Override + public void validateDataFields(RowType rowType) { + wrapped.validateDataFields(rowType); + } + + public static String getIdentifier(String wrappedFormat) { + return "cc-" + wrappedFormat; + } + + static class AbstractFactory implements FileFormatFactory { + + private final String format; + + AbstractFactory(String format) { + this.format = format; + } + + @Override + public String identifier() { + return getIdentifier(format); + } + + @Override + public FileFormat create(FormatContext formatContext) { + return new CompactedChangelogReadOnlyFormat( + getIdentifier(format), FileFormat.fromIdentifier(format, formatContext)); + } + } + + /** {@link FileFormatFactory} for compacted changelog, with orc as the real format. */ + public static class OrcFactory extends AbstractFactory { + + public OrcFactory() { + super("orc"); + } + } + + /** {@link FileFormatFactory} for compacted changelog, with parquet as the real format. */ + public static class ParquetFactory extends AbstractFactory { + + public ParquetFactory() { + super("parquet"); + } + } + + /** {@link FileFormatFactory} for compacted changelog, with avro as the real format. */ + public static class AvroFactory extends AbstractFactory { + + public AvroFactory() { + super("avro"); + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreRegister.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreRegister.java index b730d289b31b..ad501e204ce6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreRegister.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/log/LogStoreRegister.java @@ -46,11 +46,14 @@ static void registerLogSystem( ClassLoader classLoader) { Options tableOptions = Options.fromMap(options); String logStore = tableOptions.get(LOG_SYSTEM); - if (!tableOptions.get(LOG_SYSTEM).equalsIgnoreCase(NONE) - && !catalog.tableExists(identifier)) { - LogStoreRegister logStoreRegister = - getLogStoreRegister(identifier, classLoader, tableOptions, logStore); - options.putAll(logStoreRegister.registerTopic()); + if (!tableOptions.get(LOG_SYSTEM).equalsIgnoreCase(NONE)) { + try { + catalog.getTable(identifier); + } catch (Catalog.TableNotExistException e) { + LogStoreRegister logStoreRegister = + getLogStoreRegister(identifier, classLoader, tableOptions, logStore); + options.putAll(logStoreRegister.registerTopic()); + } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/CompactionDiffFollowUpScanner.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/CompactionDiffFollowUpScanner.java new file mode 100644 index 000000000000..fd909c59624b --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/CompactionDiffFollowUpScanner.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.lookup; + +import org.apache.paimon.Snapshot; +import org.apache.paimon.table.source.ScanMode; +import org.apache.paimon.table.source.snapshot.FollowUpScanner; +import org.apache.paimon.table.source.snapshot.SnapshotReader; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** {@link FollowUpScanner} for read all changed files after compact. */ +public class CompactionDiffFollowUpScanner implements FollowUpScanner { + + private static final Logger LOG = LoggerFactory.getLogger(CompactionDiffFollowUpScanner.class); + + @Override + public boolean shouldScanSnapshot(Snapshot snapshot) { + if (snapshot.commitKind() == Snapshot.CommitKind.COMPACT) { + return true; + } + + LOG.debug( + "Next snapshot id {} is not COMPACT, but is {}, check next one.", + snapshot.id(), + snapshot.commitKind()); + return false; + } + + @Override + public SnapshotReader.Plan scan(Snapshot snapshot, SnapshotReader snapshotReader) { + return snapshotReader.withMode(ScanMode.DELTA).withSnapshot(snapshot).readChanges(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java index a8660ee8c4d5..37a504c5881e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/DynamicPartitionLoader.java @@ -24,7 +24,6 @@ import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.options.Options; import org.apache.paimon.table.Table; -import org.apache.paimon.table.source.TableScan; import org.apache.paimon.types.RowType; import javax.annotation.Nullable; @@ -49,7 +48,6 @@ public class DynamicPartitionLoader implements Serializable { private final Table table; private final Duration refreshInterval; - private TableScan scan; private Comparator comparator; private LocalDateTime lastRefresh; @@ -61,7 +59,6 @@ private DynamicPartitionLoader(Table table, Duration refreshInterval) { } public void open() { - this.scan = table.newReadBuilder().newScan(); RowType partitionType = table.rowType().project(table.partitionKeys()); this.comparator = CodeGenUtils.newRecordComparator(partitionType.getFieldTypes()); } @@ -87,7 +84,10 @@ public boolean checkRefresh() { } BinaryRow previous = this.partition; - partition = scan.listPartitions().stream().max(comparator).orElse(null); + partition = + table.newReadBuilder().newScan().listPartitions().stream() + .max(comparator) + .orElse(null); lastRefresh = LocalDateTime.now(); return !Objects.equals(previous, partition); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java index 01ebbde20154..9503960fbe17 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FileStoreLookupFunction.java @@ -33,6 +33,7 @@ import org.apache.paimon.table.source.OutOfRangeException; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.FileIOUtils; +import org.apache.paimon.utils.Filter; import org.apache.paimon.utils.RowDataToObjectArrayConverter; import org.apache.paimon.shade.guava30.com.google.common.primitives.Ints; @@ -68,6 +69,7 @@ import static org.apache.paimon.CoreOptions.CONTINUOUS_DISCOVERY_INTERVAL; import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_CACHE_MODE; +import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST; import static org.apache.paimon.flink.query.RemoteTableQuery.isRemoteServiceAvailable; import static org.apache.paimon.lookup.RocksDBOptions.LOOKUP_CACHE_ROWS; import static org.apache.paimon.lookup.RocksDBOptions.LOOKUP_CONTINUOUS_DISCOVERY_INTERVAL; @@ -86,19 +88,25 @@ public class FileStoreLookupFunction implements Serializable, Closeable { private final List projectFields; private final List joinKeys; @Nullable private final Predicate predicate; + @Nullable private final RefreshBlacklist refreshBlacklist; - private transient Duration refreshInterval; private transient File path; private transient LookupTable lookupTable; - // timestamp when cache expires - private transient long nextLoadTime; + // interval of refreshing lookup table + private transient Duration refreshInterval; + // timestamp when refreshing lookup table + private transient long nextRefreshTime; protected FunctionContext functionContext; + @Nullable private Filter cacheRowFilter; + public FileStoreLookupFunction( Table table, int[] projection, int[] joinKeyIndex, @Nullable Predicate predicate) { - TableScanUtils.streamingReadingValidate(table); + if (!TableScanUtils.supportCompactDiffStreamingReading(table)) { + TableScanUtils.streamingReadingValidate(table); + } this.table = table; this.partitionLoader = DynamicPartitionLoader.of(table); @@ -126,6 +134,10 @@ public FileStoreLookupFunction( } this.predicate = predicate; + + this.refreshBlacklist = + RefreshBlacklist.create( + table.options().get(LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST.key())); } public void open(FunctionContext context) throws Exception { @@ -144,11 +156,7 @@ void open(String tmpDirectory) throws Exception { } private void open() throws Exception { - if (partitionLoader != null) { - partitionLoader.open(); - } - - this.nextLoadTime = -1; + this.nextRefreshTime = -1; Options options = Options.fromMap(table.options()); this.refreshInterval = @@ -192,7 +200,18 @@ private void open() throws Exception { this.lookupTable = FullCacheLookupTable.create(context, options.get(LOOKUP_CACHE_ROWS)); } - refreshDynamicPartition(false); + if (partitionLoader != null) { + partitionLoader.open(); + partitionLoader.checkRefresh(); + BinaryRow partition = partitionLoader.partition(); + if (partition != null) { + lookupTable.specificPartitionFilter(createSpecificPartFilter(partition)); + } + } + + if (cacheRowFilter != null) { + lookupTable.specifyCacheRowFilter(cacheRowFilter); + } lookupTable.open(); } @@ -214,15 +233,14 @@ private Predicate createProjectedPredicate(int[] projection) { public Collection lookup(RowData keyRow) { try { - checkRefresh(); + tryRefresh(); InternalRow key = new FlinkRowWrapper(keyRow); if (partitionLoader != null) { - InternalRow partition = refreshDynamicPartition(true); - if (partition == null) { + if (partitionLoader.partition() == null) { return Collections.emptyList(); } - key = JoinedRow.join(key, partition); + key = JoinedRow.join(key, partitionLoader.partition()); } List results = lookupTable.get(key); @@ -239,28 +257,6 @@ public Collection lookup(RowData keyRow) { } } - @Nullable - private BinaryRow refreshDynamicPartition(boolean reopen) throws Exception { - if (partitionLoader == null) { - return null; - } - - boolean partitionChanged = partitionLoader.checkRefresh(); - BinaryRow partition = partitionLoader.partition(); - if (partition == null) { - return null; - } - - lookupTable.specificPartitionFilter(createSpecificPartFilter(partition)); - - if (partitionChanged && reopen) { - lookupTable.close(); - lookupTable.open(); - } - - return partition; - } - private Predicate createSpecificPartFilter(BinaryRow partition) { RowType rowType = table.rowType(); List partitionKeys = table.partitionKeys(); @@ -285,20 +281,51 @@ private void reopen() { } } - private void checkRefresh() throws Exception { - if (nextLoadTime > System.currentTimeMillis()) { + @VisibleForTesting + void tryRefresh() throws Exception { + // 1. check if this time is in black list + if (refreshBlacklist != null && !refreshBlacklist.canRefresh()) { return; } - if (nextLoadTime > 0) { + + // 2. refresh dynamic partition + if (partitionLoader != null) { + boolean partitionChanged = partitionLoader.checkRefresh(); + BinaryRow partition = partitionLoader.partition(); + if (partition == null) { + // no data to be load, fast exit + return; + } + + if (partitionChanged) { + // reopen with latest partition + lookupTable.specificPartitionFilter(createSpecificPartFilter(partition)); + lookupTable.close(); + lookupTable.open(); + // no need to refresh the lookup table because it is reopened + return; + } + } + + // 3. refresh lookup table + if (shouldRefreshLookupTable()) { + lookupTable.refresh(); + nextRefreshTime = System.currentTimeMillis() + refreshInterval.toMillis(); + } + } + + private boolean shouldRefreshLookupTable() { + if (nextRefreshTime > System.currentTimeMillis()) { + return false; + } + + if (nextRefreshTime > 0) { LOG.info( "Lookup table {} has refreshed after {} second(s), refreshing", table.name(), refreshInterval.toMillis() / 1000); } - - refresh(); - - nextLoadTime = System.currentTimeMillis() + refreshInterval.toMillis(); + return true; } @VisibleForTesting @@ -306,8 +333,9 @@ LookupTable lookupTable() { return lookupTable; } - private void refresh() throws Exception { - lookupTable.refresh(); + @VisibleForTesting + long nextBlacklistCheckTime() { + return refreshBlacklist == null ? -1 : refreshBlacklist.nextBlacklistCheckTime(); } @Override @@ -359,4 +387,8 @@ protected Set getRequireCachedBucketIds() { // TODO: Implement the method when Flink support bucket shuffle for lookup join. return null; } + + protected void setCacheRowFilter(@Nullable Filter cacheRowFilter) { + this.cacheRowFilter = cacheRowFilter; + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java index 28b0da0d150c..de69c67a4c44 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/FullCacheLookupTable.java @@ -37,6 +37,7 @@ import org.apache.paimon.utils.ExecutorUtils; import org.apache.paimon.utils.FieldsComparator; import org.apache.paimon.utils.FileIOUtils; +import org.apache.paimon.utils.Filter; import org.apache.paimon.utils.MutableObjectIterator; import org.apache.paimon.utils.PartialRow; import org.apache.paimon.utils.TypeUtils; @@ -85,12 +86,14 @@ public abstract class FullCacheLookupTable implements LookupTable { private Future refreshFuture; private LookupStreamingReader reader; private Predicate specificPartition; + @Nullable private Filter cacheRowFilter; public FullCacheLookupTable(Context context) { this.table = context.table; List sequenceFields = new ArrayList<>(); + CoreOptions coreOptions = new CoreOptions(table.options()); if (table.primaryKeys().size() > 0) { - sequenceFields = new CoreOptions(table.options()).sequenceField(); + sequenceFields = coreOptions.sequenceField(); } RowType projectedType = TypeUtils.project(table.rowType(), context.projection); if (sequenceFields.size() > 0) { @@ -109,7 +112,10 @@ public FullCacheLookupTable(Context context) { projectedType = builder.build(); context = context.copy(table.rowType().getFieldIndices(projectedType.getFieldNames())); this.userDefinedSeqComparator = - UserDefinedSeqComparator.create(projectedType, sequenceFields); + UserDefinedSeqComparator.create( + projectedType, + sequenceFields, + coreOptions.sequenceFieldSortOrderIsAscending()); this.appendUdsFieldNumber = appendUdsFieldNumber.get(); } else { this.userDefinedSeqComparator = null; @@ -138,6 +144,11 @@ public void specificPartitionFilter(Predicate filter) { this.specificPartition = filter; } + @Override + public void specifyCacheRowFilter(Filter filter) { + this.cacheRowFilter = filter; + } + protected void openStateFactory() throws Exception { this.stateFactory = new RocksDBStateFactory( @@ -154,7 +165,8 @@ protected void bootstrap() throws Exception { context.table, context.projection, scanPredicate, - context.requiredCachedBucketIds); + context.requiredCachedBucketIds, + cacheRowFilter); BinaryExternalSortBuffer bulkLoadSorter = RocksDBState.createBulkLoadSorter( IOManager.create(context.tempPath.toString()), context.table.coreOptions()); @@ -334,7 +346,7 @@ static FullCacheLookupTable create(Context context, long lruCacheSize) { /** Context for {@link LookupTable}. */ public static class Context { - public final FileStoreTable table; + public final LookupFileStoreTable table; public final int[] projection; @Nullable public final Predicate tablePredicate; @Nullable public final Predicate projectedPredicate; @@ -350,7 +362,7 @@ public Context( File tempPath, List joinKey, @Nullable Set requiredCachedBucketIds) { - this.table = table; + this.table = new LookupFileStoreTable(table, joinKey); this.projection = projection; this.tablePredicate = tablePredicate; this.projectedPredicate = projectedPredicate; @@ -361,7 +373,7 @@ public Context( public Context copy(int[] newProjection) { return new Context( - table, + table.wrapped(), newProjection, tablePredicate, projectedPredicate, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/IncrementalCompactDiffSplitRead.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/IncrementalCompactDiffSplitRead.java new file mode 100644 index 000000000000..fef74127e7f4 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/IncrementalCompactDiffSplitRead.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.lookup; + +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.operation.MergeFileSplitRead; +import org.apache.paimon.operation.SplitRead; +import org.apache.paimon.reader.EmptyRecordReader; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.splitread.IncrementalDiffSplitRead; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; + +/** A {@link SplitRead} for streaming incremental diff after compaction. */ +public class IncrementalCompactDiffSplitRead extends IncrementalDiffSplitRead { + + public IncrementalCompactDiffSplitRead(MergeFileSplitRead mergeRead) { + super(mergeRead); + } + + @Override + public RecordReader createReader(DataSplit split) throws IOException { + if (split.beforeFiles().stream().noneMatch(file -> file.level() == 0)) { + return new EmptyRecordReader<>(); + } + return super.createReader(filterLevel0Files(split)); + } + + private DataSplit filterLevel0Files(DataSplit split) { + List beforeFiles = + split.beforeFiles().stream() + .filter(file -> file.level() > 0) + .collect(Collectors.toList()); + List afterFiles = + split.dataFiles().stream() + .filter(file -> file.level() > 0) + .collect(Collectors.toList()); + DataSplit.Builder builder = + new DataSplit.Builder() + .withSnapshot(split.snapshotId()) + .withPartition(split.partition()) + .withBucket(split.bucket()) + .withBucketPath(split.bucketPath()) + .withBeforeFiles(beforeFiles) + .withDataFiles(afterFiles) + .isStreaming(split.isStreaming()) + .rawConvertible(split.rawConvertible()); + + if (split.beforeDeletionFiles().isPresent()) { + builder.withBeforeDeletionFiles(split.beforeDeletionFiles().get()); + } + if (split.deletionFiles().isPresent()) { + builder.withDataDeletionFiles(split.deletionFiles().get()); + } + return builder.build(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java new file mode 100644 index 000000000000..2a140adc3240 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupCompactDiffRead.java @@ -0,0 +1,88 @@ +/* + * 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.lookup; + +import org.apache.paimon.KeyValue; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.disk.IOManager; +import org.apache.paimon.operation.MergeFileSplitRead; +import org.apache.paimon.operation.SplitRead; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.source.AbstractDataTableRead; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.InnerTableRead; +import org.apache.paimon.table.source.Split; +import org.apache.paimon.table.source.TableRead; +import org.apache.paimon.types.RowType; + +import java.io.IOException; + +import static org.apache.paimon.table.source.KeyValueTableRead.unwrap; + +/** An {@link InnerTableRead} that reads the data changed before and after compaction. */ +public class LookupCompactDiffRead extends AbstractDataTableRead { + private final SplitRead fullPhaseMergeRead; + private final SplitRead incrementalDiffRead; + + public LookupCompactDiffRead(MergeFileSplitRead mergeRead, TableSchema schema) { + super(schema); + this.incrementalDiffRead = new IncrementalCompactDiffSplitRead(mergeRead); + this.fullPhaseMergeRead = + SplitRead.convert(mergeRead, split -> unwrap(mergeRead.createReader(split))); + } + + @Override + public void applyReadType(RowType readType) { + fullPhaseMergeRead.withReadType(readType); + incrementalDiffRead.withReadType(readType); + } + + @Override + public RecordReader reader(Split split) throws IOException { + DataSplit dataSplit = (DataSplit) split; + if (dataSplit.beforeFiles().isEmpty()) { + return fullPhaseMergeRead.createReader(dataSplit); // full reading phase + } else { + return incrementalDiffRead.createReader((DataSplit) split); + } + } + + @Override + protected InnerTableRead innerWithFilter(Predicate predicate) { + fullPhaseMergeRead.withFilter(predicate); + incrementalDiffRead.withFilter(predicate); + return this; + } + + @Override + public InnerTableRead forceKeepDelete() { + fullPhaseMergeRead.forceKeepDelete(); + incrementalDiffRead.forceKeepDelete(); + return this; + } + + @Override + public TableRead withIOManager(IOManager ioManager) { + fullPhaseMergeRead.withIOManager(ioManager); + incrementalDiffRead.withIOManager(ioManager); + return this; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupDataTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupDataTableScan.java new file mode 100644 index 000000000000..908884a573c0 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupDataTableScan.java @@ -0,0 +1,87 @@ +/* + * 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.lookup; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.operation.DefaultValueAssigner; +import org.apache.paimon.table.source.DataTableStreamScan; +import org.apache.paimon.table.source.snapshot.AllDeltaFollowUpScanner; +import org.apache.paimon.table.source.snapshot.BoundedChecker; +import org.apache.paimon.table.source.snapshot.FollowUpScanner; +import org.apache.paimon.table.source.snapshot.FullStartingScanner; +import org.apache.paimon.table.source.snapshot.SnapshotReader; +import org.apache.paimon.table.source.snapshot.StartingScanner; +import org.apache.paimon.utils.SnapshotManager; + +import static org.apache.paimon.CoreOptions.StartupMode; +import static org.apache.paimon.flink.lookup.LookupFileStoreTable.LookupStreamScanMode; + +/** + * {@link org.apache.paimon.table.source.StreamTableScan} implementation for lookup streaming + * planning. + */ +public class LookupDataTableScan extends DataTableStreamScan { + + private final StartupMode startupMode; + private final LookupStreamScanMode lookupScanMode; + + public LookupDataTableScan( + CoreOptions options, + SnapshotReader snapshotReader, + SnapshotManager snapshotManager, + boolean supportStreamingReadOverwrite, + DefaultValueAssigner defaultValueAssigner, + LookupStreamScanMode lookupScanMode) { + super( + options, + snapshotReader, + snapshotManager, + supportStreamingReadOverwrite, + defaultValueAssigner); + this.startupMode = options.startupMode(); + this.lookupScanMode = lookupScanMode; + } + + @Override + protected StartingScanner createStartingScanner(boolean isStreaming) { + return startupMode != CoreOptions.StartupMode.COMPACTED_FULL + ? new FullStartingScanner(snapshotReader.snapshotManager()) + : super.createStartingScanner(isStreaming); + } + + @Override + protected FollowUpScanner createFollowUpScanner() { + switch (lookupScanMode) { + case CHANGELOG: + return super.createFollowUpScanner(); + case FILE_MONITOR: + return new AllDeltaFollowUpScanner(); + case COMPACT_DELTA_MONITOR: + return new CompactionDiffFollowUpScanner(); + default: + throw new UnsupportedOperationException( + "Unknown lookup stream scan mode: " + lookupScanMode.name()); + } + } + + @Override + protected BoundedChecker createBoundedChecker() { + return BoundedChecker.neverEnd(); // dim table should never end + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java new file mode 100644 index 000000000000..090399706330 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java @@ -0,0 +1,152 @@ +/* + * 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.lookup; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.KeyValueFileStore; +import org.apache.paimon.flink.FlinkConnectorOptions; +import org.apache.paimon.flink.utils.TableScanUtils; +import org.apache.paimon.operation.DefaultValueAssigner; +import org.apache.paimon.options.Options; +import org.apache.paimon.options.description.DescribedEnum; +import org.apache.paimon.options.description.InlineElement; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.DelegatedFileStoreTable; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.source.InnerTableRead; +import org.apache.paimon.table.source.StreamDataTableScan; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.CoreOptions.CHANGELOG_PRODUCER; +import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_CACHE_MODE; +import static org.apache.paimon.options.description.TextElement.text; + +/** {@link FileStoreTable} for lookup table. */ +public class LookupFileStoreTable extends DelegatedFileStoreTable { + + private static final long serialVersionUID = 1L; + + private final LookupStreamScanMode lookupScanMode; + + public LookupFileStoreTable(FileStoreTable wrapped, List joinKeys) { + super(wrapped); + this.lookupScanMode = lookupStreamScanMode(wrapped, joinKeys); + } + + public LookupFileStoreTable(FileStoreTable wrapped, LookupStreamScanMode lookupScanMode) { + super(wrapped); + this.lookupScanMode = lookupScanMode; + } + + @Override + public InnerTableRead newRead() { + switch (lookupScanMode) { + case CHANGELOG: + case FILE_MONITOR: + return wrapped.newRead(); + case COMPACT_DELTA_MONITOR: + return new LookupCompactDiffRead( + ((KeyValueFileStore) wrapped.store()).newRead(), wrapped.schema()); + default: + throw new UnsupportedOperationException( + "Unknown lookup stream scan mode: " + lookupScanMode.name()); + } + } + + @Override + public StreamDataTableScan newStreamScan() { + return new LookupDataTableScan( + wrapped.coreOptions(), + wrapped.newSnapshotReader(), + wrapped.snapshotManager(), + wrapped.supportStreamingReadOverwrite(), + DefaultValueAssigner.create(wrapped.schema()), + lookupScanMode); + } + + @Override + public FileStoreTable copy(Map dynamicOptions) { + return new LookupFileStoreTable(wrapped.copy(dynamicOptions), lookupScanMode); + } + + @Override + public FileStoreTable copy(TableSchema newTableSchema) { + return new LookupFileStoreTable(wrapped.copy(newTableSchema), lookupScanMode); + } + + @Override + public FileStoreTable copyWithoutTimeTravel(Map dynamicOptions) { + return new LookupFileStoreTable(wrapped.copy(dynamicOptions), lookupScanMode); + } + + @Override + public FileStoreTable copyWithLatestSchema() { + return new LookupFileStoreTable(wrapped.copyWithLatestSchema(), lookupScanMode); + } + + @Override + public FileStoreTable switchToBranch(String branchName) { + wrapped.switchToBranch(branchName); + return this; + } + + private LookupStreamScanMode lookupStreamScanMode(FileStoreTable table, List joinKeys) { + Options options = Options.fromMap(table.options()); + if (options.get(LOOKUP_CACHE_MODE) == FlinkConnectorOptions.LookupCacheMode.AUTO + && new HashSet<>(table.primaryKeys()).equals(new HashSet<>(joinKeys))) { + return LookupStreamScanMode.FILE_MONITOR; + } else if (table.primaryKeys().size() > 0 + && options.get(CHANGELOG_PRODUCER) == CoreOptions.ChangelogProducer.NONE + && TableScanUtils.supportCompactDiffStreamingReading(table)) { + return LookupStreamScanMode.COMPACT_DELTA_MONITOR; + } else { + return LookupStreamScanMode.CHANGELOG; + } + } + + /** Inner stream scan mode for lookup table. */ + public enum LookupStreamScanMode implements DescribedEnum { + CHANGELOG("changelog", "Streaming reading based on changelog or delta data files."), + FILE_MONITOR("file-monitor", "Monitor data file changes."), + COMPACT_DELTA_MONITOR( + "compact-delta-monitor", + "Streaming reading based on data changes before and after compaction."); + + private final String value; + private final String description; + + LookupStreamScanMode(String value, String description) { + this.value = value; + this.description = description; + } + + @Override + public String toString() { + return value; + } + + @Override + public InlineElement getDescription() { + return text(description); + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupStreamingReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupStreamingReader.java index ceb40c1a864f..132b30138d0a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupStreamingReader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupStreamingReader.java @@ -22,18 +22,15 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.io.SplitsParallelReadUtil; import org.apache.paimon.mergetree.compact.ConcatRecordReader; -import org.apache.paimon.options.ConfigOption; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.ReaderSupplier; import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.table.Table; import org.apache.paimon.table.source.ReadBuilder; import org.apache.paimon.table.source.Split; import org.apache.paimon.table.source.StreamTableScan; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.Filter; import org.apache.paimon.utils.FunctionWithIOException; import org.apache.paimon.utils.TypeUtils; @@ -42,10 +39,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.function.IntUnaryOperator; import java.util.stream.IntStream; @@ -56,27 +50,22 @@ /** A streaming reader to load data into {@link LookupTable}. */ public class LookupStreamingReader { - private final Table table; + private final LookupFileStoreTable table; private final int[] projection; + @Nullable private final Filter cacheRowFilter; private final ReadBuilder readBuilder; @Nullable private final Predicate projectedPredicate; private final StreamTableScan scan; - private static final List> TIME_TRAVEL_OPTIONS = - Arrays.asList( - CoreOptions.SCAN_TIMESTAMP_MILLIS, - CoreOptions.SCAN_FILE_CREATION_TIME_MILLIS, - CoreOptions.SCAN_SNAPSHOT_ID, - CoreOptions.SCAN_TAG_NAME, - CoreOptions.SCAN_VERSION); - public LookupStreamingReader( - Table table, + LookupFileStoreTable table, int[] projection, @Nullable Predicate predicate, - Set requireCachedBucketIds) { - this.table = unsetTimeTravelOptions(table); + Set requireCachedBucketIds, + @Nullable Filter cacheRowFilter) { + this.table = table; this.projection = projection; + this.cacheRowFilter = cacheRowFilter; this.readBuilder = this.table .newReadBuilder() @@ -112,21 +101,6 @@ public LookupStreamingReader( } } - private Table unsetTimeTravelOptions(Table origin) { - FileStoreTable fileStoreTable = (FileStoreTable) origin; - Map newOptions = new HashMap<>(fileStoreTable.options()); - TIME_TRAVEL_OPTIONS.stream().map(ConfigOption::key).forEach(newOptions::remove); - - CoreOptions.StartupMode startupMode = CoreOptions.fromMap(newOptions).startupMode(); - if (startupMode != CoreOptions.StartupMode.COMPACTED_FULL) { - startupMode = CoreOptions.StartupMode.LATEST_FULL; - } - newOptions.put(CoreOptions.SCAN_MODE.key(), startupMode.toString()); - - TableSchema newSchema = fileStoreTable.schema().copy(newOptions); - return fileStoreTable.copy(newSchema); - } - public RecordReader nextBatch(boolean useParallelism) throws Exception { List splits = scan.plan().splits(); CoreOptions options = CoreOptions.fromMap(table.options()); @@ -155,6 +129,10 @@ public RecordReader nextBatch(boolean useParallelism) throws Except if (projectedPredicate != null) { reader = reader.filter(projectedPredicate::test); } + + if (cacheRowFilter != null) { + reader = reader.filter(cacheRowFilter); + } return reader; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupTable.java index c13947bbd819..8ea1931b96aa 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupTable.java @@ -20,6 +20,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.utils.Filter; import java.io.Closeable; import java.io.IOException; @@ -35,4 +36,6 @@ public interface LookupTable extends Closeable { List get(InternalRow key) throws IOException; void refresh() throws Exception; + + void specifyCacheRowFilter(Filter filter); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyPartialLookupTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyPartialLookupTable.java index bdf0a1b4af77..ef5543ac9b7c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyPartialLookupTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/PrimaryKeyPartialLookupTable.java @@ -31,8 +31,8 @@ import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; import org.apache.paimon.table.source.StreamTableScan; +import org.apache.paimon.utils.Filter; import org.apache.paimon.utils.ProjectedRow; -import org.apache.paimon.utils.Projection; import javax.annotation.Nullable; @@ -40,31 +40,23 @@ import java.io.File; import java.io.IOException; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Set; -import java.util.function.Function; - -import static org.apache.paimon.CoreOptions.SCAN_BOUNDED_WATERMARK; -import static org.apache.paimon.CoreOptions.STREAM_SCAN_MODE; -import static org.apache.paimon.CoreOptions.StreamScanMode.FILE_MONITOR; /** Lookup table for primary key which supports to read the LSM tree directly. */ public class PrimaryKeyPartialLookupTable implements LookupTable { - private final Function executorFactory; + private final QueryExecutorFactory executorFactory; private final FixedBucketFromPkExtractor extractor; @Nullable private final ProjectedRow keyRearrange; @Nullable private final ProjectedRow trimmedKeyRearrange; private Predicate specificPartition; + @Nullable private Filter cacheRowFilter; private QueryExecutor queryExecutor; private PrimaryKeyPartialLookupTable( - Function executorFactory, - FileStoreTable table, - List joinKey) { + QueryExecutorFactory executorFactory, FileStoreTable table, List joinKey) { this.executorFactory = executorFactory; if (table.bucketMode() != BucketMode.HASH_FIXED) { @@ -110,7 +102,7 @@ public void specificPartitionFilter(Predicate filter) { @Override public void open() throws Exception { - this.queryExecutor = executorFactory.apply(specificPartition); + this.queryExecutor = executorFactory.create(specificPartition, cacheRowFilter); refresh(); } @@ -142,6 +134,11 @@ public void refresh() { queryExecutor.refresh(); } + @Override + public void specifyCacheRowFilter(Filter filter) { + this.cacheRowFilter = filter; + } + @Override public void close() throws IOException { if (queryExecutor != null) { @@ -156,9 +153,14 @@ public static PrimaryKeyPartialLookupTable createLocalTable( List joinKey, Set requireCachedBucketIds) { return new PrimaryKeyPartialLookupTable( - filter -> + (filter, cacheRowFilter) -> new LocalQueryExecutor( - table, projection, tempPath, filter, requireCachedBucketIds), + new LookupFileStoreTable(table, joinKey), + projection, + tempPath, + filter, + requireCachedBucketIds, + cacheRowFilter), table, joinKey); } @@ -166,7 +168,13 @@ public static PrimaryKeyPartialLookupTable createLocalTable( public static PrimaryKeyPartialLookupTable createRemoteTable( FileStoreTable table, int[] projection, List joinKey) { return new PrimaryKeyPartialLookupTable( - filter -> new RemoteQueryExecutor(table, projection), table, joinKey); + (filter, cacheRowFilter) -> new RemoteQueryExecutor(table, projection), + table, + joinKey); + } + + interface QueryExecutorFactory { + QueryExecutor create(Predicate filter, @Nullable Filter cacheRowFilter); } interface QueryExecutor extends Closeable { @@ -186,18 +194,19 @@ private LocalQueryExecutor( int[] projection, File tempPath, @Nullable Predicate filter, - Set requireCachedBucketIds) { + Set requireCachedBucketIds, + @Nullable Filter cacheRowFilter) { this.tableQuery = table.newLocalTableQuery() - .withValueProjection(Projection.of(projection).toNestedIndexes()) + .withValueProjection(projection) .withIOManager(new IOManagerImpl(tempPath.toString())); - Map dynamicOptions = new HashMap<>(); - dynamicOptions.put(STREAM_SCAN_MODE.key(), FILE_MONITOR.getValue()); - dynamicOptions.put(SCAN_BOUNDED_WATERMARK.key(), null); + if (cacheRowFilter != null) { + this.tableQuery.withCacheRowFilter(cacheRowFilter); + } + this.scan = - table.copy(dynamicOptions) - .newReadBuilder() + table.newReadBuilder() .withFilter(filter) .withBucketFilter( requireCachedBucketIds == null diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/RefreshBlacklist.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/RefreshBlacklist.java new file mode 100644 index 000000000000..e20294fe0676 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/RefreshBlacklist.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.flink.lookup; + +import org.apache.paimon.utils.DateTimeUtils; +import org.apache.paimon.utils.Pair; +import org.apache.paimon.utils.StringUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.time.format.DateTimeParseException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.TimeZone; + +/** Refresh black list for {@link FileStoreLookupFunction}. */ +public class RefreshBlacklist { + + private static final Logger LOG = LoggerFactory.getLogger(RefreshBlacklist.class); + + private final List> timePeriodsBlacklist; + + private long nextBlacklistCheckTime; + + public RefreshBlacklist(List> timePeriodsBlacklist) { + this.timePeriodsBlacklist = timePeriodsBlacklist; + this.nextBlacklistCheckTime = -1; + } + + @Nullable + public static RefreshBlacklist create(String blacklist) { + List> timePeriodsBlacklist = parseTimePeriodsBlacklist(blacklist); + if (timePeriodsBlacklist.isEmpty()) { + return null; + } + + return new RefreshBlacklist(timePeriodsBlacklist); + } + + private static List> parseTimePeriodsBlacklist(String blacklist) { + if (StringUtils.isNullOrWhitespaceOnly(blacklist)) { + return Collections.emptyList(); + } + String[] timePeriods = blacklist.split(","); + List> result = new ArrayList<>(); + for (String period : timePeriods) { + String[] times = period.split("->"); + if (times.length != 2) { + throw new IllegalArgumentException( + String.format("Incorrect time periods format: [%s].", blacklist)); + } + + long left = parseToMillis(times[0]); + long right = parseToMillis(times[1]); + if (left > right) { + throw new IllegalArgumentException( + String.format("Incorrect time period: [%s->%s].", times[0], times[1])); + } + result.add(Pair.of(left, right)); + } + return result; + } + + private static long parseToMillis(String dateTime) { + try { + return DateTimeUtils.parseTimestampData(dateTime + ":00", 3, TimeZone.getDefault()) + .getMillisecond(); + } catch (DateTimeParseException e) { + throw new IllegalArgumentException( + String.format("Date time format error: [%s].", dateTime), e); + } + } + + public boolean canRefresh() { + long currentTimeMillis = System.currentTimeMillis(); + if (currentTimeMillis < nextBlacklistCheckTime) { + return false; + } + + Pair selectedPeriod = null; + for (Pair period : timePeriodsBlacklist) { + if (period.getLeft() <= currentTimeMillis && currentTimeMillis <= period.getRight()) { + selectedPeriod = period; + break; + } + } + + if (selectedPeriod != null) { + LOG.info( + "Current time {} is in black list {}-{}, so try to refresh cache next time.", + currentTimeMillis, + selectedPeriod.getLeft(), + selectedPeriod.getRight()); + nextBlacklistCheckTime = selectedPeriod.getRight() + 1; + return false; + } + + return true; + } + + public long nextBlacklistCheckTime() { + return nextBlacklistCheckTime; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactManifestProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactManifestProcedure.java new file mode 100644 index 000000000000..3e52322a6f58 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactManifestProcedure.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.procedure; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.operation.FileStoreCommit; +import org.apache.paimon.table.FileStoreTable; + +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.ProcedureHint; +import org.apache.flink.table.procedure.ProcedureContext; + +import java.util.Collections; + +/** Compact manifest file to reduce deleted manifest entries. */ +public class CompactManifestProcedure extends ProcedureBase { + + private static final String COMMIT_USER = "Compact-Manifest-Procedure-Committer"; + + @Override + public String identifier() { + return "compact_manifest"; + } + + @ProcedureHint(argument = {@ArgumentHint(name = "table", type = @DataTypeHint("STRING"))}) + public String[] call(ProcedureContext procedureContext, String tableId) throws Exception { + + FileStoreTable table = + (FileStoreTable) + table(tableId) + .copy( + Collections.singletonMap( + CoreOptions.COMMIT_USER_PREFIX.key(), COMMIT_USER)); + + try (FileStoreCommit commit = + table.store() + .newCommit(table.coreOptions().createCommitUser()) + .ignoreEmptyCommit(false)) { + commit.compactManifest(); + } + return new String[] {"success"}; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CreateOrReplaceTagBaseProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CreateOrReplaceTagBaseProcedure.java new file mode 100644 index 000000000000..dba9d46636e6 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CreateOrReplaceTagBaseProcedure.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.procedure; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.table.Table; +import org.apache.paimon.utils.TimeUtils; + +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.ProcedureHint; +import org.apache.flink.table.procedure.ProcedureContext; + +import javax.annotation.Nullable; + +import java.time.Duration; + +/** A base procedure to create or replace a tag. */ +public abstract class CreateOrReplaceTagBaseProcedure extends ProcedureBase { + + @ProcedureHint( + argument = { + @ArgumentHint(name = "table", type = @DataTypeHint("STRING")), + @ArgumentHint(name = "tag", type = @DataTypeHint("STRING")), + @ArgumentHint( + name = "snapshot_id", + type = @DataTypeHint("BIGINT"), + isOptional = true), + @ArgumentHint( + name = "time_retained", + type = @DataTypeHint("STRING"), + isOptional = true) + }) + public String[] call( + ProcedureContext procedureContext, + String tableId, + String tagName, + @Nullable Long snapshotId, + @Nullable String timeRetained) + throws Catalog.TableNotExistException { + Table table = catalog.getTable(Identifier.fromString(tableId)); + createOrReplaceTag(table, tagName, snapshotId, toDuration(timeRetained)); + return new String[] {"Success"}; + } + + abstract void createOrReplaceTag( + Table table, String tagName, Long snapshotId, Duration timeRetained); + + @Nullable + private static Duration toDuration(@Nullable String s) { + if (s == null) { + return null; + } + + return TimeUtils.parseDuration(s); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CreateTagProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CreateTagProcedure.java index 3fb51c8d935c..b1af1c93942f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CreateTagProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CreateTagProcedure.java @@ -18,17 +18,7 @@ package org.apache.paimon.flink.procedure; -import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.catalog.Identifier; import org.apache.paimon.table.Table; -import org.apache.paimon.utils.TimeUtils; - -import org.apache.flink.table.annotation.ArgumentHint; -import org.apache.flink.table.annotation.DataTypeHint; -import org.apache.flink.table.annotation.ProcedureHint; -import org.apache.flink.table.procedure.ProcedureContext; - -import javax.annotation.Nullable; import java.time.Duration; @@ -39,46 +29,17 @@ * CALL sys.create_tag('tableId', 'tagName', snapshotId, 'timeRetained') * */ -public class CreateTagProcedure extends ProcedureBase { +public class CreateTagProcedure extends CreateOrReplaceTagBaseProcedure { public static final String IDENTIFIER = "create_tag"; - @ProcedureHint( - argument = { - @ArgumentHint(name = "table", type = @DataTypeHint("STRING")), - @ArgumentHint(name = "tag", type = @DataTypeHint("STRING")), - @ArgumentHint( - name = "snapshot_id", - type = @DataTypeHint("BIGINT"), - isOptional = true), - @ArgumentHint( - name = "time_retained", - type = @DataTypeHint("STRING"), - isOptional = true) - }) - public String[] call( - ProcedureContext procedureContext, - String tableId, - String tagName, - @Nullable Long snapshotId, - @Nullable String timeRetained) - throws Catalog.TableNotExistException { - Table table = catalog.getTable(Identifier.fromString(tableId)); + @Override + void createOrReplaceTag(Table table, String tagName, Long snapshotId, Duration timeRetained) { if (snapshotId == null) { - table.createTag(tagName, toDuration(timeRetained)); + table.createTag(tagName, timeRetained); } else { - table.createTag(tagName, snapshotId, toDuration(timeRetained)); + table.createTag(tagName, snapshotId, timeRetained); } - return new String[] {"Success"}; - } - - @Nullable - private static Duration toDuration(@Nullable String s) { - if (s == null) { - return null; - } - - return TimeUtils.parseDuration(s); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java index abbf4f48698b..ee6075a927d3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java @@ -63,6 +63,10 @@ public String identifier() { name = "expire_strategy", type = @DataTypeHint("STRING"), isOptional = true), + @ArgumentHint( + name = "max_expires", + type = @DataTypeHint("INTEGER"), + isOptional = true) }) public @DataTypeHint("ROW< expired_partitions STRING>") Row[] call( ProcedureContext procedureContext, @@ -70,7 +74,8 @@ public String identifier() { String expirationTime, String timestampFormatter, String timestampPattern, - String expireStrategy) + String expireStrategy, + Integer maxExpires) throws Catalog.TableNotExistException { FileStoreTable fileStoreTable = (FileStoreTable) table(tableId); FileStore fileStore = fileStoreTable.store(); @@ -93,6 +98,9 @@ public String identifier() { .metastoreClientFactory()) .map(MetastoreClient.Factory::create) .orElse(null)); + if (maxExpires != null) { + partitionExpire.withMaxExpires(maxExpires); + } List> expired = partitionExpire.expire(Long.MAX_VALUE); return expired == null || expired.isEmpty() ? new Row[] {Row.of("No expired partitions.")} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ExpireTagsProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ExpireTagsProcedure.java new file mode 100644 index 000000000000..3d8af1de70cc --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ExpireTagsProcedure.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.procedure; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.tag.TagTimeExpire; +import org.apache.paimon.utils.DateTimeUtils; + +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.ProcedureHint; +import org.apache.flink.table.procedure.ProcedureContext; +import org.apache.flink.types.Row; + +import javax.annotation.Nullable; + +import java.time.LocalDateTime; +import java.util.List; +import java.util.TimeZone; + +/** A procedure to expire tags by time. */ +public class ExpireTagsProcedure extends ProcedureBase { + + private static final String IDENTIFIER = "expire_tags"; + + @ProcedureHint( + argument = { + @ArgumentHint(name = "table", type = @DataTypeHint("STRING")), + @ArgumentHint( + name = "older_than", + type = @DataTypeHint("STRING"), + isOptional = true) + }) + public @DataTypeHint("ROW") Row[] call( + ProcedureContext procedureContext, String tableId, @Nullable String olderThanStr) + throws Catalog.TableNotExistException { + FileStoreTable fileStoreTable = (FileStoreTable) table(tableId); + TagTimeExpire tagTimeExpire = + fileStoreTable.store().newTagCreationManager().getTagTimeExpire(); + if (olderThanStr != null) { + LocalDateTime olderThanTime = + DateTimeUtils.parseTimestampData(olderThanStr, 3, TimeZone.getDefault()) + .toLocalDateTime(); + tagTimeExpire.withOlderThanTime(olderThanTime); + } + List expired = tagTimeExpire.expire(); + return expired.isEmpty() + ? new Row[] {Row.of("No expired tags.")} + : expired.stream().map(Row::of).toArray(Row[]::new); + } + + @Override + public String identifier() { + return IDENTIFIER; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java index cf8d7191953e..e297c0bdbb4c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MergeIntoProcedure.java @@ -102,7 +102,19 @@ public class MergeIntoProcedure extends ProcedureBase { @ArgumentHint( name = "matched_delete_condition", type = @DataTypeHint("STRING"), - isOptional = true) + isOptional = true), + @ArgumentHint( + name = "not_matched_by_source_upsert_condition", + type = @DataTypeHint("STRING"), + isOptional = true), + @ArgumentHint( + name = "not_matched_by_source_upsert_setting", + type = @DataTypeHint("STRING"), + isOptional = true), + @ArgumentHint( + name = "not_matched_by_source_delete_condition", + type = @DataTypeHint("STRING"), + isOptional = true), }) public String[] call( ProcedureContext procedureContext, @@ -115,7 +127,10 @@ public String[] call( String matchedUpsertSetting, String notMatchedInsertCondition, String notMatchedInsertValues, - String matchedDeleteCondition) { + String matchedDeleteCondition, + String notMatchedBySourceUpsertCondition, + String notMatchedBySourceUpsertSetting, + String notMatchedBySourceDeleteCondition) { targetAlias = notnull(targetAlias); sourceSqls = notnull(sourceSqls); sourceTable = notnull(sourceTable); @@ -125,6 +140,9 @@ public String[] call( notMatchedInsertCondition = notnull(notMatchedInsertCondition); notMatchedInsertValues = notnull(notMatchedInsertValues); matchedDeleteCondition = notnull(matchedDeleteCondition); + notMatchedBySourceUpsertCondition = notnull(notMatchedBySourceUpsertCondition); + notMatchedBySourceUpsertSetting = notnull(notMatchedBySourceUpsertSetting); + notMatchedBySourceDeleteCondition = notnull(notMatchedBySourceDeleteCondition); String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); @@ -166,6 +184,20 @@ public String[] call( action.withMatchedDelete(matchedDeleteCondition); } + if (!notMatchedBySourceUpsertCondition.isEmpty() + || !notMatchedBySourceUpsertSetting.isEmpty()) { + String condition = nullable(notMatchedBySourceUpsertCondition); + String values = nullable(notMatchedBySourceUpsertSetting); + checkArgument( + !"*".equals(values), + "not-matched-by-source-upsert does not support setting notMatchedBySourceUpsertSetting to *."); + action.withNotMatchedBySourceUpsert(condition, values); + } + + if (!notMatchedBySourceDeleteCondition.isEmpty()) { + action.withNotMatchedBySourceDelete(notMatchedBySourceDeleteCondition); + } + action.withStreamExecutionEnvironment(procedureContext.getExecutionEnvironment()); action.validate(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateDatabaseProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateDatabaseProcedure.java index 936d9069e3b2..8c6e8730b0c2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateDatabaseProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateDatabaseProcedure.java @@ -26,12 +26,16 @@ import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.annotation.ProcedureHint; import org.apache.flink.table.procedure.ProcedureContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.List; /** Migrate procedure to migrate all hive tables in database to paimon table. */ public class MigrateDatabaseProcedure extends ProcedureBase { + private static final Logger LOG = LoggerFactory.getLogger(MigrateDatabaseProcedure.class); + @Override public String identifier() { return "migrate_database"; @@ -64,11 +68,24 @@ public String[] call( p, ParameterUtils.parseCommaSeparatedKeyValues(properties)); + int errorCount = 0; + int successCount = 0; + for (Migrator migrator : migrators) { - migrator.executeMigrate(); - migrator.renameTable(false); + try { + migrator.executeMigrate(); + migrator.renameTable(false); + successCount++; + } catch (Exception e) { + errorCount++; + LOG.error("Call migrate_database error:" + e.getMessage()); + } } + String retStr = + String.format( + "migrate database is finished, success cnt: %s , failed cnt: %s", + String.valueOf(successCount), String.valueOf(errorCount)); - return new String[] {"Success"}; + return new String[] {retStr}; } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java index 34b016fe0d36..f2f10d087406 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/MigrateFileProcedure.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.procedure; +import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.utils.TableMigrationUtils; import org.apache.paimon.migrate.Migrator; @@ -77,7 +78,9 @@ public void migrateHandle( Identifier sourceTableId = Identifier.fromString(sourceTablePath); Identifier targetTableId = Identifier.fromString(targetPaimonTablePath); - if (!(catalog.tableExists(targetTableId))) { + try { + catalog.getTable(targetTableId); + } catch (Catalog.TableNotExistException e) { throw new IllegalArgumentException( "Target paimon table does not exist: " + targetPaimonTablePath); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java index 9bc20350d447..10ad878e0ccb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java @@ -20,12 +20,15 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.orphan.FlinkOrphanFilesClean; +import org.apache.paimon.operation.LocalOrphanFilesClean; import org.apache.flink.table.annotation.ArgumentHint; import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.annotation.ProcedureHint; import org.apache.flink.table.procedure.ProcedureContext; +import java.util.Locale; + import static org.apache.paimon.operation.OrphanFilesClean.createFileCleaner; import static org.apache.paimon.operation.OrphanFilesClean.olderThanMillis; @@ -55,29 +58,57 @@ public class RemoveOrphanFilesProcedure extends ProcedureBase { type = @DataTypeHint("STRING"), isOptional = true), @ArgumentHint(name = "dry_run", type = @DataTypeHint("BOOLEAN"), isOptional = true), - @ArgumentHint(name = "parallelism", type = @DataTypeHint("INT"), isOptional = true) + @ArgumentHint(name = "parallelism", type = @DataTypeHint("INT"), isOptional = true), + @ArgumentHint(name = "mode", type = @DataTypeHint("STRING"), isOptional = true) }) public String[] call( ProcedureContext procedureContext, String tableId, String olderThan, Boolean dryRun, - Integer parallelism) + Integer parallelism, + String mode) throws Exception { Identifier identifier = Identifier.fromString(tableId); String databaseName = identifier.getDatabaseName(); String tableName = identifier.getObjectName(); - - long deleted = - FlinkOrphanFilesClean.executeDatabaseOrphanFiles( - procedureContext.getExecutionEnvironment(), - catalog, - olderThanMillis(olderThan), - createFileCleaner(catalog, dryRun), - parallelism, - databaseName, - tableName); - return new String[] {String.valueOf(deleted)}; + if (mode == null) { + mode = "DISTRIBUTED"; + } + long deletedFiles; + try { + switch (mode.toUpperCase(Locale.ROOT)) { + case "DISTRIBUTED": + deletedFiles = + FlinkOrphanFilesClean.executeDatabaseOrphanFiles( + procedureContext.getExecutionEnvironment(), + catalog, + olderThanMillis(olderThan), + createFileCleaner(catalog, dryRun), + parallelism, + databaseName, + tableName); + break; + case "LOCAL": + deletedFiles = + LocalOrphanFilesClean.executeDatabaseOrphanFiles( + catalog, + databaseName, + tableName, + olderThanMillis(olderThan), + createFileCleaner(catalog, dryRun), + parallelism); + break; + default: + throw new IllegalArgumentException( + "Unknown mode: " + + mode + + ". Only 'DISTRIBUTED' and 'LOCAL' are supported."); + } + return new String[] {String.valueOf(deletedFiles)}; + } catch (Exception e) { + throw new RuntimeException(e); + } } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RenameTagProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RenameTagProcedure.java new file mode 100644 index 000000000000..5476e807be71 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RenameTagProcedure.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.flink.procedure; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.table.Table; + +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.ProcedureHint; +import org.apache.flink.table.procedure.ProcedureContext; + +/** + * Create tag procedure. Usage: + * + *
    
    + *  CALL sys.rename_tag('tableId', 'tagName', 'targetTagName')
    + * 
    + */ +public class RenameTagProcedure extends ProcedureBase { + private static final String IDENTIFIER = "rename_tag"; + + @ProcedureHint( + argument = { + @ArgumentHint(name = "table", type = @DataTypeHint("STRING")), + @ArgumentHint(name = "tagName", type = @DataTypeHint("STRING")), + @ArgumentHint(name = "targetTagName", type = @DataTypeHint("STRING")) + }) + public String[] call( + ProcedureContext procedureContext, String tableId, String tagName, String targetTagName) + throws Catalog.TableNotExistException { + Table table = catalog.getTable(Identifier.fromString(tableId)); + table.renameTag(tagName, targetTagName); + String ret = String.format("Rename [%s] to [%s] successfully.", tagName, targetTagName); + return new String[] {ret}; + } + + @Override + public String identifier() { + return IDENTIFIER; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ReplaceTagProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ReplaceTagProcedure.java new file mode 100644 index 000000000000..6ed6ecc0e512 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ReplaceTagProcedure.java @@ -0,0 +1,39 @@ +/* + * 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.procedure; + +import org.apache.paimon.table.Table; + +import java.time.Duration; + +/** A procedure to replace a tag. */ +public class ReplaceTagProcedure extends CreateOrReplaceTagBaseProcedure { + + private static final String IDENTIFIER = "replace_tag"; + + @Override + void createOrReplaceTag(Table table, String tagName, Long snapshotId, Duration timeRetained) { + table.replaceTag(tagName, snapshotId, timeRetained); + } + + @Override + public String identifier() { + return IDENTIFIER; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java index 5bd4cbaafac6..934ce182a09c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java @@ -67,6 +67,7 @@ public String[] call( fileStoreTable.location(), fileStoreTable.snapshotManager().branch()); if (nextSnapshotId != null) { + fileStoreTable.snapshotManager().snapshot(nextSnapshotId); consumerManager.resetConsumer(consumerId, new Consumer(nextSnapshotId)); } else { consumerManager.deleteConsumer(consumerId); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RollbackToTimestampProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RollbackToTimestampProcedure.java new file mode 100644 index 000000000000..f84dab8eab89 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RollbackToTimestampProcedure.java @@ -0,0 +1,66 @@ +/* + * 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.procedure; + +import org.apache.paimon.Snapshot; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.Table; +import org.apache.paimon.utils.Preconditions; + +import org.apache.flink.table.annotation.ArgumentHint; +import org.apache.flink.table.annotation.DataTypeHint; +import org.apache.flink.table.annotation.ProcedureHint; +import org.apache.flink.table.procedure.ProcedureContext; + +/** + * Rollback to timestamp procedure. Usage: + * + *
    
    + *  -- rollback to the snapshot which earlier or equal than timestamp.
    + *  CALL sys.rollback_to_timestamp(`table` => 'tableId', timestamp => timestamp)
    + * 
    + */ +public class RollbackToTimestampProcedure extends ProcedureBase { + + public static final String IDENTIFIER = "rollback_to_timestamp"; + + @ProcedureHint( + argument = { + @ArgumentHint(name = "table", type = @DataTypeHint("STRING")), + @ArgumentHint(name = "timestamp", type = @DataTypeHint("BIGINT")) + }) + public String[] call(ProcedureContext procedureContext, String tableId, Long timestamp) + throws Catalog.TableNotExistException { + Table table = catalog.getTable(Identifier.fromString(tableId)); + FileStoreTable fileStoreTable = (FileStoreTable) table; + Snapshot snapshot = fileStoreTable.snapshotManager().earlierOrEqualTimeMills(timestamp); + Preconditions.checkNotNull( + snapshot, String.format("count not find snapshot earlier than %s", timestamp)); + long snapshotId = snapshot.id(); + fileStoreTable.rollbackTo(snapshotId); + return new String[] {String.format("Success roll back to snapshot: %s .", snapshotId)}; + } + + @Override + public String identifier() { + return IDENTIFIER; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/query/RemoteTableQuery.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/query/RemoteTableQuery.java index 62edd6bae89c..34d993eb3db2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/query/RemoteTableQuery.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/query/RemoteTableQuery.java @@ -30,7 +30,6 @@ import org.apache.paimon.table.Table; import org.apache.paimon.table.query.TableQuery; import org.apache.paimon.utils.ProjectedRow; -import org.apache.paimon.utils.Projection; import org.apache.paimon.utils.TypeUtils; import javax.annotation.Nullable; @@ -93,12 +92,7 @@ public InternalRow lookup(BinaryRow partition, int bucket, InternalRow key) thro @Override public RemoteTableQuery withValueProjection(int[] projection) { - return withValueProjection(Projection.of(projection).toNestedIndexes()); - } - - @Override - public RemoteTableQuery withValueProjection(int[][] projection) { - this.projection = Projection.of(projection).toTopLevelIndexes(); + this.projection = projection; return this; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java index 4db56601a38e..54104130438b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/shuffle/RangeShuffle.java @@ -89,8 +89,8 @@ public class RangeShuffle { * -----------------------------BATCH-[ARange,n]-PARTITION->[RRange,m]-> * } * - *

    The streams except the sample and histogram process stream will been blocked, so the the - * sample and histogram process stream does not care about requiredExchangeMode. + *

    The streams except the sample and histogram process stream will be blocked, so the sample + * and histogram process stream does not care about requiredExchangeMode. */ public static DataStream> rangeShuffleByKey( DataStream> inputDataStream, @@ -324,8 +324,8 @@ public void endInput() { } /** - * This two-input-operator require a input with RangeBoundaries as broadcast input, and generate - * Tuple2 which includes range index and record from the other input itself as output. + * This two-input-operator require an input with RangeBoundaries as broadcast input, and + * generate Tuple2 which includes range index and record from the other input itself as output. */ private static class AssignRangeIndexOperator extends TableStreamOperator>> diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AdaptiveParallelism.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AdaptiveParallelism.java index 09b14779c766..0fd7cf9f3565 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AdaptiveParallelism.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AdaptiveParallelism.java @@ -18,7 +18,9 @@ package org.apache.paimon.flink.sink; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.BatchExecutionOptions; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; /** Get adaptive config from Flink. Only work for Flink 1.17+. */ @@ -27,4 +29,20 @@ public class AdaptiveParallelism { public static boolean isEnabled(StreamExecutionEnvironment env) { return env.getConfiguration().get(BatchExecutionOptions.ADAPTIVE_AUTO_PARALLELISM_ENABLED); } + + /** + * Get default max parallelism of AdaptiveBatchScheduler of Flink. See {@link + * org.apache.flink.runtime.scheduler.adaptivebatch.AdaptiveBatchSchedulerFactory#getDefaultMaxParallelism(Configuration, + * ExecutionConfig)}. + */ + public static int getDefaultMaxParallelism( + ReadableConfig configuration, ExecutionConfig executionConfig) { + return configuration + .getOptional(BatchExecutionOptions.ADAPTIVE_AUTO_PARALLELISM_MAX_PARALLELISM) + .orElse( + executionConfig.getParallelism() == ExecutionConfig.PARALLELISM_DEFAULT + ? BatchExecutionOptions.ADAPTIVE_AUTO_PARALLELISM_MAX_PARALLELISM + .defaultValue() + : executionConfig.getParallelism()); + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketSink.java index cf697108fd32..c2299a7e8699 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketSink.java @@ -24,18 +24,22 @@ import org.apache.paimon.table.sink.ChannelComputer; import org.apache.paimon.table.sink.PartitionKeyExtractor; import org.apache.paimon.utils.SerializableFunction; +import org.apache.paimon.utils.StringUtils; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import javax.annotation.Nullable; import java.util.Map; import static org.apache.paimon.CoreOptions.createCommitUser; +import static org.apache.paimon.flink.FlinkConnectorOptions.SINK_OPERATOR_UID_SUFFIX; +import static org.apache.paimon.flink.FlinkConnectorOptions.generateCustomUid; import static org.apache.paimon.flink.sink.FlinkStreamPartitioner.partition; /** Sink for dynamic bucket table. */ @@ -43,6 +47,8 @@ public abstract class DynamicBucketSink extends FlinkWriteSink overwritePartition) { super(table, overwritePartition); @@ -88,11 +94,20 @@ public DataStreamSink build(DataStream input, @Nullable Integer parallelis initialCommitUser, table, numAssigners, extractorFunction(), false); TupleTypeInfo> rowWithBucketType = new TupleTypeInfo<>(partitionByKeyHash.getType(), BasicTypeInfo.INT_TYPE_INFO); - DataStream> bucketAssigned = + SingleOutputStreamOperator> bucketAssigned = partitionByKeyHash - .transform("dynamic-bucket-assigner", rowWithBucketType, assignerOperator) + .transform( + DYNAMIC_BUCKET_ASSIGNER_NAME, rowWithBucketType, assignerOperator) .setParallelism(partitionByKeyHash.getParallelism()); + String uidSuffix = table.options().get(SINK_OPERATOR_UID_SUFFIX.key()); + if (!StringUtils.isNullOrWhitespaceOnly(uidSuffix)) { + bucketAssigned = + bucketAssigned.uid( + generateCustomUid( + DYNAMIC_BUCKET_ASSIGNER_NAME, table.name(), uidSuffix)); + } + // 3. shuffle by partition & bucket DataStream> partitionByBucket = partition(bucketAssigned, channelComputer2(), parallelism); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java index f810c464b58c..59f2f4b1035f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java @@ -21,10 +21,12 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.CoreOptions.ChangelogProducer; import org.apache.paimon.CoreOptions.TagCreationMode; +import org.apache.paimon.flink.compact.changelog.ChangelogCompactCoordinateOperator; +import org.apache.paimon.flink.compact.changelog.ChangelogCompactWorkerOperator; +import org.apache.paimon.flink.compact.changelog.ChangelogTaskTypeInfo; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; -import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.SerializableRunnable; @@ -32,6 +34,7 @@ import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.operators.SlotSharingGroup; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.typeutils.EitherTypeInfo; import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.CheckpointingMode; @@ -48,15 +51,14 @@ import javax.annotation.Nullable; import java.io.Serializable; -import java.util.ArrayList; import java.util.HashSet; import java.util.LinkedList; -import java.util.List; import java.util.Queue; import java.util.Set; import static org.apache.paimon.CoreOptions.FULL_COMPACTION_DELTA_COMMITS; import static org.apache.paimon.CoreOptions.createCommitUser; +import static org.apache.paimon.flink.FlinkConnectorOptions.CHANGELOG_PRECOMMIT_COMPACT; import static org.apache.paimon.flink.FlinkConnectorOptions.CHANGELOG_PRODUCER_FULL_COMPACTION_TRIGGER_INTERVAL; import static org.apache.paimon.flink.FlinkConnectorOptions.END_INPUT_WATERMARK; import static org.apache.paimon.flink.FlinkConnectorOptions.SINK_AUTO_TAG_FOR_SAVEPOINT; @@ -64,7 +66,9 @@ import static org.apache.paimon.flink.FlinkConnectorOptions.SINK_COMMITTER_MEMORY; import static org.apache.paimon.flink.FlinkConnectorOptions.SINK_COMMITTER_OPERATOR_CHAINING; import static org.apache.paimon.flink.FlinkConnectorOptions.SINK_MANAGED_WRITER_BUFFER_MEMORY; +import static org.apache.paimon.flink.FlinkConnectorOptions.SINK_OPERATOR_UID_SUFFIX; import static org.apache.paimon.flink.FlinkConnectorOptions.SINK_USE_MANAGED_MEMORY; +import static org.apache.paimon.flink.FlinkConnectorOptions.generateCustomUid; import static org.apache.paimon.flink.utils.ManagedMemoryUtils.declareManagedMemory; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -225,17 +229,32 @@ public DataStream doWrite( commitUser)) .setParallelism(parallelism == null ? input.getParallelism() : parallelism); - boolean writeMCacheEnabled = table.coreOptions().writeManifestCache().getBytes() > 0; - boolean hashDynamicMode = table.bucketMode() == BucketMode.HASH_DYNAMIC; - if (!isStreaming && (writeMCacheEnabled || hashDynamicMode)) { - assertBatchAdaptiveParallelism( - env, written.getParallelism(), writeMCacheEnabled, hashDynamicMode); + Options options = Options.fromMap(table.options()); + + String uidSuffix = options.get(SINK_OPERATOR_UID_SUFFIX); + if (options.get(SINK_OPERATOR_UID_SUFFIX) != null) { + written = written.uid(generateCustomUid(WRITER_NAME, table.name(), uidSuffix)); } - Options options = Options.fromMap(table.options()); if (options.get(SINK_USE_MANAGED_MEMORY)) { declareManagedMemory(written, options.get(SINK_MANAGED_WRITER_BUFFER_MEMORY)); } + + if (options.get(CHANGELOG_PRECOMMIT_COMPACT)) { + written = + written.transform( + "Changelog Compact Coordinator", + new EitherTypeInfo<>( + new CommittableTypeInfo(), new ChangelogTaskTypeInfo()), + new ChangelogCompactCoordinateOperator(table)) + .forceNonParallel() + .transform( + "Changelog Compact Worker", + new CommittableTypeInfo(), + new ChangelogCompactWorkerOperator(table)) + .setParallelism(written.getParallelism()); + } + return written; } @@ -284,6 +303,14 @@ protected DataStreamSink doCommit(DataStream written, String com committerOperator) .setParallelism(1) .setMaxParallelism(1); + if (options.get(SINK_OPERATOR_UID_SUFFIX) != null) { + committed = + committed.uid( + generateCustomUid( + GLOBAL_COMMITTER_NAME, + table.name(), + options.get(SINK_OPERATOR_UID_SUFFIX))); + } configureGlobalCommitter( committed, options.get(SINK_COMMITTER_CPU), options.get(SINK_COMMITTER_MEMORY)); return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); @@ -328,26 +355,6 @@ public static void assertBatchAdaptiveParallelism( assertBatchAdaptiveParallelism(env, sinkParallelism, msg); } - public static void assertBatchAdaptiveParallelism( - StreamExecutionEnvironment env, - int sinkParallelism, - boolean writeMCacheEnabled, - boolean hashDynamicMode) { - List messages = new ArrayList<>(); - if (writeMCacheEnabled) { - messages.add("Write Manifest Cache"); - } - if (hashDynamicMode) { - messages.add("Dynamic Bucket Mode"); - } - String msg = - String.format( - "Paimon Sink with %s does not support Flink's Adaptive Parallelism mode. " - + "Please manually turn it off or set Paimon `sink.parallelism` manually.", - messages); - assertBatchAdaptiveParallelism(env, sinkParallelism, msg); - } - public static void assertBatchAdaptiveParallelism( StreamExecutionEnvironment env, int sinkParallelism, String exceptionMsg) { try { 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 546f82ec1f84..dcccd0a1a988 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 @@ -44,6 +44,7 @@ import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; @@ -53,6 +54,7 @@ import static org.apache.paimon.flink.FlinkConnectorOptions.CLUSTERING_SAMPLE_FACTOR; import static org.apache.paimon.flink.FlinkConnectorOptions.CLUSTERING_STRATEGY; import static org.apache.paimon.flink.FlinkConnectorOptions.MIN_CLUSTERING_SAMPLE_FACTOR; +import static org.apache.paimon.flink.sink.FlinkSink.isStreaming; import static org.apache.paimon.flink.sink.FlinkStreamPartitioner.partition; import static org.apache.paimon.flink.sorter.TableSorter.OrderType.HILBERT; import static org.apache.paimon.flink.sorter.TableSorter.OrderType.ORDER; @@ -75,7 +77,7 @@ public class FlinkSinkBuilder { private DataStream input; @Nullable protected Map overwritePartition; - @Nullable protected Integer parallelism; + @Nullable private Integer parallelism; @Nullable private TableSortInfo tableSortInfo; // ============== for extension ============== @@ -101,9 +103,12 @@ public FlinkSinkBuilder forRow(DataStream input, DataType rowDataType) { DataFormatConverters.RowConverter converter = new DataFormatConverters.RowConverter(fieldDataTypes); this.input = - input.map((MapFunction) converter::toInternal) - .setParallelism(input.getParallelism()) - .returns(InternalTypeInfo.of(rowType)); + input.transform( + "Map", + InternalTypeInfo.of(rowType), + new StreamMapWithForwardingRecordAttributes<>( + (MapFunction) converter::toInternal)) + .setParallelism(input.getParallelism()); return this; } @@ -142,7 +147,7 @@ public FlinkSinkBuilder clusteringIfPossible( return this; } checkState(input != null, "The input stream should be specified earlier."); - if (FlinkSink.isStreaming(input) || !table.bucketMode().equals(BUCKET_UNAWARE)) { + if (isStreaming(input) || !table.bucketMode().equals(BUCKET_UNAWARE)) { LOG.warn( "Clustering is enabled; however, it has been skipped as " + "it only supports the bucket unaware table without primary keys and " @@ -208,6 +213,7 @@ public FlinkSinkBuilder clusteringIfPossible( /** Build {@link DataStreamSink}. */ public DataStreamSink build() { + setParallelismIfAdaptiveConflict(); input = trySortInput(input); DataStream input = mapToInternalRow(this.input, table.rowType()); if (table.coreOptions().localMergeEnabled() && table.schema().primaryKeys().size() > 0) { @@ -237,9 +243,12 @@ public DataStreamSink build() { protected DataStream mapToInternalRow( DataStream input, org.apache.paimon.types.RowType rowType) { - return input.map((MapFunction) FlinkRowWrapper::new) - .setParallelism(input.getParallelism()) - .returns(org.apache.paimon.flink.utils.InternalTypeInfo.fromRowType(rowType)); + return input.transform( + "Map", + org.apache.paimon.flink.utils.InternalTypeInfo.fromRowType(rowType), + new StreamMapWithForwardingRecordAttributes<>( + (MapFunction) FlinkRowWrapper::new)) + .setParallelism(input.getParallelism()); } protected DataStreamSink buildDynamicBucketSink( @@ -282,4 +291,48 @@ private DataStream trySortInput(DataStream input) { } return input; } + + private void setParallelismIfAdaptiveConflict() { + try { + boolean parallelismUndefined = parallelism == null || parallelism == -1; + boolean isStreaming = isStreaming(input); + boolean isAdaptiveParallelismEnabled = + AdaptiveParallelism.isEnabled(input.getExecutionEnvironment()); + boolean writeMCacheEnabled = table.coreOptions().writeManifestCache().getBytes() > 0; + boolean hashDynamicMode = table.bucketMode() == BucketMode.HASH_DYNAMIC; + if (parallelismUndefined + && !isStreaming + && isAdaptiveParallelismEnabled + && (writeMCacheEnabled || hashDynamicMode)) { + List messages = new ArrayList<>(); + if (writeMCacheEnabled) { + messages.add("Write Manifest Cache"); + } + if (hashDynamicMode) { + messages.add("Dynamic Bucket Mode"); + } + + String parallelismSource; + if (input.getParallelism() > 0) { + parallelismSource = "input parallelism"; + parallelism = input.getParallelism(); + } else { + parallelismSource = "AdaptiveBatchScheduler's default max parallelism"; + parallelism = + AdaptiveParallelism.getDefaultMaxParallelism( + input.getExecutionEnvironment().getConfiguration(), + input.getExecutionConfig()); + } + String msg = + String.format( + "Paimon Sink with %s does not support Flink's Adaptive Parallelism mode. " + + "Configuring sink parallelism to `%s` instead. You can also set Paimon " + + "`sink.parallelism` manually to override this configuration.", + messages, parallelismSource); + LOG.warn(msg); + } + } catch (NoClassDefFoundError ignored) { + // before 1.17, there is no adaptive parallelism + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkStreamPartitioner.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkStreamPartitioner.java index 78e532053e15..f9b81760c30e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkStreamPartitioner.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkStreamPartitioner.java @@ -24,6 +24,7 @@ import org.apache.flink.runtime.plugable.SerializationDelegate; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.transformations.PartitionTransformation; +import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -77,4 +78,14 @@ public static DataStream partition( } return new DataStream<>(input.getExecutionEnvironment(), partitioned); } + + public static DataStream rebalance(DataStream input, Integer parallelism) { + RebalancePartitioner partitioner = new RebalancePartitioner<>(); + PartitionTransformation partitioned = + new PartitionTransformation<>(input.getTransformation(), partitioner); + if (parallelism != null) { + partitioned.setParallelism(parallelism); + } + return new DataStream<>(input.getExecutionEnvironment(), partitioned); + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java index 9713421fdd6a..70fac7a83e93 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/HashBucketAssignerOperator.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.sink; +import org.apache.paimon.flink.ProcessRecordAttributesUtil; import org.apache.paimon.index.BucketAssigner; import org.apache.paimon.index.HashBucketAssigner; import org.apache.paimon.index.SimpleHashBucketAssigner; @@ -32,6 +33,7 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** Assign bucket for the input record, output record with bucket. */ @@ -100,6 +102,11 @@ public void processElement(StreamRecord streamRecord) throws Exception { output.collect(new StreamRecord<>(new Tuple2<>(value, bucket))); } + @Override + public void processRecordAttributes(RecordAttributes recordAttributes) { + ProcessRecordAttributesUtil.processWithOutput(recordAttributes, output); + } + @Override public void prepareSnapshotPreBarrier(long checkpointId) { assigner.prepareCommit(checkpointId); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java index 18d440dbceac..aba891e44100 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/LocalMergeOperator.java @@ -48,6 +48,8 @@ import java.util.List; +import static org.apache.paimon.table.PrimaryKeyTableUtils.addKeyNamePrefix; + /** * {@link AbstractStreamOperator} which buffer input record and apply merge function when the buffer * is full. Mainly to resolve data skew on primary keys. @@ -105,7 +107,7 @@ public void open() throws Exception { // partition fields. @Override public List keyFields(TableSchema schema) { - return schema.primaryKeysFields(); + return addKeyNamePrefix(schema.primaryKeysFields()); } @Override 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 513b694fb33c..7cb5d30c2f8e 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 @@ -95,15 +95,15 @@ public void initializeState(StateInitializationContext context) throws Exception catalog = catalogLoader.load(); - // Each job can only have one user name and this name must be consistent across restarts. - // We cannot use job id as commit user name here because user may change job id by creating + // Each job can only have one username and this name must be consistent across restarts. + // We cannot use job id as commit username here because user may change job id by creating // a savepoint, stop the job and then resume from savepoint. commitUser = StateUtils.getSingleValueFromState( context, "commit_user_state", String.class, initialCommitUser); state = - new StoreSinkWriteState( + new StoreSinkWriteStateImpl( context, (tableName, partition, bucket) -> ChannelComputer.select( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/BaseTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/NoopStoreSinkWriteState.java similarity index 50% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/BaseTableSource.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/NoopStoreSinkWriteState.java index 57c4b01a2876..f975bce6ea49 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/BaseTableSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/NoopStoreSinkWriteState.java @@ -16,34 +16,39 @@ * limitations under the License. */ -package org.apache.paimon.flink.source.table; +package org.apache.paimon.flink.sink; -import org.apache.paimon.flink.source.FlinkTableSource; +import javax.annotation.Nullable; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.source.ScanTableSource; +import java.util.List; -/** The {@link ScanTableSource} with push down interfaces. */ -public abstract class BaseTableSource implements ScanTableSource { +/** + * A {@link StoreSinkWriteState} which stores nothing. Currently only used for append only unaware + * bucket table writers. + */ +public class NoopStoreSinkWriteState implements StoreSinkWriteState { - private final FlinkTableSource source; + private final StateValueFilter stateValueFilter; - public BaseTableSource(FlinkTableSource source) { - this.source = source; + public NoopStoreSinkWriteState(StateValueFilter stateValueFilter) { + this.stateValueFilter = stateValueFilter; } @Override - public ChangelogMode getChangelogMode() { - return source.getChangelogMode(); + public StateValueFilter stateValueFilter() { + return stateValueFilter; } @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { - return source.getScanRuntimeProvider(runtimeProviderContext); + public @Nullable List get(String tableName, String key) { + throw new UnsupportedOperationException(); } @Override - public String asSummaryString() { - return source.asSummaryString(); + public void put(String tableName, String key, List stateValues) { + throw new UnsupportedOperationException(); } + + @Override + public void snapshotState() throws Exception {} } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java index b670b905d587..1cd10390c1a0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java @@ -21,6 +21,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.table.FileStoreTable; +import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import java.util.Map; @@ -39,6 +40,24 @@ public RowUnawareBucketSink( @Override protected OneInputStreamOperator createWriteOperator( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new RowDataStoreWriteOperator(table, logSinkFunction, writeProvider, commitUser); + return new RowDataStoreWriteOperator(table, logSinkFunction, writeProvider, commitUser) { + + @Override + protected StoreSinkWriteState createState( + StateInitializationContext context, + StoreSinkWriteState.StateValueFilter stateFilter) + throws Exception { + // No conflicts will occur in append only unaware bucket writer, so no state is + // needed. + return new NoopStoreSinkWriteState(stateFilter); + } + + @Override + protected String getCommitUser(StateInitializationContext context) throws Exception { + // No conflicts will occur in append only unaware bucket writer, so commitUser does + // not matter. + return commitUser; + } + }; } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java index b3e74a3f6125..d237f4da56cf 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreCommitter.java @@ -20,7 +20,7 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.flink.metrics.FlinkMetricRegistry; -import org.apache.paimon.flink.sink.partition.PartitionMarkDone; +import org.apache.paimon.flink.sink.partition.PartitionListeners; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.table.FileStoreTable; @@ -43,7 +43,7 @@ public class StoreCommitter implements Committer committables) throws IOException, InterruptedException { commit.commitMultiple(committables, false); calcNumBytesAndRecordsOut(committables); - if (partitionMarkDone != null) { - partitionMarkDone.notifyCommittable(committables); - } + partitionListeners.notifyCommittable(committables); } @Override public int filterAndCommit( List globalCommittables, boolean checkAppendFiles) { int committed = commit.filterAndCommitMultiple(globalCommittables, checkAppendFiles); - if (partitionMarkDone != null) { - partitionMarkDone.notifyCommittable(globalCommittables); - } + partitionListeners.notifyCommittable(globalCommittables); return committed; } @Override public Map> groupByCheckpoint(Collection committables) { - if (partitionMarkDone != null) { - try { - partitionMarkDone.snapshotState(); - } catch (Exception e) { - throw new RuntimeException(e); - } + try { + partitionListeners.snapshotState(); + } catch (Exception e) { + throw new RuntimeException(e); } Map> grouped = new HashMap<>(); @@ -146,6 +135,7 @@ public Map> groupByCheckpoint(Collection co @Override public void close() throws Exception { commit.close(); + partitionListeners.close(); } private void calcNumBytesAndRecordsOut(List committables) { 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 184288490772..bc7bb350df21 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 @@ -19,7 +19,6 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.CoreOptions; -import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; @@ -29,7 +28,6 @@ import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Preconditions; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -77,39 +75,30 @@ public StoreCompactOperator( public void initializeState(StateInitializationContext context) throws Exception { super.initializeState(context); - // Each job can only have one user name and this name must be consistent across restarts. - // We cannot use job id as commit user name here because user may change job id by creating + // Each job can only have one username and this name must be consistent across restarts. + // We cannot use job id as commit username here because user may change job id by creating // a savepoint, stop the job and then resume from savepoint. String commitUser = StateUtils.getSingleValueFromState( context, "commit_user_state", String.class, initialCommitUser); - initStateAndWriter( - context, - (tableName, partition, bucket) -> - ChannelComputer.select( - partition, - bucket, - getRuntimeContext().getNumberOfParallelSubtasks()) - == getRuntimeContext().getIndexOfThisSubtask(), - getContainingTask().getEnvironment().getIOManager(), - commitUser); - } - - @VisibleForTesting - void initStateAndWriter( - StateInitializationContext context, - StoreSinkWriteState.StateValueFilter stateFilter, - IOManager ioManager, - String commitUser) - throws Exception { - // We put state and write init in this method for convenient testing. Without construct a - // runtime context, we can test to construct a writer here - state = new StoreSinkWriteState(context, stateFilter); - + state = + new StoreSinkWriteStateImpl( + context, + (tableName, partition, bucket) -> + ChannelComputer.select( + partition, + bucket, + getRuntimeContext().getNumberOfParallelSubtasks()) + == getRuntimeContext().getIndexOfThisSubtask()); write = storeSinkWriteProvider.provide( - table, commitUser, state, ioManager, memoryPool, getMetricGroup()); + table, + commitUser, + state, + getContainingTask().getEnvironment().getIOManager(), + memoryPool, + getMetricGroup()); } @Override 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 a432a55454ab..3a5a8df5cdc7 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 @@ -84,6 +84,10 @@ public interface StoreSinkWrite { @FunctionalInterface interface Provider 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, @@ -97,6 +101,10 @@ StoreSinkWrite provide( @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, 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 bdaf7bc327be..ef8042820947 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 @@ -65,6 +65,8 @@ public class StoreSinkWriteImpl implements StoreSinkWrite { @Nullable private final MetricGroup metricGroup; + @Nullable private Boolean insertOnly; + public StoreSinkWriteImpl( FileStoreTable table, String commitUser, @@ -154,15 +156,21 @@ private TableWriteImpl newTableWrite(FileStoreTable table) { } if (memoryPoolFactory != null) { - return tableWrite.withMemoryPoolFactory(memoryPoolFactory); + tableWrite.withMemoryPoolFactory(memoryPoolFactory); } else { - return tableWrite.withMemoryPool( + tableWrite.withMemoryPool( memoryPool != null ? memoryPool : new HeapMemorySegmentPool( table.coreOptions().writeBufferSize(), table.coreOptions().pageSize())); } + + if (insertOnly != null) { + tableWrite.withInsertOnly(insertOnly); + } + + return tableWrite; } public void withCompactExecutor(ExecutorService compactExecutor) { @@ -171,6 +179,7 @@ public void withCompactExecutor(ExecutorService compactExecutor) { @Override public void withInsertOnly(boolean insertOnly) { + this.insertOnly = insertOnly; write.withInsertOnly(insertOnly); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteState.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteState.java index 072d6a1b96fc..8626a01a66e8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteState.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteState.java @@ -19,107 +19,31 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.data.BinaryRow; -import org.apache.paimon.utils.SerializationUtils; - -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.IntSerializer; -import org.apache.flink.api.common.typeutils.base.StringSerializer; -import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; -import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; -import org.apache.flink.runtime.state.StateInitializationContext; import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; /** - * States for {@link StoreSinkWrite}s. - * - *

    States are positioned first by table name and then by key name. This class should be initiated - * in a sink operator and then given to {@link StoreSinkWrite}. + * States for {@link StoreSinkWrite}s. It's a wrapper to conveniently modify states for each table + * stored in Flink states. */ -public class StoreSinkWriteState { - - private final StateValueFilter stateValueFilter; - - private final ListState> listState; - private final Map>> map; - - @SuppressWarnings("unchecked") - public StoreSinkWriteState( - StateInitializationContext context, StateValueFilter stateValueFilter) - throws Exception { - this.stateValueFilter = stateValueFilter; - TupleSerializer> listStateSerializer = - new TupleSerializer<>( - (Class>) - (Class) Tuple5.class, - new TypeSerializer[] { - StringSerializer.INSTANCE, - StringSerializer.INSTANCE, - BytePrimitiveArraySerializer.INSTANCE, - IntSerializer.INSTANCE, - BytePrimitiveArraySerializer.INSTANCE - }); - listState = - context.getOperatorStateStore() - .getUnionListState( - new ListStateDescriptor<>( - "paimon_store_sink_write_state", listStateSerializer)); - - map = new HashMap<>(); - for (Tuple5 tuple : listState.get()) { - BinaryRow partition = SerializationUtils.deserializeBinaryRow(tuple.f2); - if (stateValueFilter.filter(tuple.f0, partition, tuple.f3)) { - map.computeIfAbsent(tuple.f0, k -> new HashMap<>()) - .computeIfAbsent(tuple.f1, k -> new ArrayList<>()) - .add(new StateValue(partition, tuple.f3, tuple.f4)); - } - } - } +public interface StoreSinkWriteState { - public StateValueFilter stateValueFilter() { - return stateValueFilter; - } + StoreSinkWriteState.StateValueFilter stateValueFilter(); - public @Nullable List get(String tableName, String key) { - Map> innerMap = map.get(tableName); - return innerMap == null ? null : innerMap.get(key); - } + @Nullable + List get(String tableName, String key); - public void put(String tableName, String key, List stateValues) { - map.computeIfAbsent(tableName, k -> new HashMap<>()).put(key, stateValues); - } + void put(String tableName, String key, List stateValues); - public void snapshotState() throws Exception { - List> list = new ArrayList<>(); - for (Map.Entry>> tables : map.entrySet()) { - for (Map.Entry> entry : tables.getValue().entrySet()) { - for (StateValue stateValue : entry.getValue()) { - list.add( - Tuple5.of( - tables.getKey(), - entry.getKey(), - SerializationUtils.serializeBinaryRow(stateValue.partition()), - stateValue.bucket(), - stateValue.value())); - } - } - } - listState.update(list); - } + void snapshotState() throws Exception; /** * A state value for {@link StoreSinkWrite}. All state values should be given a partition and a * bucket so that they can be redistributed once the sink parallelism is changed. */ - public static class StateValue { + class StateValue { private final BinaryRow partition; private final int bucket; @@ -148,8 +72,7 @@ public byte[] value() { * Given the table name, partition and bucket of a {@link StateValue} in a union list state, * decide whether to keep this {@link StateValue} in this subtask. */ - public interface StateValueFilter { - + interface StateValueFilter { boolean filter(String tableName, BinaryRow partition, int bucket); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteStateImpl.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteStateImpl.java new file mode 100644 index 000000000000..a01cbcb68d38 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteStateImpl.java @@ -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. + */ + +package org.apache.paimon.flink.sink; + +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.utils.SerializationUtils; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; +import org.apache.flink.api.java.tuple.Tuple5; +import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; +import org.apache.flink.runtime.state.StateInitializationContext; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Default implementation for {@link StoreSinkWriteState}. + * + *

    States are positioned first by table name and then by key name. This class should be initiated + * in a sink operator and then given to {@link StoreSinkWrite}. + */ +public class StoreSinkWriteStateImpl implements StoreSinkWriteState { + + private final StoreSinkWriteState.StateValueFilter stateValueFilter; + + private final ListState> listState; + private final Map>> map; + + @SuppressWarnings("unchecked") + public StoreSinkWriteStateImpl( + StateInitializationContext context, + StoreSinkWriteState.StateValueFilter stateValueFilter) + throws Exception { + this.stateValueFilter = stateValueFilter; + TupleSerializer> listStateSerializer = + new TupleSerializer<>( + (Class>) + (Class) Tuple5.class, + new TypeSerializer[] { + StringSerializer.INSTANCE, + StringSerializer.INSTANCE, + BytePrimitiveArraySerializer.INSTANCE, + IntSerializer.INSTANCE, + BytePrimitiveArraySerializer.INSTANCE + }); + listState = + context.getOperatorStateStore() + .getUnionListState( + new ListStateDescriptor<>( + "paimon_store_sink_write_state", listStateSerializer)); + + map = new HashMap<>(); + for (Tuple5 tuple : listState.get()) { + BinaryRow partition = SerializationUtils.deserializeBinaryRow(tuple.f2); + if (stateValueFilter.filter(tuple.f0, partition, tuple.f3)) { + map.computeIfAbsent(tuple.f0, k -> new HashMap<>()) + .computeIfAbsent(tuple.f1, k -> new ArrayList<>()) + .add(new StoreSinkWriteState.StateValue(partition, tuple.f3, tuple.f4)); + } + } + } + + @Override + public StoreSinkWriteState.StateValueFilter stateValueFilter() { + return stateValueFilter; + } + + @Override + public @Nullable List get(String tableName, String key) { + Map> innerMap = map.get(tableName); + return innerMap == null ? null : innerMap.get(key); + } + + public void put( + String tableName, String key, List stateValues) { + map.computeIfAbsent(tableName, k -> new HashMap<>()).put(key, stateValues); + } + + public void snapshotState() throws Exception { + List> list = new ArrayList<>(); + for (Map.Entry>> tables : + map.entrySet()) { + for (Map.Entry> entry : + tables.getValue().entrySet()) { + for (StoreSinkWriteState.StateValue stateValue : entry.getValue()) { + list.add( + Tuple5.of( + tables.getKey(), + entry.getKey(), + SerializationUtils.serializeBinaryRow(stateValue.partition()), + stateValue.bucket(), + stateValue.value())); + } + } + } + listState.update(list); + } +} diff --git a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/leafnode/package.scala b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StreamMapWithForwardingRecordAttributes.java similarity index 51% rename from paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/leafnode/package.scala rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StreamMapWithForwardingRecordAttributes.java index 870aa13f6cb9..03038a042d38 100644 --- a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/leafnode/package.scala +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StreamMapWithForwardingRecordAttributes.java @@ -16,22 +16,22 @@ * limitations under the License. */ -package org.apache.paimon.spark +package org.apache.paimon.flink.sink; -import org.apache.paimon.spark.catalyst.trees.PaimonLeafLike +import org.apache.paimon.flink.ProcessRecordAttributesUtil; -import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, ParsedStatement} -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.execution.datasources.v2.V2CommandExec +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.streaming.api.operators.StreamMap; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; -package object leafnode { - trait PaimonLeafParsedStatement extends ParsedStatement with PaimonLeafLike[LogicalPlan] - - trait PaimonLeafRunnableCommand extends RunnableCommand with PaimonLeafLike[LogicalPlan] - - trait PaimonLeafCommand extends Command with PaimonLeafLike[LogicalPlan] - - trait PaimonLeafV2CommandExec extends V2CommandExec with PaimonLeafLike[SparkPlan] +/** A {@link StreamMap} that forwards received {@link RecordAttributes} to downstream operators. */ +public class StreamMapWithForwardingRecordAttributes extends StreamMap { + public StreamMapWithForwardingRecordAttributes(MapFunction mapper) { + super(mapper); + } + @Override + public void processRecordAttributes(RecordAttributes recordAttributes) { + ProcessRecordAttributesUtil.processWithOutput(recordAttributes, output); + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java index 583c1c9d161c..4e4c2ff2c67f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/SupportsRowLevelOperationFlinkTableSink.java @@ -185,11 +185,17 @@ private void validateDeletable() { table.getClass().getName())); } - MergeEngine mergeEngine = CoreOptions.fromMap(table.options()).mergeEngine(); - if (mergeEngine != DEDUPLICATE) { - throw new UnsupportedOperationException( - String.format("Merge engine %s can not support batch delete.", mergeEngine)); + CoreOptions coreOptions = CoreOptions.fromMap(table.options()); + if (coreOptions.mergeEngine() == DEDUPLICATE + || (coreOptions.mergeEngine() == PARTIAL_UPDATE + && coreOptions.partialUpdateRemoveRecordOnDelete())) { + return; } + + throw new UnsupportedOperationException( + String.format( + "Merge engine %s can not support batch delete.", + coreOptions.mergeEngine())); } private boolean canPushDownDeleteFilter() { 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 f38e0ad6bfb5..67b4720e2964 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 @@ -19,14 +19,15 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.flink.ProcessRecordAttributesUtil; import org.apache.paimon.flink.sink.StoreSinkWriteState.StateValueFilter; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.ChannelComputer; -import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import java.io.IOException; import java.util.List; @@ -56,13 +57,6 @@ public TableWriteOperator( public void initializeState(StateInitializationContext context) throws Exception { super.initializeState(context); - // Each job can only have one user name and this name must be consistent across restarts. - // We cannot use job id as commit user name here because user may change job id by creating - // a savepoint, stop the job and then resume from savepoint. - String commitUser = - StateUtils.getSingleValueFromState( - context, "commit_user_state", String.class, initialCommitUser); - boolean containLogSystem = containLogSystem(); int numTasks = getRuntimeContext().getNumberOfParallelSubtasks(); StateValueFilter stateFilter = @@ -74,27 +68,35 @@ public void initializeState(StateInitializationContext context) throws Exception return task == getRuntimeContext().getIndexOfThisSubtask(); }; - initStateAndWriter( - context, - stateFilter, - getContainingTask().getEnvironment().getIOManager(), - commitUser); + state = createState(context, stateFilter); + write = + storeSinkWriteProvider.provide( + table, + getCommitUser(context), + state, + getContainingTask().getEnvironment().getIOManager(), + memoryPool, + getMetricGroup()); } - @VisibleForTesting - void initStateAndWriter( - StateInitializationContext context, - StateValueFilter stateFilter, - IOManager ioManager, - String commitUser) + protected StoreSinkWriteState createState( + StateInitializationContext context, StoreSinkWriteState.StateValueFilter stateFilter) throws Exception { - // We put state and write init in this method for convenient testing. Without construct a - // runtime context, we can test to construct a writer here - state = new StoreSinkWriteState(context, stateFilter); + return new StoreSinkWriteStateImpl(context, stateFilter); + } - write = - storeSinkWriteProvider.provide( - table, commitUser, state, ioManager, memoryPool, getMetricGroup()); + protected String getCommitUser(StateInitializationContext context) throws Exception { + // Each job can only have one username and this name must be consistent across restarts. + // We cannot use job id as commit username here because user may change job id by creating + // a savepoint, stop the job and then resume from savepoint. + return StateUtils.getSingleValueFromState( + context, "commit_user_state", String.class, initialCommitUser); + } + + @Override + public void processRecordAttributes(RecordAttributes recordAttributes) throws Exception { + ProcessRecordAttributesUtil.processWithWrite(recordAttributes, write); + super.processRecordAttributes(recordAttributes); } protected abstract boolean containLogSystem(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java new file mode 100644 index 000000000000..eb965aa3a318 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/HmsReporter.java @@ -0,0 +1,108 @@ +/* + * 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.sink.partition; + +import org.apache.paimon.Snapshot; +import org.apache.paimon.fs.Path; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.metastore.MetastoreClient; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.ScanMode; +import org.apache.paimon.table.source.snapshot.SnapshotReader; +import org.apache.paimon.utils.Preconditions; +import org.apache.paimon.utils.SnapshotManager; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.catalog.Catalog.HIVE_LAST_UPDATE_TIME_PROP; +import static org.apache.paimon.catalog.Catalog.NUM_FILES_PROP; +import static org.apache.paimon.catalog.Catalog.NUM_ROWS_PROP; +import static org.apache.paimon.catalog.Catalog.TOTAL_SIZE_PROP; +import static org.apache.paimon.utils.PartitionPathUtils.extractPartitionSpecFromPath; + +/** Action to report the table statistic from the latest snapshot to HMS. */ +public class HmsReporter implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(HmsReporter.class); + + private final MetastoreClient metastoreClient; + private final SnapshotReader snapshotReader; + private final SnapshotManager snapshotManager; + + public HmsReporter(FileStoreTable table, MetastoreClient client) { + this.metastoreClient = + Preconditions.checkNotNull(client, "the metastore client factory is null"); + this.snapshotReader = table.newSnapshotReader(); + this.snapshotManager = table.snapshotManager(); + } + + public void report(String partition, long modifyTime) throws Exception { + Snapshot snapshot = snapshotManager.latestSnapshot(); + if (snapshot != null) { + LinkedHashMap partitionSpec = + extractPartitionSpecFromPath(new Path(partition)); + List splits = + new ArrayList<>( + snapshotReader + .withMode(ScanMode.ALL) + .withPartitionFilter(partitionSpec) + .withSnapshot(snapshot) + .read() + .dataSplits()); + long rowCount = 0; + long totalSize = 0; + long fileCount = 0; + for (DataSplit split : splits) { + List fileMetas = split.dataFiles(); + rowCount += split.rowCount(); + fileCount += fileMetas.size(); + for (DataFileMeta fileMeta : fileMetas) { + totalSize += fileMeta.fileSize(); + } + } + Map statistic = new HashMap<>(); + statistic.put(NUM_FILES_PROP, String.valueOf(fileCount)); + statistic.put(TOTAL_SIZE_PROP, String.valueOf(totalSize)); + statistic.put(NUM_ROWS_PROP, String.valueOf(rowCount)); + statistic.put(HIVE_LAST_UPDATE_TIME_PROP, String.valueOf(modifyTime / 1000)); + + LOG.info("alter partition {} with statistic {}.", partitionSpec, statistic); + metastoreClient.alterPartition(partitionSpec, statistic, modifyTime); + } + } + + @Override + public void close() throws IOException { + try { + metastoreClient.close(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListener.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListener.java new file mode 100644 index 000000000000..65d25fbc0271 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListener.java @@ -0,0 +1,32 @@ +/* + * 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.sink.partition; + +import org.apache.paimon.manifest.ManifestCommittable; + +import java.io.Closeable; +import java.util.List; + +/** The partition listener. */ +public interface PartitionListener extends Closeable { + + void notifyCommittable(List committables); + + void snapshotState() throws Exception; +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java new file mode 100644 index 000000000000..dbdf77601480 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionListeners.java @@ -0,0 +1,72 @@ +/* + * 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.sink.partition; + +import org.apache.paimon.flink.sink.Committer; +import org.apache.paimon.manifest.ManifestCommittable; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.IOUtils; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** Partition listeners. */ +public class PartitionListeners implements Closeable { + + private final List listeners; + + private PartitionListeners(List listeners) { + this.listeners = listeners; + } + + public void notifyCommittable(List committables) { + for (PartitionListener trigger : listeners) { + trigger.notifyCommittable(committables); + } + } + + public void snapshotState() throws Exception { + for (PartitionListener trigger : listeners) { + trigger.snapshotState(); + } + } + + @Override + public void close() throws IOException { + IOUtils.closeAllQuietly(listeners); + } + + public static PartitionListeners create(Committer.Context context, FileStoreTable table) + throws Exception { + List listeners = new ArrayList<>(); + + ReportHmsListener.create(context.isRestored(), context.stateStore(), table) + .ifPresent(listeners::add); + PartitionMarkDone.create( + context.streamingCheckpointEnabled(), + context.isRestored(), + context.stateStore(), + table) + .ifPresent(listeners::add); + + return new PartitionListeners(listeners); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionMarkDone.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionMarkDone.java index 39438a101b04..8714e0006e7b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionMarkDone.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/PartitionMarkDone.java @@ -33,28 +33,25 @@ import org.apache.flink.api.common.state.OperatorStateStore; -import javax.annotation.Nullable; - -import java.io.Closeable; import java.io.IOException; import java.time.Duration; import java.util.HashSet; import java.util.List; +import java.util.Optional; import java.util.Set; +import static org.apache.paimon.CoreOptions.PARTITION_MARK_DONE_WHEN_END_INPUT; import static org.apache.paimon.flink.FlinkConnectorOptions.PARTITION_IDLE_TIME_TO_DONE; -import static org.apache.paimon.flink.FlinkConnectorOptions.PARTITION_MARK_DONE_WHEN_END_INPUT; /** Mark partition done. */ -public class PartitionMarkDone implements Closeable { +public class PartitionMarkDone implements PartitionListener { private final InternalRowPartitionComputer partitionComputer; private final PartitionMarkDoneTrigger trigger; private final List actions; private final boolean waitCompaction; - @Nullable - public static PartitionMarkDone create( + public static Optional create( boolean isStreaming, boolean isRestored, OperatorStateStore stateStore, @@ -64,14 +61,15 @@ public static PartitionMarkDone create( Options options = coreOptions.toConfiguration(); if (disablePartitionMarkDone(isStreaming, table, options)) { - return null; + return Optional.empty(); } InternalRowPartitionComputer partitionComputer = new InternalRowPartitionComputer( coreOptions.partitionDefaultName(), table.schema().logicalPartitionType(), - table.partitionKeys().toArray(new String[0])); + table.partitionKeys().toArray(new String[0]), + coreOptions.legacyPartitionName()); PartitionMarkDoneTrigger trigger = PartitionMarkDoneTrigger.create(coreOptions, isRestored, stateStore); @@ -86,7 +84,8 @@ public static PartitionMarkDone create( && (coreOptions.deletionVectorsEnabled() || coreOptions.mergeEngine() == MergeEngine.FIRST_ROW); - return new PartitionMarkDone(partitionComputer, trigger, actions, waitCompaction); + return Optional.of( + new PartitionMarkDone(partitionComputer, trigger, actions, waitCompaction)); } private static boolean disablePartitionMarkDone( @@ -115,6 +114,7 @@ public PartitionMarkDone( this.waitCompaction = waitCompaction; } + @Override public void notifyCommittable(List committables) { Set partitions = new HashSet<>(); boolean endInput = false; @@ -152,6 +152,7 @@ public static void markDone(List partitions, List restore() throws Exception { List pendingPartitions = new ArrayList<>(); if (isRestored) { - pendingPartitions.addAll(pendingPartitionsState.get().iterator().next()); + Iterator> state = pendingPartitionsState.get().iterator(); + if (state.hasNext()) { + pendingPartitions.addAll(state.next()); + } } return pendingPartitions; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportHmsListener.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportHmsListener.java new file mode 100644 index 000000000000..842dd012e88e --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/partition/ReportHmsListener.java @@ -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. + */ + +package org.apache.paimon.flink.sink.partition; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.flink.FlinkConnectorOptions; +import org.apache.paimon.manifest.ManifestCommittable; +import org.apache.paimon.options.Options; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.paimon.utils.InternalRowPartitionComputer; +import org.apache.paimon.utils.PartitionPathUtils; + +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * This listener will collect data from the newly touched partition and then decide when to trigger + * a report based on the partition's idle time. + */ +public class ReportHmsListener implements PartitionListener { + + @SuppressWarnings("unchecked") + private static final ListStateDescriptor> PENDING_REPORT_STATE_DESC = + new ListStateDescriptor<>( + "pending-report-hms-partition", + new MapSerializer<>(StringSerializer.INSTANCE, LongSerializer.INSTANCE)); + + private final InternalRowPartitionComputer partitionComputer; + private final HmsReporter hmsReporter; + private final ListState> pendingPartitionsState; + private final Map pendingPartitions; + private final long idleTime; + + private ReportHmsListener( + InternalRowPartitionComputer partitionComputer, + HmsReporter hmsReporter, + OperatorStateStore store, + boolean isRestored, + long idleTime) + throws Exception { + this.partitionComputer = partitionComputer; + this.hmsReporter = hmsReporter; + this.pendingPartitionsState = store.getListState(PENDING_REPORT_STATE_DESC); + this.pendingPartitions = new HashMap<>(); + if (isRestored) { + Iterator> it = pendingPartitionsState.get().iterator(); + if (it.hasNext()) { + Map state = it.next(); + pendingPartitions.putAll(state); + } + } + this.idleTime = idleTime; + } + + public void notifyCommittable(List committables) { + Set partition = new HashSet<>(); + boolean endInput = false; + for (ManifestCommittable committable : committables) { + for (CommitMessage commitMessage : committable.fileCommittables()) { + CommitMessageImpl message = (CommitMessageImpl) commitMessage; + if (!message.newFilesIncrement().isEmpty() + || !message.compactIncrement().isEmpty()) { + partition.add( + PartitionPathUtils.generatePartitionPath( + partitionComputer.generatePartValues(message.partition()))); + } + } + if (committable.identifier() == Long.MAX_VALUE) { + endInput = true; + } + } + // append to map + long current = System.currentTimeMillis(); + partition.forEach(p -> pendingPartitions.put(p, current)); + + try { + Map partitions = reportPartition(endInput); + for (Map.Entry entry : partitions.entrySet()) { + hmsReporter.report(entry.getKey(), entry.getValue()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private Map reportPartition(boolean endInput) { + if (endInput) { + return pendingPartitions; + } + + Iterator> iterator = pendingPartitions.entrySet().iterator(); + Map result = new HashMap<>(); + long current = System.currentTimeMillis(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + if (current - entry.getValue() > idleTime) { + result.put(entry.getKey(), entry.getValue()); + iterator.remove(); + } + } + + return result; + } + + public void snapshotState() throws Exception { + pendingPartitionsState.update(Collections.singletonList(pendingPartitions)); + } + + public static Optional create( + boolean isRestored, OperatorStateStore stateStore, FileStoreTable table) + throws Exception { + + CoreOptions coreOptions = table.coreOptions(); + Options options = coreOptions.toConfiguration(); + if (options.get(FlinkConnectorOptions.PARTITION_IDLE_TIME_TO_REPORT_STATISTIC).toMillis() + <= 0) { + return Optional.empty(); + } + + if ((table.partitionKeys().isEmpty())) { + return Optional.empty(); + } + + if (!coreOptions.partitionedTableInMetastore()) { + return Optional.empty(); + } + + if (table.catalogEnvironment().metastoreClientFactory() == null) { + return Optional.empty(); + } + + InternalRowPartitionComputer partitionComputer = + new InternalRowPartitionComputer( + coreOptions.partitionDefaultName(), + table.schema().logicalPartitionType(), + table.partitionKeys().toArray(new String[0]), + coreOptions.legacyPartitionName()); + + return Optional.of( + new ReportHmsListener( + partitionComputer, + new HmsReporter( + table, + table.catalogEnvironment().metastoreClientFactory().create()), + stateStore, + isRestored, + options.get(FlinkConnectorOptions.PARTITION_IDLE_TIME_TO_REPORT_STATISTIC) + .toMillis())); + } + + @Override + public void close() throws IOException { + if (hmsReporter != null) { + hmsReporter.close(); + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java index 52ad896892a7..d4d5dd741681 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortOperator.java @@ -48,6 +48,7 @@ public class SortOperator extends TableStreamOperator private final CompressOptions spillCompression; private final int sinkParallelism; private final MemorySize maxDiskSize; + private final boolean sequenceOrder; private transient BinaryExternalSortBuffer buffer; private transient IOManager ioManager; @@ -60,7 +61,8 @@ public SortOperator( int spillSortMaxNumFiles, CompressOptions spillCompression, int sinkParallelism, - MemorySize maxDiskSize) { + MemorySize maxDiskSize, + boolean sequenceOrder) { this.keyType = keyType; this.rowType = rowType; this.maxMemory = maxMemory; @@ -70,6 +72,7 @@ public SortOperator( this.spillCompression = spillCompression; this.sinkParallelism = sinkParallelism; this.maxDiskSize = maxDiskSize; + this.sequenceOrder = sequenceOrder; } @Override @@ -100,7 +103,8 @@ void initBuffer() { pageSize, spillSortMaxNumFiles, spillCompression, - maxDiskSize); + maxDiskSize, + sequenceOrder); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java index e163ac364026..f590c2fb7fff 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/SortUtils.java @@ -163,7 +163,8 @@ public Tuple2 map(RowData value) { options.localSortMaxNumFileHandles(), options.spillCompressOptions(), sinkParallelism, - options.writeBufferSpillDiskSize())) + options.writeBufferSpillDiskSize(), + options.sequenceFieldSortOrderIsAscending())) .setParallelism(sinkParallelism) // remove the key column from every row .map( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java index 2d32bb8858e0..668aa24c145d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperator.java @@ -23,14 +23,12 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.utils.ExecutorUtils; -import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.common.operators.ProcessingTimeService.ProcessingTimeCallback; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl; import org.apache.flink.types.Either; import java.util.List; @@ -52,21 +50,16 @@ public class AppendBypassCoordinateOperator ProcessingTimeCallback { private static final long MAX_PENDING_TASKS = 5000; - private static final long EMIT_PER_BATCH = 100; private final FileStoreTable table; - private final MailboxExecutorImpl mailbox; private transient ScheduledExecutorService executorService; private transient LinkedBlockingQueue compactTasks; public AppendBypassCoordinateOperator( - FileStoreTable table, - ProcessingTimeService processingTimeService, - MailboxExecutor mailbox) { + FileStoreTable table, ProcessingTimeService processingTimeService) { this.table = table; this.processingTimeService = processingTimeService; - this.mailbox = (MailboxExecutorImpl) mailbox; this.chainingStrategy = ChainingStrategy.HEAD; } @@ -100,14 +93,12 @@ private void asyncPlan(UnawareAppendTableCompactionCoordinator coordinator) { @Override public void onProcessingTime(long time) { - while (mailbox.isIdle()) { - for (int i = 0; i < EMIT_PER_BATCH; i++) { - UnawareAppendCompactionTask task = compactTasks.poll(); - if (task == null) { - return; - } - output.collect(new StreamRecord<>(Either.Right(task))); + while (true) { + UnawareAppendCompactionTask task = compactTasks.poll(); + if (task == null) { + return; } + output.collect(new StreamRecord<>(Either.Right(task))); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java index 416865a98445..7c53e01b47e6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AppendBypassCoordinateOperatorFactory.java @@ -25,15 +25,13 @@ import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; import org.apache.flink.types.Either; /** Factory of {@link AppendBypassCoordinateOperator}. */ public class AppendBypassCoordinateOperatorFactory extends AbstractStreamOperatorFactory> - implements YieldingOperatorFactory>, - OneInputStreamOperatorFactory< - CommitT, Either> { + implements OneInputStreamOperatorFactory< + CommitT, Either> { private final FileStoreTable table; @@ -47,8 +45,7 @@ T createStreamOperator( StreamOperatorParameters> parameters) { AppendBypassCoordinateOperator operator = - new AppendBypassCoordinateOperator<>( - table, processingTimeService, getMailboxExecutor()); + new AppendBypassCoordinateOperator<>(table, processingTimeService); operator.setup( parameters.getContainingTask(), parameters.getStreamConfig(), diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BaseDataTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BaseDataTableSource.java new file mode 100644 index 000000000000..5dbbdcedd82a --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BaseDataTableSource.java @@ -0,0 +1,350 @@ +/* + * 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.source; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.CoreOptions.ChangelogProducer; +import org.apache.paimon.CoreOptions.LogChangelogMode; +import org.apache.paimon.CoreOptions.LogConsistency; +import org.apache.paimon.flink.FlinkConnectorOptions.WatermarkEmitStrategy; +import org.apache.paimon.flink.PaimonDataStreamScanProvider; +import org.apache.paimon.flink.log.LogSourceProvider; +import org.apache.paimon.flink.log.LogStoreTableFactory; +import org.apache.paimon.flink.lookup.FileStoreLookupFunction; +import org.apache.paimon.flink.lookup.LookupRuntimeProviderFactory; +import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.options.ConfigOption; +import org.apache.paimon.options.Options; +import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.DataTable; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.source.TableScan; +import org.apache.paimon.utils.Projection; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.source.LookupTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.connector.source.abilities.SupportsAggregatePushDown; +import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.AggregateExpression; +import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.types.DataType; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.IntStream; + +import static org.apache.paimon.CoreOptions.CHANGELOG_PRODUCER; +import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; +import static org.apache.paimon.CoreOptions.LOG_CONSISTENCY; +import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; +import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_ASYNC; +import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_ASYNC_THREAD_NUMBER; +import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_REMOVE_NORMALIZE; +import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_ALIGNMENT_GROUP; +import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT; +import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL; +import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_EMIT_STRATEGY; +import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_IDLE_TIMEOUT; + +/** + * Table source to create {@link StaticFileStoreSource} or {@link ContinuousFileStoreSource} under + * batch mode or streaming mode. + */ +public abstract class BaseDataTableSource extends FlinkTableSource + implements LookupTableSource, SupportsWatermarkPushDown, SupportsAggregatePushDown { + + private static final List> TIME_TRAVEL_OPTIONS = + Arrays.asList( + CoreOptions.SCAN_TIMESTAMP, + CoreOptions.SCAN_TIMESTAMP_MILLIS, + CoreOptions.SCAN_WATERMARK, + CoreOptions.SCAN_FILE_CREATION_TIME_MILLIS, + CoreOptions.SCAN_SNAPSHOT_ID, + CoreOptions.SCAN_TAG_NAME, + CoreOptions.SCAN_VERSION); + + protected final ObjectIdentifier tableIdentifier; + protected final boolean streaming; + protected final DynamicTableFactory.Context context; + @Nullable protected final LogStoreTableFactory logStoreTableFactory; + + @Nullable protected WatermarkStrategy watermarkStrategy; + protected boolean isBatchCountStar; + + public BaseDataTableSource( + ObjectIdentifier tableIdentifier, + Table table, + boolean streaming, + DynamicTableFactory.Context context, + @Nullable LogStoreTableFactory logStoreTableFactory, + @Nullable Predicate predicate, + @Nullable int[][] projectFields, + @Nullable Long limit, + @Nullable WatermarkStrategy watermarkStrategy, + boolean isBatchCountStar) { + super(table, predicate, projectFields, limit); + this.tableIdentifier = tableIdentifier; + this.streaming = streaming; + this.context = context; + this.logStoreTableFactory = logStoreTableFactory; + this.predicate = predicate; + this.projectFields = projectFields; + this.limit = limit; + this.watermarkStrategy = watermarkStrategy; + this.isBatchCountStar = isBatchCountStar; + } + + @Override + public ChangelogMode getChangelogMode() { + if (!streaming) { + // batch merge all, return insert only + return ChangelogMode.insertOnly(); + } + + if (table.primaryKeys().isEmpty()) { + return ChangelogMode.insertOnly(); + } else { + Options options = Options.fromMap(table.options()); + + if (new CoreOptions(options).mergeEngine() == FIRST_ROW) { + return ChangelogMode.insertOnly(); + } + + if (options.get(SCAN_REMOVE_NORMALIZE)) { + return ChangelogMode.all(); + } + + if (logStoreTableFactory == null + && options.get(CHANGELOG_PRODUCER) != ChangelogProducer.NONE) { + return ChangelogMode.all(); + } + + // optimization: transaction consistency and all changelog mode avoid the generation of + // normalized nodes. See FlinkTableSink.getChangelogMode validation. + return options.get(LOG_CONSISTENCY) == LogConsistency.TRANSACTIONAL + && options.get(LOG_CHANGELOG_MODE) == LogChangelogMode.ALL + ? ChangelogMode.all() + : ChangelogMode.upsert(); + } + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { + if (isBatchCountStar) { + return createCountStarScan(); + } + + LogSourceProvider logSourceProvider = null; + if (logStoreTableFactory != null) { + logSourceProvider = + logStoreTableFactory.createSourceProvider(context, scanContext, projectFields); + } + + WatermarkStrategy watermarkStrategy = this.watermarkStrategy; + Options options = Options.fromMap(table.options()); + if (watermarkStrategy != null) { + WatermarkEmitStrategy emitStrategy = options.get(SCAN_WATERMARK_EMIT_STRATEGY); + if (emitStrategy == WatermarkEmitStrategy.ON_EVENT) { + watermarkStrategy = new OnEventWatermarkStrategy(watermarkStrategy); + } + Duration idleTimeout = options.get(SCAN_WATERMARK_IDLE_TIMEOUT); + if (idleTimeout != null) { + watermarkStrategy = watermarkStrategy.withIdleness(idleTimeout); + } + String watermarkAlignGroup = options.get(SCAN_WATERMARK_ALIGNMENT_GROUP); + if (watermarkAlignGroup != null) { + watermarkStrategy = + WatermarkAlignUtils.withWatermarkAlignment( + watermarkStrategy, + watermarkAlignGroup, + options.get(SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT), + options.get(SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL)); + } + } + + FlinkSourceBuilder sourceBuilder = + new FlinkSourceBuilder(table) + .sourceName(tableIdentifier.asSummaryString()) + .sourceBounded(!streaming) + .logSourceProvider(logSourceProvider) + .projection(projectFields) + .predicate(predicate) + .limit(limit) + .watermarkStrategy(watermarkStrategy) + .dynamicPartitionFilteringFields(dynamicPartitionFilteringFields()); + + return new PaimonDataStreamScanProvider( + !streaming, + env -> + sourceBuilder + .sourceParallelism(inferSourceParallelism(env)) + .env(env) + .build()); + } + + private ScanRuntimeProvider createCountStarScan() { + TableScan scan = table.newReadBuilder().withFilter(predicate).newScan(); + List partitionEntries = scan.listPartitionEntries(); + long rowCount = partitionEntries.stream().mapToLong(PartitionEntry::recordCount).sum(); + NumberSequenceRowSource source = new NumberSequenceRowSource(rowCount, rowCount); + return new SourceProvider() { + @Override + public Source createSource() { + return source; + } + + @Override + public boolean isBounded() { + return true; + } + + @Override + public Optional getParallelism() { + return Optional.of(1); + } + }; + } + + protected abstract List dynamicPartitionFilteringFields(); + + @Override + public void applyWatermark(WatermarkStrategy watermarkStrategy) { + this.watermarkStrategy = watermarkStrategy; + } + + @Override + public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { + if (!(table instanceof FileStoreTable)) { + throw new UnsupportedOperationException( + "Currently, lookup dim table only support FileStoreTable but is " + + table.getClass().getName()); + } + + if (limit != null) { + throw new RuntimeException( + "Limit push down should not happen in Lookup source, but it is " + limit); + } + int[] projection = + projectFields == null + ? IntStream.range(0, table.rowType().getFieldCount()).toArray() + : Projection.of(projectFields).toTopLevelIndexes(); + int[] joinKey = Projection.of(context.getKeys()).toTopLevelIndexes(); + Options options = new Options(table.options()); + boolean enableAsync = options.get(LOOKUP_ASYNC); + int asyncThreadNumber = options.get(LOOKUP_ASYNC_THREAD_NUMBER); + return LookupRuntimeProviderFactory.create( + getFileStoreLookupFunction( + context, + timeTravelDisabledTable((FileStoreTable) table), + projection, + joinKey), + enableAsync, + asyncThreadNumber); + } + + protected FileStoreLookupFunction getFileStoreLookupFunction( + LookupContext context, Table table, int[] projection, int[] joinKey) { + return new FileStoreLookupFunction(table, projection, joinKey, predicate); + } + + private FileStoreTable timeTravelDisabledTable(FileStoreTable table) { + Map newOptions = new HashMap<>(table.options()); + TIME_TRAVEL_OPTIONS.stream().map(ConfigOption::key).forEach(newOptions::remove); + + CoreOptions.StartupMode startupMode = CoreOptions.fromMap(newOptions).startupMode(); + if (startupMode != CoreOptions.StartupMode.COMPACTED_FULL) { + startupMode = CoreOptions.StartupMode.LATEST_FULL; + } + newOptions.put(CoreOptions.SCAN_MODE.key(), startupMode.toString()); + + TableSchema newSchema = table.schema().copy(newOptions); + return table.copy(newSchema); + } + + @Override + public boolean applyAggregates( + List groupingSets, + List aggregateExpressions, + DataType producedDataType) { + if (isStreaming()) { + return false; + } + + if (!(table instanceof DataTable)) { + return false; + } + + if (!table.primaryKeys().isEmpty()) { + return false; + } + + CoreOptions options = ((DataTable) table).coreOptions(); + if (options.deletionVectorsEnabled()) { + return false; + } + + if (groupingSets.size() != 1) { + return false; + } + + if (groupingSets.get(0).length != 0) { + return false; + } + + if (aggregateExpressions.size() != 1) { + return false; + } + + if (!aggregateExpressions + .get(0) + .getFunctionDefinition() + .getClass() + .getName() + .equals( + "org.apache.flink.table.planner.functions.aggfunctions.Count1AggFunction")) { + return false; + } + + isBatchCountStar = true; + return true; + } + + @Override + public String asSummaryString() { + return "Paimon-DataSource"; + } + + @Override + public boolean isStreaming() { + return streaming; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java index c4544426fdc6..53a1b5f63083 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java @@ -18,68 +18,38 @@ package org.apache.paimon.flink.source; -import org.apache.paimon.CoreOptions; -import org.apache.paimon.CoreOptions.ChangelogProducer; -import org.apache.paimon.CoreOptions.LogChangelogMode; -import org.apache.paimon.CoreOptions.LogConsistency; -import org.apache.paimon.flink.FlinkConnectorOptions.WatermarkEmitStrategy; -import org.apache.paimon.flink.PaimonDataStreamScanProvider; -import org.apache.paimon.flink.log.LogSourceProvider; import org.apache.paimon.flink.log.LogStoreTableFactory; -import org.apache.paimon.flink.lookup.FileStoreLookupFunction; -import org.apache.paimon.flink.lookup.LookupRuntimeProviderFactory; -import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.stats.ColStats; +import org.apache.paimon.stats.Statistics; import org.apache.paimon.table.Table; -import org.apache.paimon.utils.Projection; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.source.LookupTableSource.LookupContext; -import org.apache.flink.table.connector.source.LookupTableSource.LookupRuntimeProvider; -import org.apache.flink.table.connector.source.ScanTableSource.ScanContext; -import org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider; +import org.apache.flink.table.connector.source.abilities.SupportsDynamicFiltering; +import org.apache.flink.table.connector.source.abilities.SupportsStatisticReport; import org.apache.flink.table.data.RowData; import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.plan.stats.ColumnStats; import org.apache.flink.table.plan.stats.TableStats; import javax.annotation.Nullable; -import java.time.Duration; +import java.util.AbstractMap; import java.util.Collections; import java.util.List; -import java.util.stream.IntStream; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; -import static org.apache.paimon.CoreOptions.CHANGELOG_PRODUCER; -import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; -import static org.apache.paimon.CoreOptions.LOG_CONSISTENCY; -import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_ASYNC; -import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_ASYNC_THREAD_NUMBER; -import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_REMOVE_NORMALIZE; -import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_ALIGNMENT_GROUP; -import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT; -import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL; -import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_EMIT_STRATEGY; -import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_WATERMARK_IDLE_TIMEOUT; import static org.apache.paimon.utils.Preconditions.checkState; /** - * Table source to create {@link StaticFileStoreSource} or {@link ContinuousFileStoreSource} under - * batch mode or change-tracking is disabled. For streaming mode with change-tracking enabled and - * FULL scan mode, it will create a {@link - * org.apache.flink.connector.base.source.hybrid.HybridSource} of {@code - * LogHybridSourceFactory.FlinkHybridFirstSource} and kafka log source created by {@link - * LogSourceProvider}. + * A {@link BaseDataTableSource} implements {@link SupportsStatisticReport} and {@link + * SupportsDynamicFiltering}. */ -public class DataTableSource extends FlinkTableSource { - - private final ObjectIdentifier tableIdentifier; - private final boolean streaming; - private final DynamicTableFactory.Context context; - @Nullable private final LogStoreTableFactory logStoreTableFactory; - - @Nullable private WatermarkStrategy watermarkStrategy; +public class DataTableSource extends BaseDataTableSource + implements SupportsStatisticReport, SupportsDynamicFiltering { @Nullable private List dynamicPartitionFilteringFields; @@ -99,7 +69,8 @@ public DataTableSource( null, null, null, - null); + null, + false); } public DataTableSource( @@ -112,103 +83,22 @@ public DataTableSource( @Nullable int[][] projectFields, @Nullable Long limit, @Nullable WatermarkStrategy watermarkStrategy, - @Nullable List dynamicPartitionFilteringFields) { - super(table, predicate, projectFields, limit); - this.tableIdentifier = tableIdentifier; - this.streaming = streaming; - this.context = context; - this.logStoreTableFactory = logStoreTableFactory; - this.predicate = predicate; - this.projectFields = projectFields; - this.limit = limit; - this.watermarkStrategy = watermarkStrategy; + @Nullable List dynamicPartitionFilteringFields, + boolean isBatchCountStar) { + super( + tableIdentifier, + table, + streaming, + context, + logStoreTableFactory, + predicate, + projectFields, + limit, + watermarkStrategy, + isBatchCountStar); this.dynamicPartitionFilteringFields = dynamicPartitionFilteringFields; } - @Override - public ChangelogMode getChangelogMode() { - if (!streaming) { - // batch merge all, return insert only - return ChangelogMode.insertOnly(); - } - - if (table.primaryKeys().isEmpty()) { - return ChangelogMode.insertOnly(); - } else { - Options options = Options.fromMap(table.options()); - - if (new CoreOptions(options).mergeEngine() == CoreOptions.MergeEngine.FIRST_ROW) { - return ChangelogMode.insertOnly(); - } - - if (options.get(SCAN_REMOVE_NORMALIZE)) { - return ChangelogMode.all(); - } - - if (logStoreTableFactory == null - && options.get(CHANGELOG_PRODUCER) != ChangelogProducer.NONE) { - return ChangelogMode.all(); - } - - // optimization: transaction consistency and all changelog mode avoid the generation of - // normalized nodes. See FlinkTableSink.getChangelogMode validation. - return options.get(LOG_CONSISTENCY) == LogConsistency.TRANSACTIONAL - && options.get(LOG_CHANGELOG_MODE) == LogChangelogMode.ALL - ? ChangelogMode.all() - : ChangelogMode.upsert(); - } - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { - LogSourceProvider logSourceProvider = null; - if (logStoreTableFactory != null) { - logSourceProvider = - logStoreTableFactory.createSourceProvider(context, scanContext, projectFields); - } - - WatermarkStrategy watermarkStrategy = this.watermarkStrategy; - Options options = Options.fromMap(table.options()); - if (watermarkStrategy != null) { - WatermarkEmitStrategy emitStrategy = options.get(SCAN_WATERMARK_EMIT_STRATEGY); - if (emitStrategy == WatermarkEmitStrategy.ON_EVENT) { - watermarkStrategy = new OnEventWatermarkStrategy(watermarkStrategy); - } - Duration idleTimeout = options.get(SCAN_WATERMARK_IDLE_TIMEOUT); - if (idleTimeout != null) { - watermarkStrategy = watermarkStrategy.withIdleness(idleTimeout); - } - String watermarkAlignGroup = options.get(SCAN_WATERMARK_ALIGNMENT_GROUP); - if (watermarkAlignGroup != null) { - watermarkStrategy = - WatermarkAlignUtils.withWatermarkAlignment( - watermarkStrategy, - watermarkAlignGroup, - options.get(SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT), - options.get(SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL)); - } - } - - FlinkSourceBuilder sourceBuilder = - new FlinkSourceBuilder(table) - .sourceName(tableIdentifier.asSummaryString()) - .sourceBounded(!streaming) - .logSourceProvider(logSourceProvider) - .projection(projectFields) - .predicate(predicate) - .limit(limit) - .watermarkStrategy(watermarkStrategy) - .dynamicPartitionFilteringFields(dynamicPartitionFilteringFields); - - return new PaimonDataStreamScanProvider( - !streaming, - env -> - sourceBuilder - .sourceParallelism(inferSourceParallelism(env)) - .env(env) - .build()); - } - @Override public DataTableSource copy() { return new DataTableSource( @@ -221,32 +111,8 @@ public DataTableSource copy() { projectFields, limit, watermarkStrategy, - dynamicPartitionFilteringFields); - } - - @Override - public void pushWatermark(WatermarkStrategy watermarkStrategy) { - this.watermarkStrategy = watermarkStrategy; - } - - @Override - public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { - if (limit != null) { - throw new RuntimeException( - "Limit push down should not happen in Lookup source, but it is " + limit); - } - int[] projection = - projectFields == null - ? IntStream.range(0, table.rowType().getFieldCount()).toArray() - : Projection.of(projectFields).toTopLevelIndexes(); - int[] joinKey = Projection.of(context.getKeys()).toTopLevelIndexes(); - Options options = new Options(table.options()); - boolean enableAsync = options.get(LOOKUP_ASYNC); - int asyncThreadNumber = options.get(LOOKUP_ASYNC_THREAD_NUMBER); - return LookupRuntimeProviderFactory.create( - new FileStoreLookupFunction(table, projection, joinKey, predicate), - enableAsync, - asyncThreadNumber); + dynamicPartitionFilteringFields, + isBatchCountStar); } @Override @@ -254,16 +120,25 @@ public TableStats reportStatistics() { if (streaming) { return TableStats.UNKNOWN; } - + Optional optionStatistics = table.statistics(); + if (optionStatistics.isPresent()) { + Statistics statistics = optionStatistics.get(); + if (statistics.mergedRecordCount().isPresent()) { + Map flinkColStats = + statistics.colStats().entrySet().stream() + .map( + entry -> + new AbstractMap.SimpleEntry<>( + entry.getKey(), + toFlinkColumnStats(entry.getValue()))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + return new TableStats(statistics.mergedRecordCount().getAsLong(), flinkColStats); + } + } scanSplitsForInference(); return new TableStats(splitStatistics.totalRowCount()); } - @Override - public String asSummaryString() { - return "Paimon-DataSource"; - } - @Override public List listAcceptedFilterFields() { // note that streaming query doesn't support dynamic filtering @@ -286,7 +161,26 @@ public void applyDynamicFiltering(List candidateFilterFields) { } @Override - public boolean isStreaming() { - return streaming; + protected List dynamicPartitionFilteringFields() { + return dynamicPartitionFilteringFields; + } + + private ColumnStats toFlinkColumnStats(ColStats colStats) { + return ColumnStats.Builder.builder() + .setNdv( + colStats.distinctCount().isPresent() + ? colStats.distinctCount().getAsLong() + : null) + .setNullCount( + colStats.nullCount().isPresent() ? colStats.nullCount().getAsLong() : null) + .setAvgLen( + colStats.avgLen().isPresent() + ? (double) colStats.avgLen().getAsLong() + : null) + .setMaxLen( + colStats.maxLen().isPresent() ? (int) colStats.maxLen().getAsLong() : null) + .setMax(colStats.max().isPresent() ? colStats.max().get() : null) + .setMin(colStats.min().isPresent() ? colStats.min().get() : null) + .build(); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java index 3131ae0e0afa..ed94043c035d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java @@ -34,6 +34,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.utils.StringUtils; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; @@ -63,6 +64,8 @@ import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType; import static org.apache.paimon.CoreOptions.StreamingReadMode.FILE; +import static org.apache.paimon.flink.FlinkConnectorOptions.SOURCE_OPERATOR_UID_SUFFIX; +import static org.apache.paimon.flink.FlinkConnectorOptions.generateCustomUid; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.paimon.utils.Preconditions.checkState; @@ -73,6 +76,7 @@ * @since 0.8 */ public class FlinkSourceBuilder { + private static final String SOURCE_NAME = "Source"; private final Table table; private final Options conf; @@ -210,6 +214,14 @@ private DataStream toDataStream(Source source) { : watermarkStrategy, sourceName, produceTypeInfo()); + + String uidSuffix = table.options().get(SOURCE_OPERATOR_UID_SUFFIX.key()); + if (!StringUtils.isNullOrWhitespaceOnly(uidSuffix)) { + dataStream = + (DataStreamSource) + dataStream.uid(generateCustomUid(SOURCE_NAME, table.name(), uidSuffix)); + } + if (parallelism != null) { dataStream.setParallelism(parallelism); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java index 7254eefaa435..2be0248f3ce8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkTableSource.java @@ -22,26 +22,23 @@ import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.LogicalTypeConversion; import org.apache.paimon.flink.PredicateConverter; +import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.PartitionPredicateVisitor; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.predicate.PredicateBuilder; import org.apache.paimon.predicate.PredicateVisitor; +import org.apache.paimon.table.DataTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.Split; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.source.LookupTableSource.LookupContext; -import org.apache.flink.table.connector.source.LookupTableSource.LookupRuntimeProvider; import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.connector.source.ScanTableSource.ScanContext; -import org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider; -import org.apache.flink.table.data.RowData; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.plan.stats.TableStats; import org.apache.flink.table.types.logical.RowType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,7 +52,11 @@ import static org.apache.paimon.options.OptionsUtils.PAIMON_PREFIX; /** A Flink {@link ScanTableSource} for paimon. */ -public abstract class FlinkTableSource { +public abstract class FlinkTableSource + implements ScanTableSource, + SupportsFilterPushDown, + SupportsProjectionPushDown, + SupportsLimitPushDown { private static final Logger LOG = LoggerFactory.getLogger(FlinkTableSource.class); @@ -85,8 +86,8 @@ public FlinkTableSource( this.limit = limit; } - /** @return The unconsumed filters. */ - public List pushFilters(List filters) { + @Override + public Result applyFilters(List filters) { List partitionKeys = table.partitionKeys(); RowType rowType = LogicalTypeConversion.toLogicalType(table.rowType()); @@ -95,7 +96,8 @@ public List pushFilters(List filters) { List unConsumedFilters = new ArrayList<>(); List consumedFilters = new ArrayList<>(); List converted = new ArrayList<>(); - PredicateVisitor visitor = new PartitionPredicateVisitor(partitionKeys); + PredicateVisitor onlyPartFieldsVisitor = + new PartitionPredicateVisitor(partitionKeys); for (ResolvedExpression filter : filters) { Optional predicateOptional = PredicateConverter.convert(rowType, filter); @@ -104,7 +106,7 @@ public List pushFilters(List filters) { unConsumedFilters.add(filter); } else { Predicate p = predicateOptional.get(); - if (isStreaming() || !p.visit(visitor)) { + if (isStreaming() || !p.visit(onlyPartFieldsVisitor)) { unConsumedFilters.add(filter); } else { consumedFilters.add(filter); @@ -115,35 +117,24 @@ public List pushFilters(List filters) { predicate = converted.isEmpty() ? null : PredicateBuilder.and(converted); LOG.info("Consumed filters: {} of {}", consumedFilters, filters); - return unConsumedFilters; + return Result.of(filters, unConsumedFilters); } - public void pushProjection(int[][] projectedFields) { + @Override + public boolean supportsNestedProjection() { + return false; + } + + @Override + public void applyProjection(int[][] projectedFields) { this.projectFields = projectedFields; } - public void pushLimit(long limit) { + @Override + public void applyLimit(long limit) { this.limit = limit; } - public abstract ChangelogMode getChangelogMode(); - - public abstract ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext); - - public abstract void pushWatermark(WatermarkStrategy watermarkStrategy); - - public abstract LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context); - - public abstract TableStats reportStatistics(); - - public abstract FlinkTableSource copy(); - - public abstract String asSummaryString(); - - public abstract List listAcceptedFilterFields(); - - public abstract void applyDynamicFiltering(List candidateFilterFields); - public abstract boolean isStreaming(); @Nullable @@ -180,9 +171,28 @@ protected Integer inferSourceParallelism(StreamExecutionEnvironment env) { protected void scanSplitsForInference() { if (splitStatistics == null) { - List splits = - table.newReadBuilder().withFilter(predicate).newScan().plan().splits(); - splitStatistics = new SplitStatistics(splits); + if (table instanceof DataTable) { + List partitionEntries = + table.newReadBuilder() + .withFilter(predicate) + .newScan() + .listPartitionEntries(); + long totalSize = 0; + long rowCount = 0; + for (PartitionEntry entry : partitionEntries) { + totalSize += entry.fileSizeInBytes(); + rowCount += entry.recordCount(); + } + long splitTargetSize = ((DataTable) table).coreOptions().splitTargetSize(); + splitStatistics = + new SplitStatistics((int) (totalSize / splitTargetSize + 1), rowCount); + } else { + List splits = + table.newReadBuilder().withFilter(predicate).newScan().plan().splits(); + splitStatistics = + new SplitStatistics( + splits.size(), splits.stream().mapToLong(Split::rowCount).sum()); + } } } @@ -192,9 +202,9 @@ protected static class SplitStatistics { private final int splitNumber; private final long totalRowCount; - protected SplitStatistics(List splits) { - this.splitNumber = splits.size(); - this.totalRowCount = splits.stream().mapToLong(Split::rowCount).sum(); + protected SplitStatistics(int splitNumber, long totalRowCount) { + this.splitNumber = splitNumber; + this.totalRowCount = totalRowCount; } public int splitNumber() { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NumberSequenceRowSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NumberSequenceRowSource.java new file mode 100644 index 000000000000..e467e37a87c2 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NumberSequenceRowSource.java @@ -0,0 +1,436 @@ +/* + * 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.source; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.NumberSequenceIterator; +import org.apache.flink.util.SplittableIterator; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.NoSuchElementException; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** A data source that produces a sequence of numbers (longs) to {@link RowData}. */ +public class NumberSequenceRowSource + implements Source< + RowData, + NumberSequenceRowSource.NumberSequenceSplit, + Collection>, + ResultTypeQueryable { + + private static final long serialVersionUID = 1L; + + /** The starting number in the sequence, inclusive. */ + private final long from; + + /** The end number in the sequence, inclusive. */ + private final long to; + + /** + * Creates a new {@code NumberSequenceSource} that produces parallel sequences covering the + * range {@code from} to {@code to} (both boundaries are inclusive). + */ + public NumberSequenceRowSource(long from, long to) { + checkArgument(from <= to, "'from' must be <= 'to'"); + this.from = from; + this.to = to; + } + + public long getFrom() { + return from; + } + + public long getTo() { + return to; + } + + // ------------------------------------------------------------------------ + // source methods + // ------------------------------------------------------------------------ + + @Override + public TypeInformation getProducedType() { + return InternalTypeInfo.of(RowType.of(new BigIntType(false))); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader( + SourceReaderContext readerContext) { + return new IteratorSourceReader<>(readerContext); + } + + @Override + public SplitEnumerator> createEnumerator( + final SplitEnumeratorContext enumContext) { + + final List splits = + splitNumberRange(from, to, enumContext.currentParallelism()); + return new IteratorSourceEnumerator<>(enumContext, splits); + } + + @Override + public SplitEnumerator> restoreEnumerator( + final SplitEnumeratorContext enumContext, + Collection checkpoint) { + return new IteratorSourceEnumerator<>(enumContext, checkpoint); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new SplitSerializer(); + } + + @Override + public SimpleVersionedSerializer> + getEnumeratorCheckpointSerializer() { + return new CheckpointSerializer(); + } + + protected List splitNumberRange(long from, long to, int numSplits) { + final NumberSequenceIterator[] subSequences = + new NumberSequenceIterator(from, to).split(numSplits); + final ArrayList splits = new ArrayList<>(subSequences.length); + + int splitId = 1; + for (NumberSequenceIterator seq : subSequences) { + if (seq.hasNext()) { + splits.add( + new NumberSequenceSplit( + String.valueOf(splitId++), seq.getCurrent(), seq.getTo())); + } + } + + return splits; + } + + // ------------------------------------------------------------------------ + // splits & checkpoint + // ------------------------------------------------------------------------ + + /** A split of the source, representing a number sub-sequence. */ + public static class NumberSequenceSplit + implements IteratorSourceSplit< + RowData, NumberSequenceRowSource.NumberSequenceIterator> { + + private final String splitId; + private final long from; + private final long to; + + public NumberSequenceSplit(String splitId, long from, long to) { + checkArgument(from <= to, "'from' must be <= 'to'"); + this.splitId = checkNotNull(splitId); + this.from = from; + this.to = to; + } + + @Override + public String splitId() { + return splitId; + } + + public long from() { + return from; + } + + public long to() { + return to; + } + + @SuppressWarnings("ClassEscapesDefinedScope") + @Override + public NumberSequenceRowSource.NumberSequenceIterator getIterator() { + return new NumberSequenceRowSource.NumberSequenceIterator(from, to); + } + + @SuppressWarnings("ClassEscapesDefinedScope") + @Override + public IteratorSourceSplit + getUpdatedSplitForIterator( + final NumberSequenceRowSource.NumberSequenceIterator iterator) { + return new NumberSequenceSplit(splitId, iterator.getCurrent(), iterator.getTo()); + } + + @Override + public String toString() { + return String.format("NumberSequenceSplit [%d, %d] (%s)", from, to, splitId); + } + } + + private static final class SplitSerializer + implements SimpleVersionedSerializer { + + private static final int CURRENT_VERSION = 1; + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(NumberSequenceSplit split) throws IOException { + checkArgument( + split.getClass() == NumberSequenceSplit.class, "cannot serialize subclasses"); + + // We will serialize 2 longs (16 bytes) plus the UFT representation of the string (2 + + // length) + final DataOutputSerializer out = + new DataOutputSerializer(split.splitId().length() + 18); + serializeV1(out, split); + return out.getCopyOfBuffer(); + } + + @Override + public NumberSequenceSplit deserialize(int version, byte[] serialized) throws IOException { + if (version != CURRENT_VERSION) { + throw new IOException("Unrecognized version: " + version); + } + final DataInputDeserializer in = new DataInputDeserializer(serialized); + return deserializeV1(in); + } + + static void serializeV1(DataOutputView out, NumberSequenceSplit split) throws IOException { + out.writeUTF(split.splitId()); + out.writeLong(split.from()); + out.writeLong(split.to()); + } + + static NumberSequenceSplit deserializeV1(DataInputView in) throws IOException { + return new NumberSequenceSplit(in.readUTF(), in.readLong(), in.readLong()); + } + } + + private static final class CheckpointSerializer + implements SimpleVersionedSerializer> { + + private static final int CURRENT_VERSION = 1; + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(Collection checkpoint) throws IOException { + // Each split needs 2 longs (16 bytes) plus the UFT representation of the string (2 + + // length) + // Assuming at most 4 digit split IDs, 22 bytes per split avoids any intermediate array + // resizing. + // plus four bytes for the length field + final DataOutputSerializer out = new DataOutputSerializer(checkpoint.size() * 22 + 4); + out.writeInt(checkpoint.size()); + for (NumberSequenceSplit split : checkpoint) { + SplitSerializer.serializeV1(out, split); + } + return out.getCopyOfBuffer(); + } + + @Override + public Collection deserialize(int version, byte[] serialized) + throws IOException { + if (version != CURRENT_VERSION) { + throw new IOException("Unrecognized version: " + version); + } + final DataInputDeserializer in = new DataInputDeserializer(serialized); + final int num = in.readInt(); + final ArrayList result = new ArrayList<>(num); + for (int remaining = num; remaining > 0; remaining--) { + result.add(SplitSerializer.deserializeV1(in)); + } + return result; + } + } + + private static class NumberSequenceIterator extends SplittableIterator { + + private static final long serialVersionUID = 1L; + + /** The last number returned by the iterator. */ + private final long to; + + /** The next number to be returned. */ + private long current; + + /** + * Creates a new splittable iterator, returning the range [from, to]. Both boundaries of the + * interval are inclusive. + * + * @param from The first number returned by the iterator. + * @param to The last number returned by the iterator. + */ + public NumberSequenceIterator(long from, long to) { + if (from > to) { + throw new IllegalArgumentException( + "The 'to' value must not be smaller than the 'from' value."); + } + + this.current = from; + this.to = to; + } + + /** + * Internal constructor to allow for empty iterators. + * + * @param from The first number returned by the iterator. + * @param to The last number returned by the iterator. + * @param unused A dummy parameter to disambiguate the constructor. + */ + @SuppressWarnings("unused") + private NumberSequenceIterator(long from, long to, boolean unused) { + this.current = from; + this.to = to; + } + + public long getCurrent() { + return this.current; + } + + public long getTo() { + return this.to; + } + + @Override + public boolean hasNext() { + return current <= to; + } + + @Override + public RowData next() { + if (current <= to) { + return GenericRowData.of(current++); + } else { + throw new NoSuchElementException(); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public NumberSequenceIterator[] split(int numPartitions) { + if (numPartitions < 1) { + throw new IllegalArgumentException("The number of partitions must be at least 1."); + } + + if (numPartitions == 1) { + return new NumberSequenceIterator[] {new NumberSequenceIterator(current, to)}; + } + + // here, numPartitions >= 2 !!! + + long elementsPerSplit; + + if (to - current + 1 >= 0) { + elementsPerSplit = (to - current + 1) / numPartitions; + } else { + // long overflow of the range. + // we compute based on half the distance, to prevent the overflow. + // in most cases it holds that: current < 0 and to > 0, except for: to == 0 and + // current + // == Long.MIN_VALUE + // the later needs a special case + final long halfDiff; // must be positive + + if (current == Long.MIN_VALUE) { + // this means to >= 0 + halfDiff = (Long.MAX_VALUE / 2 + 1) + to / 2; + } else { + long posFrom = -current; + if (posFrom > to) { + halfDiff = to + ((posFrom - to) / 2); + } else { + halfDiff = posFrom + ((to - posFrom) / 2); + } + } + elementsPerSplit = halfDiff / numPartitions * 2; + } + + // figure out how many get one in addition + long numWithExtra = -(elementsPerSplit * numPartitions) + to - current + 1; + + // based on rounding errors, we may have lost one + if (numWithExtra > numPartitions) { + elementsPerSplit++; + numWithExtra -= numPartitions; + + if (numWithExtra > numPartitions) { + throw new RuntimeException("Bug in splitting logic. Too much rounding loss."); + } + } + + NumberSequenceIterator[] iters = new NumberSequenceIterator[numPartitions]; + long curr = current; + int i = 0; + for (; i < numWithExtra; i++) { + long next = curr + elementsPerSplit + 1; + iters[i] = new NumberSequenceIterator(curr, next - 1); + curr = next; + } + for (; i < numPartitions; i++) { + long next = curr + elementsPerSplit; + iters[i] = new NumberSequenceIterator(curr, next - 1, true); + curr = next; + } + + return iters; + } + + @Override + public int getMaximumNumberOfSplits() { + if (to >= Integer.MAX_VALUE + || current <= Integer.MIN_VALUE + || to - current + 1 >= Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } else { + return (int) (to - current + 1); + } + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/StaticFileStoreSplitEnumerator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/StaticFileStoreSplitEnumerator.java index abd12aa37736..7bd8ff990f86 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/StaticFileStoreSplitEnumerator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/StaticFileStoreSplitEnumerator.java @@ -21,12 +21,15 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.flink.source.assigners.DynamicPartitionPruningAssigner; +import org.apache.paimon.flink.source.assigners.PreAssignSplitAssigner; import org.apache.paimon.flink.source.assigners.SplitAssigner; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; +import org.apache.flink.table.connector.source.DynamicFilteringEvent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,7 +42,8 @@ /** A {@link SplitEnumerator} implementation for {@link StaticFileStoreSource} input. */ public class StaticFileStoreSplitEnumerator - implements SplitEnumerator { + implements SplitEnumerator, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(StaticFileStoreSplitEnumerator.class); @@ -116,6 +120,17 @@ public Snapshot snapshot() { return snapshot; } + @Override + public void handleSourceEvent(int subtaskId, int attemptNumber, SourceEvent sourceEvent) { + // Only recognize events that don't care attemptNumber. + handleSourceEvent(subtaskId, sourceEvent); + } + + /** + * When to support a new kind of event, pay attention that whether the new event can be sent + * multiple times from different attempts of one subtask. If so, it should be handled via method + * {@link #handleSourceEvent(int, int, SourceEvent)} + */ @Override public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { if (sourceEvent instanceof ReaderConsumeProgressEvent) { @@ -128,13 +143,23 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { checkNotNull( dynamicPartitionFilteringInfo, "Cannot apply dynamic filtering because dynamicPartitionFilteringInfo hasn't been set."); - this.splitAssigner = - DynamicPartitionPruningAssigner.createDynamicPartitionPruningAssignerIfNeeded( - subtaskId, - splitAssigner, - dynamicPartitionFilteringInfo.getPartitionRowProjection(), - sourceEvent, - LOG); + + if (splitAssigner instanceof PreAssignSplitAssigner) { + this.splitAssigner = + ((PreAssignSplitAssigner) splitAssigner) + .ofDynamicPartitionPruning( + dynamicPartitionFilteringInfo.getPartitionRowProjection(), + ((DynamicFilteringEvent) sourceEvent).getData()); + } else { + this.splitAssigner = + DynamicPartitionPruningAssigner + .createDynamicPartitionPruningAssignerIfNeeded( + subtaskId, + splitAssigner, + dynamicPartitionFilteringInfo.getPartitionRowProjection(), + sourceEvent, + LOG); + } } else { LOG.error("Received unrecognized event: {}", sourceEvent); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SystemTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SystemTableSource.java index 49ed0c0b8368..e914c617fffb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SystemTableSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SystemTableSource.java @@ -32,17 +32,12 @@ import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.source.LookupTableSource; import org.apache.flink.table.connector.source.ScanTableSource.ScanContext; import org.apache.flink.table.connector.source.ScanTableSource.ScanRuntimeProvider; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.plan.stats.TableStats; import javax.annotation.Nullable; -import java.util.Collections; -import java.util.List; - /** A {@link FlinkTableSource} for system table. */ public class SystemTableSource extends FlinkTableSource { @@ -127,36 +122,6 @@ public String asSummaryString() { return "Paimon-SystemTable-Source"; } - @Override - public void pushWatermark(WatermarkStrategy watermarkStrategy) { - throw new UnsupportedOperationException(); - } - - @Override - public LookupTableSource.LookupRuntimeProvider getLookupRuntimeProvider( - LookupTableSource.LookupContext context) { - throw new UnsupportedOperationException(); - } - - @Override - public TableStats reportStatistics() { - throw new UnsupportedOperationException(); - } - - @Override - public List listAcceptedFilterFields() { - // system table doesn't support dynamic filtering - return Collections.emptyList(); - } - - @Override - public void applyDynamicFiltering(List candidateFilterFields) { - throw new UnsupportedOperationException( - String.format( - "Cannot apply dynamic filtering to Paimon system table '%s'.", - table.name())); - } - @Override public boolean isStreaming() { return isStreamingMode; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/assigners/PreAssignSplitAssigner.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/assigners/PreAssignSplitAssigner.java index 400a2e5c54eb..fbb31bd1080a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/assigners/PreAssignSplitAssigner.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/assigners/PreAssignSplitAssigner.java @@ -18,10 +18,15 @@ package org.apache.paimon.flink.source.assigners; +import org.apache.paimon.codegen.Projection; +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.flink.FlinkRowData; import org.apache.paimon.flink.source.FileStoreSourceSplit; +import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.utils.BinPacking; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.table.connector.source.DynamicFilteringData; import javax.annotation.Nullable; @@ -35,29 +40,53 @@ import java.util.Optional; import java.util.Queue; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import static org.apache.paimon.flink.utils.TableScanUtils.getSnapshotId; /** - * Pre-calculate which splits each task should process according to the weight, and then distribute - * the splits fairly. + * Pre-calculate which splits each task should process according to the weight or given + * DynamicFilteringData, and then distribute the splits fairly. */ public class PreAssignSplitAssigner implements SplitAssigner { /** Default batch splits size to avoid exceed `akka.framesize`. */ private final int splitBatchSize; + private final int parallelism; + private final Map> pendingSplitAssignment; private final AtomicInteger numberOfPendingSplits; + private final Collection splits; public PreAssignSplitAssigner( int splitBatchSize, SplitEnumeratorContext context, Collection splits) { + this(splitBatchSize, context.currentParallelism(), splits); + } + + public PreAssignSplitAssigner( + int splitBatchSize, + int parallelism, + Collection splits, + Projection partitionRowProjection, + DynamicFilteringData dynamicFilteringData) { + this( + splitBatchSize, + parallelism, + splits.stream() + .filter(s -> filter(partitionRowProjection, dynamicFilteringData, s)) + .collect(Collectors.toList())); + } + + public PreAssignSplitAssigner( + int splitBatchSize, int parallelism, Collection splits) { this.splitBatchSize = splitBatchSize; - this.pendingSplitAssignment = - createBatchFairSplitAssignment(splits, context.currentParallelism()); + this.parallelism = parallelism; + this.splits = splits; + this.pendingSplitAssignment = createBatchFairSplitAssignment(splits, parallelism); this.numberOfPendingSplits = new AtomicInteger(splits.size()); } @@ -127,4 +156,20 @@ public Optional getNextSnapshotId(int subtask) { public int numberOfRemainingSplits() { return numberOfPendingSplits.get(); } + + public SplitAssigner ofDynamicPartitionPruning( + Projection partitionRowProjection, DynamicFilteringData dynamicFilteringData) { + return new PreAssignSplitAssigner( + splitBatchSize, parallelism, splits, partitionRowProjection, dynamicFilteringData); + } + + private static boolean filter( + Projection partitionRowProjection, + DynamicFilteringData dynamicFilteringData, + FileStoreSourceSplit sourceSplit) { + DataSplit dataSplit = (DataSplit) sourceSplit.split(); + BinaryRow partition = dataSplit.partition(); + FlinkRowData projected = new FlinkRowData(partitionRowProjection.apply(partition)); + return dynamicFilteringData.contains(projected); + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java index b0b5a7784a38..8ec8d5f2c1a2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java @@ -36,8 +36,6 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.operators.StreamSource; -import org.apache.flink.streaming.api.transformations.PartitionTransformation; -import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -135,11 +133,7 @@ public static DataStream buildSource( new MultiUnawareTablesReadOperator(catalogLoader, partitionIdleTime)); } - PartitionTransformation transformation = - new PartitionTransformation<>( - source.getTransformation(), new RebalancePartitioner<>()); - - return new DataStream<>(env, transformation); + return source; } private static Long getPartitionInfo( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java index 54a90ac3670e..e398e09a8451 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java @@ -110,8 +110,7 @@ public static DataStream buildSource( isParallel, name, Boundedness.CONTINUOUS_UNBOUNDED) - .forceNonParallel() - .rebalance(); + .forceNonParallel(); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java index 4dba03cf4341..c501c2519b41 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiUnawareTablesReadOperator.java @@ -30,6 +30,8 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.time.Duration; import java.time.LocalDateTime; @@ -50,6 +52,8 @@ public class MultiUnawareTablesReadOperator MultiTableUnawareAppendCompactionTask, MultiTableUnawareAppendCompactionTask> { private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(MultiUnawareTablesReadOperator.class); + private final Catalog.Loader catalogLoader; private final Duration partitionIdleTime; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java index 97c20ed794a5..80c85f7cdb35 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/ReadOperator.java @@ -54,6 +54,9 @@ public class ReadOperator extends AbstractStreamOperator private transient IOManager ioManager; private transient FileStoreSourceReaderMetrics sourceReaderMetrics; + // we create our own gauge for currentEmitEventTimeLag, because this operator is not a FLIP-27 + // source and Flink can't automatically calculate this metric + private transient long emitEventTimeLag = FileStoreSourceReaderMetrics.UNDEFINED; private transient Counter numRecordsIn; public ReadOperator(ReadBuilder readBuilder) { @@ -65,19 +68,7 @@ public void open() throws Exception { super.open(); this.sourceReaderMetrics = new FileStoreSourceReaderMetrics(getMetricGroup()); - // we create our own gauge for currentEmitEventTimeLag, because this operator is not a - // FLIP-27 source and Flink can't automatically calculate this metric - getMetricGroup() - .gauge( - MetricNames.CURRENT_EMIT_EVENT_TIME_LAG, - () -> { - long eventTime = sourceReaderMetrics.getLatestFileCreationTime(); - if (eventTime == FileStoreSourceReaderMetrics.UNDEFINED) { - return FileStoreSourceReaderMetrics.UNDEFINED; - } else { - return System.currentTimeMillis() - eventTime; - } - }); + getMetricGroup().gauge(MetricNames.CURRENT_EMIT_EVENT_TIME_LAG, () -> emitEventTimeLag); this.numRecordsIn = InternalSourceReaderMetricGroup.wrap(getMetricGroup()) .getIOMetricGroup() @@ -108,6 +99,8 @@ public void processElement(StreamRecord record) throws Exception { try (CloseableIterator iterator = read.createReader(split).toCloseableIterator()) { while (iterator.hasNext()) { + emitEventTimeLag = System.currentTimeMillis() - eventTime; + // each Split is already counted as one input record, // so we don't need to count the first record if (firstRecord) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/PushedRichTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/PushedRichTableSource.java deleted file mode 100644 index 7e55b83f6b8b..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/PushedRichTableSource.java +++ /dev/null @@ -1,65 +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.flink.source.table; - -import org.apache.paimon.flink.source.FlinkTableSource; - -import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; -import org.apache.flink.table.expressions.ResolvedExpression; - -import java.util.List; - -/** A {@link RichTableSource} with push down. */ -public class PushedRichTableSource extends RichTableSource - implements SupportsFilterPushDown, SupportsProjectionPushDown, SupportsLimitPushDown { - - private final FlinkTableSource source; - - public PushedRichTableSource(FlinkTableSource source) { - super(source); - this.source = source; - } - - @Override - public PushedRichTableSource copy() { - return new PushedRichTableSource(source.copy()); - } - - @Override - public Result applyFilters(List filters) { - return Result.of(filters, source.pushFilters(filters)); - } - - @Override - public void applyLimit(long limit) { - source.pushLimit(limit); - } - - @Override - public boolean supportsNestedProjection() { - return false; - } - - @Override - public void applyProjection(int[][] projectedFields) { - source.pushProjection(projectedFields); - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/PushedTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/PushedTableSource.java deleted file mode 100644 index a1389b5bfa56..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/PushedTableSource.java +++ /dev/null @@ -1,65 +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.flink.source.table; - -import org.apache.paimon.flink.source.FlinkTableSource; - -import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; -import org.apache.flink.table.expressions.ResolvedExpression; - -import java.util.List; - -/** The {@link BaseTableSource} with push down. */ -public class PushedTableSource extends BaseTableSource - implements SupportsFilterPushDown, SupportsProjectionPushDown, SupportsLimitPushDown { - - private final FlinkTableSource source; - - public PushedTableSource(FlinkTableSource source) { - super(source); - this.source = source; - } - - @Override - public PushedTableSource copy() { - return new PushedTableSource(source.copy()); - } - - @Override - public Result applyFilters(List filters) { - return Result.of(filters, source.pushFilters(filters)); - } - - @Override - public void applyLimit(long limit) { - source.pushLimit(limit); - } - - @Override - public boolean supportsNestedProjection() { - return false; - } - - @Override - public void applyProjection(int[][] projectedFields) { - source.pushProjection(projectedFields); - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/RichTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/RichTableSource.java deleted file mode 100644 index 4bf0c169b71c..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/table/RichTableSource.java +++ /dev/null @@ -1,76 +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.flink.source.table; - -import org.apache.paimon.flink.source.FlinkTableSource; - -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.table.connector.source.LookupTableSource; -import org.apache.flink.table.connector.source.abilities.SupportsDynamicFiltering; -import org.apache.flink.table.connector.source.abilities.SupportsStatisticReport; -import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.plan.stats.TableStats; - -import java.util.List; - -/** The {@link BaseTableSource} with lookup, watermark, statistic and dynamic filtering. */ -public class RichTableSource extends BaseTableSource - implements LookupTableSource, - SupportsWatermarkPushDown, - SupportsStatisticReport, - SupportsDynamicFiltering { - - private final FlinkTableSource source; - - public RichTableSource(FlinkTableSource source) { - super(source); - this.source = source; - } - - @Override - public RichTableSource copy() { - return new RichTableSource(source.copy()); - } - - @Override - public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { - return source.getLookupRuntimeProvider(context); - } - - @Override - public void applyWatermark(WatermarkStrategy watermarkStrategy) { - source.pushWatermark(watermarkStrategy); - } - - @Override - public TableStats reportStatistics() { - return source.reportStatistics(); - } - - @Override - public List listAcceptedFilterFields() { - return source.listAcceptedFilterFields(); - } - - @Override - public void applyDynamicFiltering(List candidateFilterFields) { - source.applyDynamicFiltering(candidateFilterFields); - } -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java index f493dd244dd4..b59c3592a97d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableMigrationUtils.java @@ -34,7 +34,7 @@ public static Migrator getImporter( String connector, Catalog catalog, String sourceDatabase, - String souceTableName, + String sourceTableName, String targetDatabase, String targetTableName, Integer parallelism, @@ -50,7 +50,7 @@ public static Migrator getImporter( return new HiveMigrator( (HiveCatalog) catalog, sourceDatabase, - souceTableName, + sourceTableName, targetDatabase, targetTableName, parallelism, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableScanUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableScanUtils.java index a5645302f93f..30b7bbdd5dc5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableScanUtils.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableScanUtils.java @@ -20,12 +20,15 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.flink.source.FileStoreSourceSplit; +import org.apache.paimon.options.Options; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.TableScan; import java.util.HashMap; +import java.util.HashSet; import java.util.Optional; +import java.util.Set; /** Utility methods for {@link TableScan}, such as validating. */ public class TableScanUtils { @@ -59,4 +62,24 @@ public static Optional getSnapshotId(FileStoreSourceSplit split) { } return Optional.empty(); } + + /** + * Check whether streaming reading is supported based on the data changed before and after + * compact. + */ + public static boolean supportCompactDiffStreamingReading(Table table) { + CoreOptions options = CoreOptions.fromMap(table.options()); + Set compactDiffReadingEngine = + new HashSet() { + { + add(CoreOptions.MergeEngine.PARTIAL_UPDATE); + add(CoreOptions.MergeEngine.AGGREGATE); + } + }; + + return options.needLookup() + && compactDiffReadingEngine.contains(options.mergeEngine()) + && !Options.fromMap(options.toMap()) + .get(CoreOptions.PARTIAL_UPDATE_REMOVE_RECORD_ON_DELETE); + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableStatsUtil.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableStatsUtil.java new file mode 100644 index 000000000000..fe10fa63c049 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/TableStatsUtil.java @@ -0,0 +1,281 @@ +/* + * 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.utils; + +import org.apache.paimon.Snapshot; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.stats.ColStats; +import org.apache.paimon.stats.Statistics; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypeRoot; + +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBase; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBinary; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataBoolean; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataDate; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataDouble; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataLong; +import org.apache.flink.table.catalog.stats.CatalogColumnStatisticsDataString; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; + +import javax.annotation.Nullable; + +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** Utility methods for analysis table. */ +public class TableStatsUtil { + + /** create Paimon statistics. */ + @Nullable + public static Statistics createTableStats( + FileStoreTable table, CatalogTableStatistics catalogTableStatistics) { + Snapshot snapshot = table.snapshotManager().latestSnapshot(); + if (snapshot == null) { + return null; + } + return new Statistics( + snapshot.id(), + snapshot.schemaId(), + catalogTableStatistics.getRowCount(), + catalogTableStatistics.getTotalSize()); + } + + /** Create Paimon statistics from given Flink columnStatistics. */ + @Nullable + public static Statistics createTableColumnStats( + FileStoreTable table, CatalogColumnStatistics columnStatistics) { + if (!table.statistics().isPresent()) { + return null; + } + Statistics statistics = table.statistics().get(); + List fields = table.schema().fields(); + Map> tableColumnStatsMap = new HashMap<>(fields.size()); + for (DataField field : fields) { + CatalogColumnStatisticsDataBase catalogColumnStatisticsDataBase = + columnStatistics.getColumnStatisticsData().get(field.name()); + if (catalogColumnStatisticsDataBase == null) { + continue; + } + tableColumnStatsMap.put( + field.name(), getPaimonColStats(field, catalogColumnStatisticsDataBase)); + } + statistics.colStats().putAll(tableColumnStatsMap); + return statistics; + } + + /** Convert Flink ColumnStats to Paimon ColStats according to Paimon column type. */ + private static ColStats getPaimonColStats( + DataField field, CatalogColumnStatisticsDataBase colStat) { + DataTypeRoot typeRoot = field.type().getTypeRoot(); + if (colStat instanceof CatalogColumnStatisticsDataString) { + CatalogColumnStatisticsDataString stringColStat = + (CatalogColumnStatisticsDataString) colStat; + if (typeRoot.equals(DataTypeRoot.CHAR) || typeRoot.equals(DataTypeRoot.VARCHAR)) { + return ColStats.newColStats( + field.id(), + null != stringColStat.getNdv() ? stringColStat.getNdv() : null, + null, + null, + null != stringColStat.getNullCount() ? stringColStat.getNullCount() : null, + null != stringColStat.getAvgLength() + ? stringColStat.getAvgLength().longValue() + : null, + null != stringColStat.getMaxLength() ? stringColStat.getMaxLength() : null); + } + } else if (colStat instanceof CatalogColumnStatisticsDataBoolean) { + CatalogColumnStatisticsDataBoolean booleanColStat = + (CatalogColumnStatisticsDataBoolean) colStat; + if (typeRoot.equals(DataTypeRoot.BOOLEAN)) { + return ColStats.newColStats( + field.id(), + (booleanColStat.getFalseCount() > 0 ? 1L : 0) + + (booleanColStat.getTrueCount() > 0 ? 1L : 0), + null, + null, + booleanColStat.getNullCount(), + null, + null); + } + } else if (colStat instanceof CatalogColumnStatisticsDataLong) { + CatalogColumnStatisticsDataLong longColStat = (CatalogColumnStatisticsDataLong) colStat; + if (typeRoot.equals(DataTypeRoot.INTEGER)) { + return ColStats.newColStats( + field.id(), + null != longColStat.getNdv() ? longColStat.getNdv() : null, + null != longColStat.getMin() ? longColStat.getMin().intValue() : null, + null != longColStat.getMax() ? longColStat.getMax().intValue() : null, + null != longColStat.getNullCount() ? longColStat.getNullCount() : null, + null, + null); + } else if (typeRoot.equals(DataTypeRoot.TINYINT)) { + return ColStats.newColStats( + field.id(), + null != longColStat.getNdv() ? longColStat.getNdv() : null, + null != longColStat.getMin() ? longColStat.getMin().byteValue() : null, + null != longColStat.getMax() ? longColStat.getMax().byteValue() : null, + null != longColStat.getNullCount() ? longColStat.getNullCount() : null, + null, + null); + + } else if (typeRoot.equals(DataTypeRoot.SMALLINT)) { + return ColStats.newColStats( + field.id(), + null != longColStat.getNdv() ? longColStat.getNdv() : null, + null != longColStat.getMin() ? longColStat.getMin().shortValue() : null, + null != longColStat.getMax() ? longColStat.getMax().shortValue() : null, + null != longColStat.getNullCount() ? longColStat.getNullCount() : null, + null, + null); + } else if (typeRoot.equals(DataTypeRoot.BIGINT)) { + return ColStats.newColStats( + field.id(), + null != longColStat.getNdv() ? longColStat.getNdv() : null, + null != longColStat.getMin() ? longColStat.getMin() : null, + null != longColStat.getMax() ? longColStat.getMax() : null, + null != longColStat.getNullCount() ? longColStat.getNullCount() : null, + null, + null); + } else if (typeRoot.equals(DataTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE)) { + return ColStats.newColStats( + field.id(), + null != longColStat.getNdv() ? longColStat.getNdv() : null, + null != longColStat.getMin() + ? org.apache.paimon.data.Timestamp.fromSQLTimestamp( + new Timestamp(longColStat.getMin())) + : null, + null != longColStat.getMax() + ? org.apache.paimon.data.Timestamp.fromSQLTimestamp( + new Timestamp(longColStat.getMax())) + : null, + null != longColStat.getNullCount() ? longColStat.getNullCount() : null, + null, + null); + } + } else if (colStat instanceof CatalogColumnStatisticsDataDouble) { + CatalogColumnStatisticsDataDouble doubleColumnStatsData = + (CatalogColumnStatisticsDataDouble) colStat; + if (typeRoot.equals(DataTypeRoot.FLOAT)) { + return ColStats.newColStats( + field.id(), + null != doubleColumnStatsData.getNdv() + ? doubleColumnStatsData.getNdv() + : null, + null != doubleColumnStatsData.getMin() + ? doubleColumnStatsData.getMin().floatValue() + : null, + null != doubleColumnStatsData.getMax() + ? doubleColumnStatsData.getMax().floatValue() + : null, + null != doubleColumnStatsData.getNullCount() + ? doubleColumnStatsData.getNullCount() + : null, + null, + null); + } else if (typeRoot.equals(DataTypeRoot.DOUBLE)) { + return ColStats.newColStats( + field.id(), + null != doubleColumnStatsData.getNdv() + ? doubleColumnStatsData.getNdv() + : null, + null != doubleColumnStatsData.getMin() + ? doubleColumnStatsData.getMin() + : null, + null != doubleColumnStatsData.getMax() + ? doubleColumnStatsData.getMax() + : null, + null != doubleColumnStatsData.getNullCount() + ? doubleColumnStatsData.getNullCount() + : null, + null, + null); + } else if (typeRoot.equals(DataTypeRoot.DECIMAL)) { + BigDecimal max = BigDecimal.valueOf(doubleColumnStatsData.getMax()); + BigDecimal min = BigDecimal.valueOf(doubleColumnStatsData.getMin()); + return ColStats.newColStats( + field.id(), + null != doubleColumnStatsData.getNdv() + ? doubleColumnStatsData.getNdv() + : null, + null != doubleColumnStatsData.getMin() + ? Decimal.fromBigDecimal(min, min.precision(), min.scale()) + : null, + null != doubleColumnStatsData.getMax() + ? Decimal.fromBigDecimal(max, max.precision(), max.scale()) + : null, + null != doubleColumnStatsData.getNullCount() + ? doubleColumnStatsData.getNullCount() + : null, + null, + null); + } + } else if (colStat instanceof CatalogColumnStatisticsDataDate) { + CatalogColumnStatisticsDataDate dateColumnStatsData = + (CatalogColumnStatisticsDataDate) colStat; + if (typeRoot.equals(DataTypeRoot.DATE)) { + return ColStats.newColStats( + field.id(), + null != dateColumnStatsData.getNdv() ? dateColumnStatsData.getNdv() : null, + null != dateColumnStatsData.getMin() + ? new Long(dateColumnStatsData.getMin().getDaysSinceEpoch()) + .intValue() + : null, + null != dateColumnStatsData.getMax() + ? new Long(dateColumnStatsData.getMax().getDaysSinceEpoch()) + .intValue() + : null, + null != dateColumnStatsData.getNullCount() + ? dateColumnStatsData.getNullCount() + : null, + null, + null); + } + } else if (colStat instanceof CatalogColumnStatisticsDataBinary) { + CatalogColumnStatisticsDataBinary binaryColumnStatsData = + (CatalogColumnStatisticsDataBinary) colStat; + if (typeRoot.equals(DataTypeRoot.VARBINARY) || typeRoot.equals(DataTypeRoot.BINARY)) { + return ColStats.newColStats( + field.id(), + null, + null, + null, + null != binaryColumnStatsData.getNullCount() + ? binaryColumnStatsData.getNullCount() + : null, + null != binaryColumnStatsData.getAvgLength() + ? binaryColumnStatsData.getAvgLength().longValue() + : null, + null != binaryColumnStatsData.getMaxLength() + ? binaryColumnStatsData.getMaxLength() + : null); + } + } + throw new CatalogException( + String.format( + "Flink does not support convert ColumnStats '%s' for Paimon column " + + "type '%s' yet", + colStat, field.type())); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index 9eb450d33508..6fe5e74ebe0d 100644 --- a/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -21,10 +21,13 @@ org.apache.paimon.flink.action.DropPartitionActionFactory org.apache.paimon.flink.action.DeleteActionFactory org.apache.paimon.flink.action.MergeIntoActionFactory org.apache.paimon.flink.action.RollbackToActionFactory +org.apache.paimon.flink.action.RollbackToTimestampActionFactory org.apache.paimon.flink.action.CreateTagActionFactory org.apache.paimon.flink.action.CreateTagFromTimestampActionFactory org.apache.paimon.flink.action.CreateTagFromWatermarkActionFactory org.apache.paimon.flink.action.DeleteTagActionFactory +org.apache.paimon.flink.action.ExpireTagsActionFactory +org.apache.paimon.flink.action.ReplaceTagActionFactory org.apache.paimon.flink.action.ResetConsumerActionFactory org.apache.paimon.flink.action.MigrateTableActionFactory org.apache.paimon.flink.action.MigrateFileActionFactory @@ -36,6 +39,7 @@ org.apache.paimon.flink.action.MarkPartitionDoneActionFactory org.apache.paimon.flink.action.CreateBranchActionFactory org.apache.paimon.flink.action.DeleteBranchActionFactory org.apache.paimon.flink.action.FastForwardActionFactory +org.apache.paimon.flink.action.RenameTagActionFactory org.apache.paimon.flink.action.RepairActionFactory org.apache.paimon.flink.action.RewriteFileIndexActionFactory org.apache.paimon.flink.action.ExpireSnapshotsActionFactory @@ -48,12 +52,15 @@ org.apache.paimon.flink.procedure.CreateTagProcedure org.apache.paimon.flink.procedure.CreateTagFromTimestampProcedure org.apache.paimon.flink.procedure.CreateTagFromWatermarkProcedure org.apache.paimon.flink.procedure.DeleteTagProcedure +org.apache.paimon.flink.procedure.ExpireTagsProcedure +org.apache.paimon.flink.procedure.ReplaceTagProcedure org.apache.paimon.flink.procedure.CreateBranchProcedure org.apache.paimon.flink.procedure.DeleteBranchProcedure org.apache.paimon.flink.procedure.DropPartitionProcedure org.apache.paimon.flink.procedure.MergeIntoProcedure org.apache.paimon.flink.procedure.ResetConsumerProcedure org.apache.paimon.flink.procedure.RollbackToProcedure +org.apache.paimon.flink.procedure.RollbackToTimestampProcedure org.apache.paimon.flink.procedure.MigrateTableProcedure org.apache.paimon.flink.procedure.MigrateDatabaseProcedure org.apache.paimon.flink.procedure.MigrateFileProcedure @@ -67,6 +74,8 @@ org.apache.paimon.flink.procedure.privilege.DropPrivilegedUserProcedure org.apache.paimon.flink.procedure.privilege.GrantPrivilegeToUserProcedure org.apache.paimon.flink.procedure.privilege.RevokePrivilegeFromUserProcedure org.apache.paimon.flink.procedure.RepairProcedure +org.apache.paimon.flink.procedure.RenameTagProcedure org.apache.paimon.flink.procedure.FastForwardProcedure org.apache.paimon.flink.procedure.MarkPartitionDoneProcedure org.apache.paimon.flink.procedure.CloneProcedure +org.apache.paimon.flink.procedure.CompactManifestProcedure diff --git a/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.format.FileFormatFactory b/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.format.FileFormatFactory new file mode 100644 index 000000000000..6e7553d5c668 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/resources/META-INF/services/org.apache.paimon.format.FileFormatFactory @@ -0,0 +1,18 @@ +# 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. + +org.apache.paimon.flink.compact.changelog.format.CompactedChangelogReadOnlyFormat$OrcFactory +org.apache.paimon.flink.compact.changelog.format.CompactedChangelogReadOnlyFormat$ParquetFactory +org.apache.paimon.flink.compact.changelog.format.CompactedChangelogReadOnlyFormat$AvroFactory diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AbstractFlinkTableFactoryTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AbstractFlinkTableFactoryTest.java index aeb46da8d1b7..38d48fa21d2d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AbstractFlinkTableFactoryTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AbstractFlinkTableFactoryTest.java @@ -63,8 +63,7 @@ public void testSchemaEquals() { @Test public void testGetDynamicOptions() { Configuration configuration = new Configuration(); - configuration.setString("paimon.catalog1.db.T.k1", "v1"); - configuration.setString("paimon.*.db.*.k2", "v2"); + configuration.setString("k1", "v2"); ObjectIdentifier identifier = ObjectIdentifier.of("catalog1", "db", "T"); DynamicTableFactory.Context context = new FactoryUtil.DefaultDynamicTableContext( @@ -74,9 +73,25 @@ public void testGetDynamicOptions() { configuration, AbstractFlinkTableFactoryTest.class.getClassLoader(), false); - Map options = - AbstractFlinkTableFactory.getDynamicTableConfigOptions(context); - assertThat(options).isEqualTo(ImmutableMap.of("k1", "v1", "k2", "v2")); + Map options = AbstractFlinkTableFactory.getDynamicConfigOptions(context); + assertThat(options).isEqualTo(ImmutableMap.of("k1", "v2")); + + configuration = new Configuration(); + configuration.setString("k1", "v2"); + configuration.setString("k3", "v3"); + configuration.setString("paimon.catalog1.db.T.k1", "v1"); + configuration.setString("paimon.*.db.*.k2", "v2"); + identifier = ObjectIdentifier.of("catalog1", "db", "T"); + context = + new FactoryUtil.DefaultDynamicTableContext( + identifier, + null, + new HashMap<>(), + configuration, + AbstractFlinkTableFactoryTest.class.getClassLoader(), + false); + options = AbstractFlinkTableFactory.getDynamicConfigOptions(context); + assertThat(options).isEqualTo(ImmutableMap.of("k1", "v1", "k2", "v2", "k3", "v3")); } private void innerTest(RowType r1, RowType r2, boolean expectEquals) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java index 0c9f4ec6c5e9..e29f8ab56ad7 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; +import org.apache.paimon.catalog.Catalog; import org.apache.paimon.utils.BlockingIterator; import org.apache.flink.table.planner.factories.TestValuesTableFactory; @@ -305,6 +306,22 @@ public void testReadWriteBranch() throws Exception { assertThat(rows).containsExactlyInAnyOrder(Row.of(2), Row.of(1)); } + @Test + public void testBranchNotExist() throws Exception { + // create table + sql("CREATE TABLE T (id INT)"); + // insert data + batchSql("INSERT INTO T VALUES (1)"); + // create tag + paimonTable("T").createTag("tag1", 1); + // create branch + paimonTable("T").createBranch("branch1", "tag1"); + // call the FileSystemCatalog.getDataTableSchema() function + assertThatThrownBy(() -> paimonTable("T$branch_branch2")) + .isInstanceOf(Catalog.TableNotExistException.class) + .hasMessage("Table %s does not exist.", "default.T$branch_branch2"); + } + @Override protected List ddl() { return Arrays.asList( 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 eea8e3a3c217..c30e6cd5612d 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 @@ -19,10 +19,13 @@ package org.apache.paimon.flink; import org.apache.paimon.CoreOptions; +import org.apache.paimon.flink.util.AbstractTestBase; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.utils.BlockingIterator; import org.apache.paimon.utils.DateTimeUtils; +import org.apache.paimon.utils.SnapshotNotExistException; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; import org.apache.flink.util.CloseableIterator; @@ -53,30 +56,22 @@ protected List ddl() { public void testAQEWithWriteManifest() { batchSql("ALTER TABLE T SET ('write-manifest-cache' = '1 mb')"); batchSql("INSERT INTO T VALUES (1, 11, 111), (2, 22, 222)"); - assertThatThrownBy(() -> batchSql("INSERT INTO T SELECT a, b, c FROM T GROUP BY a,b,c")) - .hasMessageContaining( - "Paimon Sink with [Write Manifest Cache] does not support Flink's Adaptive Parallelism mode."); - - // work fine - batchSql( - "INSERT INTO T /*+ OPTIONS('sink.parallelism'='1') */ SELECT a, b, c FROM T GROUP BY a,b,c"); - - // work fine too - batchSql("ALTER TABLE T SET ('write-manifest-cache' = '0 b')"); batchSql("INSERT INTO T SELECT a, b, c FROM T GROUP BY a,b,c"); + assertThat(batchSql("SELECT * FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, 11, 111), + Row.of(2, 22, 222), + Row.of(1, 11, 111), + Row.of(2, 22, 222)); } @Test public void testAQEWithDynamicBucket() { batchSql("CREATE TABLE IF NOT EXISTS D_T (a INT PRIMARY KEY NOT ENFORCED, b INT, c INT)"); batchSql("INSERT INTO T VALUES (1, 11, 111), (2, 22, 222)"); - assertThatThrownBy(() -> batchSql("INSERT INTO D_T SELECT a, b, c FROM T GROUP BY a,b,c")) - .hasMessageContaining( - "Paimon Sink with [Dynamic Bucket Mode] does not support Flink's Adaptive Parallelism mode."); - - // work fine - batchSql( - "INSERT INTO D_T /*+ OPTIONS('sink.parallelism'='1') */ SELECT a, b, c FROM T GROUP BY a,b,c"); + batchSql("INSERT INTO D_T SELECT a, b, c FROM T GROUP BY a,b,c"); + assertThat(batchSql("SELECT * FROM D_T")) + .containsExactlyInAnyOrder(Row.of(1, 11, 111), Row.of(2, 22, 222)); } @Test @@ -117,8 +112,8 @@ public void testTimeTravelRead() throws Exception { assertThatThrownBy(() -> batchSql("SELECT * FROM T /*+ OPTIONS('scan.snapshot-id'='0') */")) .satisfies( anyCauseMatches( - IllegalArgumentException.class, - "The specified scan snapshotId 0 is out of available snapshotId range [1, 4].")); + SnapshotNotExistException.class, + "Specified parameter scan.snapshot-id = 0 is not exist, you can set it in range from 1 to 4.")); assertThatThrownBy( () -> @@ -126,8 +121,8 @@ public void testTimeTravelRead() throws Exception { "SELECT * FROM T /*+ OPTIONS('scan.mode'='from-snapshot-full','scan.snapshot-id'='0') */")) .satisfies( anyCauseMatches( - IllegalArgumentException.class, - "The specified scan snapshotId 0 is out of available snapshotId range [1, 4].")); + SnapshotNotExistException.class, + "Specified parameter scan.snapshot-id = 0 is not exist, you can set it in range from 1 to 4.")); assertThat( batchSql( @@ -536,4 +531,61 @@ public void testScanFromOldSchema() throws InterruptedException { DateTimeUtils.toInternal(timestamp, 0), 0))) .containsExactlyInAnyOrder(Row.of(1, "a"), Row.of(2, "b")); } + + @Test + public void testCountStarAppend() { + sql("CREATE TABLE count_append (f0 INT, f1 STRING)"); + sql("INSERT INTO count_append VALUES (1, 'a'), (2, 'b')"); + + String sql = "SELECT COUNT(*) FROM count_append"; + assertThat(sql(sql)).containsOnly(Row.of(2L)); + validateCount1PushDown(sql); + } + + @Test + public void testCountStarPartAppend() { + sql("CREATE TABLE count_part_append (f0 INT, f1 STRING, dt STRING) PARTITIONED BY (dt)"); + sql("INSERT INTO count_part_append VALUES (1, 'a', '1'), (1, 'a', '1'), (2, 'b', '2')"); + String sql = "SELECT COUNT(*) FROM count_part_append WHERE dt = '1'"; + + assertThat(sql(sql)).containsOnly(Row.of(2L)); + validateCount1PushDown(sql); + } + + @Test + public void testCountStarAppendWithDv() { + sql( + "CREATE TABLE count_append_dv (f0 INT, f1 STRING) WITH ('deletion-vectors.enabled' = 'true')"); + sql("INSERT INTO count_append_dv VALUES (1, 'a'), (2, 'b')"); + + String sql = "SELECT COUNT(*) FROM count_append_dv"; + assertThat(sql(sql)).containsOnly(Row.of(2L)); + validateCount1NotPushDown(sql); + } + + @Test + public void testCountStarPK() { + sql("CREATE TABLE count_pk (f0 INT PRIMARY KEY NOT ENFORCED, f1 STRING)"); + sql("INSERT INTO count_pk VALUES (1, 'a'), (2, 'b')"); + + String sql = "SELECT COUNT(*) FROM count_pk"; + assertThat(sql(sql)).containsOnly(Row.of(2L)); + validateCount1NotPushDown(sql); + } + + private void validateCount1PushDown(String sql) { + Transformation transformation = AbstractTestBase.translate(tEnv, sql); + while (!transformation.getInputs().isEmpty()) { + transformation = transformation.getInputs().get(0); + } + assertThat(transformation.getDescription()).contains("Count1AggFunction"); + } + + private void validateCount1NotPushDown(String sql) { + Transformation transformation = AbstractTestBase.translate(tEnv, sql); + while (!transformation.getInputs().isEmpty()) { + transformation = transformation.getInputs().get(0); + } + assertThat(transformation.getDescription()).doesNotContain("Count1AggFunction"); + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java index 6970eb043b25..1d33a9e8a6f2 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink; +import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.utils.BlockingIterator; import org.apache.paimon.utils.SnapshotManager; @@ -194,44 +195,13 @@ public void testDeleteBranchTable() throws Exception { sql("CALL sys.create_branch('default.T', 'test', 'tag1')"); sql("CALL sys.create_branch('default.T', 'test2', 'tag2')"); - assertThat(collectResult("SELECT branch_name, created_from_snapshot FROM `T$branches`")) - .containsExactlyInAnyOrder("+I[test, 1]", "+I[test2, 2]"); + assertThat(collectResult("SELECT branch_name FROM `T$branches`")) + .containsExactlyInAnyOrder("+I[test]", "+I[test2]"); sql("CALL sys.delete_branch('default.T', 'test')"); - assertThat(collectResult("SELECT branch_name, created_from_snapshot FROM `T$branches`")) - .containsExactlyInAnyOrder("+I[test2, 2]"); - } - - @Test - public void testBranchManagerGetBranchSnapshotsList() throws Exception { - sql( - "CREATE TABLE T (" - + " pt INT" - + ", k INT" - + ", v STRING" - + ", PRIMARY KEY (pt, k) NOT ENFORCED" - + " ) PARTITIONED BY (pt) WITH (" - + " 'bucket' = '2'" - + " )"); - - sql("INSERT INTO T VALUES (1, 10, 'hxh')"); - sql("INSERT INTO T VALUES (1, 20, 'hxh')"); - sql("INSERT INTO T VALUES (1, 30, 'hxh')"); - - FileStoreTable table = paimonTable("T"); - checkSnapshots(table.snapshotManager(), 1, 3); - - sql("CALL sys.create_tag('default.T', 'tag1', 1)"); - sql("CALL sys.create_tag('default.T', 'tag2', 2)"); - sql("CALL sys.create_tag('default.T', 'tag3', 3)"); - - sql("CALL sys.create_branch('default.T', 'test1', 'tag1')"); - sql("CALL sys.create_branch('default.T', 'test2', 'tag2')"); - sql("CALL sys.create_branch('default.T', 'test3', 'tag3')"); - - assertThat(collectResult("SELECT created_from_snapshot FROM `T$branches`")) - .containsExactlyInAnyOrder("+I[1]", "+I[2]", "+I[3]"); + assertThat(collectResult("SELECT branch_name FROM `T$branches`")) + .containsExactlyInAnyOrder("+I[test2]"); } @Test @@ -529,6 +499,43 @@ public void testCannotSetEmptyFallbackBranch() { .satisfies(anyCauseMatches(IllegalArgumentException.class, errMsg)); } + @Test + public void testReadBranchTableWithMultiSchemaIds() throws Exception { + sql( + "CREATE TABLE T (" + + " pt INT" + + ", k INT" + + ", v STRING" + + ", PRIMARY KEY (pt, k) NOT ENFORCED" + + " ) PARTITIONED BY (pt) WITH (" + + " 'bucket' = '2'" + + " )"); + + sql("INSERT INTO T VALUES" + " (1, 10, 'apple')," + " (1, 20, 'banana')"); + + sql("ALTER TABLE `T` ADD (v2 INT)"); + + sql("INSERT INTO T VALUES" + " (2, 10, 'cat', 2)," + " (2, 20, 'dog', 2)"); + + sql("ALTER TABLE `T` ADD (v3 INT)"); + + sql("CALL sys.create_tag('default.T', 'tag1', 2)"); + + sql("CALL sys.create_branch('default.T', 'test', 'tag1')"); + + FileStoreTable table = paimonTable("T"); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location(), "test"); + List schemaIds = schemaManager.listAllIds(); + assertThat(schemaIds.size()).isEqualTo(2); + + assertThat(collectResult("SELECT * FROM T$branch_test")) + .containsExactlyInAnyOrder( + "+I[1, 10, apple, null]", + "+I[1, 20, banana, null]", + "+I[2, 10, cat, 2]", + "+I[2, 20, dog, 2]"); + } + private List collectResult(String sql) throws Exception { List result = new ArrayList<>(); try (CloseableIterator it = tEnv.executeSql(sql).collect()) { 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 975c6a49007f..ba063248ee46 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 @@ -42,10 +42,10 @@ import java.util.stream.Collectors; import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_DML_SYNC; -import static org.apache.paimon.flink.FlinkCatalog.LAST_UPDATE_TIME_KEY; -import static org.apache.paimon.flink.FlinkCatalog.NUM_FILES_KEY; -import static org.apache.paimon.flink.FlinkCatalog.NUM_ROWS_KEY; -import static org.apache.paimon.flink.FlinkCatalog.TOTAL_SIZE_KEY; +import static org.apache.paimon.catalog.Catalog.LAST_UPDATE_TIME_PROP; +import static org.apache.paimon.catalog.Catalog.NUM_FILES_PROP; +import static org.apache.paimon.catalog.Catalog.NUM_ROWS_PROP; +import static org.apache.paimon.catalog.Catalog.TOTAL_SIZE_PROP; import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatCode; @@ -70,14 +70,23 @@ public void testSnapshotsTable() throws Exception { sql("CREATE TABLE T (a INT, b INT)"); sql("INSERT INTO T VALUES (1, 2)"); sql("INSERT INTO T VALUES (3, 4)"); + sql("INSERT INTO T VALUES (5, 6)"); List result = sql("SELECT snapshot_id, schema_id, commit_kind FROM T$snapshots"); - assertThat(result).containsExactly(Row.of(1L, 0L, "APPEND"), Row.of(2L, 0L, "APPEND")); + assertThat(result) + .containsExactly( + Row.of(1L, 0L, "APPEND"), + Row.of(2L, 0L, "APPEND"), + Row.of(3L, 0L, "APPEND")); result = sql( "SELECT snapshot_id, schema_id, commit_kind FROM T$snapshots WHERE schema_id = 0"); - assertThat(result).containsExactly(Row.of(1L, 0L, "APPEND"), Row.of(2L, 0L, "APPEND")); + assertThat(result) + .containsExactly( + Row.of(1L, 0L, "APPEND"), + Row.of(2L, 0L, "APPEND"), + Row.of(3L, 0L, "APPEND")); result = sql( @@ -87,7 +96,7 @@ public void testSnapshotsTable() throws Exception { result = sql( "SELECT snapshot_id, schema_id, commit_kind FROM T$snapshots WHERE snapshot_id > 1"); - assertThat(result).containsExactly(Row.of(2L, 0L, "APPEND")); + assertThat(result).containsExactly(Row.of(2L, 0L, "APPEND"), Row.of(3L, 0L, "APPEND")); result = sql( @@ -97,12 +106,30 @@ public void testSnapshotsTable() throws Exception { result = sql( "SELECT snapshot_id, schema_id, commit_kind FROM T$snapshots WHERE snapshot_id >= 1"); - assertThat(result).contains(Row.of(1L, 0L, "APPEND"), Row.of(2L, 0L, "APPEND")); + assertThat(result) + .contains( + Row.of(1L, 0L, "APPEND"), + Row.of(2L, 0L, "APPEND"), + Row.of(3L, 0L, "APPEND")); result = sql( "SELECT snapshot_id, schema_id, commit_kind FROM T$snapshots WHERE snapshot_id <= 2"); assertThat(result).contains(Row.of(1L, 0L, "APPEND"), Row.of(2L, 0L, "APPEND")); + + result = + sql( + "SELECT snapshot_id, schema_id, commit_kind FROM T$snapshots WHERE snapshot_id in (1, 2)"); + assertThat(result).contains(Row.of(1L, 0L, "APPEND"), Row.of(2L, 0L, "APPEND")); + + result = + sql( + "SELECT snapshot_id, schema_id, commit_kind FROM T$snapshots WHERE snapshot_id in (1, 2) or schema_id=0"); + assertThat(result) + .contains( + Row.of(1L, 0L, "APPEND"), + Row.of(2L, 0L, "APPEND"), + Row.of(3L, 0L, "APPEND")); } @Test @@ -162,7 +189,8 @@ public void testCreateSystemDatabase() { public void testChangeTableInSystemDatabase() { sql("USE sys"); assertThatCode(() -> sql("ALTER TABLE all_table_options SET ('bucket-num' = '5')")) - .hasRootCauseMessage("Can't alter system table."); + .rootCause() + .hasMessageContaining("Only support alter data table, but is: "); } @Test @@ -271,7 +299,7 @@ public void testSchemasTable() { result = sql( "SELECT schema_id, fields, partition_keys, " - + "primary_keys, options, `comment` FROM T$schemas where schema_id>0 and schema_id<3"); + + "primary_keys, options, `comment` FROM T$schemas where schema_id>0 and schema_id<3 order by schema_id"); assertThat(result.toString()) .isEqualTo( "[+I[1, [{\"id\":0,\"name\":\"a\",\"type\":\"INT NOT NULL\"}," @@ -281,6 +309,42 @@ public void testSchemasTable() { + "{\"id\":2,\"name\":\"c\",\"type\":\"STRING\"}], [], [\"a\"], {\"a.aa.aaa\":\"val1\",\"snapshot.time-retained\":\"5 h\"," + "\"b.bb.bbb\":\"val2\",\"snapshot.num-retained.max\":\"20\"}, ]]"); + // test for IN filter + result = + sql( + "SELECT schema_id, fields, partition_keys, " + + "primary_keys, options, `comment` FROM T$schemas where schema_id in (1, 3) order by schema_id"); + assertThat(result.toString()) + .isEqualTo( + "[+I[1, [{\"id\":0,\"name\":\"a\",\"type\":\"INT NOT NULL\"}," + + "{\"id\":1,\"name\":\"b\",\"type\":\"INT\"},{\"id\":2,\"name\":\"c\",\"type\":\"STRING\"}], [], [\"a\"], " + + "{\"a.aa.aaa\":\"val1\",\"snapshot.time-retained\":\"5 h\",\"b.bb.bbb\":\"val2\"}, ], " + + "+I[3, [{\"id\":0,\"name\":\"a\",\"type\":\"INT NOT NULL\"},{\"id\":1,\"name\":\"b\",\"type\":\"INT\"}," + + "{\"id\":2,\"name\":\"c\",\"type\":\"STRING\"}], [], [\"a\"], " + + "{\"a.aa.aaa\":\"val1\",\"snapshot.time-retained\":\"5 h\",\"b.bb.bbb\":\"val2\",\"snapshot.num-retained.max\":\"20\",\"snapshot.num-retained.min\":\"18\"}, ]]"); + + result = + sql( + "SELECT schema_id, fields, partition_keys, " + + "primary_keys, options, `comment` FROM T$schemas where schema_id in (1, 3) or fields='[{\"id\":0,\"name\":\"a\",\"type\":\"INT NOT NULL\"},{\"id\":1,\"name\":\"b\",\"type\":\"INT\"},{\"id\":2,\"name\":\"c\",\"type\":\"STRING\"}]' order by schema_id"); + assertThat(result.toString()) + .isEqualTo( + "[+I[0, [{\"id\":0,\"name\":\"a\",\"type\":\"INT NOT NULL\"}," + + "{\"id\":1,\"name\":\"b\",\"type\":\"INT\"},{\"id\":2,\"name\":\"c\",\"type\":\"STRING\"}], [], [\"a\"], " + + "{\"a.aa.aaa\":\"val1\",\"b.bb.bbb\":\"val2\"}, ], " + + "+I[1, [{\"id\":0,\"name\":\"a\",\"type\":\"INT NOT NULL\"}," + + "{\"id\":1,\"name\":\"b\",\"type\":\"INT\"},{\"id\":2,\"name\":\"c\",\"type\":\"STRING\"}], [], [\"a\"], " + + "{\"a.aa.aaa\":\"val1\",\"snapshot.time-retained\":\"5 h\",\"b.bb.bbb\":\"val2\"}, ], " + + "+I[2, [{\"id\":0,\"name\":\"a\",\"type\":\"INT NOT NULL\"}," + + "{\"id\":1,\"name\":\"b\",\"type\":\"INT\"},{\"id\":2,\"name\":\"c\",\"type\":\"STRING\"}], [], [\"a\"], " + + "{\"a.aa.aaa\":\"val1\",\"snapshot.time-retained\":\"5 h\",\"b.bb.bbb\":\"val2\",\"snapshot.num-retained.max\":\"20\"}, ], " + + "+I[3, [{\"id\":0,\"name\":\"a\",\"type\":\"INT NOT NULL\"},{\"id\":1,\"name\":\"b\",\"type\":\"INT\"}," + + "{\"id\":2,\"name\":\"c\",\"type\":\"STRING\"}], [], [\"a\"], " + + "{\"a.aa.aaa\":\"val1\",\"snapshot.time-retained\":\"5 h\",\"b.bb.bbb\":\"val2\",\"snapshot.num-retained.max\":\"20\",\"snapshot.num-retained.min\":\"18\"}, ], " + + "+I[4, [{\"id\":0,\"name\":\"a\",\"type\":\"INT NOT NULL\"},{\"id\":1,\"name\":\"b\",\"type\":\"INT\"},{\"id\":2,\"name\":\"c\",\"type\":\"STRING\"}], [], [\"a\"], " + + "{\"a.aa.aaa\":\"val1\",\"snapshot.time-retained\":\"5 h\",\"b.bb.bbb\":\"val2\",\"snapshot.num-retained.max\":\"20\",\"manifest.format\":\"avro\"," + + "\"snapshot.num-retained.min\":\"18\"}, ]]"); + // check with not exist schema id assertThatThrownBy( () -> @@ -554,10 +618,10 @@ void testPKTableGetPartition() throws Exception { assertThat(partitionPropertiesMap1) .allSatisfy( (par, properties) -> { - assertThat(properties.get(NUM_ROWS_KEY)).isEqualTo("2"); - assertThat(properties.get(LAST_UPDATE_TIME_KEY)).isNotBlank(); - assertThat(properties.get(NUM_FILES_KEY)).isEqualTo("1"); - assertThat(properties.get(TOTAL_SIZE_KEY)).isNotBlank(); + assertThat(properties.get(NUM_ROWS_PROP)).isEqualTo("2"); + assertThat(properties.get(LAST_UPDATE_TIME_PROP)).isNotBlank(); + assertThat(properties.get(NUM_FILES_PROP)).isEqualTo("1"); + assertThat(properties.get(TOTAL_SIZE_PROP)).isNotBlank(); }); // update p1 data sql("UPDATE PK_T SET word = 'c' WHERE id = 2"); @@ -589,8 +653,8 @@ void testNonPKTableGetPartition() throws Exception { assertThat(partitionPropertiesMap1) .allSatisfy( (par, properties) -> { - assertThat(properties.get(NUM_ROWS_KEY)).isEqualTo("1"); - assertThat(properties.get(LAST_UPDATE_TIME_KEY)).isNotBlank(); + assertThat(properties.get(NUM_ROWS_PROP)).isEqualTo("1"); + assertThat(properties.get(LAST_UPDATE_TIME_PROP)).isNotBlank(); }); // append data to p1 @@ -844,20 +908,35 @@ public void testTagsTable() throws Exception { sql("CREATE TABLE T (a INT, b INT)"); sql("INSERT INTO T VALUES (1, 2)"); sql("INSERT INTO T VALUES (3, 4)"); + sql("INSERT INTO T VALUES (5, 6)"); paimonTable("T").createTag("tag1", 1); paimonTable("T").createTag("tag2", 2); + paimonTable("T").createTag("tag3", 3); List result = sql( "SELECT tag_name, snapshot_id, schema_id, record_count FROM T$tags ORDER BY tag_name"); - - assertThat(result).containsExactly(Row.of("tag1", 1L, 0L, 1L), Row.of("tag2", 2L, 0L, 2L)); + assertThat(result) + .containsExactly( + Row.of("tag1", 1L, 0L, 1L), + Row.of("tag2", 2L, 0L, 2L), + Row.of("tag3", 3L, 0L, 3L)); result = sql( "SELECT tag_name, snapshot_id, schema_id, record_count FROM T$tags where tag_name = 'tag1' "); assertThat(result).containsExactly(Row.of("tag1", 1L, 0L, 1L)); + + result = + sql( + "SELECT tag_name, snapshot_id, schema_id, record_count FROM T$tags where tag_name in ('tag1', 'tag3')"); + assertThat(result).containsExactly(Row.of("tag1", 1L, 0L, 1L), Row.of("tag3", 3L, 0L, 3L)); + + result = + sql( + "SELECT tag_name, snapshot_id, schema_id, record_count FROM T$tags where tag_name in ('tag1') or snapshot_id=2"); + assertThat(result).containsExactly(Row.of("tag1", 1L, 0L, 1L), Row.of("tag2", 2L, 0L, 2L)); } @Test @@ -1063,13 +1142,13 @@ private static void assertPartitionUpdateTo( Long expectedNumFiles) { Map newPartitionProperties = newProperties.get(partition); Map oldPartitionProperties = oldProperties.get(partition); - assertThat(newPartitionProperties.get(NUM_ROWS_KEY)) + assertThat(newPartitionProperties.get(NUM_ROWS_PROP)) .isEqualTo(String.valueOf(expectedNumRows)); - assertThat(Long.valueOf(newPartitionProperties.get(LAST_UPDATE_TIME_KEY))) - .isGreaterThan(Long.valueOf(oldPartitionProperties.get(LAST_UPDATE_TIME_KEY))); - assertThat(newPartitionProperties.get(NUM_FILES_KEY)) + assertThat(Long.valueOf(newPartitionProperties.get(LAST_UPDATE_TIME_PROP))) + .isGreaterThan(Long.valueOf(oldPartitionProperties.get(LAST_UPDATE_TIME_PROP))); + assertThat(newPartitionProperties.get(NUM_FILES_PROP)) .isEqualTo(String.valueOf(expectedNumFiles)); - assertThat(Long.valueOf(newPartitionProperties.get(TOTAL_SIZE_KEY))) - .isGreaterThan(Long.valueOf(oldPartitionProperties.get(TOTAL_SIZE_KEY))); + assertThat(Long.valueOf(newPartitionProperties.get(TOTAL_SIZE_PROP))) + .isGreaterThan(Long.valueOf(oldPartitionProperties.get(TOTAL_SIZE_PROP))); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java index a0ce8be3af7f..cf97f7b67d4d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ContinuousFileStoreITCase.java @@ -57,33 +57,6 @@ protected List ddl() { + " WITH ('changelog-producer'='input', 'bucket' = '1')"); } - @Test - public void testSourceReuseWithoutScanPushDown() { - sEnv.executeSql("CREATE TEMPORARY TABLE print1 (a STRING) WITH ('connector'='print')"); - sEnv.executeSql("CREATE TEMPORARY TABLE print2 (b STRING) WITH ('connector'='print')"); - - StatementSet statementSet = sEnv.createStatementSet(); - statementSet.addInsertSql( - "INSERT INTO print1 SELECT a FROM T1 /*+ OPTIONS('scan.push-down' = 'false') */"); - statementSet.addInsertSql( - "INSERT INTO print2 SELECT b FROM T1 /*+ OPTIONS('scan.push-down' = 'false') */"); - assertThat(statementSet.compilePlan().explain()).contains("Reused"); - - statementSet = sEnv.createStatementSet(); - statementSet.addInsertSql( - "INSERT INTO print1 SELECT a FROM T1 /*+ OPTIONS('scan.push-down' = 'false') */ WHERE b = 'Apache'"); - statementSet.addInsertSql( - "INSERT INTO print2 SELECT b FROM T1 /*+ OPTIONS('scan.push-down' = 'false') */ WHERE a = 'Paimon'"); - assertThat(statementSet.compilePlan().explain()).contains("Reused"); - - statementSet = sEnv.createStatementSet(); - statementSet.addInsertSql( - "INSERT INTO print1 SELECT a FROM T1 /*+ OPTIONS('scan.push-down' = 'false') */ WHERE b = 'Apache' LIMIT 5"); - statementSet.addInsertSql( - "INSERT INTO print2 SELECT b FROM T1 /*+ OPTIONS('scan.push-down' = 'false') */ WHERE a = 'Paimon' LIMIT 10"); - assertThat(statementSet.compilePlan().explain()).contains("Reused"); - } - @Test public void testSourceReuseWithScanPushDown() { // source can be reused with projection applied diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java index 4e1ea424f834..239043ff79e1 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileSystemCatalogITCase.java @@ -117,7 +117,6 @@ public void testCatalogOptionsInheritAndOverride() throws Exception { + "'table-default.opt1'='value1', " + "'table-default.opt2'='value2', " + "'table-default.opt3'='value3', " - + "'fs.allow-hadoop-fallback'='false'," + "'lock.enabled'='false'" + ")", path)); @@ -134,7 +133,6 @@ public void testCatalogOptionsInheritAndOverride() throws Exception { assertThat(tableOptions).containsEntry("opt1", "value1"); assertThat(tableOptions).containsEntry("opt2", "value2"); assertThat(tableOptions).containsEntry("opt3", "value3"); - assertThat(tableOptions).doesNotContainKey("fs.allow-hadoop-fallback"); assertThat(tableOptions).doesNotContainKey("lock.enabled"); // check table options override catalog's @@ -147,7 +145,6 @@ public void testCatalogOptionsInheritAndOverride() throws Exception { assertThat(tableOptions).containsEntry("opt1", "value1"); assertThat(tableOptions).containsEntry("opt2", "value2"); assertThat(tableOptions).containsEntry("opt3", "value4"); - assertThat(tableOptions).doesNotContainKey("fs.allow-hadoop-fallback"); assertThat(tableOptions).doesNotContainKey("lock.enabled"); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkAnalyzeTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkAnalyzeTableITCase.java new file mode 100644 index 000000000000..ad8a2d45a036 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkAnalyzeTableITCase.java @@ -0,0 +1,186 @@ +/* + * 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; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.data.Decimal; +import org.apache.paimon.stats.ColStats; +import org.apache.paimon.stats.Statistics; +import org.apache.paimon.utils.DateTimeUtils; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Map; +import java.util.Optional; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT cases for analyze table. */ +public class FlinkAnalyzeTableITCase extends CatalogITCaseBase { + + @Test + public void testAnalyzeTable() throws Catalog.TableNotExistException { + sql( + "CREATE TABLE T (" + + " id STRING" + + ", name STRING" + + ", i INT" + + ", l bigint" + + ", PRIMARY KEY (id) NOT ENFORCED" + + " ) WITH (" + + " 'bucket' = '2'" + + " )"); + sql("INSERT INTO T VALUES ('1', 'a', 1, 1)"); + sql("INSERT INTO T VALUES ('2', 'aaa', 1, 2)"); + sql("ANALYZE TABLE T COMPUTE STATISTICS"); + + Optional statisticsOpt = paimonTable("T").statistics(); + assertThat(statisticsOpt.isPresent()).isTrue(); + Statistics stats = statisticsOpt.get(); + + assertThat(stats.mergedRecordCount().isPresent()).isTrue(); + Assertions.assertEquals(2L, stats.mergedRecordCount().getAsLong()); + + Assertions.assertTrue(stats.mergedRecordSize().isPresent()); + Assertions.assertTrue(stats.colStats().isEmpty()); + } + + @Test + public void testAnalyzeTableColumn() throws Catalog.TableNotExistException { + sql( + "CREATE TABLE T (" + + "id STRING, name STRING, bytes_col BYTES, int_col INT, long_col bigint,\n" + + "float_col FLOAT, double_col DOUBLE, decimal_col DECIMAL(10, 5), boolean_col BOOLEAN, date_col DATE,\n" + + "timestamp_col TIMESTAMP_LTZ, binary_col BINARY, varbinary_col VARBINARY, char_col CHAR(20), varchar_col VARCHAR(20),\n" + + "tinyint_col TINYINT, smallint_col SMALLINT" + + ", PRIMARY KEY (id) NOT ENFORCED" + + " ) WITH (" + + " 'bucket' = '2'" + + " )"); + sql( + "INSERT INTO T VALUES ('1', 'a', CAST('your_binary_data' AS BYTES), 1, 1, 1.0, 1.0, 13.12345, true, cast('2020-01-01' as date), cast('2024-01-01 00:00:00' as TIMESTAMP_LTZ), CAST('example binary1' AS BINARY), CAST('example binary1' AS VARBINARY), 'a', 'a',CAST(1 AS TINYINT), CAST(2 AS SMALLINT))"); + sql( + "INSERT INTO T VALUES ('2', 'aaa', CAST('your_binary_data' AS BYTES), 1, 1, 1.0, 5.0, 12.12345, true, cast('2021-01-02' as date), cast('2024-01-02 00:00:00' as TIMESTAMP_LTZ), CAST('example binary1' AS BINARY), CAST('example binary1' AS VARBINARY), 'aaa', 'aaa', CAST(2 AS TINYINT), CAST(4 AS SMALLINT))"); + + sql( + "INSERT INTO T VALUES ('3', 'bbbb', CAST('data' AS BYTES), 4, 19, 7.0, 1.0, 14.12345, true, cast(NULL as date), cast('2024-01-02 05:00:00' as TIMESTAMP_LTZ), CAST(NULL AS BINARY), CAST('example binary1' AS VARBINARY), 'aaa', 'aaa', CAST(NULL AS TINYINT), CAST(4 AS SMALLINT))"); + + sql( + "INSERT INTO T VALUES ('4', 'aa', CAST(NULL AS BYTES), 1, 1, 1.0, 1.0, 14.12345, false, cast(NULL as date), cast(NULL as TIMESTAMP_LTZ), CAST(NULL AS BINARY), CAST('example' AS VARBINARY), 'aba', 'aaab', CAST(NULL AS TINYINT), CAST(4 AS SMALLINT))"); + + sql("ANALYZE TABLE T COMPUTE STATISTICS FOR ALL COLUMNS"); + + Optional statisticsOpt = paimonTable("T").statistics(); + assertThat(statisticsOpt.isPresent()).isTrue(); + Statistics stats = statisticsOpt.get(); + + assertThat(stats.mergedRecordCount().isPresent()).isTrue(); + Assertions.assertEquals(4L, stats.mergedRecordCount().getAsLong()); + + Map> colStats = stats.colStats(); + Assertions.assertEquals( + ColStats.newColStats(0, 4L, null, null, 0L, 1L, 1L), colStats.get("id")); + Assertions.assertEquals( + ColStats.newColStats(1, 4L, null, null, 0L, 2L, 4L), colStats.get("name")); + + Assertions.assertEquals( + ColStats.newColStats(2, null, null, null, 1L, null, null), + colStats.get("bytes_col")); + + Assertions.assertEquals( + ColStats.newColStats(3, 2L, 1, 4, 0L, null, null), colStats.get("int_col")); + + Assertions.assertEquals( + ColStats.newColStats(4, 2L, 1L, 19L, 0L, null, null), colStats.get("long_col")); + + Assertions.assertEquals( + ColStats.newColStats(5, 2L, 1.0f, 7.0f, 0L, null, null), colStats.get("float_col")); + + Assertions.assertEquals( + ColStats.newColStats(6, 2L, 1.0d, 5.0d, 0L, null, null), + colStats.get("double_col")); + + Assertions.assertEquals( + ColStats.newColStats( + 7, + 3L, + Decimal.fromBigDecimal(new java.math.BigDecimal("12.12345"), 10, 5), + Decimal.fromBigDecimal(new java.math.BigDecimal("14.12345"), 10, 5), + 0L, + null, + null), + colStats.get("decimal_col")); + + Assertions.assertEquals( + ColStats.newColStats(8, 2L, null, null, 0L, null, null), + colStats.get("boolean_col")); + + Assertions.assertEquals( + ColStats.newColStats(9, 2L, 18262, 18629, 2L, null, null), + colStats.get("date_col")); + + Assertions.assertEquals( + ColStats.newColStats( + 10, + 3L, + DateTimeUtils.parseTimestampData("2024-01-01 00:00:00", 0), + DateTimeUtils.parseTimestampData("2024-01-02 05:00:00", 0), + 1L, + null, + null), + colStats.get("timestamp_col")); + + Assertions.assertEquals( + ColStats.newColStats(11, null, null, null, 2L, null, null), + colStats.get("binary_col")); + + Assertions.assertEquals( + ColStats.newColStats(12, null, null, null, 0L, null, null), + colStats.get("varbinary_col")); + + Assertions.assertEquals( + ColStats.newColStats(13, 3L, null, null, 0L, 20L, 20L), colStats.get("char_col")); + + Assertions.assertEquals( + ColStats.newColStats(14, 3L, null, null, 0L, 2L, 4L), colStats.get("varchar_col")); + + Assertions.assertEquals( + ColStats.newColStats( + 15, + 2L, + new Integer(1).byteValue(), + new Integer(2).byteValue(), + 2L, + null, + null), + colStats.get("tinyint_col")); + + Assertions.assertEquals( + ColStats.newColStats( + 16, + 2L, + new Integer(2).shortValue(), + new Integer(4).shortValue(), + 0L, + null, + null), + colStats.get("smallint_col")); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkBatchJobPartitionMarkdoneITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkBatchJobPartitionMarkdoneITCase.java index 9c97151ecbe7..340213aeae1b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkBatchJobPartitionMarkdoneITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkBatchJobPartitionMarkdoneITCase.java @@ -161,7 +161,7 @@ private FileStoreTable buildFileStoreTable(int[] partitions, int[] primaryKey) options.set(BUCKET, 3); options.set(PATH, getTempDirPath()); options.set(FILE_FORMAT, CoreOptions.FILE_FORMAT_AVRO); - options.set(FlinkConnectorOptions.PARTITION_MARK_DONE_WHEN_END_INPUT.key(), "true"); + options.set(CoreOptions.PARTITION_MARK_DONE_WHEN_END_INPUT.key(), "true"); Path tablePath = new CoreOptions(options.toMap()).path(); if (primaryKey.length == 0) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index f7edfa9d644f..27a89510975f 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink; import org.apache.paimon.CoreOptions; +import org.apache.paimon.TableType; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.log.LogSinkProvider; @@ -27,9 +28,12 @@ import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.Table; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.TableDescriptor; @@ -37,11 +41,17 @@ import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogDatabase; import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogMaterializedTable; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.IntervalFreshness; +import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.catalog.TestSchemaResolver; import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.catalog.exceptions.CatalogException; import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; @@ -54,6 +64,8 @@ import org.apache.flink.table.expressions.ResolvedExpression; import org.apache.flink.table.expressions.utils.ResolvedExpressionMock; import org.apache.flink.table.factories.DynamicTableFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.refresh.RefreshHandler; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -81,10 +93,12 @@ import static org.apache.paimon.flink.FlinkCatalogOptions.DISABLE_CREATE_TABLE_IN_DEFAULT_DB; import static org.apache.paimon.flink.FlinkCatalogOptions.LOG_SYSTEM_AUTO_REGISTER; import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM; +import static org.apache.paimon.flink.FlinkTestBase.createResolvedTable; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatCollection; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.Assertions.assertThrows; /** Test for {@link FlinkCatalog}. */ public class FlinkCatalogTest { @@ -99,6 +113,10 @@ public class FlinkCatalogTest { private final ObjectPath nonExistDbPath = ObjectPath.fromString("non.exist"); private final ObjectPath nonExistObjectPath = ObjectPath.fromString("db1.nonexist"); + private static final String DEFINITION_QUERY = "SELECT id, region, county FROM T"; + + private static final IntervalFreshness FRESHNESS = IntervalFreshness.ofMinute("3"); + private String warehouse; private Catalog catalog; @@ -186,6 +204,96 @@ private CatalogTable createPartitionedTable(Map options) { return new ResolvedCatalogTable(origin, resolvedSchema); } + private CatalogMaterializedTable createMaterializedTable(Map options) { + ResolvedSchema resolvedSchema = this.createSchema(); + return new ResolvedCatalogMaterializedTable( + CatalogMaterializedTable.newBuilder() + .schema(Schema.newBuilder().fromResolvedSchema(resolvedSchema).build()) + .comment("test materialized table comment") + .partitionKeys(Collections.emptyList()) + .options(options) + .definitionQuery(DEFINITION_QUERY) + .freshness(FRESHNESS) + .logicalRefreshMode(CatalogMaterializedTable.LogicalRefreshMode.AUTOMATIC) + .refreshMode(CatalogMaterializedTable.RefreshMode.CONTINUOUS) + .refreshStatus(CatalogMaterializedTable.RefreshStatus.INITIALIZING) + .build(), + resolvedSchema); + } + + @ParameterizedTest + @MethodSource("batchOptionProvider") + public void testCreateAndGetCatalogMaterializedTable(Map options) + throws Exception { + ObjectPath tablePath = path1; + CatalogMaterializedTable materializedTable = createMaterializedTable(options); + catalog.createDatabase(tablePath.getDatabaseName(), null, false); + // test create materialized table + catalog.createTable(tablePath, materializedTable, true); + + // test materialized table exist + assertThat(catalog.tableExists(tablePath)).isTrue(); + + // test get materialized table + CatalogBaseTable actualTable = catalog.getTable(tablePath); + // validate table type + assertThat(actualTable.getTableKind()) + .isEqualTo(CatalogBaseTable.TableKind.MATERIALIZED_TABLE); + + CatalogMaterializedTable actualMaterializedTable = (CatalogMaterializedTable) actualTable; + checkCreateTable(tablePath, materializedTable, actualMaterializedTable); + // test create exist materialized table + assertThrows( + TableAlreadyExistException.class, + () -> catalog.createTable(tablePath, materializedTable, false)); + } + + @ParameterizedTest + @MethodSource("batchOptionProvider") + public void testDropMaterializedTable(Map options) throws Exception { + ObjectPath tablePath = path1; + catalog.createDatabase(tablePath.getDatabaseName(), null, false); + catalog.createTable(tablePath, this.createTable(options), false); + assertThat(catalog.tableExists(tablePath)).isTrue(); + catalog.dropTable(tablePath, false); + assertThat(catalog.tableExists(tablePath)).isFalse(); + } + + @ParameterizedTest + @MethodSource("batchOptionProvider") + public void testAlterMaterializedTable(Map options) throws Exception { + ObjectPath tablePath = path1; + CatalogMaterializedTable materializedTable = createMaterializedTable(options); + catalog.createDatabase(tablePath.getDatabaseName(), null, false); + catalog.createTable(tablePath, materializedTable, true); + TestRefreshHandler refreshHandler = new TestRefreshHandler("jobID: xxx, clusterId: yyy"); + + // alter materialized table refresh handler + CatalogMaterializedTable expectedMaterializedTable = + materializedTable.copy( + CatalogMaterializedTable.RefreshStatus.ACTIVATED, + refreshHandler.asSummaryString(), + refreshHandler.toBytes()); + List tableChanges = new ArrayList<>(); + tableChanges.add( + new TableChange.ModifyRefreshStatus( + CatalogMaterializedTable.RefreshStatus.ACTIVATED)); + tableChanges.add( + new TableChange.ModifyRefreshHandler( + refreshHandler.asSummaryString(), refreshHandler.toBytes())); + catalog.alterTable(tablePath, expectedMaterializedTable, tableChanges, false); + + CatalogBaseTable updatedTable = catalog.getTable(tablePath); + checkEquals( + tablePath, + expectedMaterializedTable, + updatedTable, + Collections.singletonMap( + FlinkCatalogOptions.REGISTER_TIMEOUT.key(), + FlinkCatalogOptions.REGISTER_TIMEOUT.defaultValue().toString()), + Collections.emptySet()); + } + @ParameterizedTest @MethodSource("batchOptionProvider") public void testAlterTable(Map options) throws Exception { @@ -644,7 +752,54 @@ void testCreateTableFromTableDescriptor() throws Exception { checkCreateTable(path1, catalogTable, (CatalogTable) catalog.getTable(path1)); } - private void checkCreateTable(ObjectPath path, CatalogTable expected, CatalogTable actual) { + @Test + void testBuildPaimonTableWithCustomScheme() throws Exception { + catalog.createDatabase(path1.getDatabaseName(), null, false); + CatalogTable table = createTable(optionProvider(false).iterator().next()); + catalog.createTable(path1, table, false); + checkCreateTable(path1, table, catalog.getTable(path1)); + + List columns = + Arrays.asList( + Column.physical("first", DataTypes.STRING()), + Column.physical("second", DataTypes.INT()), + Column.physical("third", DataTypes.STRING()), + Column.physical( + "four", + DataTypes.ROW( + DataTypes.FIELD("f1", DataTypes.STRING()), + DataTypes.FIELD("f2", DataTypes.INT()), + DataTypes.FIELD( + "f3", + DataTypes.MAP( + DataTypes.STRING(), DataTypes.INT()))))); + DynamicTableFactory.Context context = + new FactoryUtil.DefaultDynamicTableContext( + ObjectIdentifier.of( + "default", path1.getDatabaseName(), path1.getObjectName()), + createResolvedTable( + new HashMap() { + { + put("path", "unsupported-scheme://foobar"); + } + }, + columns, + Collections.emptyList(), + Collections.emptyList()), + Collections.emptyMap(), + new Configuration(), + Thread.currentThread().getContextClassLoader(), + false); + + FlinkTableFactory factory = (FlinkTableFactory) catalog.getFactory().get(); + Table builtTable = factory.buildPaimonTable(context); + assertThat(builtTable).isInstanceOf(FileStoreTable.class); + assertThat(((FileStoreTable) builtTable).schema().fieldNames()) + .containsExactly("first", "second", "third", "four"); + } + + private void checkCreateTable( + ObjectPath path, CatalogBaseTable expected, CatalogBaseTable actual) { checkEquals( path, expected, @@ -661,8 +816,8 @@ private void checkAlterTable(ObjectPath path, CatalogTable expected, CatalogTabl private void checkEquals( ObjectPath path, - CatalogTable t1, - CatalogTable t2, + CatalogBaseTable t1, + CatalogBaseTable t2, Map optionsToAdd, Set optionsToRemove) { Path tablePath; @@ -681,17 +836,53 @@ private void checkEquals( options.put("path", tablePath.toString()); options.putAll(optionsToAdd); optionsToRemove.forEach(options::remove); - t1 = ((ResolvedCatalogTable) t1).copy(options); + if (t1.getTableKind() == CatalogBaseTable.TableKind.TABLE) { + t1 = ((ResolvedCatalogTable) t1).copy(options); + } else { + options.put(CoreOptions.TYPE.key(), TableType.MATERIALIZED_TABLE.toString()); + t1 = ((ResolvedCatalogMaterializedTable) t1).copy(options); + } checkEquals(t1, t2); } - private static void checkEquals(CatalogTable t1, CatalogTable t2) { + private static void checkEquals(CatalogBaseTable t1, CatalogBaseTable t2) { assertThat(t2.getTableKind()).isEqualTo(t1.getTableKind()); - assertThat(t2.getSchema()).isEqualTo(t1.getSchema()); assertThat(t2.getComment()).isEqualTo(t1.getComment()); - assertThat(t2.getPartitionKeys()).isEqualTo(t1.getPartitionKeys()); - assertThat(t2.isPartitioned()).isEqualTo(t1.isPartitioned()); assertThat(t2.getOptions()).isEqualTo(t1.getOptions()); + if (t1.getTableKind() == CatalogBaseTable.TableKind.TABLE) { + assertThat(t2.getSchema()).isEqualTo(t1.getSchema()); + assertThat(((CatalogTable) (t2)).getPartitionKeys()) + .isEqualTo(((CatalogTable) (t1)).getPartitionKeys()); + assertThat(((CatalogTable) (t2)).isPartitioned()) + .isEqualTo(((CatalogTable) (t1)).isPartitioned()); + } else { + CatalogMaterializedTable mt1 = (CatalogMaterializedTable) t1; + CatalogMaterializedTable mt2 = (CatalogMaterializedTable) t2; + assertThat( + Schema.newBuilder() + .fromResolvedSchema( + t2.getUnresolvedSchema() + .resolve(new TestSchemaResolver())) + .build()) + .isEqualTo(t1.getSchema().toSchema()); + assertThat(mt2.getPartitionKeys()).isEqualTo(mt1.getPartitionKeys()); + assertThat(mt2.isPartitioned()).isEqualTo(mt1.isPartitioned()); + // validate definition query + assertThat(mt2.getDefinitionQuery()).isEqualTo(mt1.getDefinitionQuery()); + // validate freshness + assertThat(mt2.getDefinitionFreshness()).isEqualTo(mt1.getDefinitionFreshness()); + // validate logical refresh mode + assertThat(mt2.getLogicalRefreshMode()).isEqualTo(mt1.getLogicalRefreshMode()); + // validate refresh mode + assertThat(mt2.getRefreshMode()).isEqualTo(mt1.getRefreshMode()); + // validate refresh status + assertThat(mt2.getRefreshStatus()).isEqualTo(mt1.getRefreshStatus()); + // validate refresh handler + assertThat(mt2.getRefreshHandlerDescription()) + .isEqualTo(mt1.getRefreshHandlerDescription()); + assertThat(mt2.getSerializedRefreshHandler()) + .isEqualTo(mt1.getSerializedRefreshHandler()); + } } static Stream> streamingOptionProvider() { @@ -774,4 +965,22 @@ public void unRegisterTopic() { throw new UnsupportedOperationException("Check unregister log store topic here."); } } + + private static class TestRefreshHandler implements RefreshHandler { + + private final String handlerString; + + public TestRefreshHandler(String handlerString) { + this.handlerString = handlerString; + } + + @Override + public String asSummaryString() { + return "test refresh handler"; + } + + public byte[] toBytes() { + return handlerString.getBytes(); + } + } } 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 new file mode 100644 index 000000000000..c46c4c358922 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.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.flink; + +import org.apache.paimon.Snapshot; +import org.apache.paimon.table.BucketMode; +import org.apache.paimon.utils.Pair; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExternalizedCheckpointRetention; +import org.apache.flink.configuration.StateRecoveryOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.execution.ExecutionState; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.types.Row; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test case for flink source / sink restore from savepoint. */ +@SuppressWarnings("BusyWait") +public class FlinkJobRecoveryITCase extends CatalogITCaseBase { + + private static final String MINI_CLUSTER_FIELD = "miniCluster"; + + @BeforeEach + @Override + public void before() throws IOException { + super.before(); + + // disable checkpoint + sEnv.getConfig() + .getConfiguration() + .set( + CheckpointingOptions.EXTERNALIZED_CHECKPOINT_RETENTION, + ExternalizedCheckpointRetention.RETAIN_ON_CANCELLATION) + .removeConfig(CheckpointingOptions.CHECKPOINTING_INTERVAL); + + // insert source data + batchSql("INSERT INTO source_table1 VALUES (1, 'test-1', '20241030')"); + batchSql("INSERT INTO source_table1 VALUES (2, 'test-2', '20241030')"); + batchSql("INSERT INTO source_table1 VALUES (3, 'test-3', '20241030')"); + batchSql( + "INSERT INTO source_table2 VALUES (4, 'test-4', '20241031'), (5, 'test-5', '20241031'), (6, 'test-6', '20241031')"); + } + + @Override + protected List ddl() { + return Arrays.asList( + String.format( + "CREATE CATALOG `fs_catalog` WITH ('type'='paimon', 'warehouse'='%s')", + path), + "CREATE TABLE IF NOT EXISTS `source_table1` (k INT, f1 STRING, dt STRING) WITH ('bucket'='1', 'bucket-key'='k')", + "CREATE TABLE IF NOT EXISTS `source_table2` (k INT, f1 STRING, dt STRING) WITH ('bucket'='1', 'bucket-key'='k')"); + } + + @ParameterizedTest + @EnumSource(BucketMode.class) + @Timeout(300) + public void testRestoreFromSavepointWithJobGraphChange(BucketMode bucketMode) throws Exception { + createTargetTable("target_table", bucketMode); + String beforeRecoverSql = + "INSERT INTO `target_table` /*+ OPTIONS('sink.operator-uid.suffix'='test-uid') */ SELECT * FROM source_table1 /*+ OPTIONS('source.operator-uid.suffix'='test-uid') */"; + String beforeRecoverCheckSql = "SELECT * FROM target_table"; + List beforeRecoverExpectedRows = + Arrays.asList( + Row.of(1, "test-1", "20241030"), + Row.of(2, "test-2", "20241030"), + Row.of(3, "test-3", "20241030")); + String afterRecoverSql = + "INSERT INTO `target_table` /*+ OPTIONS('sink.operator-uid.suffix'='test-uid') */ (SELECT * FROM source_table1 /*+ OPTIONS('source.operator-uid.suffix'='test-uid') */ UNION ALL SELECT * FROM source_table2)"; + String afterRecoverCheckSql = "SELECT * FROM target_table"; + List afterRecoverExpectedRows = + Arrays.asList( + Row.of(1, "test-1", "20241030"), + Row.of(2, "test-2", "20241030"), + Row.of(3, "test-3", "20241030"), + Row.of(4, "test-4", "20241031"), + Row.of(5, "test-5", "20241031"), + Row.of(6, "test-6", "20241031")); + testRecoverFromSavepoint( + beforeRecoverSql, + beforeRecoverCheckSql, + beforeRecoverExpectedRows, + afterRecoverSql, + afterRecoverCheckSql, + afterRecoverExpectedRows, + Collections.emptyList(), + Pair.of("target_table", "target_table"), + Collections.emptyMap()); + } + + @Test + @Timeout(300) + public void testRestoreFromSavepointWithIgnoreSourceState() throws Exception { + createTargetTable("target_table", BucketMode.HASH_FIXED); + String beforeRecoverSql = "INSERT INTO `target_table` SELECT * FROM source_table1"; + String beforeRecoverCheckSql = "SELECT * FROM target_table"; + List beforeRecoverExpectedRows = + Arrays.asList( + Row.of(1, "test-1", "20241030"), + Row.of(2, "test-2", "20241030"), + Row.of(3, "test-3", "20241030")); + String afterRecoverSql = + "INSERT INTO `target_table` SELECT * FROM source_table2 /*+ OPTIONS('source.operator-uid.suffix'='test-uid') */"; + String afterRecoverCheckSql = "SELECT * FROM target_table"; + List afterRecoverExpectedRows = + Arrays.asList( + Row.of(1, "test-1", "20241030"), + Row.of(2, "test-2", "20241030"), + Row.of(3, "test-3", "20241030"), + Row.of(4, "test-4", "20241031"), + Row.of(5, "test-5", "20241031"), + Row.of(6, "test-6", "20241031")); + testRecoverFromSavepoint( + beforeRecoverSql, + beforeRecoverCheckSql, + beforeRecoverExpectedRows, + afterRecoverSql, + afterRecoverCheckSql, + afterRecoverExpectedRows, + Collections.emptyList(), + Pair.of("target_table", "target_table"), + Collections.singletonMap( + StateRecoveryOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), "true")); + } + + @Test + @Timeout(300) + public void testRestoreFromSavepointWithIgnoreSinkState() throws Exception { + createTargetTable("target_table", BucketMode.HASH_FIXED); + createTargetTable("target_table2", BucketMode.HASH_FIXED); + + String beforeRecoverSql = "INSERT INTO `target_table` SELECT * FROM source_table1"; + String beforeRecoverCheckSql = "SELECT * FROM target_table"; + List beforeRecoverExpectedRows = + Arrays.asList( + Row.of(1, "test-1", "20241030"), + Row.of(2, "test-2", "20241030"), + Row.of(3, "test-3", "20241030")); + String afterRecoverSql = + "INSERT INTO `target_table2` /*+ OPTIONS('sink.operator-uid.suffix'='test-uid') */ SELECT * FROM source_table1"; + String afterRecoverCheckSql = "SELECT * FROM target_table2"; + List afterRecoverExpectedRows = + Arrays.asList( + Row.of(7, "test-7", "20241030"), + Row.of(8, "test-8", "20241030"), + Row.of(9, "test-9", "20241030")); + String updateSql = + "INSERT INTO source_table1 VALUES (7, 'test-7', '20241030'), (8, 'test-8', '20241030'), (9, 'test-9', '20241030')"; + testRecoverFromSavepoint( + beforeRecoverSql, + beforeRecoverCheckSql, + beforeRecoverExpectedRows, + afterRecoverSql, + afterRecoverCheckSql, + afterRecoverExpectedRows, + Collections.singletonList(updateSql), + Pair.of("target_table", "target_table2"), + Collections.singletonMap( + StateRecoveryOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE.key(), "true")); + } + + private void testRecoverFromSavepoint( + String beforeRecoverSql, + String beforeRecoverCheckSql, + List beforeRecoverExpectedRows, + String afterRecoverSql, + String afterRecoverCheckSql, + List afterRecoverExpectedRows, + List updateSql, + Pair targetTables, + Map recoverOptions) + throws Exception { + + //noinspection OptionalGetWithoutIsPresent + JobClient jobClient = sEnv.executeSql(beforeRecoverSql).getJobClient().get(); + String checkpointPath = + triggerCheckpointAndWaitForWrites( + jobClient, targetTables.getLeft(), beforeRecoverExpectedRows.size()); + jobClient.cancel().get(); + + List rows = batchSql(beforeRecoverCheckSql); + assertThat(rows.size()).isEqualTo(beforeRecoverExpectedRows.size()); + assertThat(rows).containsExactlyInAnyOrder(beforeRecoverExpectedRows.toArray(new Row[0])); + + for (String sql : updateSql) { + batchSql(sql); + } + + Configuration config = + sEnv.getConfig() + .getConfiguration() + .set(StateRecoveryOptions.SAVEPOINT_PATH, checkpointPath); + for (Map.Entry entry : recoverOptions.entrySet()) { + config.setString(entry.getKey(), entry.getValue()); + } + + //noinspection OptionalGetWithoutIsPresent + jobClient = sEnv.executeSql(afterRecoverSql).getJobClient().get(); + triggerCheckpointAndWaitForWrites( + jobClient, targetTables.getRight(), afterRecoverExpectedRows.size()); + jobClient.cancel().get(); + + rows = batchSql(afterRecoverCheckSql); + assertThat(rows.size()).isEqualTo(afterRecoverExpectedRows.size()); + assertThat(rows).containsExactlyInAnyOrder(afterRecoverExpectedRows.toArray(new Row[0])); + } + + private void createTargetTable(String tableName, BucketMode bucketMode) { + switch (bucketMode) { + case HASH_FIXED: + batchSql( + String.format( + "CREATE TABLE IF NOT EXISTS `%s` (k INT, f1 STRING, pt STRING, PRIMARY KEY(k, pt) NOT ENFORCED) WITH ('bucket'='2', 'commit.force-create-snapshot'='true')", + tableName)); + return; + case HASH_DYNAMIC: + batchSql( + String.format( + "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: + 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')", + tableName)); + return; + case BUCKET_UNAWARE: + batchSql( + String.format( + "CREATE TABLE IF NOT EXISTS `%s` (k INT, f1 STRING, pt STRING) WITH ('bucket'='-1', 'commit.force-create-snapshot'='true')", + tableName)); + return; + default: + throw new IllegalArgumentException("Unsupported bucket mode: " + bucketMode); + } + } + + private Snapshot waitForNewSnapshot(String tableName, long initialSnapshot) + throws InterruptedException { + Snapshot snapshot = findLatestSnapshot(tableName); + while (snapshot == null || snapshot.id() == initialSnapshot) { + Thread.sleep(2000L); + snapshot = findLatestSnapshot(tableName); + } + return snapshot; + } + + @SuppressWarnings("unchecked") + private T reflectGetMiniCluster(Object instance) + throws NoSuchFieldException, IllegalAccessException { + Field field = instance.getClass().getDeclaredField(MINI_CLUSTER_FIELD); + field.setAccessible(true); + return (T) field.get(instance); + } + + private String triggerCheckpointAndWaitForWrites( + JobClient jobClient, String targetTable, long totalRecords) throws Exception { + //noinspection resource + MiniCluster miniCluster = reflectGetMiniCluster(jobClient); + JobID jobID = jobClient.getJobID(); + JobStatus jobStatus = jobClient.getJobStatus().get(); + while (jobStatus == JobStatus.INITIALIZING || jobStatus == JobStatus.CREATED) { + Thread.sleep(2000L); + jobStatus = jobClient.getJobStatus().get(); + } + + if (jobStatus != JobStatus.RUNNING) { + throw new IllegalStateException("Job status is not RUNNING"); + } + + AtomicBoolean allTaskRunning = new AtomicBoolean(false); + while (!allTaskRunning.get()) { + allTaskRunning.set(true); + Thread.sleep(2000L); + miniCluster + .getExecutionGraph(jobID) + .thenAccept( + eg -> + eg.getAllExecutionVertices() + .forEach( + ev -> { + if (ev.getExecutionState() + != ExecutionState.RUNNING) { + allTaskRunning.set(false); + } + })) + .get(); + } + + String checkpointPath = miniCluster.triggerCheckpoint(jobID).get(); + Snapshot snapshot = waitForNewSnapshot(targetTable, -1L); + //noinspection DataFlowIssue + while (snapshot.totalRecordCount() < totalRecords) { + checkpointPath = miniCluster.triggerCheckpoint(jobID).get(); + snapshot = waitForNewSnapshot(targetTable, snapshot.id()); + } + + return checkpointPath; + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java index 46399f85632a..3e9ba2194aed 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/LookupJoinITCase.java @@ -134,45 +134,40 @@ public void testLookup(LookupCacheMode cacheMode) throws Exception { iterator.close(); } - @ParameterizedTest - @EnumSource(LookupCacheMode.class) - public void testLookupIgnoreScanOptions(LookupCacheMode cacheMode) throws Exception { - initTable(cacheMode); - sql("INSERT INTO DIM VALUES (1, 11, 111, 1111), (2, 22, 222, 2222)"); - - String scanOption; - if (ThreadLocalRandom.current().nextBoolean()) { - scanOption = "'scan.mode'='latest'"; - } else { - scanOption = "'scan.snapshot-id'='2'"; - } - String query = - "SELECT T.i, D.j, D.k1, D.k2 FROM T LEFT JOIN DIM /*+ OPTIONS(" - + scanOption - + ") */" - + " for system_time as of T.proctime AS D ON T.i = D.i"; - BlockingIterator iterator = BlockingIterator.of(sEnv.executeSql(query).collect()); - - sql("INSERT INTO T VALUES (1), (2), (3)"); - List result = iterator.collect(3); - assertThat(result) - .containsExactlyInAnyOrder( - Row.of(1, 11, 111, 1111), - Row.of(2, 22, 222, 2222), - Row.of(3, null, null, null)); - - sql("INSERT INTO DIM VALUES (2, 44, 444, 4444), (3, 33, 333, 3333)"); - Thread.sleep(2000); // wait refresh - sql("INSERT INTO T VALUES (1), (2), (3), (4)"); - result = iterator.collect(4); - assertThat(result) - .containsExactlyInAnyOrder( - Row.of(1, 11, 111, 1111), - Row.of(2, 44, 444, 4444), - Row.of(3, 33, 333, 3333), - Row.of(4, null, null, null)); - - iterator.close(); + @Test + public void testLookupIgnoreScanOptions() throws Exception { + sql( + "CREATE TABLE d (\n" + + " pt INT,\n" + + " id INT,\n" + + " data STRING,\n" + + " PRIMARY KEY (pt, id) NOT ENFORCED\n" + + ") PARTITIONED BY (pt) WITH ( 'bucket' = '1', 'continuous.discovery-interval'='1 ms' )"); + sql( + "CREATE TABLE t1 (\n" + + " pt INT,\n" + + " id INT,\n" + + " data STRING,\n" + + " `proctime` AS PROCTIME(),\n" + + " PRIMARY KEY (pt, id) NOT ENFORCED\n" + + ") PARTITIONED BY (pt) with ( 'continuous.discovery-interval'='1 ms' )"); + + sql("INSERT INTO d VALUES (1, 1, 'one'), (2, 2, 'two'), (3, 3, 'three')"); + sql("INSERT INTO t1 VALUES (1, 1, 'one'), (2, 2, 'two'), (3, 3, 'three')"); + + BlockingIterator streamIter = + streamSqlBlockIter( + "SELECT T.pt, T.id, T.data, D.pt, D.id, D.data " + + "FROM t1 AS T LEFT JOIN d /*+ OPTIONS('lookup.dynamic-partition'='max_pt()', 'scan.snapshot-id'='2') */ " + + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.id = D.id"); + + assertThat(streamIter.collect(3)) + .containsExactlyInAnyOrder( + Row.of(1, 1, "one", null, null, null), + Row.of(2, 2, "two", null, null, null), + Row.of(3, 3, "three", 3, 3, "three")); + + streamIter.close(); } @ParameterizedTest diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/MaterializedTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/MaterializedTableITCase.java new file mode 100644 index 000000000000..34f354e54de9 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/MaterializedTableITCase.java @@ -0,0 +1,640 @@ +/* + * 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; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.jobgraph.JobType; +import org.apache.flink.runtime.rest.messages.EmptyRequestBody; +import org.apache.flink.runtime.rest.messages.JobMessageParameters; +import org.apache.flink.runtime.rest.messages.MessageHeaders; +import org.apache.flink.runtime.rest.messages.RequestBody; +import org.apache.flink.runtime.rest.messages.ResponseBody; +import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointConfigHeaders; +import org.apache.flink.runtime.rest.messages.checkpoints.CheckpointConfigInfo; +import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo; +import org.apache.flink.runtime.rest.util.RestMapperUtils; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogMaterializedTable; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogBaseTable; +import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.gateway.api.endpoint.EndpointVersion; +import org.apache.flink.table.gateway.api.operation.OperationHandle; +import org.apache.flink.table.gateway.api.results.TableInfo; +import org.apache.flink.table.gateway.api.session.SessionEnvironment; +import org.apache.flink.table.gateway.api.session.SessionHandle; +import org.apache.flink.table.gateway.api.utils.SqlGatewayException; +import org.apache.flink.table.gateway.rest.util.SqlGatewayRestEndpointExtension; +import org.apache.flink.table.gateway.service.SqlGatewayServiceImpl; +import org.apache.flink.table.gateway.service.utils.IgnoreExceptionHandler; +import org.apache.flink.table.gateway.service.utils.SqlGatewayServiceExtension; +import org.apache.flink.table.gateway.workflow.scheduler.EmbeddedQuartzScheduler; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; +import org.apache.flink.table.refresh.ContinuousRefreshHandler; +import org.apache.flink.table.refresh.ContinuousRefreshHandlerSerializer; +import org.apache.flink.table.shaded.org.quartz.JobKey; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.testutils.executor.TestExecutorExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.concurrent.ExecutorThreadFactory; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.ArrayList; +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 java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +import static org.apache.flink.table.catalog.CommonCatalogOptions.TABLE_CATALOG_STORE_KIND; +import static org.apache.flink.table.factories.FactoryUtil.WORKFLOW_SCHEDULER_TYPE; +import static org.apache.flink.table.gateway.service.utils.SqlGatewayServiceTestUtil.awaitOperationTermination; +import static org.apache.flink.table.gateway.service.utils.SqlGatewayServiceTestUtil.fetchAllResults; +import static org.apache.flink.test.util.TestUtils.waitUntilAllTasksAreRunning; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test the support of Materialized Table. */ +public class MaterializedTableITCase { + private static final String FILE_CATALOG_STORE = "file_store"; + private static final String TEST_CATALOG_PREFIX = "test_catalog"; + protected static final String TEST_DEFAULT_DATABASE = "default"; + + private static final AtomicLong COUNTER = new AtomicLong(0); + + @RegisterExtension + @Order(1) + static final MiniClusterExtension MINI_CLUSTER = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(2) + .build()); + + @RegisterExtension + @Order(2) + protected static final SqlGatewayServiceExtension SQL_GATEWAY_SERVICE_EXTENSION = + new SqlGatewayServiceExtension(MINI_CLUSTER::getClientConfiguration); + + @RegisterExtension + @Order(3) + protected static final TestExecutorExtension EXECUTOR_EXTENSION = + new TestExecutorExtension<>( + () -> + Executors.newCachedThreadPool( + new ExecutorThreadFactory( + "SqlGatewayService Test Pool", + IgnoreExceptionHandler.INSTANCE))); + + @RegisterExtension + @Order(4) + protected static final SqlGatewayRestEndpointExtension SQL_GATEWAY_REST_ENDPOINT_EXTENSION = + new SqlGatewayRestEndpointExtension(SQL_GATEWAY_SERVICE_EXTENSION::getService); + + protected static SqlGatewayServiceImpl service; + private static SessionEnvironment defaultSessionEnvironment; + private static Path baseCatalogPath; + + private String paimonWarehousePath; + protected String paimonCatalogName; + + protected SessionHandle sessionHandle; + + protected RestClusterClient restClusterClient; + + @BeforeAll + static void setUp(@TempDir Path temporaryFolder) throws Exception { + service = (SqlGatewayServiceImpl) SQL_GATEWAY_SERVICE_EXTENSION.getService(); + + // initialize file catalog store path + Path fileCatalogStore = temporaryFolder.resolve(FILE_CATALOG_STORE); + Files.createDirectory(fileCatalogStore); + Map catalogStoreOptions = new HashMap<>(); + catalogStoreOptions.put(TABLE_CATALOG_STORE_KIND.key(), "file"); + catalogStoreOptions.put("table.catalog-store.file.path", fileCatalogStore.toString()); + + // initialize catalog base path + baseCatalogPath = temporaryFolder.resolve(TEST_CATALOG_PREFIX); + Files.createDirectory(baseCatalogPath); + + // workflow scheduler config + Map workflowSchedulerConfig = new HashMap<>(); + workflowSchedulerConfig.put(WORKFLOW_SCHEDULER_TYPE.key(), "embedded"); + workflowSchedulerConfig.put( + "sql-gateway.endpoint.rest.address", + SQL_GATEWAY_REST_ENDPOINT_EXTENSION.getTargetAddress()); + workflowSchedulerConfig.put( + "sql-gateway.endpoint.rest.port", + String.valueOf(SQL_GATEWAY_REST_ENDPOINT_EXTENSION.getTargetPort())); + + // Session conf for testing purpose + Map testConf = new HashMap<>(); + testConf.put("k1", "v1"); + testConf.put("k2", "v2"); + + defaultSessionEnvironment = + SessionEnvironment.newBuilder() + .addSessionConfig(catalogStoreOptions) + .addSessionConfig(workflowSchedulerConfig) + .addSessionConfig(testConf) + .setSessionEndpointVersion(new EndpointVersion() {}) + .build(); + } + + @BeforeEach + void before(@InjectClusterClient RestClusterClient injectClusterClient) throws Exception { + String randomStr = String.valueOf(COUNTER.incrementAndGet()); + // initialize warehouse path with random uuid + Path fileCatalogPath = baseCatalogPath.resolve(randomStr); + Files.createDirectory(fileCatalogPath); + + paimonWarehousePath = fileCatalogPath.toString(); + paimonCatalogName = TEST_CATALOG_PREFIX + randomStr; + // initialize session handle, create paimon catalog and register it to catalog + // store + sessionHandle = initializeSession(); + + // init rest cluster client + restClusterClient = injectClusterClient; + } + + @AfterEach + void after() throws Exception { + Set tableInfos = + service.listTables( + sessionHandle, + paimonCatalogName, + TEST_DEFAULT_DATABASE, + Collections.singleton(CatalogBaseTable.TableKind.TABLE)); + + // drop all materialized tables + for (TableInfo tableInfo : tableInfos) { + ResolvedCatalogBaseTable resolvedTable = + service.getTable(sessionHandle, tableInfo.getIdentifier()); + if (CatalogBaseTable.TableKind.MATERIALIZED_TABLE == resolvedTable.getTableKind()) { + String dropTableDDL = + String.format( + "DROP MATERIALIZED TABLE %s", + tableInfo.getIdentifier().asSerializableString()); + OperationHandle dropTableHandle; + dropTableHandle = + service.executeStatement( + sessionHandle, dropTableDDL, -1, new Configuration()); + awaitOperationTermination(service, sessionHandle, dropTableHandle); + } + } + } + + @Test + void testCreateMaterializedTableInContinuousMode() throws Exception { + String materializedTableDDL = + "CREATE MATERIALIZED TABLE users_shops" + + " PARTITIONED BY (ds)\n" + + " WITH(\n" + + " 'format' = 'debezium-json'\n" + + " )\n" + + " FRESHNESS = INTERVAL '30' SECOND\n" + + " AS SELECT \n" + + " user_id,\n" + + " shop_id,\n" + + " ds,\n" + + " SUM (payment_amount_cents) AS payed_buy_fee_sum,\n" + + " SUM (1) AS pv\n" + + " FROM (\n" + + " SELECT user_id, shop_id, DATE_FORMAT(order_created_at, 'yyyy-MM-dd') AS ds, payment_amount_cents FROM datagenSource" + + " ) AS tmp\n" + + " GROUP BY (user_id, shop_id, ds)"; + OperationHandle materializedTableHandle = + service.executeStatement( + sessionHandle, materializedTableDDL, -1, new Configuration()); + awaitOperationTermination(service, sessionHandle, materializedTableHandle); + + // validate materialized table: schema, refresh mode, refresh status, refresh handler, + // doesn't check the data because it generates randomly. + ResolvedCatalogMaterializedTable actualMaterializedTable = + (ResolvedCatalogMaterializedTable) + service.getTable( + sessionHandle, + ObjectIdentifier.of( + paimonCatalogName, TEST_DEFAULT_DATABASE, "users_shops")); + + // Expected schema + ResolvedSchema expectedSchema = + ResolvedSchema.of( + Arrays.asList( + Column.physical("user_id", DataTypes.BIGINT()), + Column.physical("shop_id", DataTypes.BIGINT()), + Column.physical("ds", DataTypes.STRING()), + Column.physical("payed_buy_fee_sum", DataTypes.BIGINT()), + Column.physical("pv", DataTypes.INT().notNull()))); + + assertThat(actualMaterializedTable.getResolvedSchema()).isEqualTo(expectedSchema); + assertThat(actualMaterializedTable.getFreshness()).isEqualTo(Duration.ofSeconds(30)); + assertThat(actualMaterializedTable.getLogicalRefreshMode()) + .isEqualTo(CatalogMaterializedTable.LogicalRefreshMode.AUTOMATIC); + assertThat(actualMaterializedTable.getRefreshMode()) + .isEqualTo(CatalogMaterializedTable.RefreshMode.CONTINUOUS); + assertThat(actualMaterializedTable.getRefreshStatus()) + .isEqualTo(CatalogMaterializedTable.RefreshStatus.ACTIVATED); + assertThat(actualMaterializedTable.getRefreshHandlerDescription()).isNotEmpty(); + assertThat(actualMaterializedTable.getSerializedRefreshHandler()).isNotEmpty(); + + ContinuousRefreshHandler activeRefreshHandler = + ContinuousRefreshHandlerSerializer.INSTANCE.deserialize( + actualMaterializedTable.getSerializedRefreshHandler(), + getClass().getClassLoader()); + + waitUntilAllTasksAreRunning( + restClusterClient, JobID.fromHexString(activeRefreshHandler.getJobId())); + + // verify the background job is running + String describeJobDDL = String.format("DESCRIBE JOB '%s'", activeRefreshHandler.getJobId()); + OperationHandle describeJobHandle = + service.executeStatement(sessionHandle, describeJobDDL, -1, new Configuration()); + awaitOperationTermination(service, sessionHandle, describeJobHandle); + List jobResults = fetchAllResults(service, sessionHandle, describeJobHandle); + assertThat(jobResults.get(0).getString(2).toString()).isEqualTo("RUNNING"); + + // get checkpoint interval + long checkpointInterval = + getCheckpointIntervalConfig(restClusterClient, activeRefreshHandler.getJobId()); + assertThat(checkpointInterval).isEqualTo(30 * 1000); + } + + @Test + void testAlterMaterializedTableRefresh() throws Exception { + long timeout = Duration.ofSeconds(20).toMillis(); + long pause = Duration.ofSeconds(2).toMillis(); + + List data = new ArrayList<>(); + data.add(Row.of(1L, 1L, 1L, "2024-01-01")); + data.add(Row.of(2L, 2L, 2L, "2024-01-02")); + data.add(Row.of(3L, 3L, 3L, "2024-01-02")); + createAndVerifyCreateMaterializedTableWithData( + "my_materialized_table", + data, + Collections.singletonMap("ds", "yyyy-MM-dd"), + CatalogMaterializedTable.RefreshMode.CONTINUOUS); + + // remove the last element + data.remove(2); + + long currentTime = System.currentTimeMillis(); + String alterStatement = + "ALTER MATERIALIZED TABLE my_materialized_table REFRESH PARTITION (ds = '2024-01-02')"; + OperationHandle alterHandle = + service.executeStatement(sessionHandle, alterStatement, -1, new Configuration()); + awaitOperationTermination(service, sessionHandle, alterHandle); + List result = fetchAllResults(service, sessionHandle, alterHandle); + assertThat(result.size()).isEqualTo(1); + String jobId = result.get(0).getString(0).toString(); + + // 1. verify a new job is created + verifyRefreshJobCreated(restClusterClient, jobId, currentTime); + // 2. verify the new job overwrite the data + try (ExecutionInBatchModeRunner ignored = new ExecutionInBatchModeRunner()) { + org.apache.paimon.utils.CommonTestUtils.waitUtil( + () -> + fetchTableData(sessionHandle, "SELECT * FROM my_materialized_table") + .size() + == data.size(), + Duration.ofMillis(timeout), + Duration.ofMillis(pause), + "Failed to verify the data in materialized table."); + assertThat( + fetchTableData( + sessionHandle, + "SELECT * FROM my_materialized_table where ds = '2024-01-02'") + .size()) + .isEqualTo(1); + } + } + + @Test + void testDropMaterializedTable() throws Exception { + createAndVerifyCreateMaterializedTableWithData( + "users_shops", + Collections.emptyList(), + Collections.emptyMap(), + CatalogMaterializedTable.RefreshMode.FULL); + + JobKey jobKey = + JobKey.jobKey( + "quartz_job_" + + ObjectIdentifier.of( + paimonCatalogName, + TEST_DEFAULT_DATABASE, + "users_shops") + .asSerializableString(), + "default_group"); + EmbeddedQuartzScheduler embeddedWorkflowScheduler = + SQL_GATEWAY_REST_ENDPOINT_EXTENSION + .getSqlGatewayRestEndpoint() + .getQuartzScheduler(); + + // verify refresh workflow is created + assertThat(embeddedWorkflowScheduler.getQuartzScheduler().checkExists(jobKey)).isTrue(); + + // Drop materialized table using drop table statement + String dropTableUsingMaterializedTableDDL = "DROP TABLE users_shops"; + OperationHandle dropTableUsingMaterializedTableHandle = + service.executeStatement( + sessionHandle, dropTableUsingMaterializedTableDDL, -1, new Configuration()); + + assertThatThrownBy( + () -> + awaitOperationTermination( + service, + sessionHandle, + dropTableUsingMaterializedTableHandle)) + .rootCause() + .isInstanceOf(ValidationException.class) + .hasMessage( + String.format( + "Table with identifier '%s' does not exist.", + ObjectIdentifier.of( + paimonCatalogName, + TEST_DEFAULT_DATABASE, + "users_shops") + .asSummaryString())); + + // drop materialized table + String dropMaterializedTableDDL = "DROP MATERIALIZED TABLE IF EXISTS users_shops"; + OperationHandle dropMaterializedTableHandle = + service.executeStatement( + sessionHandle, dropMaterializedTableDDL, -1, new Configuration()); + awaitOperationTermination(service, sessionHandle, dropMaterializedTableHandle); + + // verify materialized table metadata is removed + assertThatThrownBy( + () -> + service.getTable( + sessionHandle, + ObjectIdentifier.of( + paimonCatalogName, + TEST_DEFAULT_DATABASE, + "users_shops"))) + .isInstanceOf(SqlGatewayException.class) + .hasMessageContaining("Failed to getTable."); + + // verify refresh workflow is removed + assertThat(embeddedWorkflowScheduler.getQuartzScheduler().checkExists(jobKey)).isFalse(); + } + + private long getCheckpointIntervalConfig(RestClusterClient restClusterClient, String jobId) + throws Exception { + CheckpointConfigInfo checkpointConfigInfo = + sendJobRequest( + restClusterClient, + CheckpointConfigHeaders.getInstance(), + EmptyRequestBody.getInstance(), + jobId); + return RestMapperUtils.getStrictObjectMapper() + .readTree( + RestMapperUtils.getStrictObjectMapper() + .writeValueAsString(checkpointConfigInfo)) + .get("interval") + .asLong(); + } + + private static + P sendJobRequest( + RestClusterClient restClusterClient, + MessageHeaders headers, + R requestBody, + String jobId) + throws Exception { + M jobMessageParameters = headers.getUnresolvedMessageParameters(); + jobMessageParameters.jobPathParameter.resolve(JobID.fromHexString(jobId)); + + return restClusterClient + .sendRequest(headers, jobMessageParameters, requestBody) + .get(5, TimeUnit.SECONDS); + } + + private SessionHandle initializeSession() { + SessionHandle sessionHandle = service.openSession(defaultSessionEnvironment); + String catalogDDL = + String.format( + "CREATE CATALOG %s\n" + + "WITH (\n" + + " 'type' = 'paimon',\n" + + " 'warehouse' = '%s'" + + " )", + paimonCatalogName, paimonWarehousePath); + service.configureSession(sessionHandle, catalogDDL, -1); + service.configureSession( + sessionHandle, String.format("USE CATALOG %s", paimonCatalogName), -1); + + // create source table + String dataGenSource = + "CREATE TEMPORARY TABLE datagenSource (\n" + + " order_id BIGINT,\n" + + " order_number VARCHAR(20),\n" + + " user_id BIGINT,\n" + + " shop_id BIGINT,\n" + + " product_id BIGINT,\n" + + " status BIGINT,\n" + + " order_type BIGINT,\n" + + " order_created_at TIMESTAMP,\n" + + " payment_amount_cents BIGINT\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'datagen',\n" + + " 'rows-per-second' = '10'\n" + + ")"; + service.configureSession(sessionHandle, dataGenSource, -1); + return sessionHandle; + } + + public void createAndVerifyCreateMaterializedTableWithData( + String materializedTableName, + List data, + Map partitionFormatter, + CatalogMaterializedTable.RefreshMode refreshMode) + throws Exception { + long timeout = Duration.ofSeconds(20).toMillis(); + long pause = Duration.ofSeconds(2).toMillis(); + + String dataId = TestValuesTableFactory.registerData(data); + String sourceDdl = + String.format( + "CREATE TEMPORARY TABLE IF NOT EXISTS my_source (\n" + + " order_id BIGINT,\n" + + " user_id BIGINT,\n" + + " shop_id BIGINT,\n" + + " order_created_at STRING\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true',\n" + + " 'data-id' = '%s'\n" + + ")", + dataId); + OperationHandle sourceHandle = + service.executeStatement(sessionHandle, sourceDdl, -1, new Configuration()); + awaitOperationTermination(service, sessionHandle, sourceHandle); + + String partitionFields = + partitionFormatter != null && !partitionFormatter.isEmpty() + ? partitionFormatter.entrySet().stream() + .map( + e -> + String.format( + "'partition.fields.%s.date-formatter' = '%s'", + e.getKey(), e.getValue())) + .collect(Collectors.joining(",\n", "", ",\n")) + : "\n"; + String materializedTableDDL = + String.format( + "CREATE MATERIALIZED TABLE %s" + + " PARTITIONED BY (ds)\n" + + " WITH(\n" + + " %s" + + " 'format' = 'debezium-json'\n" + + " )\n" + + " FRESHNESS = INTERVAL '30' SECOND\n" + + " REFRESH_MODE = %s\n" + + " AS SELECT \n" + + " user_id,\n" + + " shop_id,\n" + + " ds,\n" + + " COUNT(order_id) AS order_cnt\n" + + " FROM (\n" + + " SELECT user_id, shop_id, order_created_at AS ds, order_id FROM my_source" + + " ) AS tmp\n" + + " GROUP BY (user_id, shop_id, ds)", + materializedTableName, partitionFields, refreshMode.toString()); + + OperationHandle materializedTableHandle = + service.executeStatement( + sessionHandle, materializedTableDDL, -1, new Configuration()); + awaitOperationTermination(service, sessionHandle, materializedTableHandle); + try (ExecutionInBatchModeRunner ignore = new ExecutionInBatchModeRunner()) { + // verify data exists in materialized table + CommonTestUtils.waitUtil( + () -> + fetchTableData( + sessionHandle, + String.format( + "SELECT * FROM %s", + materializedTableName)) + .size() + == data.size(), + Duration.ofMillis(timeout), + Duration.ofMillis(pause), + "Failed to verify the data in materialized table."); + } + } + + /** + * A helper runner to wrap code with try-with-resource clause. All session execution will be + * executed in flink batch runtime mode. + */ + protected class ExecutionInBatchModeRunner implements AutoCloseable { + private final String oldMode; + + ExecutionInBatchModeRunner() { + this.oldMode = service.getSessionConfig(sessionHandle).get("execution.runtime-mode"); + service.configureSession(sessionHandle, "SET 'execution.runtime-mode' = 'batch'", -1); + } + + @Override + public void close() throws Exception { + if (oldMode != null) { + service.configureSession( + sessionHandle, + String.format("SET 'execution.runtime-mode' = '%s'", oldMode), + -1); + } + } + } + + public List fetchTableData(SessionHandle sessionHandle, String query) { + Configuration configuration = new Configuration(); + OperationHandle queryHandle = + service.executeStatement(sessionHandle, query, -1, configuration); + + return fetchAllResults(service, sessionHandle, queryHandle); + } + + public void verifyRefreshJobCreated( + RestClusterClient restClusterClient, String jobId, long startTime) throws Exception { + long timeout = Duration.ofSeconds(20).toMillis(); + long pause = Duration.ofSeconds(2).toMillis(); + + // 1. verify a new job is created + Optional job = + restClusterClient.listJobs().get(timeout, TimeUnit.MILLISECONDS).stream() + .filter(j -> j.getJobId().toString().equals(jobId)) + .findFirst(); + assertThat(job).isPresent(); + assertThat(job.get().getStartTime()).isGreaterThan(startTime); + + // 2. verify the new job is a batch job + JobDetailsInfo jobDetailsInfo = + restClusterClient + .getJobDetails(JobID.fromHexString(jobId)) + .get(timeout, TimeUnit.MILLISECONDS); + assertThat(jobDetailsInfo.getJobType()).isEqualTo(JobType.BATCH); + + // 3. verify the new job is finished + CommonTestUtils.waitUtil( + () -> { + try { + return JobStatus.FINISHED.equals( + restClusterClient + .getJobStatus(JobID.fromHexString(jobId)) + .get(5, TimeUnit.SECONDS)); + } catch (Exception ignored) { + } + return false; + }, + Duration.ofMillis(timeout), + Duration.ofMillis(pause), + "Failed to verify whether the job is finished."); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PartialUpdateITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PartialUpdateITCase.java index 4505487fae1b..0bedbfd024d1 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PartialUpdateITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PartialUpdateITCase.java @@ -621,4 +621,68 @@ public void testIgnoreDelete(boolean localMerge) throws Exception { Row.ofKind(RowKind.UPDATE_AFTER, 1, "A", "apple")); iterator.close(); } + + @Test + public void testRemoveRecordOnDelete() { + sql( + "CREATE TABLE remove_record_on_delete (pk INT PRIMARY KEY NOT ENFORCED, a STRING, b STRING) WITH (" + + " 'merge-engine' = 'partial-update'," + + " 'partial-update.remove-record-on-delete' = 'true'" + + ")"); + + sql("INSERT INTO remove_record_on_delete VALUES (1, CAST (NULL AS STRING), 'apple')"); + + // delete record + sql("DELETE FROM remove_record_on_delete WHERE pk = 1"); + + // batch read + assertThat(sql("SELECT * FROM remove_record_on_delete")).isEmpty(); + + // insert records + sql("INSERT INTO remove_record_on_delete VALUES (1, CAST (NULL AS STRING), 'apache')"); + sql("INSERT INTO remove_record_on_delete VALUES (1, 'A', CAST (NULL AS STRING))"); + + // batch read + assertThat(sql("SELECT * FROM remove_record_on_delete")) + .containsExactlyInAnyOrder(Row.of(1, "A", "apache")); + } + + @Test + public void testRemoveRecordOnDeleteLookup() throws Exception { + sql( + "CREATE TABLE remove_record_on_delete (pk INT PRIMARY KEY NOT ENFORCED, a STRING, b STRING) WITH (" + + " 'merge-engine' = 'partial-update'," + + " 'partial-update.remove-record-on-delete' = 'true'," + + " 'changelog-producer' = 'lookup'" + + ")"); + + sql("INSERT INTO remove_record_on_delete VALUES (1, CAST (NULL AS STRING), 'apple')"); + + // delete record + sql("DELETE FROM remove_record_on_delete WHERE pk = 1"); + + // batch read + assertThat(sql("SELECT * FROM remove_record_on_delete")).isEmpty(); + + // insert records + sql("INSERT INTO remove_record_on_delete VALUES (1, CAST (NULL AS STRING), 'apache')"); + sql("INSERT INTO remove_record_on_delete VALUES (1, 'A', CAST (NULL AS STRING))"); + + // batch read + assertThat(sql("SELECT * FROM remove_record_on_delete")) + .containsExactlyInAnyOrder(Row.of(1, "A", "apache")); + + // streaming read results has -U + BlockingIterator iterator = + streamSqlBlockIter( + "SELECT * FROM remove_record_on_delete /*+ OPTIONS('scan.timestamp-millis' = '0') */"); + assertThat(iterator.collect(5)) + .containsExactly( + Row.ofKind(RowKind.INSERT, 1, null, "apple"), + Row.ofKind(RowKind.DELETE, 1, null, "apple"), + Row.ofKind(RowKind.INSERT, 1, null, "apache"), + Row.ofKind(RowKind.UPDATE_BEFORE, 1, null, "apache"), + Row.ofKind(RowKind.UPDATE_AFTER, 1, "A", "apache")); + iterator.close(); + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PredicateITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PredicateITCase.java index f50f7db60f02..0668c17eea7c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PredicateITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PredicateITCase.java @@ -21,6 +21,8 @@ import org.apache.flink.types.Row; import org.junit.jupiter.api.Test; +import java.util.concurrent.ThreadLocalRandom; + import static org.assertj.core.api.Assertions.assertThat; /** Predicate ITCase. */ @@ -50,6 +52,42 @@ public void testAppendFilterBucket() throws Exception { innerTestAllFields(); } + @Test + public void testIntegerFilter() { + int rand = ThreadLocalRandom.current().nextInt(3); + String fileFormat; + if (rand == 0) { + fileFormat = "avro"; + } else if (rand == 1) { + fileFormat = "parquet"; + } else { + fileFormat = "orc"; + } + + sql( + "CREATE TABLE T (" + + "a TINYINT," + + "b SMALLINT," + + "c INT," + + "d BIGINT" + + ") WITH (" + + "'file.format' = '%s'" + + ")", + fileFormat); + sql( + "INSERT INTO T VALUES (CAST (1 AS TINYINT), CAST (1 AS SMALLINT), 1, 1), " + + "(CAST (2 AS TINYINT), CAST (2 AS SMALLINT), 2, 2)"); + + Row expectedResult = Row.of((byte) 1, (short) 1, 1, 1L); + assertThat(sql("SELECT * FROM T WHERE a = CAST (1 AS TINYINT)")) + .containsExactly(expectedResult); + assertThat(sql("SELECT * FROM T WHERE b = CAST (1 AS SMALLINT)")) + .containsExactly(expectedResult); + assertThat(sql("SELECT * FROM T WHERE c = 1")).containsExactly(expectedResult); + assertThat(sql("SELECT * FROM T WHERE d = CAST (1 AS BIGINT)")) + .containsExactly(expectedResult); + } + private void writeRecords() throws Exception { sql("INSERT INTO T VALUES (1, 2), (3, 4), (5, 6), (7, 8), (9, 10)"); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java index 1a8da3c5b7f9..027eada9224c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java @@ -37,8 +37,11 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; +import java.nio.file.Files; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -48,8 +51,10 @@ import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; /** Tests for changelog table with primary keys. */ public class PrimaryKeyFileStoreTableITCase extends AbstractTestBase { @@ -73,7 +78,7 @@ public void before() throws IOException { private String createCatalogSql(String catalogName, String warehouse) { String defaultPropertyString = ""; - if (tableDefaultProperties.size() > 0) { + if (!tableDefaultProperties.isEmpty()) { defaultPropertyString = ", "; defaultPropertyString += tableDefaultProperties.entrySet().stream() @@ -95,7 +100,7 @@ private String createCatalogSql(String catalogName, String warehouse) { // ------------------------------------------------------------------------ @Test - @Timeout(1200) + @Timeout(180) public void testFullCompactionTriggerInterval() throws Exception { innerTestChangelogProducing( Arrays.asList( @@ -104,7 +109,7 @@ public void testFullCompactionTriggerInterval() throws Exception { } @Test - @Timeout(1200) + @Timeout(180) public void testFullCompactionWithLongCheckpointInterval() throws Exception { // create table TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().parallelism(1).build(); @@ -163,7 +168,7 @@ public void testFullCompactionWithLongCheckpointInterval() throws Exception { } @Test - @Timeout(1200) + @Timeout(180) public void testLookupChangelog() throws Exception { innerTestChangelogProducing(Collections.singletonList("'changelog-producer' = 'lookup'")); } @@ -221,6 +226,9 @@ public void testTableReadWriteBranch() throws Exception { } assertThat(actualBranch) .containsExactlyInAnyOrder("+I[1, A]", "+I[10, v10]", "+I[11, v11]", "+I[12, v12]"); + + it.close(); + branchIt.close(); } private void innerTestChangelogProducing(List options) throws Exception { @@ -237,7 +245,10 @@ private void innerTestChangelogProducing(List options) throws Exception "CREATE TABLE T ( k INT, v STRING, PRIMARY KEY (k) NOT ENFORCED ) " + "WITH ( " + "'bucket' = '2', " - + String.join(",", options) + // producers will very quickly produce snapshots, + // so consumers should also discover new snapshots quickly + + "'continuous.discovery-interval' = '1ms', " + + String.join(", ", options) + ")"); Path inputPath = new Path(path, "input"); @@ -297,9 +308,7 @@ private void innerTestChangelogProducing(List options) throws Exception public void testBatchJobWithConflictAndRestart() throws Exception { TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().allowRestart(10).build(); tEnv.executeSql( - "CREATE CATALOG mycat WITH ( 'type' = 'paimon', 'warehouse' = '" - + getTempDirPath() - + "' )"); + "CREATE CATALOG mycat WITH ( 'type' = 'paimon', 'warehouse' = '" + path + "' )"); tEnv.executeSql("USE CATALOG mycat"); tEnv.executeSql( "CREATE TABLE t ( k INT, v INT, PRIMARY KEY (k) NOT ENFORCED ) " @@ -329,19 +338,287 @@ public void testBatchJobWithConflictAndRestart() throws Exception { } } + @Timeout(60) + @ParameterizedTest() + @ValueSource(booleans = {false, true}) + public void testRecreateTableWithException(boolean isReloadData) throws Exception { + TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().build(); + bEnv.executeSql(createCatalogSql("testCatalog", path + "/warehouse")); + bEnv.executeSql("USE CATALOG testCatalog"); + bEnv.executeSql( + "CREATE TABLE t ( pt INT, k INT, v INT, PRIMARY KEY (pt, k) NOT ENFORCED ) " + + "PARTITIONED BY (pt) " + + "WITH (" + + " 'bucket' = '2'\n" + + " ,'continuous.discovery-interval' = '1s'\n" + + ")"); + + TableEnvironment sEnv = + tableEnvironmentBuilder() + .streamingMode() + .parallelism(4) + .checkpointIntervalMs(1000) + .build(); + sEnv.executeSql(createCatalogSql("testCatalog", path + "/warehouse")); + sEnv.executeSql("USE CATALOG testCatalog"); + CloseableIterator it = sEnv.executeSql("SELECT * FROM t").collect(); + + // first write + List values = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + values.add(String.format("(0, %d, %d)", i, i)); + values.add(String.format("(1, %d, %d)", i, i)); + } + bEnv.executeSql("INSERT INTO t VALUES " + String.join(", ", values)).await(); + List expected = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + expected.add(Row.ofKind(RowKind.INSERT, 0, i, i)); + expected.add(Row.ofKind(RowKind.INSERT, 1, i, i)); + } + assertStreamingResult(it, expected); + + // second write + values.clear(); + for (int i = 0; i < 10; i++) { + values.add(String.format("(0, %d, %d)", i, i + 1)); + values.add(String.format("(1, %d, %d)", i, i + 1)); + } + bEnv.executeSql("INSERT INTO t VALUES " + String.join(", ", values)).await(); + + // start a read job + for (int i = 0; i < 10; i++) { + expected.add(Row.ofKind(RowKind.UPDATE_BEFORE, 0, i, i)); + expected.add(Row.ofKind(RowKind.UPDATE_BEFORE, 1, i, i)); + expected.add(Row.ofKind(RowKind.UPDATE_AFTER, 0, i, i + 1)); + expected.add(Row.ofKind(RowKind.UPDATE_AFTER, 1, i, i + 1)); + } + assertStreamingResult(it, expected.subList(20, 60)); + + // delete table and recreate a same table + bEnv.executeSql("DROP TABLE t"); + bEnv.executeSql( + "CREATE TABLE t ( pt INT, k INT, v INT, PRIMARY KEY (pt, k) NOT ENFORCED ) " + + "PARTITIONED BY (pt) " + + "WITH (" + + " 'bucket' = '2'\n" + + ")"); + + // if reload data, it will generate a new snapshot for recreated table + if (isReloadData) { + bEnv.executeSql("INSERT INTO t VALUES " + String.join(", ", values)).await(); + } + assertThatCode(it::next) + .rootCause() + .hasMessageContaining( + "The next expected snapshot is too big! Most possible cause might be the table had been recreated."); + } + + @Test + @Timeout(120) + public void testChangelogCompactInBatchWrite() throws Exception { + TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().build(); + String catalogDdl = + "CREATE CATALOG mycat WITH ( 'type' = 'paimon', 'warehouse' = '" + path + "' )"; + bEnv.executeSql(catalogDdl); + bEnv.executeSql("USE CATALOG mycat"); + bEnv.executeSql( + "CREATE TABLE t ( pt INT, k INT, v INT, PRIMARY KEY (pt, k) NOT ENFORCED ) " + + "PARTITIONED BY (pt) " + + "WITH (" + + " 'bucket' = '10',\n" + + " 'changelog-producer' = 'lookup',\n" + + " 'changelog.precommit-compact' = 'true',\n" + + " 'snapshot.num-retained.min' = '3',\n" + + " 'snapshot.num-retained.max' = '3'\n" + + ")"); + + TableEnvironment sEnv = + tableEnvironmentBuilder().streamingMode().checkpointIntervalMs(1000).build(); + sEnv.executeSql(catalogDdl); + sEnv.executeSql("USE CATALOG mycat"); + + List values = new ArrayList<>(); + for (int i = 0; i < 1000; i++) { + values.add(String.format("(0, %d, %d)", i, i)); + values.add(String.format("(1, %d, %d)", i, i)); + } + bEnv.executeSql("INSERT INTO t VALUES " + String.join(", ", values)).await(); + + List compactedChangelogs2 = listAllFilesWithPrefix("compacted-changelog-"); + assertThat(compactedChangelogs2).hasSize(2); + assertThat(listAllFilesWithPrefix("changelog-")).isEmpty(); + + List expected = new ArrayList<>(); + for (int i = 0; i < 1000; i++) { + expected.add(Row.ofKind(RowKind.INSERT, 0, i, i)); + expected.add(Row.ofKind(RowKind.INSERT, 1, i, i)); + } + assertStreamingResult( + sEnv.executeSql("SELECT * FROM t /*+ OPTIONS('scan.snapshot-id' = '1') */"), + expected); + + values.clear(); + for (int i = 0; i < 1000; i++) { + values.add(String.format("(0, %d, %d)", i, i + 1)); + values.add(String.format("(1, %d, %d)", i, i + 1)); + } + bEnv.executeSql("INSERT INTO t VALUES " + String.join(", ", values)).await(); + + assertThat(listAllFilesWithPrefix("compacted-changelog-")).hasSize(4); + assertThat(listAllFilesWithPrefix("changelog-")).isEmpty(); + + for (int i = 0; i < 1000; i++) { + expected.add(Row.ofKind(RowKind.UPDATE_BEFORE, 0, i, i)); + expected.add(Row.ofKind(RowKind.UPDATE_BEFORE, 1, i, i)); + expected.add(Row.ofKind(RowKind.UPDATE_AFTER, 0, i, i + 1)); + expected.add(Row.ofKind(RowKind.UPDATE_AFTER, 1, i, i + 1)); + } + assertStreamingResult( + sEnv.executeSql("SELECT * FROM t /*+ OPTIONS('scan.snapshot-id' = '1') */"), + expected); + + values.clear(); + for (int i = 0; i < 1000; i++) { + values.add(String.format("(0, %d, %d)", i, i + 2)); + values.add(String.format("(1, %d, %d)", i, i + 2)); + } + bEnv.executeSql("INSERT INTO t VALUES " + String.join(", ", values)).await(); + + assertThat(listAllFilesWithPrefix("compacted-changelog-")).hasSize(4); + assertThat(listAllFilesWithPrefix("changelog-")).isEmpty(); + LocalFileIO fileIO = LocalFileIO.create(); + for (String p : compactedChangelogs2) { + assertThat(fileIO.exists(new Path(p))).isFalse(); + } + + expected = expected.subList(2000, 6000); + for (int i = 0; i < 1000; i++) { + expected.add(Row.ofKind(RowKind.UPDATE_BEFORE, 0, i, i + 1)); + expected.add(Row.ofKind(RowKind.UPDATE_BEFORE, 1, i, i + 1)); + expected.add(Row.ofKind(RowKind.UPDATE_AFTER, 0, i, i + 2)); + expected.add(Row.ofKind(RowKind.UPDATE_AFTER, 1, i, i + 2)); + } + assertStreamingResult( + sEnv.executeSql("SELECT * FROM t /*+ OPTIONS('scan.snapshot-id' = '1') */"), + expected); + } + + @Test + @Timeout(120) + public void testChangelogCompactInStreamWrite() throws Exception { + TableEnvironment sEnv = + tableEnvironmentBuilder() + .streamingMode() + .checkpointIntervalMs(2000) + .parallelism(4) + .build(); + + sEnv.executeSql(createCatalogSql("testCatalog", path + "/warehouse")); + sEnv.executeSql("USE CATALOG testCatalog"); + sEnv.executeSql( + "CREATE TABLE t ( pt INT, k INT, v INT, PRIMARY KEY (pt, k) NOT ENFORCED ) " + + "PARTITIONED BY (pt) " + + "WITH (" + + " 'bucket' = '10',\n" + + " 'changelog-producer' = 'lookup',\n" + + " 'changelog.precommit-compact' = 'true'\n" + + ")"); + + Path inputPath = new Path(path, "input"); + LocalFileIO.create().mkdirs(inputPath); + sEnv.executeSql( + "CREATE TABLE `default_catalog`.`default_database`.`s` ( pt INT, k INT, v INT, PRIMARY KEY (pt, k) NOT ENFORCED) " + + "WITH ( 'connector' = 'filesystem', 'format' = 'testcsv', 'path' = '" + + inputPath + + "', 'source.monitor-interval' = '500ms' )"); + + sEnv.executeSql("INSERT INTO t SELECT * FROM `default_catalog`.`default_database`.`s`"); + CloseableIterator it = sEnv.executeSql("SELECT * FROM t").collect(); + + // write initial data + List values = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + values.add(String.format("(0, %d, %d)", i, i)); + values.add(String.format("(1, %d, %d)", i, i)); + } + sEnv.executeSql( + "INSERT INTO `default_catalog`.`default_database`.`s` VALUES " + + String.join(", ", values)) + .await(); + + List expected = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + expected.add(Row.ofKind(RowKind.INSERT, 0, i, i)); + expected.add(Row.ofKind(RowKind.INSERT, 1, i, i)); + } + assertStreamingResult(it, expected); + + List compactedChangelogs2 = listAllFilesWithPrefix("compacted-changelog-"); + assertThat(compactedChangelogs2).hasSize(2); + assertThat(listAllFilesWithPrefix("changelog-")).isEmpty(); + + // write update data + values.clear(); + for (int i = 0; i < 100; i++) { + values.add(String.format("(0, %d, %d)", i, i + 1)); + values.add(String.format("(1, %d, %d)", i, i + 1)); + } + sEnv.executeSql( + "INSERT INTO `default_catalog`.`default_database`.`s` VALUES " + + String.join(", ", values)) + .await(); + for (int i = 0; i < 100; i++) { + expected.add(Row.ofKind(RowKind.UPDATE_BEFORE, 0, i, i)); + expected.add(Row.ofKind(RowKind.UPDATE_BEFORE, 1, i, i)); + expected.add(Row.ofKind(RowKind.UPDATE_AFTER, 0, i, i + 1)); + expected.add(Row.ofKind(RowKind.UPDATE_AFTER, 1, i, i + 1)); + } + assertStreamingResult(it, expected.subList(200, 600)); + assertThat(listAllFilesWithPrefix("compacted-changelog-")).hasSize(4); + assertThat(listAllFilesWithPrefix("changelog-")).isEmpty(); + } + + private List listAllFilesWithPrefix(String prefix) throws Exception { + try (Stream stream = Files.walk(java.nio.file.Paths.get(path))) { + return stream.filter(Files::isRegularFile) + .filter(p -> p.getFileName().toString().startsWith(prefix)) + .map(java.nio.file.Path::toString) + .collect(Collectors.toList()); + } + } + + private void assertStreamingResult(TableResult result, List expected) throws Exception { + List actual = new ArrayList<>(); + try (CloseableIterator it = result.collect()) { + while (actual.size() < expected.size() && it.hasNext()) { + actual.add(it.next()); + } + } + assertThat(actual).hasSameElementsAs(expected); + } + + private void assertStreamingResult(CloseableIterator it, List expected) { + List actual = new ArrayList<>(); + while (actual.size() < expected.size() && it.hasNext()) { + actual.add(it.next()); + } + + assertThat(actual).hasSameElementsAs(expected); + } + // ------------------------------------------------------------------------ // Random Tests // ------------------------------------------------------------------------ @Test - @Timeout(1200) + @Timeout(180) public void testNoChangelogProducerBatchRandom() throws Exception { TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().build(); testNoChangelogProducerRandom(bEnv, 1, false); } @Test - @Timeout(1200) + @Timeout(180) public void testNoChangelogProducerStreamingRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = @@ -354,14 +631,14 @@ public void testNoChangelogProducerStreamingRandom() throws Exception { } @Test - @Timeout(1200) + @Timeout(180) public void testFullCompactionChangelogProducerBatchRandom() throws Exception { TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().build(); testFullCompactionChangelogProducerRandom(bEnv, 1, false); } @Test - @Timeout(1200) + @Timeout(180) public void testFullCompactionChangelogProducerStreamingRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = @@ -374,7 +651,7 @@ public void testFullCompactionChangelogProducerStreamingRandom() throws Exceptio } @Test - @Timeout(1200) + @Timeout(180) public void testStandAloneFullCompactJobRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = @@ -387,14 +664,14 @@ public void testStandAloneFullCompactJobRandom() throws Exception { } @Test - @Timeout(1200) + @Timeout(180) public void testLookupChangelogProducerBatchRandom() throws Exception { TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().build(); testLookupChangelogProducerRandom(bEnv, 1, false); } @Test - @Timeout(1200) + @Timeout(180) public void testLookupChangelogProducerStreamingRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = @@ -407,7 +684,7 @@ public void testLookupChangelogProducerStreamingRandom() throws Exception { } @Test - @Timeout(1200) + @Timeout(180) public void testStandAloneLookupJobRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = @@ -458,10 +735,10 @@ private void testFullCompactionChangelogProducerRandom( enableFailure, "'bucket' = '4'," + String.format( - "'write-buffer-size' = '%s',", - random.nextBoolean() ? "512kb" : "1mb") - + "'changelog-producer' = 'full-compaction'," - + "'full-compaction.delta-commits' = '3'"); + "'write-buffer-size' = '%s'," + + "'changelog-producer' = 'full-compaction'," + + "'full-compaction.delta-commits' = '3'", + random.nextBoolean() ? "4mb" : "8mb")); // sleep for a random amount of time to check // if we can first read complete records then read incremental records correctly @@ -482,14 +759,17 @@ private void testLookupChangelogProducerRandom( tEnv, numProducers, enableFailure, - "'bucket' = '4'," - + String.format( - "'write-buffer-size' = '%s',", - random.nextBoolean() ? "512kb" : "1mb") - + "'changelog-producer' = 'lookup'," - + String.format("'lookup-wait' = '%s',", random.nextBoolean()) - + String.format( - "'deletion-vectors.enabled' = '%s'", enableDeletionVectors)); + String.format( + "'bucket' = '4', " + + "'writer-buffer-size' = '%s', " + + "'changelog-producer' = 'lookup', " + + "'lookup-wait' = '%s', " + + "'deletion-vectors.enabled' = '%s', " + + "'changelog.precommit-compact' = '%s'", + random.nextBoolean() ? "4mb" : "8mb", + random.nextBoolean(), + enableDeletionVectors, + random.nextBoolean())); // sleep for a random amount of time to check // if we can first read complete records then read incremental records correctly @@ -508,11 +788,11 @@ private void testStandAloneFullCompactJobRandom( false, "'bucket' = '4'," + String.format( - "'write-buffer-size' = '%s',", - random.nextBoolean() ? "512kb" : "1mb") - + "'changelog-producer' = 'full-compaction'," - + "'full-compaction.delta-commits' = '3'," - + "'write-only' = 'true'"); + "'write-buffer-size' = '%s'," + + "'changelog-producer' = 'full-compaction'," + + "'full-compaction.delta-commits' = '3'," + + "'write-only' = 'true'", + random.nextBoolean() ? "4mb" : "8mb")); // sleep for a random amount of time to check // if dedicated compactor job can find first snapshot to compact correctly @@ -547,11 +827,11 @@ private void testStandAloneLookupJobRandom( false, "'bucket' = '4'," + String.format( - "'write-buffer-size' = '%s',", - random.nextBoolean() ? "512kb" : "1mb") - + "'changelog-producer' = 'lookup'," - + String.format("'lookup-wait' = '%s',", random.nextBoolean()) - + "'write-only' = 'true'"); + "'write-buffer-size' = '%s'," + + "'changelog-producer' = 'lookup'," + + "'lookup-wait' = '%s'," + + "'write-only' = 'true'", + random.nextBoolean() ? "4mb" : "8mb", random.nextBoolean())); // sleep for a random amount of time to check // if dedicated compactor job can find first snapshot to compact correctly @@ -616,6 +896,10 @@ private void checkChangelogTestResult(int numProducers) throws Exception { private List testRandom( TableEnvironment tEnv, int numProducers, boolean enableFailure, String tableProperties) throws Exception { + // producers will very quickly produce snapshots, + // so consumers should also discover new snapshots quickly + tableProperties += ",'continuous.discovery-interval' = '1ms'"; + String failingName = UUID.randomUUID().toString(); String failingPath = FailingFileIO.getFailingPath(failingName, path); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java index d1e9b23e1b54..10de1ae4839f 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java @@ -76,7 +76,6 @@ import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; import static org.apache.paimon.CoreOptions.SOURCE_SPLIT_OPEN_FILE_COST; import static org.apache.paimon.CoreOptions.SOURCE_SPLIT_TARGET_SIZE; -import static org.apache.paimon.flink.AbstractFlinkTableFactory.buildPaimonTable; import static org.apache.paimon.flink.FlinkConnectorOptions.INFER_SCAN_MAX_PARALLELISM; import static org.apache.paimon.flink.FlinkConnectorOptions.INFER_SCAN_PARALLELISM; import static org.apache.paimon.flink.FlinkConnectorOptions.SCAN_PARALLELISM; @@ -1827,7 +1826,10 @@ private void testSinkParallelism(Integer configParallelism, int expectedParallel DynamicTableSink tableSink = new FlinkTableSink( - context.getObjectIdentifier(), buildPaimonTable(context), context, null); + context.getObjectIdentifier(), + new FlinkTableFactory().buildPaimonTable(context), + context, + null); assertThat(tableSink).isInstanceOf(FlinkTableSink.class); // 2. get sink provider diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java index e1655bcb6ba9..08f79efccb76 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SchemaChangeITCase.java @@ -1007,6 +1007,36 @@ public void testAlterTableNonPhysicalColumn() { .doesNotContain("schema"); } + @Test + public void testSequenceFieldSortOrder() { + // test default condition which get the largest record + sql( + "CREATE TABLE T1 (a STRING PRIMARY KEY NOT ENFORCED, b STRING, c STRING) WITH ('sequence.field'='c')"); + sql("INSERT INTO T1 VALUES ('a', 'b', 'l')"); + sql("INSERT INTO T1 VALUES ('a', 'd', 'n')"); + sql("INSERT INTO T1 VALUES ('a', 'e', 'm')"); + List sql = sql("select * from T1"); + assertThat(sql("select * from T1").toString()).isEqualTo("[+I[a, d, n]]"); + + // test for get small record + sql( + "CREATE TABLE T2 (a STRING PRIMARY KEY NOT ENFORCED, b STRING, c BIGINT) WITH ('sequence.field'='c', 'sequence.field.sort-order'='descending')"); + sql("INSERT INTO T2 VALUES ('a', 'b', 1)"); + sql("INSERT INTO T2 VALUES ('a', 'd', 3)"); + sql("INSERT INTO T2 VALUES ('a', 'e', 2)"); + sql = sql("select * from T2"); + assertThat(sql("select * from T2").toString()).isEqualTo("[+I[a, b, 1]]"); + + // test for get largest record + sql( + "CREATE TABLE T3 (a STRING PRIMARY KEY NOT ENFORCED, b STRING, c DOUBLE) WITH ('sequence.field'='c', 'sequence.field.sort-order'='ascending')"); + sql("INSERT INTO T3 VALUES ('a', 'b', 1.0)"); + sql("INSERT INTO T3 VALUES ('a', 'd', 3.0)"); + sql("INSERT INTO T3 VALUES ('a', 'e', 2.0)"); + sql = sql("select * from T3"); + assertThat(sql("select * from T3").toString()).isEqualTo("[+I[a, d, 3.0]]"); + } + @Test public void testAlterTableMetadataComment() { sql("CREATE TABLE T (a INT, name VARCHAR METADATA COMMENT 'header1', b INT)"); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java index b14f1c041bbd..cb323542d4c1 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/UnawareBucketAppendOnlyTableITCase.java @@ -19,14 +19,25 @@ package org.apache.paimon.flink; import org.apache.paimon.Snapshot; +import org.apache.paimon.data.InternalRow; import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.reader.RecordReaderIterator; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.utils.FailingFileIO; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.File; import java.time.Duration; @@ -346,6 +357,94 @@ public void testFileIndex() { .containsExactlyInAnyOrder(Row.of(2, "c", "BBB"), Row.of(3, "c", "BBB")); } + @Timeout(60) + @Test + public void testStatelessWriter() throws Exception { + FileStoreTable table = + FileStoreTableFactory.create( + LocalFileIO.create(), new Path(path, "default.db/append_table")); + + StreamExecutionEnvironment env = + streamExecutionEnvironmentBuilder() + .streamingMode() + .parallelism(2) + .checkpointIntervalMs(500) + .build(); + DataStream source = + env.addSource(new TestStatelessWriterSource(table)).setParallelism(2).forward(); + + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + tEnv.registerCatalog("mycat", sEnv.getCatalog("PAIMON").get()); + tEnv.executeSql("USE CATALOG mycat"); + tEnv.createTemporaryView("S", tEnv.fromDataStream(source).as("id")); + + tEnv.executeSql("INSERT INTO append_table SELECT id, 'test' FROM S").await(); + assertThat(batchSql("SELECT * FROM append_table")) + .containsExactlyInAnyOrder(Row.of(1, "test"), Row.of(2, "test")); + } + + private static class TestStatelessWriterSource extends RichParallelSourceFunction { + + private final FileStoreTable table; + + private volatile boolean isRunning = true; + + private TestStatelessWriterSource(FileStoreTable table) { + this.table = table; + } + + @Override + public void run(SourceContext sourceContext) throws Exception { + int taskId = getRuntimeContext().getIndexOfThisSubtask(); + // wait some time in parallelism #2, + // so that it does not commit in the same checkpoint with parallelism #1 + int waitCount = (taskId == 0 ? 0 : 10); + + while (isRunning) { + synchronized (sourceContext.getCheckpointLock()) { + if (taskId == 0) { + if (waitCount == 0) { + sourceContext.collect(1); + } else if (countNumRecords() >= 1) { + // wait for the record to commit before exiting + break; + } + } else { + int numRecords = countNumRecords(); + if (numRecords >= 1) { + if (waitCount == 0) { + sourceContext.collect(2); + } else if (countNumRecords() >= 2) { + // make sure the next checkpoint is successful + break; + } + } + } + waitCount--; + } + Thread.sleep(1000); + } + } + + private int countNumRecords() throws Exception { + int ret = 0; + RecordReader reader = + table.newRead().createReader(table.newSnapshotReader().read()); + try (RecordReaderIterator it = new RecordReaderIterator<>(reader)) { + while (it.hasNext()) { + it.next(); + ret++; + } + } + return ret; + } + + @Override + public void cancel() { + isRunning = false; + } + } + @Override protected List ddl() { return Arrays.asList( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java index ba95113d9adb..bc849f0a135f 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCase.java @@ -272,7 +272,7 @@ public void testUnawareBucketStreamingCompact() throws Exception { checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); - // repairing that the ut don't specify the real parition of table + // repairing that the ut don't specify the real partition of table runActionForUnawareTable(true); // first compaction, snapshot will be 3 @@ -314,7 +314,7 @@ public void testUnawareBucketBatchCompact() throws Exception { checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); - // repairing that the ut don't specify the real parition of table + // repairing that the ut don't specify the real partition of table runActionForUnawareTable(false); // first compaction, snapshot will be 3. @@ -366,7 +366,7 @@ public void testSpecifyNonPartitionField() throws Exception { rowData(1, 100, 15, BinaryString.fromString("20221209"))); Assertions.assertThatThrownBy(() -> runAction(false)) - .hasMessage("Only parition key can be specialized in compaction action."); + .hasMessage("Only partition key can be specialized in compaction action."); } @Test diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java index ef921ad666ca..e2243ddf269a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java @@ -27,6 +27,7 @@ import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.flink.table.api.TableException; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -39,6 +40,7 @@ import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.init; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.testStreamingRead; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; /** IT cases for consumer management actions. */ public class ConsumerActionITCase extends ActionITCaseBase { @@ -144,6 +146,48 @@ public void testResetConsumer(String invoker) throws Exception { } Optional consumer3 = consumerManager.consumer("myid"); assertThat(consumer3).isNotPresent(); + + // reset consumer to a not exist snapshot id + List args1 = + Arrays.asList( + "reset_consumer", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--consumer_id", + "myid", + "--next_snapshot", + "10"); + switch (invoker) { + case "action": + assertThrows( + RuntimeException.class, + () -> createAction(ResetConsumerAction.class, args1).run()); + break; + case "procedure_indexed": + assertThrows( + TableException.class, + () -> + executeSQL( + String.format( + "CALL sys.reset_consumer('%s.%s', 'myid', 10)", + database, tableName))); + break; + case "procedure_named": + assertThrows( + TableException.class, + () -> + executeSQL( + String.format( + "CALL sys.reset_consumer(`table` => '%s.%s', consumer_id => 'myid', next_snapshot_id => cast(10 as bigint))", + database, tableName))); + break; + default: + throw new UnsupportedOperationException(invoker); + } } @ParameterizedTest @@ -206,7 +250,7 @@ public void testResetBranchConsumer(String invoker) throws Exception { "--consumer_id", "myid", "--next_snapshot", - "1"); + "3"); // reset consumer switch (invoker) { case "action": @@ -215,13 +259,13 @@ public void testResetBranchConsumer(String invoker) throws Exception { case "procedure_indexed": executeSQL( String.format( - "CALL sys.reset_consumer('%s.%s', 'myid', 1)", + "CALL sys.reset_consumer('%s.%s', 'myid', 3)", database, branchTableName)); break; case "procedure_named": executeSQL( String.format( - "CALL sys.reset_consumer(`table` => '%s.%s', consumer_id => 'myid', next_snapshot_id => cast(1 as bigint))", + "CALL sys.reset_consumer(`table` => '%s.%s', consumer_id => 'myid', next_snapshot_id => cast(3 as bigint))", database, branchTableName)); break; default: @@ -229,7 +273,7 @@ public void testResetBranchConsumer(String invoker) throws Exception { } Optional consumer2 = consumerManager.consumer("myid"); assertThat(consumer2).isPresent(); - assertThat(consumer2.get().nextSnapshot()).isEqualTo(1); + assertThat(consumer2.get().nextSnapshot()).isEqualTo(3); // delete consumer switch (invoker) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ExpireTagsActionTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ExpireTagsActionTest.java new file mode 100644 index 000000000000..5a156ced25be --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ExpireTagsActionTest.java @@ -0,0 +1,108 @@ +/* + * 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.data.Timestamp; +import org.apache.paimon.table.FileStoreTable; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.LocalDateTime; + +import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.bEnv; +import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.init; +import static org.assertj.core.api.Assertions.assertThat; + +/** IT cases for {@link ExpireTagsAction}. */ +public class ExpireTagsActionTest extends ActionITCaseBase { + + @BeforeEach + public void setUp() { + init(warehouse); + } + + @Test + public void testExpireTags() throws Exception { + bEnv.executeSql( + "CREATE TABLE T (id STRING, name STRING," + + " PRIMARY KEY (id) NOT ENFORCED)" + + " WITH ('bucket'='1', 'write-only'='true')"); + + FileStoreTable table = getFileStoreTable("T"); + + // generate 5 snapshots + for (int i = 1; i <= 5; i++) { + bEnv.executeSql("INSERT INTO T VALUES ('" + i + "', '" + i + "')").await(); + } + assertThat(table.snapshotManager().snapshotCount()).isEqualTo(5); + + bEnv.executeSql("CALL sys.create_tag('default.T', 'tag-1', 1)").await(); + bEnv.executeSql("CALL sys.create_tag('default.T', 'tag-2', 2, '1h')").await(); + bEnv.executeSql("CALL sys.create_tag('default.T', 'tag-3', 3, '1h')").await(); + assertThat(table.tagManager().tags().size()).isEqualTo(3); + + createAction( + ExpireTagsAction.class, + "expire_tags", + "--warehouse", + warehouse, + "--table", + database + ".T") + .run(); + // no tags expired + assertThat(table.tagManager().tags().size()).isEqualTo(3); + + bEnv.executeSql("CALL sys.create_tag('default.T', 'tag-4', 4, '1s')").await(); + bEnv.executeSql("CALL sys.create_tag('default.T', 'tag-5', 5, '1s')").await(); + assertThat(table.tagManager().tags().size()).isEqualTo(5); + + Thread.sleep(2000); + createAction( + ExpireTagsAction.class, + "expire_tags", + "--warehouse", + warehouse, + "--table", + database + ".T") + .run(); + // tag-4,tag-5 expires + assertThat(table.tagManager().tags().size()).isEqualTo(3); + assertThat(table.tagManager().tagExists("tag-4")).isFalse(); + assertThat(table.tagManager().tagExists("tag-5")).isFalse(); + + // tag-3 as the base older_than time + LocalDateTime olderThanTime = table.tagManager().tag("tag-3").getTagCreateTime(); + java.sql.Timestamp timestamp = + new java.sql.Timestamp(Timestamp.fromLocalDateTime(olderThanTime).getMillisecond()); + createAction( + ExpireTagsAction.class, + "expire_tags", + "--warehouse", + warehouse, + "--table", + database + ".T", + "--older_than", + timestamp.toString()) + .run(); + // tag-1,tag-2 expires. tag-1 expired by its file creation time. + assertThat(table.tagManager().tags().size()).isEqualTo(1); + assertThat(table.tagManager().tagExists("tag-3")).isTrue(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MergeIntoActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MergeIntoActionITCase.java index 41d607fac5f6..3907c0398532 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MergeIntoActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MergeIntoActionITCase.java @@ -571,6 +571,123 @@ public void testIllegalSourceNameSqlCase() { .satisfies(anyCauseMatches(ValidationException.class, "Object 'S' not found")); } + @ParameterizedTest + @MethodSource("testArguments") + public void testNotMatchedBySourceUpsert(boolean qualified, String invoker) throws Exception { + sEnv.executeSql("DROP TABLE T"); + prepareTargetTable(CoreOptions.ChangelogProducer.INPUT); + + // build MergeIntoAction + MergeIntoActionBuilder action = new MergeIntoActionBuilder(warehouse, database, "T"); + action.withSourceSqls("CREATE TEMPORARY VIEW SS AS SELECT k, v, 'unknown', dt FROM S") + .withSourceTable(qualified ? "default.SS" : "SS") + .withMergeCondition("T.k = SS.k AND T.dt = SS.dt") + .withNotMatchedBySourceUpsert( + "dt < '02-28'", "v = v || '_nmu', last_action = 'not_matched_upsert'"); + + String procedureStatement = ""; + if ("procedure_indexed".equals(invoker)) { + procedureStatement = + String.format( + "CALL sys.merge_into('%s.T', '', '%s', '%s', 'T.k = SS.k AND T.dt = SS.dt', '', '', '', '', '', 'dt < ''02-28''', 'v = v || ''_nmu'', last_action = ''not_matched_upsert''')", + database, + "CREATE TEMPORARY VIEW SS AS SELECT k, v, ''unknown'', dt FROM S", + qualified ? "default.SS" : "SS"); + } else if ("procedure_named".equals(invoker)) { + procedureStatement = + String.format( + "CALL sys.merge_into(" + + "target_table => '%s.T', " + + "source_sqls => '%s', " + + "source_table => '%s', " + + "merge_condition => 'T.k = SS.k AND T.dt = SS.dt', " + + "not_matched_by_source_upsert_condition => 'dt < ''02-28'''," + + "not_matched_by_source_upsert_setting => 'v = v || ''_nmu'', last_action = ''not_matched_upsert''')", + database, + "CREATE TEMPORARY VIEW SS AS SELECT k, v, ''unknown'', dt FROM S", + qualified ? "default.SS" : "SS"); + } + + List streamingExpected = + Arrays.asList( + changelogRow("+U", 2, "v_2_nmu", "not_matched_upsert", "02-27"), + changelogRow("+U", 3, "v_3_nmu", "not_matched_upsert", "02-27")); + + List batchExpected = + Arrays.asList( + changelogRow("+I", 1, "v_1", "creation", "02-27"), + changelogRow("+I", 2, "v_2_nmu", "not_matched_upsert", "02-27"), + changelogRow("+I", 3, "v_3_nmu", "not_matched_upsert", "02-27"), + changelogRow("+I", 4, "v_4", "creation", "02-27"), + changelogRow("+I", 5, "v_5", "creation", "02-28"), + changelogRow("+I", 6, "v_6", "creation", "02-28"), + changelogRow("+I", 7, "v_7", "creation", "02-28"), + changelogRow("+I", 8, "v_8", "creation", "02-28"), + changelogRow("+I", 9, "v_9", "creation", "02-28"), + changelogRow("+I", 10, "v_10", "creation", "02-28")); + + if ("action".equals(invoker)) { + validateActionRunResult(action.build(), streamingExpected, batchExpected); + } else { + validateProcedureResult(procedureStatement, streamingExpected, batchExpected); + } + } + + @ParameterizedTest + @MethodSource("testArguments") + public void testNotMatchedBySourceDelete(boolean qualified, String invoker) throws Exception { + // build MergeIntoAction + MergeIntoActionBuilder action = new MergeIntoActionBuilder(warehouse, database, "T"); + action.withSourceSqls("CREATE TEMPORARY VIEW SS AS SELECT k, v, 'unknown', dt FROM S") + .withSourceTable(qualified ? "default.SS" : "SS") + .withMergeCondition("T.k = SS.k AND T.dt = SS.dt") + .withNotMatchedBySourceDelete(null); + + String procedureStatement = ""; + if ("procedure_indexed".equals(invoker)) { + procedureStatement = + String.format( + "CALL sys.merge_into('%s.T', '', '%s', '%s', 'T.k = SS.k AND T.dt = SS.dt', '', '', '', '', '', '', '', 'TRUE')", + database, + "CREATE TEMPORARY VIEW SS AS SELECT k, v, ''unknown'', dt FROM S", + qualified ? "default.SS" : "SS"); + } else if ("procedure_named".equals(invoker)) { + procedureStatement = + String.format( + "CALL sys.merge_into(" + + "target_table => '%s.T', " + + "source_sqls => '%s', " + + "source_table => '%s', " + + "merge_condition => 'T.k = SS.k AND T.dt = SS.dt', " + + "not_matched_by_source_delete_condition => 'TRUE')", + database, + "CREATE TEMPORARY VIEW SS AS SELECT k, v, ''unknown'', dt FROM S", + qualified ? "default.SS" : "SS"); + } + + List streamingExpected = + Arrays.asList( + changelogRow("-D", 2, "v_2", "creation", "02-27"), + changelogRow("-D", 3, "v_3", "creation", "02-27"), + changelogRow("-D", 5, "v_5", "creation", "02-28"), + changelogRow("-D", 6, "v_6", "creation", "02-28"), + changelogRow("-D", 9, "v_9", "creation", "02-28"), + changelogRow("-D", 10, "v_10", "creation", "02-28")); + + List batchExpected = + Arrays.asList( + changelogRow("+I", 1, "v_1", "creation", "02-27"), + changelogRow("+I", 4, "v_4", "creation", "02-27"), + changelogRow("+I", 7, "v_7", "creation", "02-28"), + changelogRow("+I", 8, "v_8", "creation", "02-28")); + + if ("action".equals(invoker)) { + validateActionRunResult(action.build(), streamingExpected, batchExpected); + } else { + validateProcedureResult(procedureStatement, streamingExpected, batchExpected); + } + } + private void validateActionRunResult( MergeIntoAction action, List streamingExpected, List batchExpected) throws Exception { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCase.java index 82c09c6674ed..938a8ce1be7a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCase.java @@ -271,4 +271,71 @@ public void testCleanWithBranch(boolean isNamedArgument) throws Exception { ImmutableList actualDeleteFile = ImmutableList.copyOf(executeSQL(procedure)); assertThat(actualDeleteFile).containsOnly(Row.of("4")); } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testRunWithMode(boolean isNamedArgument) throws Exception { + createTableAndWriteData(tableName); + + List args = + new ArrayList<>( + Arrays.asList( + "remove_orphan_files", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName)); + RemoveOrphanFilesAction action1 = createAction(RemoveOrphanFilesAction.class, args); + assertThatCode(action1::run).doesNotThrowAnyException(); + + args.add("--older_than"); + args.add("2023-12-31 23:59:59"); + RemoveOrphanFilesAction action2 = createAction(RemoveOrphanFilesAction.class, args); + assertThatCode(action2::run).doesNotThrowAnyException(); + + String withoutOlderThan = + String.format( + isNamedArgument + ? "CALL sys.remove_orphan_files(`table` => '%s.%s')" + : "CALL sys.remove_orphan_files('%s.%s')", + database, + tableName); + CloseableIterator withoutOlderThanCollect = executeSQL(withoutOlderThan); + assertThat(ImmutableList.copyOf(withoutOlderThanCollect)).containsOnly(Row.of("0")); + + String withLocalMode = + String.format( + isNamedArgument + ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59', dry_run => true, parallelism => 5, mode => 'local')" + : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59', true, 5, 'local')", + database, + tableName); + ImmutableList actualLocalRunDeleteFile = + ImmutableList.copyOf(executeSQL(withLocalMode)); + assertThat(actualLocalRunDeleteFile).containsOnly(Row.of("2")); + + String withDistributedMode = + String.format( + isNamedArgument + ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59', dry_run => true, parallelism => 5, mode => 'distributed')" + : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59', true, 5, 'distributed')", + database, + tableName); + ImmutableList actualDistributedRunDeleteFile = + ImmutableList.copyOf(executeSQL(withDistributedMode)); + assertThat(actualDistributedRunDeleteFile).containsOnly(Row.of("2")); + + String withInvalidMode = + String.format( + isNamedArgument + ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59', dry_run => true, parallelism => 5, mode => 'unknown')" + : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59', true, 5, 'unknown')", + database, + tableName); + assertThatCode(() -> executeSQL(withInvalidMode)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Unknown mode"); + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ReplaceTagActionTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ReplaceTagActionTest.java new file mode 100644 index 000000000000..00b43b9e11c9 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ReplaceTagActionTest.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action; + +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.TagManager; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.bEnv; +import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.init; +import static org.assertj.core.api.Assertions.assertThat; + +/** IT cases for {@link ReplaceTagAction}. */ +public class ReplaceTagActionTest extends ActionITCaseBase { + + @BeforeEach + public void setUp() { + init(warehouse); + } + + @Test + public void testReplaceTag() throws Exception { + bEnv.executeSql( + "CREATE TABLE T (id INT, name STRING," + + " PRIMARY KEY (id) NOT ENFORCED)" + + " WITH ('bucket'='1')"); + + FileStoreTable table = getFileStoreTable("T"); + TagManager tagManager = table.tagManager(); + + bEnv.executeSql("INSERT INTO T VALUES (1, 'a')").await(); + bEnv.executeSql("INSERT INTO T VALUES (2, 'b')").await(); + assertThat(table.snapshotManager().snapshotCount()).isEqualTo(2); + + Assertions.assertThatThrownBy( + () -> + bEnv.executeSql( + "CALL sys.replace_tag(`table` => 'default.T', tag => 'test_tag')")) + .hasMessageContaining("Tag name 'test_tag' does not exist."); + + bEnv.executeSql("CALL sys.create_tag(`table` => 'default.T', tag => 'test_tag')"); + assertThat(tagManager.tagExists("test_tag")).isEqualTo(true); + assertThat(tagManager.tag("test_tag").trimToSnapshot().id()).isEqualTo(2); + assertThat(tagManager.tag("test_tag").getTagTimeRetained()).isEqualTo(null); + + // replace tag with new time_retained + createAction( + ReplaceTagAction.class, + "replace_tag", + "--warehouse", + warehouse, + "--database", + database, + "--table", + "T", + "--tag_name", + "test_tag", + "--time_retained", + "1 d") + .run(); + assertThat(tagManager.tag("test_tag").getTagTimeRetained().toHours()).isEqualTo(24); + + // replace tag with new snapshot and time_retained + createAction( + ReplaceTagAction.class, + "replace_tag", + "--warehouse", + warehouse, + "--database", + database, + "--table", + "T", + "--tag_name", + "test_tag", + "--snapshot", + "1", + "--time_retained", + "2 d") + .run(); + assertThat(tagManager.tag("test_tag").trimToSnapshot().id()).isEqualTo(1); + assertThat(tagManager.tag("test_tag").getTagTimeRetained().toHours()).isEqualTo(48); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RollbackToActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RollbackToActionITCase.java index 859f8deda4d1..7dc9fec643cf 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RollbackToActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RollbackToActionITCase.java @@ -160,4 +160,59 @@ public void rollbackToTagTest(String invoker) throws Exception { "SELECT * FROM `" + tableName + "`", Arrays.asList(Row.of(1L, "Hi"), Row.of(2L, "Apache"))); } + + @ParameterizedTest + @ValueSource(strings = {"action", "procedure_named", "procedure_indexed"}) + public void rollbackToTimestampTest(String invoker) throws Exception { + FileStoreTable table = + createFileStoreTable( + ROW_TYPE, + Collections.emptyList(), + Collections.singletonList("k"), + Collections.emptyList(), + Collections.emptyMap()); + StreamWriteBuilder writeBuilder = table.newStreamWriteBuilder().withCommitUser(commitUser); + write = writeBuilder.newWrite(); + commit = writeBuilder.newCommit(); + + writeData(rowData(1L, BinaryString.fromString("Hi"))); + writeData(rowData(2L, BinaryString.fromString("Apache"))); + long timestamp = System.currentTimeMillis(); + writeData(rowData(2L, BinaryString.fromString("Paimon"))); + + switch (invoker) { + case "action": + createAction( + RollbackToTimestampAction.class, + "rollback_to_timestamp", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--timestamp", + timestamp + "") + .run(); + break; + case "procedure_indexed": + executeSQL( + String.format( + "CALL sys.rollback_to_timestamp('%s.%s', %s)", + database, tableName, timestamp)); + break; + case "procedure_named": + executeSQL( + String.format( + "CALL sys.rollback_to_timestamp(`table` => '%s.%s', `timestamp` => %s)", + database, tableName, timestamp)); + break; + default: + throw new UnsupportedOperationException(invoker); + } + + testBatchRead( + "SELECT * FROM `" + tableName + "`", + Arrays.asList(Row.of(1L, "Hi"), Row.of(2L, "Apache"))); + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/TagActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/TagActionITCase.java index aa98de38715b..7310a68df7a2 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/TagActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/TagActionITCase.java @@ -240,6 +240,101 @@ public void testCreateAndDeleteTag(String invoker) throws Exception { assertThat(tagManager.tagExists("tag3")).isFalse(); } + @ParameterizedTest(name = "{0}") + @ValueSource(strings = {"action", "procedure_indexed", "procedure_named"}) + public void testRenameTag(String invoker) throws Exception { + init(warehouse); + + RowType rowType = + RowType.of( + new DataType[] {DataTypes.BIGINT(), DataTypes.STRING()}, + new String[] {"k", "v"}); + FileStoreTable table = + createFileStoreTable( + rowType, + Collections.emptyList(), + Collections.singletonList("k"), + Collections.emptyList(), + Collections.emptyMap()); + + StreamWriteBuilder writeBuilder = table.newStreamWriteBuilder().withCommitUser(commitUser); + write = writeBuilder.newWrite(); + commit = writeBuilder.newCommit(); + + // 3 snapshots + writeData(rowData(1L, BinaryString.fromString("Hi"))); + writeData(rowData(2L, BinaryString.fromString("Hello"))); + writeData(rowData(3L, BinaryString.fromString("Paimon"))); + + TagManager tagManager = new TagManager(table.fileIO(), table.location()); + switch (invoker) { + case "action": + createAction( + CreateTagAction.class, + "create_tag", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--tag_name", + "tag2") + .run(); + break; + case "procedure_indexed": + executeSQL( + String.format( + "CALL sys.create_tag('%s.%s', 'tag2', 2)", database, tableName)); + break; + case "procedure_named": + executeSQL( + String.format( + "CALL sys.create_tag(`table` => '%s.%s', tag => 'tag2', snapshot_id => cast(2 as bigint))", + database, tableName)); + break; + default: + throw new UnsupportedOperationException(invoker); + } + assertThat(tagManager.tagExists("tag2")).isTrue(); + + switch (invoker) { + case "action": + createAction( + RenameTagAction.class, + "rename_tag", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--tag_name", + "tag2", + "--target_tag_name", + "tag3") + .run(); + break; + case "procedure_indexed": + executeSQL( + String.format( + "CALL sys.rename_tag('%s.%s', 'tag2', 'tag3')", + database, tableName)); + break; + case "procedure_named": + executeSQL( + String.format( + "CALL sys.rename_tag(`table` => '%s.%s', tagName => 'tag2', targetTagName => 'tag3')", + database, tableName)); + break; + default: + throw new UnsupportedOperationException(invoker); + } + + assertThat(tagManager.tagExists("tag2")).isFalse(); + assertThat(tagManager.tagExists("tag3")).isTrue(); + } + @ParameterizedTest(name = "{0}") @ValueSource(strings = {"action", "procedure_indexed", "procedure_named"}) public void testCreateLatestTag(String invoker) throws Exception { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/UnawareBucketCompactorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/UnawareBucketCompactorTest.java index 69229ddce2f6..1ec26afabc55 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/UnawareBucketCompactorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/UnawareBucketCompactorTest.java @@ -34,8 +34,10 @@ import org.apache.paimon.types.DataTypes; import org.apache.paimon.utils.ExecutorThreadFactory; +import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.SimpleCounter; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -48,6 +50,8 @@ import java.util.concurrent.Executors; import static org.apache.paimon.operation.metrics.CompactionMetrics.AVG_COMPACTION_TIME; +import static org.apache.paimon.operation.metrics.CompactionMetrics.COMPACTION_COMPLETED_COUNT; +import static org.apache.paimon.operation.metrics.CompactionMetrics.COMPACTION_QUEUED_COUNT; import static org.apache.paimon.operation.metrics.CompactionMetrics.COMPACTION_THREAD_BUSY; /** Test for {@link UnawareBucketCompactor}. */ @@ -65,7 +69,8 @@ public void testGaugeCollection() throws Exception { Executors.newSingleThreadScheduledExecutor( new ExecutorThreadFactory( Thread.currentThread().getName() + "-append-only-compact-worker")); - Map map = new HashMap<>(); + Map gaugeMap = new HashMap<>(); + Map counterMap = new HashMap<>(); UnawareBucketCompactor unawareBucketCompactor = new UnawareBucketCompactor( (FileStoreTable) catalog.getTable(identifier()), @@ -74,7 +79,7 @@ public void testGaugeCollection() throws Exception { new FileStoreSourceReaderTest.DummyMetricGroup() { @Override public > G gauge(String name, G gauge) { - map.put(name, gauge); + gaugeMap.put(name, gauge); return null; } @@ -87,6 +92,13 @@ public MetricGroup addGroup(String name) { public MetricGroup addGroup(String key, String value) { return this; } + + @Override + public Counter counter(String name) { + SimpleCounter counter = new SimpleCounter(); + counterMap.put(name, counter); + return counter; + } }); for (int i = 0; i < 320; i++) { @@ -94,11 +106,15 @@ public MetricGroup addGroup(String key, String value) { Thread.sleep(250); } - double compactionThreadBusy = (double) map.get(COMPACTION_THREAD_BUSY).getValue(); - double compactionAvrgTime = (double) map.get(AVG_COMPACTION_TIME).getValue(); + double compactionThreadBusy = (double) gaugeMap.get(COMPACTION_THREAD_BUSY).getValue(); + double compactionAvgTime = (double) gaugeMap.get(AVG_COMPACTION_TIME).getValue(); + long compactionsCompletedCount = counterMap.get(COMPACTION_COMPLETED_COUNT).getCount(); + long compactionsQueuedCount = counterMap.get(COMPACTION_QUEUED_COUNT).getCount(); Assertions.assertThat(compactionThreadBusy).isGreaterThan(45).isLessThan(55); - Assertions.assertThat(compactionAvrgTime).isGreaterThan(120).isLessThan(140); + Assertions.assertThat(compactionAvgTime).isGreaterThan(120).isLessThan(140); + Assertions.assertThat(compactionsCompletedCount).isEqualTo(320L); + Assertions.assertThat(compactionsQueuedCount).isEqualTo(0L); } protected Catalog getCatalog() { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializerTest.java new file mode 100644 index 000000000000..906fac850973 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTaskSerializerTest.java @@ -0,0 +1,94 @@ +/* + * 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.compact.changelog; + +import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.BinaryRowWriter; +import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.manifest.FileSource; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.UUID; + +import static org.apache.paimon.mergetree.compact.MergeTreeCompactManagerTest.row; +import static org.apache.paimon.stats.StatsTestUtils.newSimpleStats; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link ChangelogCompactTaskSerializer}. */ +public class ChangelogCompactTaskSerializerTest { + private final ChangelogCompactTaskSerializer serializer = new ChangelogCompactTaskSerializer(); + + @Test + public void testSerializer() throws Exception { + BinaryRow partition = new BinaryRow(1); + BinaryRowWriter writer = new BinaryRowWriter(partition); + writer.writeInt(0, 0); + writer.complete(); + + ChangelogCompactTask task = + new ChangelogCompactTask( + 1L, + partition, + new HashMap>() { + { + put(0, newFiles(20)); + put(1, newFiles(20)); + } + }, + new HashMap>() { + { + put(0, newFiles(10)); + put(1, newFiles(10)); + } + }); + ChangelogCompactTask serializeTask = serializer.deserialize(1, serializer.serialize(task)); + assertThat(task).isEqualTo(serializeTask); + } + + private List newFiles(int num) { + List list = new ArrayList<>(); + for (int i = 0; i < num; i++) { + list.add(newFile()); + } + return list; + } + + private DataFileMeta newFile() { + return new DataFileMeta( + UUID.randomUUID().toString(), + 0, + 1, + row(0), + row(0), + newSimpleStats(0, 1), + newSimpleStats(0, 1), + 0, + 1, + 0, + 0, + 0L, + null, + FileSource.APPEND, + null); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/iceberg/FlinkIcebergITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/iceberg/FlinkIcebergITCaseBase.java new file mode 100644 index 000000000000..413a404c41ed --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/iceberg/FlinkIcebergITCaseBase.java @@ -0,0 +1,416 @@ +/* + * 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.iceberg; + +import org.apache.paimon.flink.util.AbstractTestBase; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.io.CloseableIterable; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.ArrayList; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT cases for Paimon Iceberg compatibility. */ +public abstract class FlinkIcebergITCaseBase extends AbstractTestBase { + + @ParameterizedTest + @ValueSource(strings = {"orc", "parquet", "avro"}) + public void testPrimaryKeyTable(String format) throws Exception { + String warehouse = getTempDirPath(); + TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().parallelism(2).build(); + tEnv.executeSql( + "CREATE CATALOG paimon WITH (\n" + + " 'type' = 'paimon',\n" + + " 'warehouse' = '" + + warehouse + + "'\n" + + ")"); + tEnv.executeSql( + "CREATE TABLE paimon.`default`.T (\n" + + " pt INT,\n" + + " k INT,\n" + + " v1 INT,\n" + + " v2 STRING,\n" + + " PRIMARY KEY (pt, k) NOT ENFORCED\n" + + ") PARTITIONED BY (pt) WITH (\n" + + " 'metadata.iceberg.storage' = 'hadoop-catalog',\n" + // make sure all changes are visible in iceberg metadata + + " 'full-compaction.delta-commits' = '1',\n" + + " 'file.format' = '" + + format + + "'\n" + + ")"); + tEnv.executeSql( + "INSERT INTO paimon.`default`.T VALUES " + + "(1, 10, 100, 'apple'), " + + "(1, 11, 110, 'banana'), " + + "(2, 20, 200, 'cat'), " + + "(2, 21, 210, 'dog')") + .await(); + + tEnv.executeSql( + "CREATE CATALOG iceberg WITH (\n" + + " 'type' = 'iceberg',\n" + + " 'catalog-type' = 'hadoop',\n" + + " 'warehouse' = '" + + warehouse + + "/iceberg',\n" + + " 'cache-enabled' = 'false'\n" + + ")"); + assertThat( + collect( + tEnv.executeSql( + "SELECT v1, k, v2, pt FROM iceberg.`default`.T ORDER BY pt, k"))) + .containsExactly( + Row.of(100, 10, "apple", 1), + Row.of(110, 11, "banana", 1), + Row.of(200, 20, "cat", 2), + Row.of(210, 21, "dog", 2)); + + tEnv.executeSql( + "INSERT INTO paimon.`default`.T VALUES " + + "(1, 10, 101, 'red'), " + + "(1, 12, 121, 'green'), " + + "(2, 20, 201, 'blue'), " + + "(2, 22, 221, 'yellow')") + .await(); + assertThat( + collect( + tEnv.executeSql( + "SELECT v1, k, v2, pt FROM iceberg.`default`.T ORDER BY pt, k"))) + .containsExactly( + Row.of(101, 10, "red", 1), + Row.of(110, 11, "banana", 1), + Row.of(121, 12, "green", 1), + Row.of(201, 20, "blue", 2), + Row.of(210, 21, "dog", 2), + Row.of(221, 22, "yellow", 2)); + } + + @ParameterizedTest + @ValueSource(strings = {"orc", "parquet", "avro"}) + public void testAppendOnlyTable(String format) throws Exception { + String warehouse = getTempDirPath(); + TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().parallelism(2).build(); + tEnv.executeSql( + "CREATE CATALOG paimon WITH (\n" + + " 'type' = 'paimon',\n" + + " 'warehouse' = '" + + warehouse + + "'\n" + + ")"); + tEnv.executeSql( + "CREATE TABLE paimon.`default`.cities (\n" + + " country STRING,\n" + + " name STRING\n" + + ") WITH (\n" + + " 'metadata.iceberg.storage' = 'hadoop-catalog',\n" + + " 'file.format' = '" + + format + + "'\n" + + ")"); + tEnv.executeSql( + "INSERT INTO paimon.`default`.cities VALUES " + + "('usa', 'new york'), " + + "('germany', 'berlin'), " + + "('usa', 'chicago'), " + + "('germany', 'hamburg')") + .await(); + + tEnv.executeSql( + "CREATE CATALOG iceberg WITH (\n" + + " 'type' = 'iceberg',\n" + + " 'catalog-type' = 'hadoop',\n" + + " 'warehouse' = '" + + warehouse + + "/iceberg',\n" + + " 'cache-enabled' = 'false'\n" + + ")"); + assertThat(collect(tEnv.executeSql("SELECT name, country FROM iceberg.`default`.cities"))) + .containsExactlyInAnyOrder( + Row.of("new york", "usa"), + Row.of("chicago", "usa"), + Row.of("berlin", "germany"), + Row.of("hamburg", "germany")); + + tEnv.executeSql( + "INSERT INTO paimon.`default`.cities VALUES " + + "('usa', 'houston'), " + + "('germany', 'munich')") + .await(); + assertThat( + collect( + tEnv.executeSql( + "SELECT name FROM iceberg.`default`.cities WHERE country = 'germany'"))) + .containsExactlyInAnyOrder(Row.of("berlin"), Row.of("hamburg"), Row.of("munich")); + } + + @ParameterizedTest + @ValueSource(strings = {"orc", "parquet", "avro"}) + public void testFilterAllTypes(String format) throws Exception { + String warehouse = getTempDirPath(); + TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().parallelism(2).build(); + tEnv.executeSql( + "CREATE CATALOG paimon WITH (\n" + + " 'type' = 'paimon',\n" + + " 'warehouse' = '" + + warehouse + + "'\n" + + ")"); + tEnv.executeSql( + "CREATE TABLE paimon.`default`.T (\n" + + " pt INT,\n" + + " id INT," + + " v_int INT,\n" + + " v_boolean BOOLEAN,\n" + + " v_bigint BIGINT,\n" + + " v_float FLOAT,\n" + + " v_double DOUBLE,\n" + + " v_decimal DECIMAL(8, 3),\n" + + " v_varchar STRING,\n" + + " v_varbinary VARBINARY(20),\n" + + " v_date DATE,\n" + // it seems that Iceberg Flink connector has some bug when filtering a + // timestamp_ltz, so we don't test it here + + " v_timestamp TIMESTAMP(6)\n" + + ") PARTITIONED BY (pt) WITH (\n" + + " 'metadata.iceberg.storage' = 'hadoop-catalog',\n" + + " 'file.format' = '" + + format + + "'\n" + + ")"); + tEnv.executeSql( + "INSERT INTO paimon.`default`.T VALUES " + + "(1, 1, 1, true, 10, CAST(100.0 AS FLOAT), 1000.0, 123.456, 'cat', CAST('B_cat' AS VARBINARY(20)), DATE '2024-10-10', TIMESTAMP '2024-10-10 11:22:33.123456'), " + + "(2, 2, 2, false, 20, CAST(200.0 AS FLOAT), 2000.0, 234.567, 'dog', CAST('B_dog' AS VARBINARY(20)), DATE '2024-10-20', TIMESTAMP '2024-10-20 11:22:33.123456'), " + + "(3, 3, CAST(NULL AS INT), CAST(NULL AS BOOLEAN), CAST(NULL AS BIGINT), CAST(NULL AS FLOAT), CAST(NULL AS DOUBLE), CAST(NULL AS DECIMAL(8, 3)), CAST(NULL AS STRING), CAST(NULL AS VARBINARY(20)), CAST(NULL AS DATE), CAST(NULL AS TIMESTAMP(6)))") + .await(); + + tEnv.executeSql( + "CREATE CATALOG iceberg WITH (\n" + + " 'type' = 'iceberg',\n" + + " 'catalog-type' = 'hadoop',\n" + + " 'warehouse' = '" + + warehouse + + "/iceberg',\n" + + " 'cache-enabled' = 'false'\n" + + ")"); + tEnv.executeSql("USE CATALOG iceberg"); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where pt = 1"))) + .containsExactly(Row.of(1)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_int = 1"))) + .containsExactly(Row.of(1)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_boolean = true"))) + .containsExactly(Row.of(1)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_bigint = 10"))) + .containsExactly(Row.of(1)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_float = 100.0"))) + .containsExactly(Row.of(1)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_double = 1000.0"))) + .containsExactly(Row.of(1)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_decimal = 123.456"))) + .containsExactly(Row.of(1)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_varchar = 'cat'"))) + .containsExactly(Row.of(1)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_date = '2024-10-10'"))) + .containsExactly(Row.of(1)); + assertThat( + collect( + tEnv.executeSql( + "SELECT id FROM T where v_timestamp = TIMESTAMP '2024-10-10 11:22:33.123456'"))) + .containsExactly(Row.of(1)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_int IS NULL"))) + .containsExactly(Row.of(3)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_boolean IS NULL"))) + .containsExactly(Row.of(3)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_bigint IS NULL"))) + .containsExactly(Row.of(3)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_float IS NULL"))) + .containsExactly(Row.of(3)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_double IS NULL"))) + .containsExactly(Row.of(3)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_decimal IS NULL"))) + .containsExactly(Row.of(3)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_varchar IS NULL"))) + .containsExactly(Row.of(3)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_varbinary IS NULL"))) + .containsExactly(Row.of(3)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_date IS NULL"))) + .containsExactly(Row.of(3)); + assertThat(collect(tEnv.executeSql("SELECT id FROM T where v_timestamp IS NULL"))) + .containsExactly(Row.of(3)); + } + + @ParameterizedTest + // orc writer does not write timestamp_ltz correctly, however we won't fix it due to + // compatibility concern, so we don't test orc here + @ValueSource(strings = {"parquet"}) + public void testFilterTimestampLtz(String format) throws Exception { + String warehouse = getTempDirPath(); + TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().parallelism(2).build(); + tEnv.executeSql( + "CREATE CATALOG paimon WITH (\n" + + " 'type' = 'paimon',\n" + + " 'warehouse' = '" + + warehouse + + "'\n" + + ")"); + tEnv.executeSql( + "CREATE TABLE paimon.`default`.T (\n" + + " id INT," + + " v_timestampltz TIMESTAMP_LTZ(6)\n" + + ") WITH (\n" + + " 'metadata.iceberg.storage' = 'hadoop-catalog',\n" + + " 'file.format' = '" + + format + + "'\n" + + ")"); + tEnv.executeSql( + "INSERT INTO paimon.`default`.T VALUES " + + "(1, CAST(TO_TIMESTAMP_LTZ(1100000000321, 3) AS TIMESTAMP_LTZ(6))), " + + "(2, CAST(TO_TIMESTAMP_LTZ(1200000000321, 3) AS TIMESTAMP_LTZ(6))), " + + "(3, CAST(NULL AS TIMESTAMP_LTZ(6)))") + .await(); + + HadoopCatalog icebergCatalog = + new HadoopCatalog(new Configuration(), warehouse + "/iceberg"); + TableIdentifier icebergIdentifier = TableIdentifier.of("default", "T"); + org.apache.iceberg.Table icebergTable = icebergCatalog.loadTable(icebergIdentifier); + + CloseableIterable result = + IcebergGenerics.read(icebergTable) + .where(Expressions.equal("v_timestampltz", 1100000000321000L)) + .build(); + List actual = new ArrayList<>(); + for (Record record : result) { + actual.add(record.get(0)); + } + result.close(); + assertThat(actual).containsExactly(1); + + result = + IcebergGenerics.read(icebergTable) + .where(Expressions.isNull("v_timestampltz")) + .build(); + actual = new ArrayList<>(); + for (Record record : result) { + actual.add(record.get(0)); + } + result.close(); + assertThat(actual).containsExactly(3); + } + + @ParameterizedTest + @ValueSource(strings = {"orc", "parquet", "avro"}) + public void testDropAndRecreateTable(String format) throws Exception { + String warehouse = getTempDirPath(); + TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().parallelism(2).build(); + tEnv.executeSql( + "CREATE CATALOG paimon WITH (\n" + + " 'type' = 'paimon',\n" + + " 'warehouse' = '" + + warehouse + + "'\n" + + ")"); + String createTableDdl = + "CREATE TABLE paimon.`default`.cities (\n" + + " country STRING,\n" + + " name STRING\n" + + ") WITH (\n" + + " 'metadata.iceberg.storage' = 'hadoop-catalog',\n" + + " 'file.format' = '" + + format + + "'\n" + + ")"; + tEnv.executeSql(createTableDdl); + tEnv.executeSql( + "INSERT INTO paimon.`default`.cities VALUES " + + "('usa', 'new york'), " + + "('germany', 'berlin')") + .await(); + + tEnv.executeSql( + "CREATE CATALOG iceberg WITH (\n" + + " 'type' = 'iceberg',\n" + + " 'catalog-type' = 'hadoop',\n" + + " 'warehouse' = '" + + warehouse + + "/iceberg',\n" + + " 'cache-enabled' = 'false'\n" + + ")"); + assertThat(collect(tEnv.executeSql("SELECT name, country FROM iceberg.`default`.cities"))) + .containsExactlyInAnyOrder(Row.of("new york", "usa"), Row.of("berlin", "germany")); + + tEnv.executeSql( + "INSERT INTO paimon.`default`.cities VALUES " + + "('usa', 'chicago'), " + + "('germany', 'hamburg')") + .await(); + assertThat(collect(tEnv.executeSql("SELECT name, country FROM iceberg.`default`.cities"))) + .containsExactlyInAnyOrder( + Row.of("new york", "usa"), + Row.of("chicago", "usa"), + Row.of("berlin", "germany"), + Row.of("hamburg", "germany")); + + tEnv.executeSql("DROP TABLE paimon.`default`.cities"); + tEnv.executeSql(createTableDdl); + tEnv.executeSql( + "INSERT INTO paimon.`default`.cities VALUES " + + "('usa', 'houston'), " + + "('germany', 'munich')") + .await(); + assertThat(collect(tEnv.executeSql("SELECT name, country FROM iceberg.`default`.cities"))) + .containsExactlyInAnyOrder(Row.of("houston", "usa"), Row.of("munich", "germany")); + + tEnv.executeSql( + "INSERT INTO paimon.`default`.cities VALUES " + + "('usa', 'san francisco'), " + + "('germany', 'cologne')") + .await(); + assertThat( + collect( + tEnv.executeSql( + "SELECT name FROM iceberg.`default`.cities WHERE country = 'germany'"))) + .containsExactlyInAnyOrder(Row.of("munich"), Row.of("cologne")); + } + + private List collect(TableResult result) throws Exception { + List rows = new ArrayList<>(); + try (CloseableIterator it = result.collect()) { + while (it.hasNext()) { + rows.add(it.next()); + } + } + return rows; + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/FileStoreLookupFunctionTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/FileStoreLookupFunctionTest.java index f8c8adcb2bf3..2b219f6a712a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/FileStoreLookupFunctionTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/FileStoreLookupFunctionTest.java @@ -34,6 +34,7 @@ import org.apache.paimon.service.ServiceManager; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.Table; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.StreamTableWrite; import org.apache.paimon.table.sink.TableCommitImpl; @@ -52,6 +53,9 @@ import java.net.InetSocketAddress; import java.nio.file.Path; import java.time.Duration; +import java.time.Instant; +import java.time.ZoneId; +import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -59,8 +63,11 @@ import java.util.Random; import java.util.UUID; +import static org.apache.paimon.flink.FlinkConnectorOptions.LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST; import static org.apache.paimon.service.ServiceManager.PRIMARY_KEY_LOOKUP; +import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; /** Tests for {@link FileStoreLookupFunction}. */ public class FileStoreLookupFunctionTest { @@ -91,6 +98,18 @@ private void createLookupFunction( boolean dynamicPartition, boolean refreshAsync) throws Exception { + table = createFileStoreTable(isPartition, dynamicPartition, refreshAsync); + lookupFunction = createLookupFunction(table, joinEqualPk); + lookupFunction.open(tempDir.toString()); + } + + private FileStoreLookupFunction createLookupFunction(Table table, boolean joinEqualPk) { + return new FileStoreLookupFunction( + table, new int[] {0, 1}, joinEqualPk ? new int[] {0, 1} : new int[] {1}, null); + } + + private FileStoreTable createFileStoreTable( + boolean isPartition, boolean dynamicPartition, boolean refreshAsync) throws Exception { SchemaManager schemaManager = new SchemaManager(fileIO, tablePath); Options conf = new Options(); conf.set(FlinkConnectorOptions.LOOKUP_REFRESH_ASYNC, refreshAsync); @@ -106,7 +125,6 @@ private void createLookupFunction( RowType.of( new DataType[] {DataTypes.INT(), DataTypes.INT(), DataTypes.BIGINT()}, new String[] {"pt", "k", "v"}); - Schema schema = new Schema( rowType.getFields(), @@ -115,17 +133,8 @@ private void createLookupFunction( conf.toMap(), ""); TableSchema tableSchema = schemaManager.createTable(schema); - table = - FileStoreTableFactory.create( - fileIO, new org.apache.paimon.fs.Path(tempDir.toString()), tableSchema); - - lookupFunction = - new FileStoreLookupFunction( - table, - new int[] {0, 1}, - joinEqualPk ? new int[] {0, 1} : new int[] {1}, - null); - lookupFunction.open(tempDir.toString()); + return FileStoreTableFactory.create( + fileIO, new org.apache.paimon.fs.Path(tempDir.toString()), tableSchema); } @AfterEach @@ -214,6 +223,68 @@ public void testLookupDynamicPartition() throws Exception { .isEqualTo(0); } + @Test + public void testParseWrongTimePeriodsBlacklist() throws Exception { + Table table = createFileStoreTable(false, false, false); + + Table table1 = + table.copy( + Collections.singletonMap( + LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST.key(), + "2024-10-31 12:00,2024-10-31 16:00")); + assertThatThrownBy(() -> createLookupFunction(table1, true)) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Incorrect time periods format: [2024-10-31 12:00,2024-10-31 16:00].")); + + Table table2 = + table.copy( + Collections.singletonMap( + LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST.key(), + "20241031 12:00->20241031 16:00")); + assertThatThrownBy(() -> createLookupFunction(table2, true)) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Date time format error: [20241031 12:00]")); + + Table table3 = + table.copy( + Collections.singletonMap( + LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST.key(), + "2024-10-31 12:00->2024-10-31 16:00,2024-10-31 20:00->2024-10-31 18:00")); + assertThatThrownBy(() -> createLookupFunction(table3, true)) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "Incorrect time period: [2024-10-31 20:00->2024-10-31 18:00]")); + } + + @Test + public void testCheckRefreshInBlacklist() throws Exception { + Instant now = Instant.now(); + Instant start = Instant.ofEpochSecond(now.getEpochSecond() / 60 * 60); + Instant end = start.plusSeconds(30 * 60); + + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm"); + String left = start.atZone(ZoneId.systemDefault()).format(formatter); + String right = end.atZone(ZoneId.systemDefault()).format(formatter); + + Table table = + createFileStoreTable(false, false, false) + .copy( + Collections.singletonMap( + LOOKUP_REFRESH_TIME_PERIODS_BLACKLIST.key(), + left + "->" + right)); + + FileStoreLookupFunction lookupFunction = createLookupFunction(table, true); + + lookupFunction.tryRefresh(); + + assertThat(lookupFunction.nextBlacklistCheckTime()).isEqualTo(end.toEpochMilli() + 1); + } + private void commit(List messages) throws Exception { TableCommitImpl commit = table.newCommit(commitUser); commit.commit(messages); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java index 14643542e73d..46c61a15bd8a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/lookup/LookupTableTest.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.JoinedRow; @@ -558,6 +559,129 @@ public void testNoPrimaryKeyTableFilter() throws Exception { assertRow(result.get(1), 1, 11, 111); } + @Test + public void testPkTableWithCacheRowFilter() throws Exception { + FileStoreTable storeTable = createTable(singletonList("f0"), new Options()); + writeWithBucketAssigner( + storeTable, row -> 0, GenericRow.of(1, 11, 111), GenericRow.of(2, 22, 222)); + + FullCacheLookupTable.Context context = + new FullCacheLookupTable.Context( + storeTable, + new int[] {0, 1, 2}, + null, + null, + tempDir.toFile(), + singletonList("f0"), + null); + table = FullCacheLookupTable.create(context, ThreadLocalRandom.current().nextInt(2) * 10); + assertThat(table).isInstanceOf(PrimaryKeyLookupTable.class); + table.specifyCacheRowFilter(row -> row.getInt(0) < 2); + table.open(); + + List res = table.get(GenericRow.of(1)); + assertThat(res).hasSize(1); + assertRow(res.get(0), 1, 11, 111); + + res = table.get(GenericRow.of(2)); + assertThat(res).isEmpty(); + + writeWithBucketAssigner( + storeTable, row -> 0, GenericRow.of(0, 0, 0), GenericRow.of(3, 33, 333)); + res = table.get(GenericRow.of(0)); + assertThat(res).isEmpty(); + + table.refresh(); + res = table.get(GenericRow.of(0)); + assertThat(res).hasSize(1); + assertRow(res.get(0), 0, 0, 0); + + res = table.get(GenericRow.of(3)); + assertThat(res).isEmpty(); + } + + @Test + public void testNoPkTableWithCacheRowFilter() throws Exception { + FileStoreTable storeTable = createTable(emptyList(), new Options()); + writeWithBucketAssigner( + storeTable, row -> 0, GenericRow.of(1, 11, 111), GenericRow.of(2, 22, 222)); + + FullCacheLookupTable.Context context = + new FullCacheLookupTable.Context( + storeTable, + new int[] {0, 1, 2}, + null, + null, + tempDir.toFile(), + singletonList("f0"), + null); + table = FullCacheLookupTable.create(context, ThreadLocalRandom.current().nextInt(2) * 10); + assertThat(table).isInstanceOf(NoPrimaryKeyLookupTable.class); + table.specifyCacheRowFilter(row -> row.getInt(0) < 2); + table.open(); + + List res = table.get(GenericRow.of(1)); + assertThat(res).hasSize(1); + assertRow(res.get(0), 1, 11, 111); + + res = table.get(GenericRow.of(2)); + assertThat(res).isEmpty(); + + writeWithBucketAssigner( + storeTable, row -> 0, GenericRow.of(0, 0, 0), GenericRow.of(3, 33, 333)); + res = table.get(GenericRow.of(0)); + assertThat(res).isEmpty(); + + table.refresh(); + res = table.get(GenericRow.of(0)); + assertThat(res).hasSize(1); + assertRow(res.get(0), 0, 0, 0); + + res = table.get(GenericRow.of(3)); + assertThat(res).isEmpty(); + } + + @Test + public void testSecKeyTableWithCacheRowFilter() throws Exception { + FileStoreTable storeTable = createTable(singletonList("f0"), new Options()); + writeWithBucketAssigner( + storeTable, row -> 0, GenericRow.of(1, 11, 111), GenericRow.of(2, 22, 222)); + + FullCacheLookupTable.Context context = + new FullCacheLookupTable.Context( + storeTable, + new int[] {0, 1, 2}, + null, + null, + tempDir.toFile(), + singletonList("f1"), + null); + table = FullCacheLookupTable.create(context, ThreadLocalRandom.current().nextInt(2) * 10); + assertThat(table).isInstanceOf(SecondaryIndexLookupTable.class); + table.specifyCacheRowFilter(row -> row.getInt(1) < 22); + table.open(); + + List res = table.get(GenericRow.of(11)); + assertThat(res).hasSize(1); + assertRow(res.get(0), 1, 11, 111); + + res = table.get(GenericRow.of(22)); + assertThat(res).isEmpty(); + + writeWithBucketAssigner( + storeTable, row -> 0, GenericRow.of(0, 0, 0), GenericRow.of(3, 33, 333)); + res = table.get(GenericRow.of(0)); + assertThat(res).isEmpty(); + + table.refresh(); + res = table.get(GenericRow.of(0)); + assertThat(res).hasSize(1); + assertRow(res.get(0), 0, 0, 0); + + res = table.get(GenericRow.of(33)); + assertThat(res).isEmpty(); + } + @Test public void testPartialLookupTable() throws Exception { FileStoreTable dimTable = createDimTable(); @@ -591,6 +715,27 @@ public void testPartialLookupTable() throws Exception { assertThat(result).hasSize(0); } + @Test + public void testPartialLookupTableWithRowFilter() throws Exception { + Options options = new Options(); + options.set(CoreOptions.BUCKET.key(), "2"); + options.set(CoreOptions.BUCKET_KEY.key(), "f0"); + FileStoreTable dimTable = createTable(singletonList("f0"), options); + write(dimTable, GenericRow.of(1, 11, 111), GenericRow.of(2, 22, 222)); + + PrimaryKeyPartialLookupTable table = + PrimaryKeyPartialLookupTable.createLocalTable( + dimTable, new int[] {0, 2}, tempDir.toFile(), ImmutableList.of("f0"), null); + table.specifyCacheRowFilter(row -> row.getInt(0) < 2); + table.open(); + + List result = table.get(row(1, 11)); + assertThat(result).hasSize(1); + + result = table.get(row(2, 22)); + assertThat(result).isEmpty(); + } + @Test public void testPartialLookupTableWithProjection() throws Exception { FileStoreTable dimTable = createDimTable(); @@ -722,6 +867,108 @@ public void testPKLookupTableRefreshAsync(boolean refreshAsync) throws Exception table.close(); } + @Test + public void testFullCacheLookupTableWithForceLookup() throws Exception { + Options options = new Options(); + options.set(CoreOptions.MERGE_ENGINE, CoreOptions.MergeEngine.PARTIAL_UPDATE); + options.set( + FlinkConnectorOptions.LOOKUP_CACHE_MODE, + FlinkConnectorOptions.LookupCacheMode.FULL); + options.set(CoreOptions.WRITE_ONLY, true); + options.set(CoreOptions.FORCE_LOOKUP, true); + options.set(CoreOptions.BUCKET, 1); + FileStoreTable storeTable = createTable(singletonList("f0"), options); + FileStoreTable compactTable = + storeTable.copy(Collections.singletonMap(CoreOptions.WRITE_ONLY.key(), "false")); + FullCacheLookupTable.Context context = + new FullCacheLookupTable.Context( + storeTable, + new int[] {0, 1, 2}, + null, + null, + tempDir.toFile(), + singletonList("f0"), + null); + table = FullCacheLookupTable.create(context, ThreadLocalRandom.current().nextInt(2) * 10); + + // initialize + write(storeTable, ioManager, GenericRow.of(1, 11, 111)); + compact(compactTable, BinaryRow.EMPTY_ROW, 0, ioManager, true); + table.open(); + + List result = table.get(row(1)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 1, 11, 111); + + // first write + write(storeTable, GenericRow.of(1, null, 222)); + table.refresh(); + result = table.get(row(1)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 1, 11, 111); // old value because there is no compact + + // only L0 occur compact + compact(compactTable, BinaryRow.EMPTY_ROW, 0, ioManager, false); + table.refresh(); + result = table.get(row(1)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 1, 11, 222); // get new value after compact + + // second write + write(storeTable, GenericRow.of(1, 22, null)); + table.refresh(); + result = table.get(row(1)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 1, 11, 222); // old value + + // full compact + compact(compactTable, BinaryRow.EMPTY_ROW, 0, ioManager, true); + table.refresh(); + result = table.get(row(1)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 1, 22, 222); // new value + } + + @Test + public void testPartialLookupTableWithForceLookup() throws Exception { + Options options = new Options(); + options.set(CoreOptions.MERGE_ENGINE, CoreOptions.MergeEngine.PARTIAL_UPDATE); + options.set(CoreOptions.CHANGELOG_PRODUCER, CoreOptions.ChangelogProducer.NONE); + options.set(CoreOptions.FORCE_LOOKUP, true); + options.set(CoreOptions.BUCKET, 1); + FileStoreTable dimTable = createTable(singletonList("f0"), options); + + PrimaryKeyPartialLookupTable table = + PrimaryKeyPartialLookupTable.createLocalTable( + dimTable, + new int[] {0, 1, 2}, + tempDir.toFile(), + ImmutableList.of("f0"), + null); + table.open(); + + List result = table.get(row(1, -1)); + assertThat(result).hasSize(0); + + write(dimTable, ioManager, GenericRow.of(1, -1, 11), GenericRow.of(2, -2, 22)); + result = table.get(row(1)); + assertThat(result).hasSize(0); + + table.refresh(); + result = table.get(row(1)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 1, -1, 11); + result = table.get(row(2)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 2, -2, 22); + + write(dimTable, ioManager, GenericRow.of(1, null, 111)); + table.refresh(); + result = table.get(row(1)); + assertThat(result).hasSize(1); + assertRow(result.get(0), 1, -1, 111); + } + private FileStoreTable createDimTable() throws Exception { FileIO fileIO = LocalFileIO.create(); org.apache.paimon.fs.Path tablePath = diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactManifestProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactManifestProcedureITCase.java new file mode 100644 index 000000000000..89cdc48d85a0 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactManifestProcedureITCase.java @@ -0,0 +1,133 @@ +/* + * 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.procedure; + +import org.apache.paimon.flink.CatalogITCaseBase; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Objects; + +/** IT Case for {@link CompactManifestProcedure}. */ +public class CompactManifestProcedureITCase extends CatalogITCaseBase { + + @Test + public void testManifestCompactProcedure() { + sql( + "CREATE TABLE T (" + + " k INT," + + " v STRING," + + " hh INT," + + " dt STRING" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'file.format' = 'parquet'," + + " 'manifest.full-compaction-threshold-size' = '10000 T'," + + " 'bucket' = '-1'" + + ")"); + + sql( + "INSERT INTO T VALUES (1, '10', 15, '20221208'), (4, '100', 16, '20221208'), (5, '1000', 15, '20221209')"); + + sql( + "INSERT OVERWRITE T VALUES (1, '10', 15, '20221208'), (4, '100', 16, '20221208'), (5, '1000', 15, '20221209')"); + + sql( + "INSERT OVERWRITE T VALUES (1, '10', 15, '20221208'), (4, '100', 16, '20221208'), (5, '1000', 15, '20221209')"); + + sql( + "INSERT OVERWRITE T VALUES (1, '101', 15, '20221208'), (4, '1001', 16, '20221208'), (5, '10001', 15, '20221209')"); + + Assertions.assertThat( + sql("SELECT sum(num_deleted_files) FROM T$manifests").get(0).getField(0)) + .isEqualTo(9L); + + Assertions.assertThat( + Objects.requireNonNull( + sql("CALL sys.compact_manifest(`table` => 'default.T')") + .get(0) + .getField(0)) + .toString()) + .isEqualTo("success"); + + Assertions.assertThat( + sql("SELECT sum(num_deleted_files) FROM T$manifests").get(0).getField(0)) + .isEqualTo(0L); + + Assertions.assertThat(sql("SELECT * FROM T ORDER BY k").toString()) + .isEqualTo( + "[+I[1, 101, 15, 20221208], +I[4, 1001, 16, 20221208], +I[5, 10001, 15, 20221209]]"); + } + + @Test + public void testManifestCompactProcedureWithBranch() { + sql( + "CREATE TABLE T (" + + " k INT," + + " v STRING," + + " hh INT," + + " dt STRING" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'manifest.full-compaction-threshold-size' = '10000 T'," + + " 'bucket' = '-1'" + + ")"); + + sql( + "INSERT INTO `T` VALUES (1, '10', 15, '20221208'), (4, '100', 16, '20221208'), (5, '1000', 15, '20221209')"); + + sql("CALL sys.create_tag('default.T', 'tag1', 1)"); + + sql("call sys.create_branch('default.T', 'branch1', 'tag1')"); + + sql( + "INSERT OVERWRITE T$branch_branch1 VALUES (1, '10', 15, '20221208'), (4, '100', 16, '20221208'), (5, '1000', 15, '20221209')"); + + sql( + "INSERT OVERWRITE T$branch_branch1 VALUES (1, '10', 15, '20221208'), (4, '100', 16, '20221208'), (5, '1000', 15, '20221209')"); + + sql( + "INSERT OVERWRITE T$branch_branch1 VALUES (1, '101', 15, '20221208'), (4, '1001', 16, '20221208'), (5, '10001', 15, '20221209')"); + + Assertions.assertThat( + sql("SELECT sum(num_deleted_files) FROM T$branch_branch1$manifests") + .get(0) + .getField(0)) + .isEqualTo(9L); + + Assertions.assertThat( + Objects.requireNonNull( + sql("CALL sys.compact_manifest(`table` => 'default.T$branch_branch1')") + .get(0) + .getField(0)) + .toString()) + .isEqualTo("success"); + + Assertions.assertThat( + sql("SELECT sum(num_deleted_files) FROM T$branch_branch1$manifests") + .get(0) + .getField(0)) + .isEqualTo(0L); + + Assertions.assertThat(sql("SELECT * FROM T$branch_branch1 ORDER BY k").toString()) + .isEqualTo( + "[+I[1, 101, 15, 20221208], +I[4, 1001, 16, 20221208], +I[5, 10001, 15, 20221209]]"); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CreateTagsProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CreateTagsProcedureITCase.java index aef952b6c561..77e2b74c5d50 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CreateTagsProcedureITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CreateTagsProcedureITCase.java @@ -58,6 +58,30 @@ public void testCreateTags() { .containsExactlyInAnyOrder("+I[k, 2024-01-01]"); } + @Test + public void testRenameTag() { + sql( + "CREATE TABLE T (" + + " k STRING," + + " dt STRING," + + " PRIMARY KEY (k, dt) NOT ENFORCED" + + ") PARTITIONED BY (dt) WITH (" + + " 'bucket' = '1'" + + ")"); + sql("insert into T values('k', '2024-01-01')"); + sql("insert into T values('k2', '2024-01-02')"); + + sql("CALL sys.create_tag('default.T', 'tag1')"); + + assertThat(sql("select tag_name from `T$tags`").stream().map(Row::toString)) + .containsExactlyInAnyOrder("+I[tag1]"); + + sql("CALL sys.rename_tag('default.T', 'tag1', 'tag2')"); + + assertThat(sql("select tag_name from `T$tags`").stream().map(Row::toString)) + .containsExactlyInAnyOrder("+I[tag2]"); + } + @Test public void testThrowSnapshotNotExistException() { sql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedureITCase.java index 71a6dc466ee0..bc2e84902f35 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedureITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedureITCase.java @@ -347,6 +347,61 @@ public void testPartitionExpireWithTimePartition() throws Exception { .containsExactlyInAnyOrder("Never-expire:9999-09-09:99:99"); } + @Test + public void testSortAndLimitExpirePartition() throws Exception { + sql( + "CREATE TABLE T (" + + " k STRING," + + " dt STRING," + + " hm STRING," + + " PRIMARY KEY (k, dt, hm) NOT ENFORCED" + + ") PARTITIONED BY (dt, hm) WITH (" + + " 'bucket' = '1'" + + ")"); + FileStoreTable table = paimonTable("T"); + // Test there are no expired partitions. + assertThat( + callExpirePartitions( + "CALL sys.expire_partitions(" + + "`table` => 'default.T'" + + ", expiration_time => '1 d'" + + ", timestamp_formatter => 'yyyy-MM-dd')")) + .containsExactlyInAnyOrder("No expired partitions."); + + sql("INSERT INTO T VALUES ('3', '2024-06-02', '02:00')"); + sql("INSERT INTO T VALUES ('2', '2024-06-02', '01:00')"); + sql("INSERT INTO T VALUES ('4', '2024-06-03', '01:00')"); + sql("INSERT INTO T VALUES ('1', '2024-06-01', '01:00')"); + // This partition never expires. + sql("INSERT INTO T VALUES ('Never-expire', '9999-09-09', '99:99')"); + + Function consumerReadResult = + (InternalRow row) -> + row.getString(0) + ":" + row.getString(1) + ":" + row.getString(2); + assertThat(read(table, consumerReadResult)) + .containsExactlyInAnyOrder( + "1:2024-06-01:01:00", + "2:2024-06-02:01:00", + "3:2024-06-02:02:00", + "4:2024-06-03:01:00", + "Never-expire:9999-09-09:99:99"); + + // Show a list of expired partitions. + assertThat( + callExpirePartitions( + "CALL sys.expire_partitions(" + + "`table` => 'default.T'" + + ", expiration_time => '1 d'" + + ", timestamp_formatter => 'yyyy-MM-dd', max_expires => 3)")) + .containsExactly( + "dt=2024-06-01, hm=01:00", + "dt=2024-06-02, hm=01:00", + "dt=2024-06-02, hm=02:00"); + + assertThat(read(table, consumerReadResult)) + .containsExactlyInAnyOrder("4:2024-06-03:01:00", "Never-expire:9999-09-09:99:99"); + } + /** Return a list of expired partitions. */ public List callExpirePartitions(String callSql) { return sql(callSql).stream() diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpireTagsProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpireTagsProcedureITCase.java new file mode 100644 index 000000000000..4a89531b22a0 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ExpireTagsProcedureITCase.java @@ -0,0 +1,141 @@ +/* + * 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.procedure; + +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.flink.CatalogITCaseBase; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.SnapshotManager; + +import org.apache.flink.types.Row; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT Case for {@link ExpireTagsProcedure}. */ +public class ExpireTagsProcedureITCase extends CatalogITCaseBase { + + @Test + public void testExpireTagsByTagCreateTimeAndTagTimeRetained() throws Exception { + sql( + "CREATE TABLE T (id STRING, name STRING," + + " PRIMARY KEY (id) NOT ENFORCED)" + + " WITH ('bucket'='1', 'write-only'='true')"); + + FileStoreTable table = paimonTable("T"); + SnapshotManager snapshotManager = table.snapshotManager(); + + // generate 5 snapshots + for (int i = 1; i <= 5; i++) { + sql("INSERT INTO T VALUES ('" + i + "', '" + i + "')"); + } + checkSnapshots(snapshotManager, 1, 5); + + sql("CALL sys.create_tag(`table` => 'default.T', tag => 'tag-1', snapshot_id => 1)"); + sql( + "CALL sys.create_tag(`table` => 'default.T', tag => 'tag-2', snapshot_id => 2, time_retained => '1h')"); + + // no tags expired + assertThat(sql("CALL sys.expire_tags(`table` => 'default.T')")) + .containsExactly(Row.of("No expired tags.")); + + sql( + "CALL sys.create_tag(`table` => 'default.T', tag => 'tag-3', snapshot_id => 3, time_retained => '1s')"); + sql( + "CALL sys.create_tag(`table` => 'default.T', tag => 'tag-4', snapshot_id => 4, time_retained => '1s')"); + + Thread.sleep(2000); + // tag-3,tag-4 expired + assertThat(sql("CALL sys.expire_tags(`table` => 'default.T')")) + .containsExactlyInAnyOrder(Row.of("tag-3"), Row.of("tag-4")); + } + + @Test + public void testExpireTagsByOlderThanTime() throws Exception { + sql( + "CREATE TABLE T (id STRING, name STRING," + + " PRIMARY KEY (id) NOT ENFORCED)" + + " WITH ('bucket'='1', 'write-only'='true')"); + + FileStoreTable table = paimonTable("T"); + SnapshotManager snapshotManager = table.snapshotManager(); + + // generate 5 snapshots + for (int i = 1; i <= 5; i++) { + sql("INSERT INTO T VALUES ('" + i + "', '" + i + "')"); + } + checkSnapshots(snapshotManager, 1, 5); + + sql("CALL sys.create_tag(`table` => 'default.T', tag => 'tag-1', snapshot_id => 1)"); + sql( + "CALL sys.create_tag(`table` => 'default.T', tag => 'tag-2', snapshot_id => 2, time_retained => '1d')"); + sql( + "CALL sys.create_tag(`table` => 'default.T', tag => 'tag-3', snapshot_id => 3, time_retained => '1d')"); + sql( + "CALL sys.create_tag(`table` => 'default.T', tag => 'tag-4', snapshot_id => 4, time_retained => '1d')"); + List sql = sql("select count(tag_name) from `T$tags`"); + assertThat(sql("select count(tag_name) from `T$tags`")).containsExactly(Row.of(4L)); + + // no tags expired + assertThat(sql("CALL sys.expire_tags(`table` => 'default.T')")) + .containsExactlyInAnyOrder(Row.of("No expired tags.")); + + // tag-2 as the base older_than time. + // tag-1 expired by its file creation time. + LocalDateTime olderThanTime1 = table.tagManager().tag("tag-2").getTagCreateTime(); + java.sql.Timestamp timestamp1 = + new java.sql.Timestamp( + Timestamp.fromLocalDateTime(olderThanTime1).getMillisecond()); + assertThat( + sql( + "CALL sys.expire_tags(`table` => 'default.T', older_than => '" + + timestamp1.toString() + + "')")) + .containsExactlyInAnyOrder(Row.of("tag-1")); + + sql( + "CALL sys.create_tag(`table` => 'default.T', tag => 'tag-5', snapshot_id => 5, time_retained => '1s')"); + Thread.sleep(1000); + + // tag-4 as the base older_than time. + // tag-2,tag-3,tag-5 expired, tag-5 reached its tagTimeRetained. + LocalDateTime olderThanTime2 = table.tagManager().tag("tag-4").getTagCreateTime(); + java.sql.Timestamp timestamp2 = + new java.sql.Timestamp( + Timestamp.fromLocalDateTime(olderThanTime2).getMillisecond()); + assertThat( + sql( + "CALL sys.expire_tags(`table` => 'default.T', older_than => '" + + timestamp2.toString() + + "')")) + .containsExactlyInAnyOrder(Row.of("tag-2"), Row.of("tag-3"), Row.of("tag-5")); + + assertThat(sql("select tag_name from `T$tags`")).containsExactly(Row.of("tag-4")); + } + + private void checkSnapshots(SnapshotManager sm, int earliest, int latest) throws IOException { + assertThat(sm.snapshotCount()).isEqualTo(latest - earliest + 1); + assertThat(sm.earliestSnapshotId()).isEqualTo(earliest); + assertThat(sm.latestSnapshotId()).isEqualTo(latest); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ProcedureTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ProcedureTest.java index c24d4105a557..74e3aeeac53b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ProcedureTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ProcedureTest.java @@ -98,7 +98,8 @@ public void testProcedureHasNamedArgument() { } private Method getMethodFromName(Class clazz, String methodName) { - Method[] methods = clazz.getDeclaredMethods(); + // get all methods of current and parent class + Method[] methods = clazz.getMethods(); for (Method method : methods) { if (method.getName().equals(methodName)) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ReplaceTagProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ReplaceTagProcedureITCase.java new file mode 100644 index 000000000000..8a4eb791a6ad --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/ReplaceTagProcedureITCase.java @@ -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. + */ + +package org.apache.paimon.flink.procedure; + +import org.apache.paimon.flink.CatalogITCaseBase; + +import org.apache.flink.types.Row; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT Case for {@link ReplaceTagProcedure}. */ +public class ReplaceTagProcedureITCase extends CatalogITCaseBase { + + @Test + public void testExpireTagsByTagCreateTimeAndTagTimeRetained() throws Exception { + sql( + "CREATE TABLE T (id INT, name STRING," + + " PRIMARY KEY (id) NOT ENFORCED)" + + " WITH ('bucket'='1')"); + + sql("INSERT INTO T VALUES (1, 'a')"); + sql("INSERT INTO T VALUES (2, 'b')"); + assertThat(paimonTable("T").snapshotManager().snapshotCount()).isEqualTo(2); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL sys.replace_tag(`table` => 'default.T', tag => 'test_tag')")) + .hasMessageContaining("Tag name 'test_tag' does not exist."); + + sql("CALL sys.create_tag(`table` => 'default.T', tag => 'test_tag')"); + assertThat(sql("select tag_name,snapshot_id,time_retained from `T$tags`")) + .containsExactly(Row.of("test_tag", 2L, null)); + + // replace tag with new time_retained + sql( + "CALL sys.replace_tag(`table` => 'default.T', tag => 'test_tag'," + + " time_retained => '1 d')"); + assertThat(sql("select tag_name,snapshot_id,time_retained from `T$tags`")) + .containsExactly(Row.of("test_tag", 2L, "PT24H")); + + // replace tag with new snapshot and time_retained + sql( + "CALL sys.replace_tag(`table` => 'default.T', tag => 'test_tag'," + + " snapshot => 1, time_retained => '2 d')"); + assertThat(sql("select tag_name,snapshot_id,time_retained from `T$tags`")) + .containsExactly(Row.of("test_tag", 2L, "PT48H")); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java index b2764fc37c6e..668d651236fd 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.GenericRow; +import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.utils.TestingMetricUtils; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.io.CompactIncrement; @@ -37,6 +38,7 @@ import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.ThrowingConsumer; +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; import org.apache.flink.api.common.ExecutionConfig; @@ -281,6 +283,34 @@ public void testRestoreCommitUser() throws Exception { Assertions.assertThat(actual).hasSameElementsAs(Lists.newArrayList(commitUser)); } + @Test + public void testRestoreEmptyMarkDoneState() throws Exception { + FileStoreTable table = createFileStoreTable(o -> {}, Collections.singletonList("b")); + + String commitUser = UUID.randomUUID().toString(); + + // 1. Generate operatorSubtaskState + OperatorSubtaskState snapshot; + { + OneInputStreamOperatorTestHarness testHarness = + createLossyTestHarness(table, commitUser); + + testHarness.open(); + snapshot = writeAndSnapshot(table, commitUser, 1, 1, testHarness); + testHarness.close(); + } + // 2. enable mark done. + table = + table.copy( + ImmutableMap.of( + FlinkConnectorOptions.PARTITION_IDLE_TIME_TO_DONE.key(), "1h")); + + // 3. restore from state. + OneInputStreamOperatorTestHarness testHarness = + createLossyTestHarness(table); + testHarness.initializeState(snapshot); + } + @Test public void testCommitInputEnd() throws Exception { FileStoreTable table = createFileStoreTable(); @@ -565,7 +595,7 @@ public void testCalcDataBytesSend() throws Exception { table, commit, Committer.createContext("", metricGroup, true, false, null)); committer.commit(Collections.singletonList(manifestCommittable)); CommitterMetrics metrics = committer.getCommitterMetrics(); - assertThat(metrics.getNumBytesOutCounter().getCount()).isEqualTo(529); + assertThat(metrics.getNumBytesOutCounter().getCount()).isEqualTo(533); assertThat(metrics.getNumRecordsOutCounter().getCount()).isEqualTo(2); committer.close(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTestBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTestBase.java index 77b53ba7069d..a69f8dbd3a35 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTestBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTestBase.java @@ -88,10 +88,15 @@ protected void assertResults(FileStoreTable table, String... expected) { } protected FileStoreTable createFileStoreTable() throws Exception { - return createFileStoreTable(options -> {}); + return createFileStoreTable(options -> {}, Collections.emptyList()); } protected FileStoreTable createFileStoreTable(Consumer setOptions) throws Exception { + return createFileStoreTable(setOptions, Collections.emptyList()); + } + + protected FileStoreTable createFileStoreTable( + Consumer setOptions, List partitionKeys) throws Exception { Options conf = new Options(); conf.set(CoreOptions.PATH, tablePath.toString()); conf.setString("bucket", "1"); @@ -101,7 +106,7 @@ protected FileStoreTable createFileStoreTable(Consumer setOptions) thro schemaManager.createTable( new Schema( ROW_TYPE.getFields(), - Collections.emptyList(), + partitionKeys, Collections.emptyList(), conf.toMap(), "")); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java index 7cacb6c2931f..9f6e8159a1be 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactionTaskSimpleSerializerTest.java @@ -79,6 +79,7 @@ private DataFileMeta newFile() { 0, 0L, null, - FileSource.APPEND); + FileSource.APPEND, + null); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java index 84246d00aecc..c335568344b3 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java @@ -36,17 +36,15 @@ import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; -import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorStateStore; import org.apache.flink.streaming.api.transformations.OneInputTransformation; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -90,20 +88,14 @@ private boolean testSpillable( ((OneInputTransformation) written.getTransformation()) .getOperatorFactory()) .getOperator()); - StateInitializationContextImpl context = - new StateInitializationContextImpl( - null, - new MockOperatorStateStore() { - @Override - public ListState getUnionListState( - ListStateDescriptor stateDescriptor) throws Exception { - return getListState(stateDescriptor); - } - }, - null, - null, - null); - operator.initStateAndWriter(context, (a, b, c) -> true, new IOManagerAsync(), "123"); + + TypeSerializer serializer = + new CommittableTypeInfo().createSerializer(new ExecutionConfig()); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(operator); + harness.setup(serializer); + harness.initializeEmptyState(); + return ((KeyValueFileStoreWrite) ((StoreSinkWriteImpl) operator.write).write.getWrite()) .bufferSpillable(); } 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 eb7756718738..3f2daedffd48 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 @@ -27,12 +27,10 @@ import org.apache.paimon.table.TableTestBase; import org.apache.paimon.table.sink.SinkRecord; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.state.StateInitializationContextImpl; -import org.apache.flink.streaming.api.operators.collect.utils.MockOperatorStateStore; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.data.RowData; import org.assertj.core.api.Assertions; import org.junit.jupiter.params.ParameterizedTest; @@ -50,36 +48,28 @@ public void testCompactExactlyOnce(boolean streamingMode) throws Exception { CompactRememberStoreWrite compactRememberStoreWrite = new CompactRememberStoreWrite(streamingMode); - StoreCompactOperator storeCompactOperator = + StoreCompactOperator operator = new StoreCompactOperator( - (FileStoreTable) getTableDefault(), + getTableDefault(), (table, commitUser, state, ioManager, memoryPool, metricGroup) -> compactRememberStoreWrite, "10086"); - storeCompactOperator.open(); - StateInitializationContextImpl context = - new StateInitializationContextImpl( - null, - new MockOperatorStateStore() { - @Override - public ListState getUnionListState( - ListStateDescriptor stateDescriptor) throws Exception { - return getListState(stateDescriptor); - } - }, - null, - null, - null); - storeCompactOperator.initStateAndWriter( - context, (a, b, c) -> true, new IOManagerAsync(), "123"); - - storeCompactOperator.processElement(new StreamRecord<>(data(0))); - storeCompactOperator.processElement(new StreamRecord<>(data(0))); - storeCompactOperator.processElement(new StreamRecord<>(data(1))); - storeCompactOperator.processElement(new StreamRecord<>(data(1))); - storeCompactOperator.processElement(new StreamRecord<>(data(2))); - storeCompactOperator.prepareCommit(true, 1); + TypeSerializer serializer = + new CommittableTypeInfo().createSerializer(new ExecutionConfig()); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(operator); + harness.setup(serializer); + harness.initializeEmptyState(); + harness.open(); + + harness.processElement(new StreamRecord<>(data(0))); + harness.processElement(new StreamRecord<>(data(0))); + harness.processElement(new StreamRecord<>(data(1))); + harness.processElement(new StreamRecord<>(data(1))); + harness.processElement(new StreamRecord<>(data(2))); + + operator.prepareCommit(true, 1); Assertions.assertThat(compactRememberStoreWrite.compactTime).isEqualTo(3); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java index 5338b3886001..19c22d137c7f 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/AddDonePartitionActionTest.java @@ -26,6 +26,7 @@ import java.util.HashSet; import java.util.LinkedHashMap; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; @@ -61,6 +62,15 @@ public void markDone(LinkedHashMap partitionSpec) throw new UnsupportedOperationException(); } + @Override + public void alterPartition( + LinkedHashMap partitionSpec, + Map parameters, + long modifyTime) + throws Exception { + throw new UnsupportedOperationException(); + } + @Override public void close() throws Exception { closed.set(true); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java new file mode 100644 index 000000000000..0050f3083a8e --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/HmsReporterTest.java @@ -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. + */ + +package org.apache.paimon.flink.sink.partition; + +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.metastore.MetastoreClient; +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.table.sink.BatchTableCommit; +import org.apache.paimon.table.sink.BatchTableWrite; +import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.utils.PartitionPathUtils; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +/** Test for {@link HmsReporter}. */ +public class HmsReporterTest { + + @TempDir java.nio.file.Path tempDir; + + @Test + public void testReportAction() throws Exception { + Path tablePath = new Path(tempDir.toString(), "table"); + SchemaManager schemaManager = new SchemaManager(LocalFileIO.create(), tablePath); + Schema schema = + new Schema( + Lists.newArrayList( + new DataField(0, "c1", DataTypes.STRING()), + new DataField(1, "c2", DataTypes.STRING()), + new DataField(2, "c3", DataTypes.STRING())), + Collections.singletonList("c1"), + Collections.emptyList(), + Maps.newHashMap(), + ""); + schemaManager.createTable(schema); + + FileStoreTable table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath); + BatchTableWrite writer = table.newBatchWriteBuilder().newWrite(); + writer.write( + GenericRow.of( + BinaryString.fromString("a"), + BinaryString.fromString("a"), + BinaryString.fromString("a"))); + writer.write( + GenericRow.of( + BinaryString.fromString("b"), + BinaryString.fromString("a"), + BinaryString.fromString("a"))); + List messages = writer.prepareCommit(); + BatchTableCommit committer = table.newBatchWriteBuilder().newCommit(); + committer.commit(messages); + AtomicBoolean closed = new AtomicBoolean(false); + Map> partitionParams = Maps.newHashMap(); + + MetastoreClient client = + new MetastoreClient() { + @Override + public void addPartition(BinaryRow partition) throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void addPartition(LinkedHashMap partitionSpec) + throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void deletePartition(LinkedHashMap partitionSpec) + throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void markDone(LinkedHashMap partitionSpec) + throws Exception { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartition( + LinkedHashMap partitionSpec, + Map parameters, + long modifyTime) + throws Exception { + partitionParams.put( + PartitionPathUtils.generatePartitionPath(partitionSpec), + parameters); + } + + @Override + public void close() throws Exception { + closed.set(true); + } + }; + + HmsReporter action = new HmsReporter(table, client); + long time = 1729598544974L; + action.report("c1=a/", time); + Assertions.assertThat(partitionParams).containsKey("c1=a/"); + Assertions.assertThat(partitionParams.get("c1=a/")) + .isEqualTo( + ImmutableMap.of( + "numFiles", + "1", + "totalSize", + "591", + "numRows", + "1", + "transient_lastDdlTime", + String.valueOf(time / 1000))); + action.close(); + Assertions.assertThat(closed).isTrue(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java index 6dc14c9d0342..f0f4596c61bb 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java @@ -51,7 +51,7 @@ import static java.util.Collections.singletonList; import static org.apache.paimon.CoreOptions.DELETION_VECTORS_ENABLED; import static org.apache.paimon.CoreOptions.PARTITION_MARK_DONE_ACTION; -import static org.apache.paimon.flink.FlinkConnectorOptions.PARTITION_MARK_DONE_WHEN_END_INPUT; +import static org.apache.paimon.CoreOptions.PARTITION_MARK_DONE_WHEN_END_INPUT; import static org.assertj.core.api.Assertions.assertThat; class PartitionMarkDoneTest extends TableTestBase { @@ -86,7 +86,7 @@ private void innerTest(boolean deletionVectors) throws Exception { Path location = catalog.getTableLocation(identifier); Path successFile = new Path(location, "a=0/_SUCCESS"); PartitionMarkDone markDone = - PartitionMarkDone.create(false, false, new MockOperatorStateStore(), table); + PartitionMarkDone.create(false, false, new MockOperatorStateStore(), table).get(); notifyCommits(markDone, true); assertThat(table.fileIO().exists(successFile)).isEqualTo(deletionVectors); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sorter/SortOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sorter/SortOperatorTest.java index 155e259e02bb..c74c1c2c17a4 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sorter/SortOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sorter/SortOperatorTest.java @@ -65,7 +65,8 @@ public void testSort() throws Exception { 128, CompressOptions.defaultOptions(), 1, - MemorySize.MAX_VALUE) {}; + MemorySize.MAX_VALUE, + true) {}; OneInputStreamOperatorTestHarness harness = createTestHarness(sortOperator); harness.open(); @@ -114,7 +115,8 @@ public void testCloseSortOperator() throws Exception { 128, CompressOptions.defaultOptions(), 1, - MemorySize.MAX_VALUE) {}; + MemorySize.MAX_VALUE, + true) {}; OneInputStreamOperatorTestHarness harness = createTestHarness(sortOperator); harness.open(); File[] files = harness.getEnvironment().getIOManager().getSpillingDirectories(); 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 ad30f6388c90..92c8dd94a14e 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 @@ -115,7 +115,8 @@ private DataSplit dataSplit(int partition, int bucket, String... fileNames) { 0, // not used 0L, // not used null, // not used - FileSource.APPEND)); + FileSource.APPEND, + null)); } return DataSplit.builder() .withSnapshot(1) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitSerializerTest.java index f2c0732a2208..9c884cd5c33d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitSerializerTest.java @@ -88,7 +88,8 @@ public static DataFileMeta newFile(int level) { level, 0L, null, - FileSource.APPEND); + FileSource.APPEND, + null); } public static FileStoreSourceSplit newSourceSplit( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FlinkTableSourceTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FlinkTableSourceTest.java index 95f5721c4651..cff9ab6f4d25 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FlinkTableSourceTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FlinkTableSourceTest.java @@ -52,13 +52,14 @@ public void testApplyFilterNonPartitionTable() throws Exception { Schema schema = Schema.newBuilder().column("col1", DataTypes.INT()).build(); TableSchema tableSchema = new SchemaManager(fileIO, tablePath).createTable(schema); Table table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath, tableSchema); - FlinkTableSource tableSource = + DataTableSource tableSource = new DataTableSource( ObjectIdentifier.of("catalog1", "db1", "T"), table, false, null, null); // col1 = 1 List filters = ImmutableList.of(col1Equal1()); - Assertions.assertThat(tableSource.pushFilters(filters)).isEqualTo(filters); + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) + .isEqualTo(filters); } @Test @@ -81,54 +82,57 @@ public void testApplyPartitionTable() throws Exception { // col1 = 1 && p1 = 1 => [p1 = 1] List filters = ImmutableList.of(col1Equal1(), p1Equal1()); - Assertions.assertThat(tableSource.pushFilters(filters)) + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) .isEqualTo(ImmutableList.of(filters.get(0))); // col1 = 1 && p2 like '%a' => None filters = ImmutableList.of(col1Equal1(), p2Like("%a")); - Assertions.assertThat(tableSource.pushFilters(filters)).isEqualTo(filters); + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) + .isEqualTo(filters); // col1 = 1 && p2 like 'a%' => [p2 like 'a%'] filters = ImmutableList.of(col1Equal1(), p2Like("a%")); - Assertions.assertThat(tableSource.pushFilters(filters)) + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) .isEqualTo(ImmutableList.of(filters.get(0))); // rand(42) > 0.1 => None filters = ImmutableList.of(rand()); - Assertions.assertThat(tableSource.pushFilters(filters)).isEqualTo(filters); + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) + .isEqualTo(filters); // upper(p1) = "A" => [upper(p1) = "A"] filters = ImmutableList.of(upperP2EqualA()); - Assertions.assertThat(tableSource.pushFilters(filters)).isEqualTo(filters); + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) + .isEqualTo(filters); // col1 = 1 && (p2 like 'a%' or p1 = 1) => [p2 like 'a%' or p1 = 1] filters = ImmutableList.of(col1Equal1(), or(p2Like("a%"), p1Equal1())); - Assertions.assertThat(tableSource.pushFilters(filters)) + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) .isEqualTo(ImmutableList.of(filters.get(0))); // col1 = 1 && (p2 like '%a' or p1 = 1) => None filters = ImmutableList.of(col1Equal1(), or(p2Like("%a"), p1Equal1())); - Assertions.assertThat(tableSource.pushFilters(filters)) + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) .containsExactlyInAnyOrder(filters.toArray(new ResolvedExpression[0])); // col1 = 1 && (p2 like 'a%' && p1 = 1) => [p2 like 'a%' && p1 = 1] filters = ImmutableList.of(col1Equal1(), and(p2Like("a%"), p1Equal1())); - Assertions.assertThat(tableSource.pushFilters(filters)) + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) .isEqualTo(ImmutableList.of(filters.get(0))); // col1 = 1 && (p2 like '%a' && p1 = 1) => None filters = ImmutableList.of(col1Equal1(), and(p2Like("%a"), p1Equal1())); - Assertions.assertThat(tableSource.pushFilters(filters)) + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) .containsExactlyInAnyOrder(filters.toArray(new ResolvedExpression[0])); // p2 like 'a%' && (col1 = 1 or p1 = 1) => [col1 = 1 or p1 = 1] filters = ImmutableList.of(p2Like("a%"), or(col1Equal1(), p1Equal1())); - Assertions.assertThat(tableSource.pushFilters(filters)) + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) .isEqualTo(ImmutableList.of(filters.get(1))); // p2 like 'a%' && (col1 = 1 && p1 = 1) => [col1 = 1 && p1 = 1] filters = ImmutableList.of(p2Like("a%"), and(col1Equal1(), p1Equal1())); - Assertions.assertThat(tableSource.pushFilters(filters)) + Assertions.assertThat(tableSource.applyFilters(filters).getRemainingFilters()) .isEqualTo(ImmutableList.of(filters.get(1))); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java new file mode 100644 index 000000000000..8404d994fa9f --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/IteratorSourcesITCase.java @@ -0,0 +1,98 @@ +/* + * 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.source; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamUtils; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.TestLogger; +import org.junit.ClassRule; +import org.junit.Test; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.Assert.fail; + +/** + * An integration test for the sources based on iterators. + * + *

    This test uses the {@link NumberSequenceRowSource} as a concrete iterator source + * implementation, but covers all runtime-related aspects for all the iterator-based sources + * together. + */ +public class IteratorSourcesITCase extends TestLogger { + + private static final int PARALLELISM = 4; + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .build()); + + // ------------------------------------------------------------------------ + + @Test + public void testParallelSourceExecution() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + final DataStream stream = + env.fromSource( + new NumberSequenceRowSource(1L, 1_000L), + WatermarkStrategy.noWatermarks(), + "iterator source"); + + final List result = + DataStreamUtils.collectBoundedStream(stream, "Iterator Source Test"); + + verifySequence(result, 1L, 1_000L); + } + + // ------------------------------------------------------------------------ + // test utils + // ------------------------------------------------------------------------ + + private static void verifySequence( + final List sequence, final long from, final long to) { + if (sequence.size() != to - from + 1) { + fail(String.format("Expected: Sequence [%d, %d]. Found: %s", from, to, sequence)); + } + + final List list = + sequence.stream() + .map(r -> r.getLong(0)) + .sorted(Long::compareTo) + .collect(Collectors.toList()); + + int pos = 0; + for (long value = from; value <= to; value++, pos++) { + if (value != list.get(pos)) { + fail(String.format("Expected: Sequence [%d, %d]. Found: %s", from, to, list)); + } + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/NumberSequenceRowSourceTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/NumberSequenceRowSourceTest.java new file mode 100644 index 000000000000..5f17b842fb77 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/NumberSequenceRowSourceTest.java @@ -0,0 +1,195 @@ +/* + * 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.source; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.metrics.groups.SourceReaderMetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.SimpleUserCodeClassLoader; +import org.apache.flink.util.UserCodeClassLoader; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.assertj.core.api.Assertions.fail; + +/** Tests for the {@link NumberSequenceRowSource}. */ +class NumberSequenceRowSourceTest { + + @Test + void testReaderCheckpoints() throws Exception { + final long from = 177; + final long mid = 333; + final long to = 563; + final long elementsPerCycle = (to - from) / 3; + + final TestingReaderOutput out = new TestingReaderOutput<>(); + + SourceReader reader = createReader(); + reader.addSplits( + Arrays.asList( + new NumberSequenceRowSource.NumberSequenceSplit("split-1", from, mid), + new NumberSequenceRowSource.NumberSequenceSplit("split-2", mid + 1, to))); + + long remainingInCycle = elementsPerCycle; + while (reader.pollNext(out) != InputStatus.END_OF_INPUT) { + if (--remainingInCycle <= 0) { + remainingInCycle = elementsPerCycle; + // checkpoint + List splits = reader.snapshotState(1L); + + // re-create and restore + reader = createReader(); + if (splits.isEmpty()) { + reader.notifyNoMoreSplits(); + } else { + reader.addSplits(splits); + } + } + } + + final List result = out.getEmittedRecords(); + validateSequence(result, from, to); + } + + private static void validateSequence( + final List sequence, final long from, final long to) { + if (sequence.size() != to - from + 1) { + failSequence(sequence, from, to); + } + + long nextExpected = from; + for (RowData next : sequence) { + if (next.getLong(0) != nextExpected++) { + failSequence(sequence, from, to); + } + } + } + + private static void failSequence(final List sequence, final long from, final long to) { + fail( + String.format( + "Expected: A sequence [%d, %d], but found: sequence (size %d) : %s", + from, to, sequence.size(), sequence)); + } + + private static SourceReader + createReader() { + // the arguments passed in the source constructor matter only to the enumerator + return new NumberSequenceRowSource(0L, 0L).createReader(new DummyReaderContext()); + } + + // ------------------------------------------------------------------------ + // test utils / mocks + // + // the "flink-connector-test-utils module has proper mocks and utils, + // but cannot be used here, because it would create a cyclic dependency. + // ------------------------------------------------------------------------ + + private static final class DummyReaderContext implements SourceReaderContext { + + @Override + public SourceReaderMetricGroup metricGroup() { + return UnregisteredMetricsGroup.createSourceReaderMetricGroup(); + } + + @Override + public Configuration getConfiguration() { + return new Configuration(); + } + + @Override + public String getLocalHostName() { + return "localhost"; + } + + @Override + public int getIndexOfSubtask() { + return 0; + } + + @Override + public void sendSplitRequest() {} + + @Override + public void sendSourceEventToCoordinator(SourceEvent sourceEvent) {} + + @Override + public UserCodeClassLoader getUserCodeClassLoader() { + return SimpleUserCodeClassLoader.create(getClass().getClassLoader()); + } + + @Override + public int currentParallelism() { + return 1; + } + } + + private static final class TestingReaderOutput implements ReaderOutput { + + private final ArrayList emittedRecords = new ArrayList<>(); + + @Override + public void collect(E record) { + emittedRecords.add(record); + } + + @Override + public void collect(E record, long timestamp) { + collect(record); + } + + @Override + public void emitWatermark(Watermark watermark) { + throw new UnsupportedOperationException(); + } + + @Override + public void markIdle() { + throw new UnsupportedOperationException(); + } + + @Override + public void markActive() { + throw new UnsupportedOperationException(); + } + + @Override + public SourceOutput createOutputForSplit(String splitId) { + return this; + } + + @Override + public void releaseOutputForSplit(String splitId) {} + + public ArrayList getEmittedRecords() { + return emittedRecords; + } + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java index aab8666fe998..85679e5fd30a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java @@ -107,7 +107,10 @@ public TestChangelogDataReadWrite(String root) { "default", CoreOptions.FILE_FORMAT.defaultValue().toString(), CoreOptions.DATA_FILE_PREFIX.defaultValue(), - CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue()); + CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), + CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), + CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), + CoreOptions.FILE_COMPRESSION.defaultValue()); this.snapshotManager = new SnapshotManager(LocalFileIO.create(), new Path(root)); this.commitUser = UUID.randomUUID().toString(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java index 0f6d66815d41..61a03a29a21b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/OperatorSourceTest.java @@ -203,6 +203,7 @@ public void testReadOperatorMetricsRegisterAndUpdate() throws Exception { readerOperatorMetricGroup, "currentEmitEventTimeLag") .getValue()) .isEqualTo(-1L); + harness.processElement(new StreamRecord<>(splits.get(0))); assertThat( (Long) @@ -211,13 +212,22 @@ public void testReadOperatorMetricsRegisterAndUpdate() throws Exception { "currentFetchEventTimeLag") .getValue()) .isGreaterThan(0); + long emitEventTimeLag = + (Long) + TestingMetricUtils.getGauge( + readerOperatorMetricGroup, "currentEmitEventTimeLag") + .getValue(); + assertThat(emitEventTimeLag).isGreaterThan(0); + + // wait for a while and read metrics again, metrics should not change + Thread.sleep(100); assertThat( (Long) TestingMetricUtils.getGauge( readerOperatorMetricGroup, "currentEmitEventTimeLag") .getValue()) - .isGreaterThan(0); + .isEqualTo(emitEventTimeLag); } private T testReadSplit( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/FileStoreTableStatisticsTestBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/FileStoreTableStatisticsTestBase.java index 8b69ef285f3a..826bf28d1248 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/FileStoreTableStatisticsTestBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/FileStoreTableStatisticsTestBase.java @@ -18,13 +18,18 @@ package org.apache.paimon.flink.source.statistics; +import org.apache.paimon.FileStore; +import org.apache.paimon.Snapshot; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.flink.source.DataTableSource; import org.apache.paimon.fs.Path; +import org.apache.paimon.operation.FileStoreCommit; import org.apache.paimon.predicate.PredicateBuilder; import org.apache.paimon.schema.Schema; +import org.apache.paimon.stats.ColStats; +import org.apache.paimon.stats.Statistics; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.StreamTableCommit; import org.apache.paimon.table.sink.StreamTableWrite; @@ -33,6 +38,7 @@ import org.apache.paimon.types.VarCharType; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.plan.stats.ColumnStats; import org.apache.flink.table.plan.stats.TableStats; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -40,6 +46,8 @@ import org.junit.jupiter.api.io.TempDir; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import java.util.UUID; /** Statistics tests for {@link FileStoreTable}. */ @@ -60,9 +68,66 @@ public void before() { @Test public void testTableScanStatistics() throws Exception { FileStoreTable table = writeData(); + Map> colStatsMap = new HashMap<>(); + colStatsMap.put("pt", ColStats.newColStats(0, 2L, 1, 2, 0L, null, null)); + colStatsMap.put("a", ColStats.newColStats(1, 9L, 10, 90, 0L, null, null)); + colStatsMap.put("b", ColStats.newColStats(2, 7L, 100L, 900L, 2L, null, null)); + colStatsMap.put( + "c", + ColStats.newColStats( + 3, + 7L, + BinaryString.fromString("S1"), + BinaryString.fromString("S8"), + 2L, + null, + null)); + + FileStore fileStore = table.store(); + FileStoreCommit fileStoreCommit = fileStore.newCommit(commitUser); + Snapshot latestSnapshot = fileStore.snapshotManager().latestSnapshot(); + Statistics colStats = + new Statistics( + latestSnapshot.id(), latestSnapshot.schemaId(), 9L, null, colStatsMap); + fileStoreCommit.commitStatistics(colStats, Long.MAX_VALUE); + fileStoreCommit.close(); DataTableSource scanSource = new DataTableSource(identifier, table, false, null, null); Assertions.assertThat(scanSource.reportStatistics().getRowCount()).isEqualTo(9L); - // TODO validate column statistics + Map expectedColStats = new HashMap<>(); + expectedColStats.put( + "pt", + ColumnStats.Builder.builder() + .setNdv(2L) + .setMin(1) + .setMax(2) + .setNullCount(0L) + .build()); + expectedColStats.put( + "a", + ColumnStats.Builder.builder() + .setNdv(9L) + .setMin(10) + .setMax(90) + .setNullCount(0L) + .build()); + expectedColStats.put( + "b", + ColumnStats.Builder.builder() + .setNdv(7L) + .setMin(100L) + .setMax(900L) + .setNullCount(2L) + .build()); + expectedColStats.put( + "c", + ColumnStats.Builder.builder() + .setNdv(7L) + .setMin(BinaryString.fromString("S1")) + .setMax(BinaryString.fromString("S8")) + .setNullCount(2L) + .build()); + Assertions.assertThat(scanSource.reportStatistics().getColumnStats()) + .isEqualTo(expectedColStats); } @Test @@ -87,9 +152,68 @@ public void testTableFilterPartitionStatistics() throws Exception { null, null, null, - null); + null, + false); Assertions.assertThat(partitionFilterSource.reportStatistics().getRowCount()).isEqualTo(5L); - // TODO validate column statistics + Map> colStatsMap = new HashMap<>(); + colStatsMap.put("pt", ColStats.newColStats(0, 1L, 1, 2, 0L, null, null)); + colStatsMap.put("a", ColStats.newColStats(1, 5L, 10, 90, 0L, null, null)); + colStatsMap.put("b", ColStats.newColStats(2, 3L, 100L, 900L, 2L, null, null)); + colStatsMap.put( + "c", + ColStats.newColStats( + 3, + 3L, + BinaryString.fromString("S1"), + BinaryString.fromString("S7"), + 2L, + null, + null)); + + FileStore fileStore = table.store(); + FileStoreCommit fileStoreCommit = fileStore.newCommit(commitUser); + Snapshot latestSnapshot = fileStore.snapshotManager().latestSnapshot(); + Statistics colStats = + new Statistics( + latestSnapshot.id(), latestSnapshot.schemaId(), 9L, null, colStatsMap); + fileStoreCommit.commitStatistics(colStats, Long.MAX_VALUE); + fileStoreCommit.close(); + + Map expectedColStats = new HashMap<>(); + expectedColStats.put( + "pt", + ColumnStats.Builder.builder() + .setNdv(1L) + .setMin(1) + .setMax(2) + .setNullCount(0L) + .build()); + expectedColStats.put( + "a", + ColumnStats.Builder.builder() + .setNdv(5L) + .setMin(10) + .setMax(90) + .setNullCount(0L) + .build()); + expectedColStats.put( + "b", + ColumnStats.Builder.builder() + .setNdv(3L) + .setMin(100L) + .setMax(900L) + .setNullCount(2L) + .build()); + expectedColStats.put( + "c", + ColumnStats.Builder.builder() + .setNdv(3L) + .setMin(BinaryString.fromString("S1")) + .setMax(BinaryString.fromString("S7")) + .setNullCount(2L) + .build()); + Assertions.assertThat(partitionFilterSource.reportStatistics().getColumnStats()) + .isEqualTo(expectedColStats); } @Test @@ -107,9 +231,68 @@ public void testTableFilterKeyStatistics() throws Exception { null, null, null, - null); + null, + false); Assertions.assertThat(keyFilterSource.reportStatistics().getRowCount()).isEqualTo(2L); - // TODO validate column statistics + Map> colStatsMap = new HashMap<>(); + colStatsMap.put("pt", ColStats.newColStats(0, 1L, 2, 2, 0L, null, null)); + colStatsMap.put("a", ColStats.newColStats(1, 1L, 50, 50, 0L, null, null)); + colStatsMap.put("b", ColStats.newColStats(2, 1L, null, null, 1L, null, null)); + colStatsMap.put( + "c", + ColStats.newColStats( + 3, + 1L, + BinaryString.fromString("S5"), + BinaryString.fromString("S5"), + 0L, + null, + null)); + + FileStore fileStore = table.store(); + FileStoreCommit fileStoreCommit = fileStore.newCommit(commitUser); + Snapshot latestSnapshot = fileStore.snapshotManager().latestSnapshot(); + Statistics colStats = + new Statistics( + latestSnapshot.id(), latestSnapshot.schemaId(), 9L, null, colStatsMap); + fileStoreCommit.commitStatistics(colStats, Long.MAX_VALUE); + fileStoreCommit.close(); + + Map expectedColStats = new HashMap<>(); + expectedColStats.put( + "pt", + ColumnStats.Builder.builder() + .setNdv(1L) + .setMin(2) + .setMax(2) + .setNullCount(0L) + .build()); + expectedColStats.put( + "a", + ColumnStats.Builder.builder() + .setNdv(1L) + .setMin(50) + .setMax(50) + .setNullCount(0L) + .build()); + expectedColStats.put( + "b", + ColumnStats.Builder.builder() + .setNdv(1L) + .setMin(null) + .setMax(null) + .setNullCount(1L) + .build()); + expectedColStats.put( + "c", + ColumnStats.Builder.builder() + .setNdv(1L) + .setMin(BinaryString.fromString("S5")) + .setMax(BinaryString.fromString("S5")) + .setNullCount(0L) + .build()); + Assertions.assertThat(keyFilterSource.reportStatistics().getColumnStats()) + .isEqualTo(expectedColStats); } @Test @@ -127,9 +310,68 @@ public void testTableFilterValueStatistics() throws Exception { null, null, null, - null); + null, + false); Assertions.assertThat(keyFilterSource.reportStatistics().getRowCount()).isEqualTo(4L); - // TODO validate column statistics + Map> colStatsMap = new HashMap<>(); + colStatsMap.put("pt", ColStats.newColStats(0, 4L, 2, 2, 0L, null, null)); + colStatsMap.put("a", ColStats.newColStats(1, 4L, 50, 50, 0L, null, null)); + colStatsMap.put("b", ColStats.newColStats(2, 4L, null, null, 1L, null, null)); + colStatsMap.put( + "c", + ColStats.newColStats( + 3, + 4L, + BinaryString.fromString("S5"), + BinaryString.fromString("S8"), + 0L, + null, + null)); + + FileStore fileStore = table.store(); + FileStoreCommit fileStoreCommit = fileStore.newCommit(commitUser); + Snapshot latestSnapshot = fileStore.snapshotManager().latestSnapshot(); + Statistics colStats = + new Statistics( + latestSnapshot.id(), latestSnapshot.schemaId(), 9L, null, colStatsMap); + fileStoreCommit.commitStatistics(colStats, Long.MAX_VALUE); + fileStoreCommit.close(); + + Map expectedColStats = new HashMap<>(); + expectedColStats.put( + "pt", + ColumnStats.Builder.builder() + .setNdv(4L) + .setMin(2) + .setMax(2) + .setNullCount(0L) + .build()); + expectedColStats.put( + "a", + ColumnStats.Builder.builder() + .setNdv(4L) + .setMin(50) + .setMax(50) + .setNullCount(0L) + .build()); + expectedColStats.put( + "b", + ColumnStats.Builder.builder() + .setNdv(4L) + .setMin(null) + .setMax(null) + .setNullCount(1L) + .build()); + expectedColStats.put( + "c", + ColumnStats.Builder.builder() + .setNdv(4L) + .setMin(BinaryString.fromString("S5")) + .setMax(BinaryString.fromString("S8")) + .setNullCount(0L) + .build()); + Assertions.assertThat(keyFilterSource.reportStatistics().getColumnStats()) + .isEqualTo(expectedColStats); } protected FileStoreTable writeData() throws Exception { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java index 8e4643ef5200..f5d4121672b0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java @@ -51,7 +51,8 @@ public void testTableFilterValueStatistics() throws Exception { null, null, null, - null); + null, + false); Assertions.assertThat(keyFilterSource.reportStatistics().getRowCount()).isEqualTo(9L); // TODO validate column statistics } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java index f09ab0924d07..ce0017eb1874 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/AbstractTestBase.java @@ -21,6 +21,7 @@ import org.apache.paimon.utils.FileIOUtils; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; @@ -33,14 +34,22 @@ import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.CollectModifyOperation; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.QueryOperation; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; import java.io.File; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.nio.file.Path; import java.time.Duration; +import java.util.Collections; +import java.util.List; import java.util.UUID; /** Similar to Flink's AbstractTestBase but using Junit5. */ @@ -303,4 +312,39 @@ public StreamExecutionEnvironment build() { return env; } } + + public static Transformation translate(TableEnvironment env, String statement) { + TableEnvironmentImpl envImpl = (TableEnvironmentImpl) env; + List operations = envImpl.getParser().parse(statement); + + if (operations.size() != 1) { + throw new RuntimeException("No operation after parsing for " + statement); + } + + Operation operation = operations.get(0); + if (operation instanceof QueryOperation) { + QueryOperation queryOperation = (QueryOperation) operation; + CollectModifyOperation sinkOperation = new CollectModifyOperation(queryOperation); + List> transformations; + try { + Method translate = + TableEnvironmentImpl.class.getDeclaredMethod("translate", List.class); + translate.setAccessible(true); + //noinspection unchecked + transformations = + (List>) + translate.invoke(envImpl, Collections.singletonList(sinkOperation)); + } catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) { + throw new RuntimeException(e); + } + + if (transformations.size() != 1) { + throw new RuntimeException("No transformation after translating for " + statement); + } + + return transformations.get(0); + } else { + throw new RuntimeException(); + } + } } diff --git a/paimon-format/pom.xml b/paimon-format/pom.xml index e96fddcd8480..5804a704b0cf 100644 --- a/paimon-format/pom.xml +++ b/paimon-format/pom.xml @@ -32,13 +32,10 @@ under the License. Paimon : Format - 1.13.1 - 1.9.2 2.5 1.6 3.12.0 2.8.1 - 3.19.6 diff --git a/paimon-format/src/main/java/org/apache/orc/impl/RecordReaderImpl.java b/paimon-format/src/main/java/org/apache/orc/impl/RecordReaderImpl.java new file mode 100644 index 000000000000..6c3af4e50043 --- /dev/null +++ b/paimon-format/src/main/java/org/apache/orc/impl/RecordReaderImpl.java @@ -0,0 +1,1889 @@ +/* + * 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.orc.impl; + +import org.apache.paimon.fileindex.FileIndexResult; +import org.apache.paimon.fileindex.bitmap.BitmapIndexResult; +import org.apache.paimon.utils.RoaringBitmap32; + +import org.apache.commons.lang3.ArrayUtils; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf; +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument; +import org.apache.hadoop.hive.ql.util.TimestampUtils; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.io.Text; +import org.apache.orc.BooleanColumnStatistics; +import org.apache.orc.CollectionColumnStatistics; +import org.apache.orc.ColumnStatistics; +import org.apache.orc.CompressionCodec; +import org.apache.orc.DataReader; +import org.apache.orc.DateColumnStatistics; +import org.apache.orc.DecimalColumnStatistics; +import org.apache.orc.DoubleColumnStatistics; +import org.apache.orc.IntegerColumnStatistics; +import org.apache.orc.OrcConf; +import org.apache.orc.OrcFile; +import org.apache.orc.OrcFilterContext; +import org.apache.orc.OrcProto; +import org.apache.orc.Reader; +import org.apache.orc.RecordReader; +import org.apache.orc.StringColumnStatistics; +import org.apache.orc.StripeInformation; +import org.apache.orc.TimestampColumnStatistics; +import org.apache.orc.TypeDescription; +import org.apache.orc.filter.BatchFilter; +import org.apache.orc.impl.filter.FilterFactory; +import org.apache.orc.impl.reader.ReaderEncryption; +import org.apache.orc.impl.reader.StripePlanner; +import org.apache.orc.impl.reader.tree.BatchReader; +import org.apache.orc.impl.reader.tree.TypeReader; +import org.apache.orc.util.BloomFilter; +import org.apache.orc.util.BloomFilterIO; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.ZoneOffset; +import java.time.chrono.ChronoLocalDate; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.SortedSet; +import java.util.TimeZone; +import java.util.TreeSet; +import java.util.function.Consumer; + +/* This file is based on source code from the ORC Project (http://orc.apache.org/), licensed by the Apache + * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for + * additional information regarding copyright ownership. */ + +/** An orc RecordReaderImpl. */ +public class RecordReaderImpl implements RecordReader { + static final Logger LOG = LoggerFactory.getLogger(RecordReaderImpl.class); + private static final boolean isLogDebugEnabled = LOG.isDebugEnabled(); + // as public for use with test cases + public static final OrcProto.ColumnStatistics EMPTY_COLUMN_STATISTICS = + OrcProto.ColumnStatistics.newBuilder() + .setNumberOfValues(0) + .setHasNull(false) + .setBytesOnDisk(0) + .build(); + protected final Path path; + private final long firstRow; + private final List stripes = new ArrayList<>(); + private OrcProto.StripeFooter stripeFooter; + private final long totalRowCount; + protected final TypeDescription schema; + // the file included columns indexed by the file's column ids. + private final boolean[] fileIncluded; + private final long rowIndexStride; + private long rowInStripe = 0; + // position of the follow reader within the stripe + private long followRowInStripe = 0; + private int currentStripe = -1; + private long rowBaseInStripe = 0; + private long rowCountInStripe = 0; + private final BatchReader reader; + private final OrcIndex indexes; + // identifies the columns requiring row indexes + private final boolean[] rowIndexColsToRead; + private final SargApplier sargApp; + // an array about which row groups aren't skipped + private boolean[] includedRowGroups = null; + private final DataReader dataReader; + private final int maxDiskRangeChunkLimit; + private final StripePlanner planner; + // identifies the type of read, ALL(read everything), LEADERS(read only the filter columns) + private final TypeReader.ReadPhase startReadPhase; + // identifies that follow columns bytes must be read + private boolean needsFollowColumnsRead; + private final boolean noSelectedVector; + // identifies whether the file has bad bloom filters that we should not use. + private final boolean skipBloomFilters; + private final FileIndexResult fileIndexResult; + static final String[] BAD_CPP_BLOOM_FILTER_VERSIONS = { + "1.6.0", "1.6.1", "1.6.2", "1.6.3", "1.6.4", "1.6.5", "1.6.6", "1.6.7", "1.6.8", "1.6.9", + "1.6.10", "1.6.11", "1.7.0" + }; + + /** + * Given a list of column names, find the given column and return the index. + * + * @param evolution the mapping from reader to file schema + * @param columnName the fully qualified column name to look for + * @return the file column number or -1 if the column wasn't found in the file schema + * @throws IllegalArgumentException if the column was not found in the reader schema + */ + static int findColumns(SchemaEvolution evolution, String columnName) { + TypeDescription fileColumn = findColumnType(evolution, columnName); + return fileColumn == null ? -1 : fileColumn.getId(); + } + + static TypeDescription findColumnType(SchemaEvolution evolution, String columnName) { + try { + TypeDescription readerColumn = + evolution + .getReaderBaseSchema() + .findSubtype(columnName, evolution.isSchemaEvolutionCaseAware); + return evolution.getFileType(readerColumn); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException( + "Filter could not find column with name: " + + columnName + + " on " + + evolution.getReaderBaseSchema(), + e); + } + } + + /** + * Given a column name such as 'a.b.c', this method returns the column 'a.b.c' if present in the + * file. In case 'a.b.c' is not found in file then it tries to look for 'a.b', then 'a'. If none + * are present then it shall return null. + * + * @param evolution the mapping from reader to file schema + * @param columnName the fully qualified column name to look for + * @return the file column type or null in case none of the branch columns are present in the + * file + * @throws IllegalArgumentException if the column was not found in the reader schema + */ + static TypeDescription findMostCommonColumn(SchemaEvolution evolution, String columnName) { + try { + TypeDescription readerColumn = + evolution + .getReaderBaseSchema() + .findSubtype(columnName, evolution.isSchemaEvolutionCaseAware); + TypeDescription fileColumn; + do { + fileColumn = evolution.getFileType(readerColumn); + if (fileColumn == null) { + readerColumn = readerColumn.getParent(); + } else { + return fileColumn; + } + } while (readerColumn != null); + return null; + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException( + "Filter could not find column with name: " + + columnName + + " on " + + evolution.getReaderBaseSchema(), + e); + } + } + + /** + * Find the mapping from predicate leaves to columns. + * + * @param sargLeaves the search argument that we need to map + * @param evolution the mapping from reader to file schema + * @return an array mapping the sarg leaves to concrete column numbers in the file + */ + public static int[] mapSargColumnsToOrcInternalColIdx( + List sargLeaves, SchemaEvolution evolution) { + int[] result = new int[sargLeaves.size()]; + for (int i = 0; i < sargLeaves.size(); ++i) { + int colNum = -1; + try { + String colName = sargLeaves.get(i).getColumnName(); + colNum = findColumns(evolution, colName); + } catch (IllegalArgumentException e) { + LOG.debug("{}", e.getMessage()); + } + result[i] = colNum; + } + return result; + } + + public RecordReaderImpl( + ReaderImpl fileReader, Reader.Options options, FileIndexResult fileIndexResult) + throws IOException { + this.fileIndexResult = fileIndexResult; + OrcFile.WriterVersion writerVersion = fileReader.getWriterVersion(); + SchemaEvolution evolution; + if (options.getSchema() == null) { + LOG.info("Reader schema not provided -- using file schema " + fileReader.getSchema()); + evolution = new SchemaEvolution(fileReader.getSchema(), null, options); + } else { + + // Now that we are creating a record reader for a file, validate that + // the schema to read is compatible with the file schema. + // + evolution = new SchemaEvolution(fileReader.getSchema(), options.getSchema(), options); + if (LOG.isDebugEnabled() && evolution.hasConversion()) { + LOG.debug( + "ORC file " + + fileReader.path.toString() + + " has data type conversion --\n" + + "reader schema: " + + options.getSchema().toString() + + "\n" + + "file schema: " + + fileReader.getSchema()); + } + } + this.noSelectedVector = !options.useSelected(); + LOG.debug("noSelectedVector={}", this.noSelectedVector); + this.schema = evolution.getReaderSchema(); + this.path = fileReader.path; + this.rowIndexStride = fileReader.rowIndexStride; + boolean ignoreNonUtf8BloomFilter = + OrcConf.IGNORE_NON_UTF8_BLOOM_FILTERS.getBoolean(fileReader.conf); + ReaderEncryption encryption = fileReader.getEncryption(); + this.fileIncluded = evolution.getFileIncluded(); + SearchArgument sarg = options.getSearchArgument(); + boolean[] rowIndexCols = new boolean[evolution.getFileIncluded().length]; + if (sarg != null && rowIndexStride > 0) { + sargApp = + new SargApplier( + sarg, + rowIndexStride, + evolution, + writerVersion, + fileReader.useUTCTimestamp, + fileReader.writerUsedProlepticGregorian(), + fileReader.options.getConvertToProlepticGregorian()); + sargApp.setRowIndexCols(rowIndexCols); + } else { + sargApp = null; + } + + long rows = 0; + long skippedRows = 0; + long offset = options.getOffset(); + long maxOffset = options.getMaxOffset(); + for (StripeInformation stripe : fileReader.getStripes()) { + long stripeStart = stripe.getOffset(); + if (offset > stripeStart) { + skippedRows += stripe.getNumberOfRows(); + } else if (stripeStart < maxOffset) { + this.stripes.add(stripe); + rows += stripe.getNumberOfRows(); + } + } + this.maxDiskRangeChunkLimit = + OrcConf.ORC_MAX_DISK_RANGE_CHUNK_LIMIT.getInt(fileReader.conf); + Boolean zeroCopy = options.getUseZeroCopy(); + if (zeroCopy == null) { + zeroCopy = OrcConf.USE_ZEROCOPY.getBoolean(fileReader.conf); + } + if (options.getDataReader() != null) { + this.dataReader = options.getDataReader().clone(); + } else { + InStream.StreamOptions unencryptedOptions = + InStream.options() + .withCodec(OrcCodecPool.getCodec(fileReader.getCompressionKind())) + .withBufferSize(fileReader.getCompressionSize()); + DataReaderProperties.Builder builder = + DataReaderProperties.builder() + .withCompression(unencryptedOptions) + .withFileSystemSupplier(fileReader.getFileSystemSupplier()) + .withPath(fileReader.path) + .withMaxDiskRangeChunkLimit(maxDiskRangeChunkLimit) + .withZeroCopy(zeroCopy) + .withMinSeekSize(options.minSeekSize()) + .withMinSeekSizeTolerance(options.minSeekSizeTolerance()); + FSDataInputStream file = fileReader.takeFile(); + if (file != null) { + builder.withFile(file); + } + this.dataReader = RecordReaderUtils.createDefaultDataReader(builder.build()); + } + firstRow = skippedRows; + totalRowCount = rows; + Boolean skipCorrupt = options.getSkipCorruptRecords(); + if (skipCorrupt == null) { + skipCorrupt = OrcConf.SKIP_CORRUPT_DATA.getBoolean(fileReader.conf); + } + + String[] filterCols = null; + Consumer filterCallBack = null; + String filePath = + options.allowPluginFilters() + ? fileReader.getFileSystem().makeQualified(fileReader.path).toString() + : null; + BatchFilter filter = + FilterFactory.createBatchFilter( + options, + evolution.getReaderBaseSchema(), + evolution.isSchemaEvolutionCaseAware(), + fileReader.getFileVersion(), + false, + filePath, + fileReader.conf); + if (filter != null) { + // If a filter is determined then use this + filterCallBack = filter; + filterCols = filter.getColumnNames(); + } + + // Map columnNames to ColumnIds + SortedSet filterColIds = new TreeSet<>(); + if (filterCols != null) { + for (String colName : filterCols) { + TypeDescription expandCol = findColumnType(evolution, colName); + // If the column is not present in the file then this can be ignored from read. + if (expandCol == null || expandCol.getId() == -1) { + // Add -1 to filter columns so that the NullTreeReader is invoked during the + // LEADERS phase + filterColIds.add(-1); + // Determine the common parent and include these + expandCol = findMostCommonColumn(evolution, colName); + } + while (expandCol != null && expandCol.getId() != -1) { + // classify the column and the parent branch as LEAD + filterColIds.add(expandCol.getId()); + rowIndexCols[expandCol.getId()] = true; + expandCol = expandCol.getParent(); + } + } + this.startReadPhase = TypeReader.ReadPhase.LEADERS; + LOG.debug( + "Using startReadPhase: {} with filter columns: {}", + startReadPhase, + filterColIds); + } else { + this.startReadPhase = TypeReader.ReadPhase.ALL; + } + + this.rowIndexColsToRead = ArrayUtils.contains(rowIndexCols, true) ? rowIndexCols : null; + TreeReaderFactory.ReaderContext readerContext = + new TreeReaderFactory.ReaderContext() + .setSchemaEvolution(evolution) + .setFilterCallback(filterColIds, filterCallBack) + .skipCorrupt(skipCorrupt) + .fileFormat(fileReader.getFileVersion()) + .useUTCTimestamp(fileReader.useUTCTimestamp) + .setProlepticGregorian( + fileReader.writerUsedProlepticGregorian(), + fileReader.options.getConvertToProlepticGregorian()) + .setEncryption(encryption); + reader = TreeReaderFactory.createRootReader(evolution.getReaderSchema(), readerContext); + skipBloomFilters = hasBadBloomFilters(fileReader.getFileTail().getFooter()); + + int columns = evolution.getFileSchema().getMaximumId() + 1; + indexes = + new OrcIndex( + new OrcProto.RowIndex[columns], + new OrcProto.Stream.Kind[columns], + new OrcProto.BloomFilterIndex[columns]); + + planner = + new StripePlanner( + evolution.getFileSchema(), + encryption, + dataReader, + writerVersion, + ignoreNonUtf8BloomFilter, + maxDiskRangeChunkLimit, + filterColIds); + + try { + advanceToNextRow(reader, 0L, true); + } catch (Exception e) { + // Try to close since this happens in constructor. + close(); + long stripeId = stripes.size() == 0 ? 0 : stripes.get(0).getStripeId(); + throw new IOException( + String.format("Problem opening stripe %d footer in %s.", stripeId, path), e); + } + } + + /** + * Check if the file has inconsistent bloom filters. We will skip using them in the following + * reads. + * + * @return true if it has. + */ + private boolean hasBadBloomFilters(OrcProto.Footer footer) { + // Only C++ writer in old releases could have bad bloom filters. + if (footer.getWriter() != 1) { + return false; + } + // 'softwareVersion' is added in 1.5.13, 1.6.11, and 1.7.0. + // 1.6.x releases before 1.6.11 won't have it. On the other side, the C++ writer + // supports writing bloom filters since 1.6.0. So files written by the C++ writer + // and with 'softwareVersion' unset would have bad bloom filters. + if (!footer.hasSoftwareVersion()) { + return true; + } + String fullVersion = footer.getSoftwareVersion(); + String version = fullVersion; + // Deal with snapshot versions, e.g. 1.6.12-SNAPSHOT. + if (fullVersion.contains("-")) { + version = fullVersion.substring(0, fullVersion.indexOf('-')); + } + for (String v : BAD_CPP_BLOOM_FILTER_VERSIONS) { + if (v.equals(version)) { + return true; + } + } + return false; + } + + /** An orc PositionProvider impl. */ + public static final class PositionProviderImpl implements PositionProvider { + private final OrcProto.RowIndexEntry entry; + private int index; + + public PositionProviderImpl(OrcProto.RowIndexEntry entry) { + this(entry, 0); + } + + public PositionProviderImpl(OrcProto.RowIndexEntry entry, int startPos) { + this.entry = entry; + this.index = startPos; + } + + @Override + public long getNext() { + return entry.getPositions(index++); + } + } + + /** An orc PositionProvider impl. */ + public static final class ZeroPositionProvider implements PositionProvider { + @Override + public long getNext() { + return 0; + } + } + + public OrcProto.StripeFooter readStripeFooter(StripeInformation stripe) throws IOException { + return dataReader.readStripeFooter(stripe); + } + + enum Location { + BEFORE, + MIN, + MIDDLE, + MAX, + AFTER + } + + static class ValueRange { + final Comparable lower; + final Comparable upper; + final boolean onlyLowerBound; + final boolean onlyUpperBound; + final boolean hasNulls; + final boolean hasValue; + final boolean comparable; + + ValueRange( + PredicateLeaf predicate, + T lower, + T upper, + boolean hasNulls, + boolean onlyLowerBound, + boolean onlyUpperBound, + boolean hasValue, + boolean comparable) { + PredicateLeaf.Type type = predicate.getType(); + this.lower = getBaseObjectForComparison(type, lower); + this.upper = getBaseObjectForComparison(type, upper); + this.hasNulls = hasNulls; + this.onlyLowerBound = onlyLowerBound; + this.onlyUpperBound = onlyUpperBound; + this.hasValue = hasValue; + this.comparable = comparable; + } + + ValueRange( + PredicateLeaf predicate, + T lower, + T upper, + boolean hasNulls, + boolean onlyLowerBound, + boolean onlyUpperBound) { + this( + predicate, + lower, + upper, + hasNulls, + onlyLowerBound, + onlyUpperBound, + lower != null, + lower != null); + } + + ValueRange(PredicateLeaf predicate, T lower, T upper, boolean hasNulls) { + this(predicate, lower, upper, hasNulls, false, false); + } + + /** + * A value range where the data is either missing or all null. + * + * @param predicate the predicate to test + * @param hasNulls whether there are nulls + */ + ValueRange(PredicateLeaf predicate, boolean hasNulls) { + this(predicate, null, null, hasNulls, false, false); + } + + boolean hasValues() { + return hasValue; + } + + /** + * Whether min or max is provided for comparison. + * + * @return is it comparable + */ + boolean isComparable() { + return hasValue && comparable; + } + + /** + * value range is invalid if the column statistics are non-existent. + * + * @see ColumnStatisticsImpl#isStatsExists() this method is similar to isStatsExists + * @return value range is valid or not + */ + boolean isValid() { + return hasValue || hasNulls; + } + + /** + * Given a point and min and max, determine if the point is before, at the min, in the + * middle, at the max, or after the range. + * + * @param point the point to test + * @return the location of the point + */ + Location compare(Comparable point) { + int minCompare = point.compareTo(lower); + if (minCompare < 0) { + return Location.BEFORE; + } else if (minCompare == 0) { + return onlyLowerBound ? Location.BEFORE : Location.MIN; + } + int maxCompare = point.compareTo(upper); + if (maxCompare > 0) { + return Location.AFTER; + } else if (maxCompare == 0) { + return onlyUpperBound ? Location.AFTER : Location.MAX; + } + return Location.MIDDLE; + } + + /** + * Is this range a single point? + * + * @return true if min == max + */ + boolean isSingleton() { + return lower != null && !onlyLowerBound && !onlyUpperBound && lower.equals(upper); + } + + /** + * Add the null option to the truth value, if the range includes nulls. + * + * @param value the original truth value + * @return the truth value extended with null if appropriate + */ + SearchArgument.TruthValue addNull(SearchArgument.TruthValue value) { + if (hasNulls) { + switch (value) { + case YES: + return SearchArgument.TruthValue.YES_NULL; + case NO: + return SearchArgument.TruthValue.NO_NULL; + case YES_NO: + return SearchArgument.TruthValue.YES_NO_NULL; + default: + return value; + } + } else { + return value; + } + } + } + + /** + * Get the maximum value out of an index entry. Includes option to specify if timestamp column + * stats values should be in UTC. + * + * @param index the index entry + * @param predicate the kind of predicate + * @param useUTCTimestamp use UTC for timestamps + * @return the object for the maximum value or null if there isn't one + */ + static ValueRange getValueRange( + ColumnStatistics index, PredicateLeaf predicate, boolean useUTCTimestamp) { + if (index.getNumberOfValues() == 0) { + return new ValueRange<>(predicate, index.hasNull()); + } else if (index instanceof IntegerColumnStatistics) { + IntegerColumnStatistics stats = (IntegerColumnStatistics) index; + Long min = stats.getMinimum(); + Long max = stats.getMaximum(); + return new ValueRange<>(predicate, min, max, stats.hasNull()); + } else if (index instanceof CollectionColumnStatistics) { + CollectionColumnStatistics stats = (CollectionColumnStatistics) index; + Long min = stats.getMinimumChildren(); + Long max = stats.getMaximumChildren(); + return new ValueRange<>(predicate, min, max, stats.hasNull()); + } else if (index instanceof DoubleColumnStatistics) { + DoubleColumnStatistics stats = (DoubleColumnStatistics) index; + Double min = stats.getMinimum(); + Double max = stats.getMaximum(); + return new ValueRange<>(predicate, min, max, stats.hasNull()); + } else if (index instanceof StringColumnStatistics) { + StringColumnStatistics stats = (StringColumnStatistics) index; + return new ValueRange<>( + predicate, + stats.getLowerBound(), + stats.getUpperBound(), + stats.hasNull(), + stats.getMinimum() == null, + stats.getMaximum() == null); + } else if (index instanceof DateColumnStatistics) { + DateColumnStatistics stats = (DateColumnStatistics) index; + ChronoLocalDate min = stats.getMinimumLocalDate(); + ChronoLocalDate max = stats.getMaximumLocalDate(); + return new ValueRange<>(predicate, min, max, stats.hasNull()); + } else if (index instanceof DecimalColumnStatistics) { + DecimalColumnStatistics stats = (DecimalColumnStatistics) index; + HiveDecimal min = stats.getMinimum(); + HiveDecimal max = stats.getMaximum(); + return new ValueRange<>(predicate, min, max, stats.hasNull()); + } else if (index instanceof TimestampColumnStatistics) { + TimestampColumnStatistics stats = (TimestampColumnStatistics) index; + Timestamp min = useUTCTimestamp ? stats.getMinimumUTC() : stats.getMinimum(); + Timestamp max = useUTCTimestamp ? stats.getMaximumUTC() : stats.getMaximum(); + return new ValueRange<>(predicate, min, max, stats.hasNull()); + } else if (index instanceof BooleanColumnStatistics) { + BooleanColumnStatistics stats = (BooleanColumnStatistics) index; + Boolean min = stats.getFalseCount() == 0; + Boolean max = stats.getTrueCount() != 0; + return new ValueRange<>(predicate, min, max, stats.hasNull()); + } else { + return new ValueRange( + predicate, null, null, index.hasNull(), false, false, true, false); + } + } + + /** + * Evaluate a predicate with respect to the statistics from the column that is referenced in the + * predicate. + * + * @param statsProto the statistics for the column mentioned in the predicate + * @param predicate the leaf predicate we need to evaluation + * @param bloomFilter the bloom filter + * @param writerVersion the version of software that wrote the file + * @param type what is the kind of this column + * @return the set of truth values that may be returned for the given predicate. + */ + static SearchArgument.TruthValue evaluatePredicateProto( + OrcProto.ColumnStatistics statsProto, + PredicateLeaf predicate, + OrcProto.Stream.Kind kind, + OrcProto.ColumnEncoding encoding, + OrcProto.BloomFilter bloomFilter, + OrcFile.WriterVersion writerVersion, + TypeDescription type) { + return evaluatePredicateProto( + statsProto, + predicate, + kind, + encoding, + bloomFilter, + writerVersion, + type, + true, + false); + } + + /** + * Evaluate a predicate with respect to the statistics from the column that is referenced in the + * predicate. Includes option to specify if timestamp column stats values should be in UTC and + * if the file writer used proleptic Gregorian calendar. + * + * @param statsProto the statistics for the column mentioned in the predicate + * @param predicate the leaf predicate we need to evaluation + * @param bloomFilter the bloom filter + * @param writerVersion the version of software that wrote the file + * @param type what is the kind of this column + * @param writerUsedProlepticGregorian file written using the proleptic Gregorian calendar + * @param useUTCTimestamp + * @return the set of truth values that may be returned for the given predicate. + */ + static SearchArgument.TruthValue evaluatePredicateProto( + OrcProto.ColumnStatistics statsProto, + PredicateLeaf predicate, + OrcProto.Stream.Kind kind, + OrcProto.ColumnEncoding encoding, + OrcProto.BloomFilter bloomFilter, + OrcFile.WriterVersion writerVersion, + TypeDescription type, + boolean writerUsedProlepticGregorian, + boolean useUTCTimestamp) { + ColumnStatistics cs = + ColumnStatisticsImpl.deserialize( + null, statsProto, writerUsedProlepticGregorian, true); + ValueRange range = getValueRange(cs, predicate, useUTCTimestamp); + + // files written before ORC-135 stores timestamp wrt to local timezone causing issues with + // PPD. + // disable PPD for timestamp for all old files + TypeDescription.Category category = type.getCategory(); + if (category == TypeDescription.Category.TIMESTAMP) { + if (!writerVersion.includes(OrcFile.WriterVersion.ORC_135)) { + LOG.debug( + "Not using predication pushdown on {} because it doesn't " + + "include ORC-135. Writer version: {}", + predicate.getColumnName(), + writerVersion); + return range.addNull(SearchArgument.TruthValue.YES_NO); + } + if (predicate.getType() != PredicateLeaf.Type.TIMESTAMP + && predicate.getType() != PredicateLeaf.Type.DATE + && predicate.getType() != PredicateLeaf.Type.STRING) { + return range.addNull(SearchArgument.TruthValue.YES_NO); + } + } else if (writerVersion == OrcFile.WriterVersion.ORC_135 + && category == TypeDescription.Category.DECIMAL + && type.getPrecision() <= TypeDescription.MAX_DECIMAL64_PRECISION) { + // ORC 1.5.0 to 1.5.5, which use WriterVersion.ORC_135, have broken + // min and max values for decimal64. See ORC-517. + LOG.debug( + "Not using predicate push down on {}, because the file doesn't" + + " include ORC-517. Writer version: {}", + predicate.getColumnName(), + writerVersion); + return SearchArgument.TruthValue.YES_NO_NULL; + } else if ((category == TypeDescription.Category.DOUBLE + || category == TypeDescription.Category.FLOAT) + && cs instanceof DoubleColumnStatistics) { + DoubleColumnStatistics dstas = (DoubleColumnStatistics) cs; + if (Double.isNaN(dstas.getSum())) { + LOG.debug( + "Not using predication pushdown on {} because stats contain NaN values", + predicate.getColumnName()); + return dstas.hasNull() + ? SearchArgument.TruthValue.YES_NO_NULL + : SearchArgument.TruthValue.YES_NO; + } + } + return evaluatePredicateRange( + predicate, + range, + BloomFilterIO.deserialize( + kind, encoding, writerVersion, type.getCategory(), bloomFilter), + useUTCTimestamp); + } + + /** + * Evaluate a predicate with respect to the statistics from the column that is referenced in the + * predicate. + * + * @param stats the statistics for the column mentioned in the predicate + * @param predicate the leaf predicate we need to evaluation + * @return the set of truth values that may be returned for the given predicate. + */ + public static SearchArgument.TruthValue evaluatePredicate( + ColumnStatistics stats, PredicateLeaf predicate, BloomFilter bloomFilter) { + return evaluatePredicate(stats, predicate, bloomFilter, false); + } + + /** + * Evaluate a predicate with respect to the statistics from the column that is referenced in the + * predicate. Includes option to specify if timestamp column stats values should be in UTC. + * + * @param stats the statistics for the column mentioned in the predicate + * @param predicate the leaf predicate we need to evaluation + * @param bloomFilter + * @param useUTCTimestamp + * @return the set of truth values that may be returned for the given predicate. + */ + public static SearchArgument.TruthValue evaluatePredicate( + ColumnStatistics stats, + PredicateLeaf predicate, + BloomFilter bloomFilter, + boolean useUTCTimestamp) { + ValueRange range = getValueRange(stats, predicate, useUTCTimestamp); + + return evaluatePredicateRange(predicate, range, bloomFilter, useUTCTimestamp); + } + + static SearchArgument.TruthValue evaluatePredicateRange( + PredicateLeaf predicate, + ValueRange range, + BloomFilter bloomFilter, + boolean useUTCTimestamp) { + if (!range.isValid()) { + return SearchArgument.TruthValue.YES_NO_NULL; + } + + // if we didn't have any values, everything must have been null + if (!range.hasValues()) { + if (predicate.getOperator() == PredicateLeaf.Operator.IS_NULL) { + return SearchArgument.TruthValue.YES; + } else { + return SearchArgument.TruthValue.NULL; + } + } else if (!range.isComparable()) { + return range.hasNulls + ? SearchArgument.TruthValue.YES_NO_NULL + : SearchArgument.TruthValue.YES_NO; + } + + SearchArgument.TruthValue result; + Comparable baseObj = (Comparable) predicate.getLiteral(); + // Predicate object and stats objects are converted to the type of the predicate object. + Comparable predObj = getBaseObjectForComparison(predicate.getType(), baseObj); + + result = evaluatePredicateMinMax(predicate, predObj, range); + if (shouldEvaluateBloomFilter(predicate, result, bloomFilter)) { + return evaluatePredicateBloomFilter( + predicate, predObj, bloomFilter, range.hasNulls, useUTCTimestamp); + } else { + return result; + } + } + + private static boolean shouldEvaluateBloomFilter( + PredicateLeaf predicate, SearchArgument.TruthValue result, BloomFilter bloomFilter) { + // evaluate bloom filter only when + // 1) Bloom filter is available + // 2) Min/Max evaluation yield YES or MAYBE + // 3) Predicate is EQUALS or IN list + return bloomFilter != null + && result != SearchArgument.TruthValue.NO_NULL + && result != SearchArgument.TruthValue.NO + && (predicate.getOperator().equals(PredicateLeaf.Operator.EQUALS) + || predicate.getOperator().equals(PredicateLeaf.Operator.NULL_SAFE_EQUALS) + || predicate.getOperator().equals(PredicateLeaf.Operator.IN)); + } + + private static SearchArgument.TruthValue evaluatePredicateMinMax( + PredicateLeaf predicate, Comparable predObj, ValueRange range) { + Location loc; + + switch (predicate.getOperator()) { + case NULL_SAFE_EQUALS: + loc = range.compare(predObj); + if (loc == Location.BEFORE || loc == Location.AFTER) { + return SearchArgument.TruthValue.NO; + } else { + return SearchArgument.TruthValue.YES_NO; + } + case EQUALS: + loc = range.compare(predObj); + if (range.isSingleton() && loc == Location.MIN) { + return range.addNull(SearchArgument.TruthValue.YES); + } else if (loc == Location.BEFORE || loc == Location.AFTER) { + return range.addNull(SearchArgument.TruthValue.NO); + } else { + return range.addNull(SearchArgument.TruthValue.YES_NO); + } + case LESS_THAN: + loc = range.compare(predObj); + if (loc == Location.AFTER) { + return range.addNull(SearchArgument.TruthValue.YES); + } else if (loc == Location.BEFORE || loc == Location.MIN) { + return range.addNull(SearchArgument.TruthValue.NO); + } else { + return range.addNull(SearchArgument.TruthValue.YES_NO); + } + case LESS_THAN_EQUALS: + loc = range.compare(predObj); + if (loc == Location.AFTER + || loc == Location.MAX + || (loc == Location.MIN && range.isSingleton())) { + return range.addNull(SearchArgument.TruthValue.YES); + } else if (loc == Location.BEFORE) { + return range.addNull(SearchArgument.TruthValue.NO); + } else { + return range.addNull(SearchArgument.TruthValue.YES_NO); + } + case IN: + if (range.isSingleton()) { + // for a single value, look through to see if that value is in the + // set + for (Object arg : predicate.getLiteralList()) { + predObj = getBaseObjectForComparison(predicate.getType(), (Comparable) arg); + if (range.compare(predObj) == Location.MIN) { + return range.addNull(SearchArgument.TruthValue.YES); + } + } + return range.addNull(SearchArgument.TruthValue.NO); + } else { + // are all of the values outside of the range? + for (Object arg : predicate.getLiteralList()) { + predObj = getBaseObjectForComparison(predicate.getType(), (Comparable) arg); + loc = range.compare(predObj); + if (loc == Location.MIN || loc == Location.MIDDLE || loc == Location.MAX) { + return range.addNull(SearchArgument.TruthValue.YES_NO); + } + } + return range.addNull(SearchArgument.TruthValue.NO); + } + case BETWEEN: + List args = predicate.getLiteralList(); + if (args == null || args.isEmpty()) { + return range.addNull(SearchArgument.TruthValue.YES_NO); + } + Comparable predObj1 = + getBaseObjectForComparison(predicate.getType(), (Comparable) args.get(0)); + + loc = range.compare(predObj1); + if (loc == Location.BEFORE || loc == Location.MIN) { + Comparable predObj2 = + getBaseObjectForComparison( + predicate.getType(), (Comparable) args.get(1)); + Location loc2 = range.compare(predObj2); + if (loc2 == Location.AFTER || loc2 == Location.MAX) { + return range.addNull(SearchArgument.TruthValue.YES); + } else if (loc2 == Location.BEFORE) { + return range.addNull(SearchArgument.TruthValue.NO); + } else { + return range.addNull(SearchArgument.TruthValue.YES_NO); + } + } else if (loc == Location.AFTER) { + return range.addNull(SearchArgument.TruthValue.NO); + } else { + return range.addNull(SearchArgument.TruthValue.YES_NO); + } + case IS_NULL: + // min = null condition above handles the all-nulls YES case + return range.hasNulls + ? SearchArgument.TruthValue.YES_NO + : SearchArgument.TruthValue.NO; + default: + return range.addNull(SearchArgument.TruthValue.YES_NO); + } + } + + private static SearchArgument.TruthValue evaluatePredicateBloomFilter( + PredicateLeaf predicate, + final Object predObj, + BloomFilter bloomFilter, + boolean hasNull, + boolean useUTCTimestamp) { + switch (predicate.getOperator()) { + case NULL_SAFE_EQUALS: + // null safe equals does not return *_NULL variant. So set hasNull to false + return checkInBloomFilter(bloomFilter, predObj, false, useUTCTimestamp); + case EQUALS: + return checkInBloomFilter(bloomFilter, predObj, hasNull, useUTCTimestamp); + case IN: + for (Object arg : predicate.getLiteralList()) { + // if atleast one value in IN list exist in bloom filter, qualify the row + // group/stripe + Object predObjItem = + getBaseObjectForComparison(predicate.getType(), (Comparable) arg); + SearchArgument.TruthValue result = + checkInBloomFilter(bloomFilter, predObjItem, hasNull, useUTCTimestamp); + if (result == SearchArgument.TruthValue.YES_NO_NULL + || result == SearchArgument.TruthValue.YES_NO) { + return result; + } + } + return hasNull ? SearchArgument.TruthValue.NO_NULL : SearchArgument.TruthValue.NO; + default: + return hasNull + ? SearchArgument.TruthValue.YES_NO_NULL + : SearchArgument.TruthValue.YES_NO; + } + } + + private static SearchArgument.TruthValue checkInBloomFilter( + BloomFilter bf, Object predObj, boolean hasNull, boolean useUTCTimestamp) { + SearchArgument.TruthValue result = + hasNull ? SearchArgument.TruthValue.NO_NULL : SearchArgument.TruthValue.NO; + + if (predObj instanceof Long) { + if (bf.testLong((Long) predObj)) { + result = SearchArgument.TruthValue.YES_NO_NULL; + } + } else if (predObj instanceof Double) { + if (bf.testDouble((Double) predObj)) { + result = SearchArgument.TruthValue.YES_NO_NULL; + } + } else if (predObj instanceof String + || predObj instanceof Text + || predObj instanceof HiveDecimalWritable + || predObj instanceof BigDecimal) { + if (bf.testString(predObj.toString())) { + result = SearchArgument.TruthValue.YES_NO_NULL; + } + } else if (predObj instanceof Timestamp) { + if (useUTCTimestamp) { + if (bf.testLong(((Timestamp) predObj).getTime())) { + result = SearchArgument.TruthValue.YES_NO_NULL; + } + } else { + if (bf.testLong( + SerializationUtils.convertToUtc( + TimeZone.getDefault(), ((Timestamp) predObj).getTime()))) { + result = SearchArgument.TruthValue.YES_NO_NULL; + } + } + } else if (predObj instanceof ChronoLocalDate) { + if (bf.testLong(((ChronoLocalDate) predObj).toEpochDay())) { + result = SearchArgument.TruthValue.YES_NO_NULL; + } + } else { + // if the predicate object is null and if hasNull says there are no nulls then return NO + if (predObj == null && !hasNull) { + result = SearchArgument.TruthValue.NO; + } else { + result = SearchArgument.TruthValue.YES_NO_NULL; + } + } + + if (result == SearchArgument.TruthValue.YES_NO_NULL && !hasNull) { + result = SearchArgument.TruthValue.YES_NO; + } + + LOG.debug("Bloom filter evaluation: {}", result); + + return result; + } + + /** An exception for when we can't cast things appropriately. */ + static class SargCastException extends IllegalArgumentException { + + SargCastException(String string) { + super(string); + } + } + + private static Comparable getBaseObjectForComparison(PredicateLeaf.Type type, Comparable obj) { + if (obj == null) { + return null; + } + switch (type) { + case BOOLEAN: + if (obj instanceof Boolean) { + return obj; + } else { + // will only be true if the string conversion yields "true", all other values + // are + // considered false + return Boolean.valueOf(obj.toString()); + } + case DATE: + if (obj instanceof ChronoLocalDate) { + return obj; + } else if (obj instanceof java.sql.Date) { + return ((java.sql.Date) obj).toLocalDate(); + } else if (obj instanceof Date) { + return LocalDateTime.ofInstant(((Date) obj).toInstant(), ZoneOffset.UTC) + .toLocalDate(); + } else if (obj instanceof String) { + return LocalDate.parse((String) obj); + } else if (obj instanceof Timestamp) { + return ((Timestamp) obj).toLocalDateTime().toLocalDate(); + } + // always string, but prevent the comparison to numbers (are they + // days/seconds/milliseconds?) + break; + case DECIMAL: + if (obj instanceof Boolean) { + return new HiveDecimalWritable( + (Boolean) obj ? HiveDecimal.ONE : HiveDecimal.ZERO); + } else if (obj instanceof Integer) { + return new HiveDecimalWritable((Integer) obj); + } else if (obj instanceof Long) { + return new HiveDecimalWritable(((Long) obj)); + } else if (obj instanceof Float || obj instanceof Double || obj instanceof String) { + return new HiveDecimalWritable(obj.toString()); + } else if (obj instanceof BigDecimal) { + return new HiveDecimalWritable(HiveDecimal.create((BigDecimal) obj)); + } else if (obj instanceof HiveDecimal) { + return new HiveDecimalWritable((HiveDecimal) obj); + } else if (obj instanceof HiveDecimalWritable) { + return obj; + } else if (obj instanceof Timestamp) { + return new HiveDecimalWritable( + Double.toString(TimestampUtils.getDouble((Timestamp) obj))); + } + break; + case FLOAT: + if (obj instanceof Number) { + // widening conversion + return ((Number) obj).doubleValue(); + } else if (obj instanceof HiveDecimal) { + return ((HiveDecimal) obj).doubleValue(); + } else if (obj instanceof String) { + return Double.valueOf(obj.toString()); + } else if (obj instanceof Timestamp) { + return TimestampUtils.getDouble((Timestamp) obj); + } + break; + case LONG: + if (obj instanceof Number) { + // widening conversion + return ((Number) obj).longValue(); + } else if (obj instanceof HiveDecimal) { + return ((HiveDecimal) obj).longValue(); + } else if (obj instanceof String) { + return Long.valueOf(obj.toString()); + } + break; + case STRING: + if (obj instanceof ChronoLocalDate) { + ChronoLocalDate date = (ChronoLocalDate) obj; + return date.format( + DateTimeFormatter.ISO_LOCAL_DATE.withChronology(date.getChronology())); + } + return (obj.toString()); + case TIMESTAMP: + if (obj instanceof Timestamp) { + return obj; + } else if (obj instanceof Integer) { + return new Timestamp(((Number) obj).longValue()); + } else if (obj instanceof Float) { + return TimestampUtils.doubleToTimestamp(((Float) obj).doubleValue()); + } else if (obj instanceof Double) { + return TimestampUtils.doubleToTimestamp((Double) obj); + } else if (obj instanceof HiveDecimal) { + return TimestampUtils.decimalToTimestamp((HiveDecimal) obj); + } else if (obj instanceof HiveDecimalWritable) { + return TimestampUtils.decimalToTimestamp( + ((HiveDecimalWritable) obj).getHiveDecimal()); + } else if (obj instanceof Date) { + return new Timestamp(((Date) obj).getTime()); + } else if (obj instanceof ChronoLocalDate) { + return new Timestamp( + ((ChronoLocalDate) obj) + .atTime(LocalTime.MIDNIGHT) + .toInstant(ZoneOffset.UTC) + .getEpochSecond() + * 1000L); + } + // float/double conversion to timestamp is interpreted as seconds whereas integer + // conversion + // to timestamp is interpreted as milliseconds by default. The integer to timestamp + // casting + // is also config driven. The filter operator changes its promotion based on config: + // "int.timestamp.conversion.in.seconds". Disable PPD for integer cases. + break; + default: + break; + } + + throw new SargCastException( + String.format( + "ORC SARGS could not convert from %s to %s", + obj.getClass().getSimpleName(), type)); + } + + /** search argument applier. */ + public static class SargApplier { + public static final boolean[] READ_ALL_RGS = null; + public static final boolean[] READ_NO_RGS = new boolean[0]; + + private final OrcFile.WriterVersion writerVersion; + private final SearchArgument sarg; + private final List sargLeaves; + private final int[] filterColumns; + private final long rowIndexStride; + // same as the above array, but indices are set to true + private final SchemaEvolution evolution; + private final long[] exceptionCount; + private final boolean useUTCTimestamp; + private final boolean writerUsedProlepticGregorian; + private final boolean convertToProlepticGregorian; + + /** + * @deprecated Use the constructor having full parameters. This exists for backward + * compatibility. + */ + public SargApplier( + SearchArgument sarg, + long rowIndexStride, + SchemaEvolution evolution, + OrcFile.WriterVersion writerVersion, + boolean useUTCTimestamp) { + this(sarg, rowIndexStride, evolution, writerVersion, useUTCTimestamp, false, false); + } + + public SargApplier( + SearchArgument sarg, + long rowIndexStride, + SchemaEvolution evolution, + OrcFile.WriterVersion writerVersion, + boolean useUTCTimestamp, + boolean writerUsedProlepticGregorian, + boolean convertToProlepticGregorian) { + this.writerVersion = writerVersion; + this.sarg = sarg; + sargLeaves = sarg.getLeaves(); + this.writerUsedProlepticGregorian = writerUsedProlepticGregorian; + this.convertToProlepticGregorian = convertToProlepticGregorian; + filterColumns = mapSargColumnsToOrcInternalColIdx(sargLeaves, evolution); + this.rowIndexStride = rowIndexStride; + this.evolution = evolution; + exceptionCount = new long[sargLeaves.size()]; + this.useUTCTimestamp = useUTCTimestamp; + } + + public void setRowIndexCols(boolean[] rowIndexCols) { + // included will not be null, row options will fill the array with + // trues if null + for (int i : filterColumns) { + // filter columns may have -1 as index which could be partition + // column in SARG. + if (i > 0) { + rowIndexCols[i] = true; + } + } + } + + /** + * Pick the row groups that we need to load from the current stripe. + * + * @return an array with a boolean for each row group or null if all of the row groups must + * be read. + * @throws IOException + */ + public boolean[] pickRowGroups( + StripeInformation stripe, + OrcProto.RowIndex[] indexes, + OrcProto.Stream.Kind[] bloomFilterKinds, + List encodings, + OrcProto.BloomFilterIndex[] bloomFilterIndices, + boolean returnNone, + long rowBaseInStripe, + FileIndexResult fileIndexResult) + throws IOException { + long rowsInStripe = stripe.getNumberOfRows(); + int groupsInStripe = (int) ((rowsInStripe + rowIndexStride - 1) / rowIndexStride); + boolean[] result = new boolean[groupsInStripe]; // TODO: avoid alloc? + SearchArgument.TruthValue[] leafValues = + new SearchArgument.TruthValue[sargLeaves.size()]; + boolean hasSelected = false; + boolean hasSkipped = false; + SearchArgument.TruthValue[] exceptionAnswer = + new SearchArgument.TruthValue[leafValues.length]; + RoaringBitmap32 bitmap = null; + if (fileIndexResult instanceof BitmapIndexResult) { + bitmap = ((BitmapIndexResult) fileIndexResult).get(); + } + for (int rowGroup = 0; rowGroup < result.length; ++rowGroup) { + for (int pred = 0; pred < leafValues.length; ++pred) { + int columnIx = filterColumns[pred]; + if (columnIx == -1) { + // the column is a virtual column + leafValues[pred] = SearchArgument.TruthValue.YES_NO_NULL; + } else if (exceptionAnswer[pred] != null) { + leafValues[pred] = exceptionAnswer[pred]; + } else { + if (indexes[columnIx] == null) { + LOG.warn("Index is not populated for " + columnIx); + return READ_ALL_RGS; + } + OrcProto.RowIndexEntry entry = indexes[columnIx].getEntry(rowGroup); + if (entry == null) { + throw new AssertionError( + "RG is not populated for " + columnIx + " rg " + rowGroup); + } + OrcProto.ColumnStatistics stats = EMPTY_COLUMN_STATISTICS; + if (entry.hasStatistics()) { + stats = entry.getStatistics(); + } + OrcProto.BloomFilter bf = null; + OrcProto.Stream.Kind bfk = null; + if (bloomFilterIndices != null && bloomFilterIndices[columnIx] != null) { + bfk = bloomFilterKinds[columnIx]; + bf = bloomFilterIndices[columnIx].getBloomFilter(rowGroup); + } + if (evolution != null && evolution.isPPDSafeConversion(columnIx)) { + PredicateLeaf predicate = sargLeaves.get(pred); + try { + leafValues[pred] = + evaluatePredicateProto( + stats, + predicate, + bfk, + encodings.get(columnIx), + bf, + writerVersion, + evolution.getFileSchema().findSubtype(columnIx), + writerUsedProlepticGregorian, + useUTCTimestamp); + } catch (Exception e) { + exceptionCount[pred] += 1; + if (e instanceof SargCastException) { + LOG.info( + "Skipping ORC PPD - " + + e.getMessage() + + " on " + + predicate); + } else { + final String reason = + e.getClass().getSimpleName() + + " when evaluating predicate." + + " Skipping ORC PPD." + + " Stats: " + + stats + + " Predicate: " + + predicate; + LOG.warn(reason, e); + } + boolean hasNoNull = stats.hasHasNull() && !stats.getHasNull(); + if (predicate + .getOperator() + .equals(PredicateLeaf.Operator.NULL_SAFE_EQUALS) + || hasNoNull) { + exceptionAnswer[pred] = SearchArgument.TruthValue.YES_NO; + } else { + exceptionAnswer[pred] = SearchArgument.TruthValue.YES_NO_NULL; + } + leafValues[pred] = exceptionAnswer[pred]; + } + } else { + leafValues[pred] = SearchArgument.TruthValue.YES_NO_NULL; + } + if (LOG.isTraceEnabled()) { + LOG.trace("Stats = " + stats); + LOG.trace( + "Setting " + sargLeaves.get(pred) + " to " + leafValues[pred]); + } + } + } + result[rowGroup] = sarg.evaluate(leafValues).isNeeded(); + if (bitmap != null) { + long firstRow = rowBaseInStripe + rowIndexStride * rowGroup; + long lastRow = Math.min(firstRow + rowIndexStride, firstRow + rowsInStripe); + result[rowGroup] &= bitmap.rangeCardinality(firstRow, lastRow) > 0; + } + hasSelected = hasSelected || result[rowGroup]; + hasSkipped = hasSkipped || (!result[rowGroup]); + if (LOG.isDebugEnabled()) { + LOG.debug( + "Row group " + + (rowIndexStride * rowGroup) + + " to " + + (rowIndexStride * (rowGroup + 1) - 1) + + " is " + + (result[rowGroup] ? "" : "not ") + + "included."); + } + } + + return hasSkipped + ? ((hasSelected || !returnNone) ? result : READ_NO_RGS) + : READ_ALL_RGS; + } + + /** + * Get the count of exceptions for testing. + * + * @return + */ + long[] getExceptionCount() { + return exceptionCount; + } + } + + /** + * Pick the row groups that we need to load from the current stripe. + * + * @return an array with a boolean for each row group or null if all of the row groups must be + * read. + * @throws IOException + */ + protected boolean[] pickRowGroups() throws IOException { + // Read the Row Indicies if required + if (rowIndexColsToRead != null) { + readCurrentStripeRowIndex(); + } + + // In the absence of SArg all rows groups should be included + if (sargApp == null) { + return null; + } + return sargApp.pickRowGroups( + stripes.get(currentStripe), + indexes.getRowGroupIndex(), + skipBloomFilters ? null : indexes.getBloomFilterKinds(), + stripeFooter.getColumnsList(), + skipBloomFilters ? null : indexes.getBloomFilterIndex(), + false, + rowBaseInStripe, + fileIndexResult); + } + + private void clearStreams() { + planner.clearStreams(); + } + + /** + * Read the current stripe into memory. + * + * @throws IOException + */ + private void readStripe() throws IOException { + StripeInformation stripe = beginReadStripe(); + planner.parseStripe(stripe, fileIncluded); + includedRowGroups = pickRowGroups(); + + // move forward to the first unskipped row + if (includedRowGroups != null) { + while (rowInStripe < rowCountInStripe + && !includedRowGroups[(int) (rowInStripe / rowIndexStride)]) { + rowInStripe = Math.min(rowCountInStripe, rowInStripe + rowIndexStride); + } + } + + // if we haven't skipped the whole stripe, read the data + if (rowInStripe < rowCountInStripe) { + planner.readData(indexes, includedRowGroups, false, startReadPhase); + reader.startStripe(planner, startReadPhase); + needsFollowColumnsRead = true; + // if we skipped the first row group, move the pointers forward + if (rowInStripe != 0) { + seekToRowEntry(reader, (int) (rowInStripe / rowIndexStride), startReadPhase); + } + } + } + + private StripeInformation beginReadStripe() throws IOException { + StripeInformation stripe = stripes.get(currentStripe); + stripeFooter = readStripeFooter(stripe); + clearStreams(); + // setup the position in the stripe + rowCountInStripe = stripe.getNumberOfRows(); + rowInStripe = 0; + followRowInStripe = 0; + rowBaseInStripe = 0; + for (int i = 0; i < currentStripe; ++i) { + rowBaseInStripe += stripes.get(i).getNumberOfRows(); + } + // reset all of the indexes + OrcProto.RowIndex[] rowIndex = indexes.getRowGroupIndex(); + for (int i = 0; i < rowIndex.length; ++i) { + rowIndex[i] = null; + } + return stripe; + } + + /** + * Read the next stripe until we find a row that we don't skip. + * + * @throws IOException + */ + private void advanceStripe() throws IOException { + rowInStripe = rowCountInStripe; + while (rowInStripe >= rowCountInStripe && currentStripe < stripes.size() - 1) { + currentStripe += 1; + readStripe(); + } + } + + /** + * Determine the RowGroup based on the supplied row id. + * + * @param rowIdx Row for which the row group is being determined + * @return Id of the RowGroup that the row belongs to + */ + private int computeRGIdx(long rowIdx) { + return rowIndexStride == 0 ? 0 : (int) (rowIdx / rowIndexStride); + } + + /** + * Skip over rows that we aren't selecting, so that the next row is one that we will read. + * + * @param nextRow the row we want to go to + * @throws IOException + */ + private boolean advanceToNextRow(BatchReader reader, long nextRow, boolean canAdvanceStripe) + throws IOException { + long nextRowInStripe = nextRow - rowBaseInStripe; + // check for row skipping + if (rowIndexStride != 0 + && includedRowGroups != null + && nextRowInStripe < rowCountInStripe) { + int rowGroup = computeRGIdx(nextRowInStripe); + if (!includedRowGroups[rowGroup]) { + while (rowGroup < includedRowGroups.length && !includedRowGroups[rowGroup]) { + rowGroup += 1; + } + if (rowGroup >= includedRowGroups.length) { + if (canAdvanceStripe) { + advanceStripe(); + } + return canAdvanceStripe; + } + nextRowInStripe = Math.min(rowCountInStripe, rowGroup * rowIndexStride); + } + } + if (nextRowInStripe >= rowCountInStripe) { + if (canAdvanceStripe) { + advanceStripe(); + } + return canAdvanceStripe; + } + if (nextRowInStripe != rowInStripe) { + if (rowIndexStride != 0) { + int rowGroup = (int) (nextRowInStripe / rowIndexStride); + seekToRowEntry(reader, rowGroup, startReadPhase); + reader.skipRows(nextRowInStripe - rowGroup * rowIndexStride, startReadPhase); + } else { + reader.skipRows(nextRowInStripe - rowInStripe, startReadPhase); + } + rowInStripe = nextRowInStripe; + } + return true; + } + + @Override + public boolean nextBatch(VectorizedRowBatch batch) throws IOException { + try { + int batchSize; + // do...while is required to handle the case where the filter eliminates all rows in the + // batch, we never return an empty batch unless the file is exhausted + do { + if (rowInStripe >= rowCountInStripe) { + currentStripe += 1; + if (currentStripe >= stripes.size()) { + batch.size = 0; + return false; + } + // Read stripe in Memory + readStripe(); + followRowInStripe = rowInStripe; + } + + batchSize = computeBatchSize(batch.getMaxSize()); + reader.setVectorColumnCount(batch.getDataColumnCount()); + reader.nextBatch(batch, batchSize, startReadPhase); + if (startReadPhase == TypeReader.ReadPhase.LEADERS && batch.size > 0) { + // At least 1 row has been selected and as a result we read the follow columns + // into the + // row batch + reader.nextBatch( + batch, batchSize, prepareFollowReaders(rowInStripe, followRowInStripe)); + followRowInStripe = rowInStripe + batchSize; + } + rowInStripe += batchSize; + advanceToNextRow(reader, rowInStripe + rowBaseInStripe, true); + // batch.size can be modified by filter so only batchSize can tell if we actually + // read rows + } while (batchSize != 0 && batch.size == 0); + + if (noSelectedVector) { + // In case selected vector is not supported we leave the size to be read size. In + // this case + // the non filter columns might be read selectively, however the filter after the + // reader + // should eliminate rows that don't match predicate conditions + batch.size = batchSize; + batch.selectedInUse = false; + } + + return batchSize != 0; + } catch (IOException e) { + // Rethrow exception with file name in log message + throw new IOException("Error reading file: " + path, e); + } + } + + /** + * This method prepares the non-filter column readers for next batch. This involves the + * following 1. Determine position 2. Perform IO if required 3. Position the non-filter readers + * + *

    This method is repositioning the non-filter columns and as such this method shall never + * have to deal with navigating the stripe forward or skipping row groups, all of this should + * have already taken place based on the filter columns. + * + * @param toFollowRow The rowIdx identifies the required row position within the stripe for + * follow read + * @param fromFollowRow Indicates the current position of the follow read, exclusive + * @return the read phase for reading non-filter columns, this shall be FOLLOWERS_AND_PARENTS in + * case of a seek otherwise will be FOLLOWERS + */ + private TypeReader.ReadPhase prepareFollowReaders(long toFollowRow, long fromFollowRow) + throws IOException { + // 1. Determine the required row group and skip rows needed from the RG start + int needRG = computeRGIdx(toFollowRow); + // The current row is not yet read so we -1 to compute the previously read row group + int readRG = computeRGIdx(fromFollowRow - 1); + long skipRows; + if (needRG == readRG && toFollowRow >= fromFollowRow) { + // In case we are skipping forward within the same row group, we compute skip rows from + // the + // current position + skipRows = toFollowRow - fromFollowRow; + } else { + // In all other cases including seeking backwards, we compute the skip rows from the + // start of + // the required row group + skipRows = toFollowRow - (needRG * rowIndexStride); + } + + // 2. Plan the row group idx for the non-filter columns if this has not already taken place + if (needsFollowColumnsRead) { + needsFollowColumnsRead = false; + planner.readFollowData(indexes, includedRowGroups, needRG, false); + reader.startStripe(planner, TypeReader.ReadPhase.FOLLOWERS); + } + + // 3. Position the non-filter readers to the required RG and skipRows + TypeReader.ReadPhase result = TypeReader.ReadPhase.FOLLOWERS; + if (needRG != readRG || toFollowRow < fromFollowRow) { + // When having to change a row group or in case of back navigation, seek both the filter + // parents and non-filter. This will re-position the parents present vector. This is + // needed + // to determine the number of non-null values to skip on the non-filter columns. + seekToRowEntry(reader, needRG, TypeReader.ReadPhase.FOLLOWERS_AND_PARENTS); + // skip rows on both the filter parents and non-filter as both have been positioned in + // the + // previous step + reader.skipRows(skipRows, TypeReader.ReadPhase.FOLLOWERS_AND_PARENTS); + result = TypeReader.ReadPhase.FOLLOWERS_AND_PARENTS; + } else if (skipRows > 0) { + // in case we are only skipping within the row group, position the filter parents back + // to the + // position of the follow. This is required to determine the non-null values to skip on + // the + // non-filter columns. + seekToRowEntry(reader, readRG, TypeReader.ReadPhase.LEADER_PARENTS); + reader.skipRows( + fromFollowRow - (readRG * rowIndexStride), TypeReader.ReadPhase.LEADER_PARENTS); + // Move both the filter parents and non-filter forward, this will compute the correct + // non-null skips on follow children + reader.skipRows(skipRows, TypeReader.ReadPhase.FOLLOWERS_AND_PARENTS); + result = TypeReader.ReadPhase.FOLLOWERS_AND_PARENTS; + } + // Identifies the read level that should be performed for the read + // FOLLOWERS_WITH_PARENTS indicates repositioning identifying both non-filter and filter + // parents + // FOLLOWERS indicates read only of the non-filter level without the parents, which is used + // during + // contiguous read. During a contiguous read no skips are needed and the non-null + // information of + // the parent is available in the column vector for use during non-filter read + return result; + } + + private int computeBatchSize(long targetBatchSize) { + final int batchSize; + // In case of PPD, batch size should be aware of row group boundaries. If only a subset of + // row + // groups are selected then marker position is set to the end of range (subset of row groups + // within strip). Batch size computed out of marker position makes sure that batch size is + // aware of row group boundary and will not cause overflow when reading rows + // illustration of this case is here https://issues.apache.org/jira/browse/HIVE-6287 + if (rowIndexStride != 0 + && (includedRowGroups != null || startReadPhase != TypeReader.ReadPhase.ALL) + && rowInStripe < rowCountInStripe) { + int startRowGroup = (int) (rowInStripe / rowIndexStride); + if (includedRowGroups != null && !includedRowGroups[startRowGroup]) { + while (startRowGroup < includedRowGroups.length + && !includedRowGroups[startRowGroup]) { + startRowGroup += 1; + } + } + + int endRowGroup = startRowGroup; + // We force row group boundaries when dealing with filters. We adjust the end row group + // to + // be the next row group even if more than one are possible selections. + if (includedRowGroups != null && startReadPhase == TypeReader.ReadPhase.ALL) { + while (endRowGroup < includedRowGroups.length && includedRowGroups[endRowGroup]) { + endRowGroup += 1; + } + } else { + endRowGroup += 1; + } + + final long markerPosition = Math.min((endRowGroup * rowIndexStride), rowCountInStripe); + batchSize = (int) Math.min(targetBatchSize, (markerPosition - rowInStripe)); + + if (isLogDebugEnabled && batchSize < targetBatchSize) { + LOG.debug("markerPosition: " + markerPosition + " batchSize: " + batchSize); + } + } else { + batchSize = (int) Math.min(targetBatchSize, (rowCountInStripe - rowInStripe)); + } + return batchSize; + } + + @Override + public void close() throws IOException { + clearStreams(); + dataReader.close(); + } + + @Override + public long getRowNumber() { + return rowInStripe + rowBaseInStripe + firstRow; + } + + /** + * Return the fraction of rows that have been read from the selected. section of the file + * + * @return fraction between 0.0 and 1.0 of rows consumed + */ + @Override + public float getProgress() { + return ((float) rowBaseInStripe + rowInStripe) / totalRowCount; + } + + private int findStripe(long rowNumber) { + for (int i = 0; i < stripes.size(); i++) { + StripeInformation stripe = stripes.get(i); + if (stripe.getNumberOfRows() > rowNumber) { + return i; + } + rowNumber -= stripe.getNumberOfRows(); + } + throw new IllegalArgumentException("Seek after the end of reader range"); + } + + private void readCurrentStripeRowIndex() throws IOException { + planner.readRowIndex(rowIndexColsToRead, indexes); + } + + public OrcIndex readRowIndex(int stripeIndex, boolean[] included, boolean[] readCols) + throws IOException { + // Use the cached objects if the read request matches the cached request + if (stripeIndex == currentStripe + && (readCols == null || Arrays.equals(readCols, rowIndexColsToRead))) { + if (rowIndexColsToRead != null) { + return indexes; + } else { + return planner.readRowIndex(readCols, indexes); + } + } else { + StripePlanner copy = new StripePlanner(planner); + if (included == null) { + included = new boolean[schema.getMaximumId() + 1]; + Arrays.fill(included, true); + } + copy.parseStripe(stripes.get(stripeIndex), included); + return copy.readRowIndex(readCols, null); + } + } + + private void seekToRowEntry(BatchReader reader, int rowEntry, TypeReader.ReadPhase readPhase) + throws IOException { + OrcProto.RowIndex[] rowIndices = indexes.getRowGroupIndex(); + PositionProvider[] index = new PositionProvider[rowIndices.length]; + for (int i = 0; i < index.length; ++i) { + if (rowIndices[i] != null) { + OrcProto.RowIndexEntry entry = rowIndices[i].getEntry(rowEntry); + // This is effectively a test for pre-ORC-569 files. + if (rowEntry == 0 && entry.getPositionsCount() == 0) { + index[i] = new ZeroPositionProvider(); + } else { + index[i] = new PositionProviderImpl(entry); + } + } + } + reader.seek(index, readPhase); + } + + @Override + public void seekToRow(long rowNumber) throws IOException { + if (rowNumber < 0) { + throw new IllegalArgumentException("Seek to a negative row number " + rowNumber); + } else if (rowNumber < firstRow) { + throw new IllegalArgumentException("Seek before reader range " + rowNumber); + } + // convert to our internal form (rows from the beginning of slice) + rowNumber -= firstRow; + + // move to the right stripe + int rightStripe = findStripe(rowNumber); + if (rightStripe != currentStripe) { + currentStripe = rightStripe; + readStripe(); + } + if (rowIndexColsToRead == null) { + // Read the row indexes only if they were not already read as part of readStripe() + readCurrentStripeRowIndex(); + } + + // if we aren't to the right row yet, advance in the stripe. + advanceToNextRow(reader, rowNumber, true); + } + + private static final String TRANSLATED_SARG_SEPARATOR = "_"; + + public static String encodeTranslatedSargColumn(int rootColumn, Integer indexInSourceTable) { + return rootColumn + + TRANSLATED_SARG_SEPARATOR + + ((indexInSourceTable == null) ? -1 : indexInSourceTable); + } + + public static int[] mapTranslatedSargColumns( + List types, List sargLeaves) { + int[] result = new int[sargLeaves.size()]; + OrcProto.Type lastRoot = null; // Root will be the same for everyone as of now. + String lastRootStr = null; + for (int i = 0; i < result.length; ++i) { + String[] rootAndIndex = + sargLeaves.get(i).getColumnName().split(TRANSLATED_SARG_SEPARATOR); + assert rootAndIndex.length == 2; + String rootStr = rootAndIndex[0], indexStr = rootAndIndex[1]; + int index = Integer.parseInt(indexStr); + // First, check if the column even maps to anything. + if (index == -1) { + result[i] = -1; + continue; + } + assert index >= 0; + // Then, find the root type if needed. + if (!rootStr.equals(lastRootStr)) { + lastRoot = types.get(Integer.parseInt(rootStr)); + lastRootStr = rootStr; + } + // Subtypes of the root types correspond, in order, to the columns in the table schema + // (disregarding schema evolution that doesn't presently work). Get the index for the + // corresponding subtype. + result[i] = lastRoot.getSubtypes(index); + } + return result; + } + + public CompressionCodec getCompressionCodec() { + return dataReader.getCompressionOptions().getCodec(); + } + + public int getMaxDiskRangeChunkLimit() { + return maxDiskRangeChunkLimit; + } + + /** + * Get sargApplier for testing. + * + * @return sargApplier in record reader. + */ + SargApplier getSargApp() { + return sargApp; + } +} diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroFileFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroFileFormat.java index dec7d4f447f8..63a51c0a13a9 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroFileFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroFileFormat.java @@ -53,20 +53,23 @@ public class AvroFileFormat extends FileFormat { public static final String IDENTIFIER = "avro"; - public static final ConfigOption AVRO_OUTPUT_CODEC = - ConfigOptions.key("codec") + private static final ConfigOption AVRO_OUTPUT_CODEC = + ConfigOptions.key("avro.codec") .stringType() .defaultValue(SNAPPY_CODEC) .withDescription("The compression codec for avro"); - public static final ConfigOption> AVRO_ROW_NAME_MAPPING = - ConfigOptions.key("row-name-mapping").mapType().defaultValue(new HashMap<>()); + private static final ConfigOption> AVRO_ROW_NAME_MAPPING = + ConfigOptions.key("avro.row-name-mapping").mapType().defaultValue(new HashMap<>()); - private final FormatContext context; + private final Options options; + private final int zstdLevel; public AvroFileFormat(FormatContext context) { super(IDENTIFIER); - this.context = context; + + this.options = getIdentifierPrefixOptions(context.options()); + this.zstdLevel = context.zstdLevel(); } @Override @@ -95,13 +98,12 @@ public void validateDataFields(RowType rowType) { } private CodecFactory createCodecFactory(String compression) { - Options options = context.formatOptions(); if (options.contains(AVRO_OUTPUT_CODEC)) { return CodecFactory.fromString(options.get(AVRO_OUTPUT_CODEC)); } if (compression.equalsIgnoreCase("zstd")) { - return CodecFactory.zstandardCodec(context.zstdLevel()); + return CodecFactory.zstandardCodec(zstdLevel); } return CodecFactory.fromString(options.get(AVRO_OUTPUT_CODEC)); } @@ -117,8 +119,7 @@ private RowAvroWriterFactory(RowType rowType) { (out, compression) -> { Schema schema = AvroSchemaConverter.convertToSchema( - rowType, - context.formatOptions().get(AVRO_ROW_NAME_MAPPING)); + rowType, options.get(AVRO_ROW_NAME_MAPPING)); AvroRowDatumWriter datumWriter = new AvroRowDatumWriter(rowType); DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java index e2e2c057e1a6..810a6ce59e0e 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcFileFormat.java @@ -84,7 +84,7 @@ public class OrcFileFormat extends FileFormat { public OrcFileFormat(FormatContext formatContext) { super(IDENTIFIER); - this.orcProperties = getOrcProperties(formatContext.formatOptions(), formatContext); + this.orcProperties = getOrcProperties(formatContext.options(), formatContext); Configuration conf; Configuration cachedConf = configCache.getIfPresent(orcProperties); if (cachedConf != null) { @@ -163,11 +163,9 @@ public FormatWriterFactory createWriterFactory(RowType type) { return new OrcWriterFactory(vectorizer, orcProperties, writerConf, writeBatchSize); } - private static Properties getOrcProperties(Options options, FormatContext formatContext) { + private Properties getOrcProperties(Options options, FormatContext formatContext) { Properties orcProperties = new Properties(); - Properties properties = new Properties(); - options.addAllToProperties(properties); - properties.forEach((k, v) -> orcProperties.put(IDENTIFIER + "." + k, v)); + orcProperties.putAll(getIdentifierPrefixOptions(options).toMap()); if (!orcProperties.containsKey(OrcConf.COMPRESSION_ZSTD_LEVEL.getAttribute())) { orcProperties.setProperty( diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java index 6b1590f03109..4b4c967f4de2 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcReaderFactory.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.columnar.ColumnarRow; import org.apache.paimon.data.columnar.ColumnarRowIterator; import org.apache.paimon.data.columnar.VectorizedColumnBatch; +import org.apache.paimon.fileindex.FileIndexResult; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.OrcFormatReaderContext; import org.apache.paimon.format.fs.HadoopReadOnlyFileSystem; @@ -45,6 +46,8 @@ import org.apache.orc.RecordReader; import org.apache.orc.StripeInformation; import org.apache.orc.TypeDescription; +import org.apache.orc.impl.ReaderImpl; +import org.apache.orc.impl.RecordReaderImpl; import javax.annotation.Nullable; @@ -104,7 +107,8 @@ public OrcVectorizedReader createReader(FormatReaderFactory.Context context) context.fileIO(), context.filePath(), 0, - context.fileSize()); + context.fileSize(), + context.fileIndex()); return new OrcVectorizedReader(orcReader, poolOfBatches); } @@ -253,9 +257,10 @@ private static RecordReader createRecordReader( FileIO fileIO, org.apache.paimon.fs.Path path, long splitStart, - long splitLength) + long splitLength, + FileIndexResult fileIndexResult) throws IOException { - org.apache.orc.Reader orcReader = createReader(conf, fileIO, path); + org.apache.orc.Reader orcReader = createReader(conf, fileIO, path, fileIndexResult); try { // get offset and length for the stripes that start in the split Pair offsetAndLength = @@ -335,7 +340,8 @@ private static Pair getOffsetAndLengthForSplit( public static org.apache.orc.Reader createReader( org.apache.hadoop.conf.Configuration conf, FileIO fileIO, - org.apache.paimon.fs.Path path) + org.apache.paimon.fs.Path path, + FileIndexResult fileIndexResult) throws IOException { // open ORC file and create reader org.apache.hadoop.fs.Path hPath = new org.apache.hadoop.fs.Path(path.toUri()); @@ -345,6 +351,11 @@ public static org.apache.orc.Reader createReader( // configure filesystem from Paimon FileIO readerOptions.filesystem(new HadoopReadOnlyFileSystem(fileIO)); - return OrcFile.createReader(hPath, readerOptions); + return new ReaderImpl(hPath, readerOptions) { + @Override + public RecordReader rows(Options options) throws IOException { + return new RecordReaderImpl(this, options, fileIndexResult); + } + }; } } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/filter/OrcSimpleStatsExtractor.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/filter/OrcSimpleStatsExtractor.java index dc6fd69dd0ed..e3c741cb8e36 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/filter/OrcSimpleStatsExtractor.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/filter/OrcSimpleStatsExtractor.java @@ -74,7 +74,8 @@ public SimpleColStats[] extract(FileIO fileIO, Path path) throws IOException { @Override public Pair extractWithFileInfo(FileIO fileIO, Path path) throws IOException { - try (Reader reader = OrcReaderFactory.createReader(new Configuration(), fileIO, path)) { + try (Reader reader = + OrcReaderFactory.createReader(new Configuration(), fileIO, path, null)) { long rowCount = reader.getNumberOfRows(); ColumnStatistics[] columnStatistics = reader.getStatistics(); TypeDescription schema = reader.getSchema(); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ColumnConfigParser.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ColumnConfigParser.java new file mode 100644 index 000000000000..a4e33807f6ee --- /dev/null +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ColumnConfigParser.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.format.parquet; + +import org.apache.hadoop.conf.Configuration; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Function; + +/** + * Parses the specified key-values in the format of root.key#column.path from a {@link + * Configuration} object. + * + *

    NOTE: The file was copied from Apache parquet project. + */ +public class ColumnConfigParser { + + private static class ConfigHelper { + private final String prefix; + private final Function function; + private final BiConsumer consumer; + + public ConfigHelper( + String prefix, Function function, BiConsumer consumer) { + this.prefix = prefix; + this.function = function; + this.consumer = consumer; + } + + public void processKey(String key) { + if (key.startsWith(prefix)) { + String columnPath = key.substring(prefix.length()); + T value = function.apply(key); + consumer.accept(columnPath, value); + } + } + } + + private final List> helpers = new ArrayList<>(); + + public ColumnConfigParser withColumnConfig( + String rootKey, Function function, BiConsumer consumer) { + helpers.add(new ConfigHelper(rootKey + '#', function, consumer)); + return this; + } + + public void parseConfig(Configuration conf) { + for (Map.Entry entry : conf) { + for (ConfigHelper helper : helpers) { + // We retrieve the value from function instead of parsing from the string here to + // use the exact + // implementations + // in Configuration + helper.processKey(entry.getKey()); + } + } + } +} diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetFileFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetFileFormat.java index c8ce5ccf5c9a..7c8bf6be3af2 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetFileFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetFileFormat.java @@ -42,32 +42,31 @@ /** Parquet {@link FileFormat}. */ public class ParquetFileFormat extends FileFormat { - private final FormatContext formatContext; + private final Options options; + private final int readBatchSize; public ParquetFileFormat(FormatContext formatContext) { super(IDENTIFIER); - this.formatContext = formatContext; + + this.options = getParquetConfiguration(formatContext); + this.readBatchSize = formatContext.readBatchSize(); } @VisibleForTesting - Options formatOptions() { - return formatContext.formatOptions(); + Options getOptions() { + return options; } @Override public FormatReaderFactory createReaderFactory( RowType projectedRowType, List filters) { return new ParquetReaderFactory( - getParquetConfiguration(formatContext), - projectedRowType, - formatContext.readBatchSize(), - ParquetFilters.convert(filters)); + options, projectedRowType, readBatchSize, ParquetFilters.convert(filters)); } @Override public FormatWriterFactory createWriterFactory(RowType type) { - return new ParquetWriterFactory( - new RowDataParquetBuilder(type, getParquetConfiguration(formatContext))); + return new ParquetWriterFactory(new RowDataParquetBuilder(type, options)); } @Override @@ -81,11 +80,8 @@ public Optional createStatsExtractor( return Optional.of(new ParquetSimpleStatsExtractor(type, statsCollectors)); } - public static Options getParquetConfiguration(FormatContext context) { - Options parquetOptions = new Options(); - context.formatOptions() - .toMap() - .forEach((key, value) -> parquetOptions.setString(IDENTIFIER + "." + key, value)); + private Options getParquetConfiguration(FormatContext context) { + Options parquetOptions = getIdentifierPrefixOptions(context.options()); if (!parquetOptions.containsKey("parquet.compression.codec.zstd.level")) { parquetOptions.set( diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java index b0715bb5389d..2a62c0bc8947 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetReaderFactory.java @@ -33,7 +33,10 @@ import org.apache.paimon.options.Options; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.reader.RecordReader.RecordIterator; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; +import org.apache.paimon.types.MapType; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Pool; @@ -45,6 +48,7 @@ import org.apache.parquet.hadoop.ParquetInputFormat; import org.apache.parquet.io.ColumnIOFactory; import org.apache.parquet.io.MessageColumnIO; +import org.apache.parquet.schema.ConversionPatterns; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; @@ -55,11 +59,17 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Set; +import static org.apache.paimon.format.parquet.ParquetSchemaConverter.LIST_ELEMENT_NAME; +import static org.apache.paimon.format.parquet.ParquetSchemaConverter.LIST_NAME; +import static org.apache.paimon.format.parquet.ParquetSchemaConverter.MAP_KEY_NAME; +import static org.apache.paimon.format.parquet.ParquetSchemaConverter.MAP_REPEATED_NAME; +import static org.apache.paimon.format.parquet.ParquetSchemaConverter.MAP_VALUE_NAME; import static org.apache.paimon.format.parquet.reader.ParquetSplitReaderUtil.buildFieldsList; import static org.apache.paimon.format.parquet.reader.ParquetSplitReaderUtil.createColumnReader; import static org.apache.paimon.format.parquet.reader.ParquetSplitReaderUtil.createWritableColumnVector; @@ -103,7 +113,8 @@ public ParquetReader createReader(FormatReaderFactory.Context context) throws IO ParquetFileReader reader = new ParquetFileReader( ParquetInputFile.fromPath(context.fileIO(), context.filePath()), - builder.build()); + builder.build(), + context.fileIndex()); MessageType fileSchema = reader.getFileMetaData().getSchema(); MessageType requestedSchema = clipParquetSchema(fileSchema); reader.setRequestedSchema(requestedSchema); @@ -155,13 +166,59 @@ private MessageType clipParquetSchema(GroupType parquetSchema) { ParquetSchemaConverter.convertToParquetType(fieldName, projectedTypes[i]); unknownFieldsIndices.add(i); } else { - types[i] = parquetSchema.getType(fieldName); + Type parquetType = parquetSchema.getType(fieldName); + types[i] = clipParquetType(projectedTypes[i], parquetType); } } return Types.buildMessage().addFields(types).named("paimon-parquet"); } + /** Clips `parquetType` by `readType`. */ + private Type clipParquetType(DataType readType, Type parquetType) { + switch (readType.getTypeRoot()) { + case ROW: + RowType rowType = (RowType) readType; + GroupType rowGroup = (GroupType) parquetType; + List rowGroupFields = new ArrayList<>(); + for (DataField field : rowType.getFields()) { + String fieldName = field.name(); + if (rowGroup.containsField(fieldName)) { + Type type = rowGroup.getType(fieldName); + rowGroupFields.add(clipParquetType(field.type(), type)); + } else { + // todo: support nested field missing + throw new RuntimeException("field " + fieldName + " is missing"); + } + } + return rowGroup.withNewFields(rowGroupFields); + case MAP: + MapType mapType = (MapType) readType; + GroupType mapGroup = (GroupType) parquetType; + GroupType keyValue = mapGroup.getType(MAP_REPEATED_NAME).asGroupType(); + return ConversionPatterns.mapType( + mapGroup.getRepetition(), + mapGroup.getName(), + MAP_REPEATED_NAME, + clipParquetType(mapType.getKeyType(), keyValue.getType(MAP_KEY_NAME)), + keyValue.containsField(MAP_VALUE_NAME) + ? clipParquetType( + mapType.getValueType(), keyValue.getType(MAP_VALUE_NAME)) + : null); + case ARRAY: + ArrayType arrayType = (ArrayType) readType; + GroupType arrayGroup = (GroupType) parquetType; + GroupType list = arrayGroup.getType(LIST_NAME).asGroupType(); + return ConversionPatterns.listOfElements( + arrayGroup.getRepetition(), + arrayGroup.getName(), + clipParquetType( + arrayType.getElementType(), list.getType(LIST_ELEMENT_NAME))); + default: + return parquetType; + } + } + private void checkSchema(MessageType fileSchema, MessageType requestedSchema) throws IOException, UnsupportedOperationException { if (projectedFields.length != requestedSchema.getFieldCount()) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java index 345c9944c9ec..5cf81c375a37 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetSchemaConverter.java @@ -18,8 +18,11 @@ package org.apache.paimon.format.parquet; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypeRoot; import org.apache.paimon.types.DecimalType; import org.apache.paimon.types.IntType; import org.apache.paimon.types.LocalZonedTimestampType; @@ -47,14 +50,13 @@ public class ParquetSchemaConverter { static final String MAP_REPEATED_NAME = "key_value"; + static final String MAP_KEY_NAME = "key"; + static final String MAP_VALUE_NAME = "value"; + static final String LIST_NAME = "list"; static final String LIST_ELEMENT_NAME = "element"; public static MessageType convertToParquetMessageType(String name, RowType rowType) { - Type[] types = new Type[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); i++) { - types[i] = convertToParquetType(rowType.getFieldNames().get(i), rowType.getTypeAt(i)); - } - return new MessageType(name, types); + return new MessageType(name, convertToParquetTypes(rowType)); } public static Type convertToParquetType(String name, DataType type) { @@ -149,8 +151,8 @@ private static Type convertToParquetType( repetition, name, MAP_REPEATED_NAME, - convertToParquetType("key", keyType), - convertToParquetType("value", mapType.getValueType())); + convertToParquetType(MAP_KEY_NAME, keyType), + convertToParquetType(MAP_VALUE_NAME, mapType.getValueType())); case MULTISET: MultisetType multisetType = (MultisetType) type; DataType elementType = multisetType.getElementType(); @@ -163,8 +165,8 @@ private static Type convertToParquetType( repetition, name, MAP_REPEATED_NAME, - convertToParquetType("key", elementType), - convertToParquetType("value", new IntType(false))); + convertToParquetType(MAP_KEY_NAME, elementType), + convertToParquetType(MAP_VALUE_NAME, new IntType(false))); case ROW: RowType rowType = (RowType) type; return new GroupType(repetition, name, convertToParquetTypes(rowType)); @@ -194,8 +196,47 @@ private static Type createTimestampWithLogicalType( private static List convertToParquetTypes(RowType rowType) { List types = new ArrayList<>(rowType.getFieldCount()); - for (int i = 0; i < rowType.getFieldCount(); i++) { - types.add(convertToParquetType(rowType.getFieldNames().get(i), rowType.getTypeAt(i))); + for (DataField field : rowType.getFields()) { + Type parquetType = convertToParquetType(field.name(), field.type()); + Type typeWithId = parquetType.withId(field.id()); + if (field.type().getTypeRoot() == DataTypeRoot.ARRAY) { + GroupType groupType = (GroupType) parquetType; + GroupType wrapperType = (GroupType) groupType.getFields().get(0); + Type elementTypeWithId = + wrapperType + .getFields() + .get(0) + .withId(SpecialFields.getArrayElementFieldId(field.id())); + typeWithId = + ConversionPatterns.listOfElements( + groupType.getRepetition(), + groupType.getName(), + elementTypeWithId) + .withId(field.id()); + } else if (field.type().getTypeRoot() == DataTypeRoot.MAP + || field.type().getTypeRoot() == DataTypeRoot.MULTISET) { + GroupType groupType = (GroupType) parquetType; + GroupType wrapperType = (GroupType) groupType.getFields().get(0); + Type keyTypeWithId = + wrapperType + .getFields() + .get(0) + .withId(SpecialFields.getMapKeyFieldId(field.id())); + Type valueTypeWithId = + wrapperType + .getFields() + .get(1) + .withId(SpecialFields.getMapValueFieldId(field.id())); + typeWithId = + ConversionPatterns.mapType( + groupType.getRepetition(), + groupType.getName(), + MAP_REPEATED_NAME, + keyTypeWithId, + valueTypeWithId) + .withId(field.id()); + } + types.add(typeWithId); } return types; } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetUtil.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetUtil.java index 055fe83f7c66..82d19e448878 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetUtil.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/ParquetUtil.java @@ -18,6 +18,7 @@ package org.apache.paimon.format.parquet; +import org.apache.paimon.fileindex.FileIndexResult; import org.apache.paimon.format.SimpleStatsExtractor; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; @@ -48,7 +49,7 @@ public class ParquetUtil { */ public static Pair>, SimpleStatsExtractor.FileInfo> extractColumnStats(FileIO fileIO, Path path) throws IOException { - try (ParquetFileReader reader = getParquetReader(fileIO, path)) { + try (ParquetFileReader reader = getParquetReader(fileIO, path, null)) { ParquetMetadata parquetMetadata = reader.getFooter(); List blockMetaDataList = parquetMetadata.getBlocks(); Map> resultStats = new HashMap<>(); @@ -77,9 +78,12 @@ public class ParquetUtil { * @param path the path of parquet files to be read * @return parquet reader, used for reading footer, status, etc. */ - public static ParquetFileReader getParquetReader(FileIO fileIO, Path path) throws IOException { + public static ParquetFileReader getParquetReader( + FileIO fileIO, Path path, FileIndexResult fileIndexResult) throws IOException { return new ParquetFileReader( - ParquetInputFile.fromPath(fileIO, path), ParquetReadOptions.builder().build()); + ParquetInputFile.fromPath(fileIO, path), + ParquetReadOptions.builder().build(), + fileIndexResult); } static void assertStatsClass( diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetSplitReaderUtil.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetSplitReaderUtil.java index 90abaa992c17..860ec54fa88b 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetSplitReaderUtil.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetSplitReaderUtil.java @@ -370,12 +370,12 @@ private static List getAllColumnDescriptorByType( } public static List buildFieldsList( - List childrens, List fieldNames, MessageColumnIO columnIO) { + List children, List fieldNames, MessageColumnIO columnIO) { List list = new ArrayList<>(); - for (int i = 0; i < childrens.size(); i++) { + for (int i = 0; i < children.size(); i++) { list.add( constructField( - childrens.get(i), lookupColumnByName(columnIO, fieldNames.get(i)))); + children.get(i), lookupColumnByName(columnIO, fieldNames.get(i)))); } return list; } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/RowDataParquetBuilder.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/RowDataParquetBuilder.java index 26d38a1c1863..da55f94942fd 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/RowDataParquetBuilder.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/writer/RowDataParquetBuilder.java @@ -19,6 +19,7 @@ package org.apache.paimon.format.parquet.writer; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.format.parquet.ColumnConfigParser; import org.apache.paimon.options.Options; import org.apache.paimon.types.RowType; @@ -46,33 +47,61 @@ public RowDataParquetBuilder(RowType rowType, Options options) { @Override public ParquetWriter createWriter(OutputFile out, String compression) throws IOException { - return new ParquetRowDataBuilder(out, rowType) - .withConf(conf) - .withCompressionCodec(CompressionCodecName.fromConf(getCompression(compression))) - .withRowGroupSize( - conf.getLong( - ParquetOutputFormat.BLOCK_SIZE, ParquetWriter.DEFAULT_BLOCK_SIZE)) - .withPageSize( - conf.getInt(ParquetOutputFormat.PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE)) - .withDictionaryPageSize( - conf.getInt( - ParquetOutputFormat.DICTIONARY_PAGE_SIZE, - ParquetProperties.DEFAULT_DICTIONARY_PAGE_SIZE)) - .withMaxPaddingSize( - conf.getInt( - ParquetOutputFormat.MAX_PADDING_BYTES, - ParquetWriter.MAX_PADDING_SIZE_DEFAULT)) - .withDictionaryEncoding( - conf.getBoolean( - ParquetOutputFormat.ENABLE_DICTIONARY, - ParquetProperties.DEFAULT_IS_DICTIONARY_ENABLED)) - .withValidation(conf.getBoolean(ParquetOutputFormat.VALIDATION, false)) - .withWriterVersion( - ParquetProperties.WriterVersion.fromString( - conf.get( - ParquetOutputFormat.WRITER_VERSION, - ParquetProperties.DEFAULT_WRITER_VERSION.toString()))) - .build(); + ParquetRowDataBuilder builder = + new ParquetRowDataBuilder(out, rowType) + .withConf(conf) + .withCompressionCodec( + CompressionCodecName.fromConf(getCompression(compression))) + .withRowGroupSize( + conf.getLong( + ParquetOutputFormat.BLOCK_SIZE, + ParquetWriter.DEFAULT_BLOCK_SIZE)) + .withPageSize( + conf.getInt( + ParquetOutputFormat.PAGE_SIZE, + ParquetWriter.DEFAULT_PAGE_SIZE)) + .withDictionaryPageSize( + conf.getInt( + ParquetOutputFormat.DICTIONARY_PAGE_SIZE, + ParquetProperties.DEFAULT_DICTIONARY_PAGE_SIZE)) + .withMaxPaddingSize( + conf.getInt( + ParquetOutputFormat.MAX_PADDING_BYTES, + ParquetWriter.MAX_PADDING_SIZE_DEFAULT)) + .withDictionaryEncoding( + conf.getBoolean( + ParquetOutputFormat.ENABLE_DICTIONARY, + ParquetProperties.DEFAULT_IS_DICTIONARY_ENABLED)) + .withValidation(conf.getBoolean(ParquetOutputFormat.VALIDATION, false)) + .withWriterVersion( + ParquetProperties.WriterVersion.fromString( + conf.get( + ParquetOutputFormat.WRITER_VERSION, + ParquetProperties.DEFAULT_WRITER_VERSION + .toString()))) + .withBloomFilterEnabled( + conf.getBoolean( + ParquetOutputFormat.BLOOM_FILTER_ENABLED, + ParquetProperties.DEFAULT_BLOOM_FILTER_ENABLED)); + new ColumnConfigParser() + .withColumnConfig( + ParquetOutputFormat.ENABLE_DICTIONARY, + key -> conf.getBoolean(key, false), + builder::withDictionaryEncoding) + .withColumnConfig( + ParquetOutputFormat.BLOOM_FILTER_ENABLED, + key -> conf.getBoolean(key, false), + builder::withBloomFilterEnabled) + .withColumnConfig( + ParquetOutputFormat.BLOOM_FILTER_EXPECTED_NDV, + key -> conf.getLong(key, -1L), + builder::withBloomFilterNDV) + .withColumnConfig( + ParquetOutputFormat.BLOOM_FILTER_FPP, + key -> conf.getDouble(key, ParquetProperties.DEFAULT_BLOOM_FILTER_FPP), + builder::withBloomFilterFPP) + .parseConfig(conf); + return builder.build(); } public String getCompression(String compression) { diff --git a/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java b/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java index cacc241fd24b..96cf2fe726cf 100644 --- a/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java +++ b/paimon-format/src/main/java/org/apache/parquet/filter2/predicate/ParquetFilters.java @@ -172,6 +172,11 @@ private static Comparable toParquetObject(Object value) { } if (value instanceof Number) { + if (value instanceof Byte) { + return ((Byte) value).intValue(); + } else if (value instanceof Short) { + return ((Short) value).intValue(); + } return (Comparable) value; } else if (value instanceof String) { return Binary.fromString((String) value); diff --git a/paimon-format/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/paimon-format/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java index aca1f021b98f..e3fc118ad674 100644 --- a/paimon-format/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java +++ b/paimon-format/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java @@ -18,10 +18,13 @@ package org.apache.parquet.hadoop; +import org.apache.paimon.fileindex.FileIndexResult; +import org.apache.paimon.fileindex.bitmap.BitmapIndexResult; import org.apache.paimon.format.parquet.ParquetInputFile; import org.apache.paimon.format.parquet.ParquetInputStream; import org.apache.paimon.fs.FileRange; import org.apache.paimon.fs.VectoredReadable; +import org.apache.paimon.utils.RoaringBitmap32; import org.apache.hadoop.fs.Path; import org.apache.parquet.ParquetReadOptions; @@ -92,6 +95,7 @@ import java.util.Objects; import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; import java.util.zip.CRC32; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -225,10 +229,14 @@ private static ParquetMetadata readFooter( private DictionaryPageReader nextDictionaryReader = null; private InternalFileDecryptor fileDecryptor = null; + private FileIndexResult fileIndexResult; - public ParquetFileReader(InputFile file, ParquetReadOptions options) throws IOException { + public ParquetFileReader( + InputFile file, ParquetReadOptions options, FileIndexResult fileIndexResult) + throws IOException { this.converter = new ParquetMetadataConverter(options); this.file = (ParquetInputFile) file; + this.fileIndexResult = fileIndexResult; this.f = this.file.newStream(); this.options = options; try { @@ -333,22 +341,38 @@ public String getFile() { private List filterRowGroups(List blocks) throws IOException { FilterCompat.Filter recordFilter = options.getRecordFilter(); - if (checkRowIndexOffsetExists(blocks) && FilterCompat.isFilteringRequired(recordFilter)) { - // set up data filters based on configured levels - List levels = new ArrayList<>(); + if (checkRowIndexOffsetExists(blocks)) { + if (FilterCompat.isFilteringRequired(recordFilter)) { + // set up data filters based on configured levels + List levels = new ArrayList<>(); - if (options.useStatsFilter()) { - levels.add(STATISTICS); - } + if (options.useStatsFilter()) { + levels.add(STATISTICS); + } - if (options.useDictionaryFilter()) { - levels.add(DICTIONARY); - } + if (options.useDictionaryFilter()) { + levels.add(DICTIONARY); + } - if (options.useBloomFilter()) { - levels.add(BLOOMFILTER); + if (options.useBloomFilter()) { + levels.add(BLOOMFILTER); + } + blocks = RowGroupFilter.filterRowGroups(levels, recordFilter, blocks, this); + } + if (fileIndexResult instanceof BitmapIndexResult) { + RoaringBitmap32 bitmap = ((BitmapIndexResult) fileIndexResult).get(); + blocks = + blocks.stream() + .filter( + it -> { + long rowIndexOffset = it.getRowIndexOffset(); + return bitmap.rangeCardinality( + rowIndexOffset, + rowIndexOffset + it.getRowCount()) + > 0; + }) + .collect(Collectors.toList()); } - return RowGroupFilter.filterRowGroups(levels, recordFilter, blocks, this); } return blocks; diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java index d02a0514c13e..46bf6afe6613 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcFileFormatTest.java @@ -38,7 +38,7 @@ public class OrcFileFormatTest { @Test public void testAbsent() { Options options = new Options(); - options.setString("haha", "1"); + options.setString("orc.haha", "1"); OrcFileFormat orc = new OrcFileFormatFactory().create(new FormatContext(options, 1024, 1024)); assertThat(orc.orcProperties().getProperty(IDENTIFIER + ".haha", "")).isEqualTo("1"); @@ -47,8 +47,8 @@ public void testAbsent() { @Test public void testPresent() { Options options = new Options(); - options.setString("haha", "1"); - options.setString("compress", "zlib"); + options.setString("orc.haha", "1"); + options.setString("orc.compress", "zlib"); OrcFileFormat orc = new OrcFileFormatFactory().create(new FormatContext(options, 1024, 1024)); assertThat(orc.orcProperties().getProperty(IDENTIFIER + ".haha", "")).isEqualTo("1"); diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcBulkWriterTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcBulkWriterTest.java index 899f77dc4a2e..1510a9382324 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcBulkWriterTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcBulkWriterTest.java @@ -43,7 +43,7 @@ class OrcBulkWriterTest { void testRowBatch(@TempDir java.nio.file.Path tempDir) throws IOException { Options options = new Options(); options.set(CoreOptions.WRITE_BATCH_SIZE, 1); - FileFormat orc = FileFormat.getFileFormat(options, "orc"); + FileFormat orc = FileFormat.fromIdentifier("orc", options); Assertions.assertThat(orc).isInstanceOf(OrcFileFormat.class); RowType rowType = diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java index b5f74ea831a6..8110998576cb 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/writer/OrcZstdTest.java @@ -59,9 +59,9 @@ class OrcZstdTest { @Test void testWriteOrcWithZstd(@TempDir java.nio.file.Path tempDir) throws IOException { Options options = new Options(); - options.set("compress", "zstd"); - options.set("stripe.size", "31457280"); - options.set("compression.zstd.level", "1"); + options.set("orc.compress", "zstd"); + options.set("orc.stripe.size", "31457280"); + options.set("orc.compression.zstd.level", "1"); OrcFileFormat orc = new OrcFileFormatFactory() .create(new FileFormatFactory.FormatContext(options, 1024, 1024)); @@ -92,9 +92,9 @@ void testWriteOrcWithZstd(@TempDir java.nio.file.Path tempDir) throws IOExceptio Assertions.assertThat(formatWriter).isInstanceOf(OrcBulkWriter.class); Options optionsWithLowLevel = new Options(); - optionsWithLowLevel.set("compress", "zstd"); - optionsWithLowLevel.set("stripe.size", "31457280"); - optionsWithLowLevel.set("compression.zstd.level", "1"); + optionsWithLowLevel.set("orc.compress", "zstd"); + optionsWithLowLevel.set("orc.stripe.size", "31457280"); + optionsWithLowLevel.set("orc.compression.zstd.level", "1"); Random random = new Random(); for (int i = 0; i < 1000; i++) { diff --git a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFileFormatTest.java b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFileFormatTest.java index 61c24026e86d..b51f8fc05dcb 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFileFormatTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFileFormatTest.java @@ -34,30 +34,26 @@ import java.util.ArrayList; import java.util.List; -import static org.apache.paimon.format.parquet.ParquetFileFormat.getParquetConfiguration; -import static org.apache.paimon.format.parquet.ParquetFileFormatFactory.IDENTIFIER; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link ParquetFileFormatFactory}. */ public class ParquetFileFormatTest { - private static final ConfigOption KEY1 = - ConfigOptions.key("k1").stringType().defaultValue("absent"); @Test - public void testAbsent() { - Options options = new Options(); - ParquetFileFormat parquet = - new ParquetFileFormatFactory().create(new FormatContext(options, 1024, 1024)); - assertThat(parquet.formatOptions().getString(KEY1)).isEqualTo("absent"); - } + public void testConfiguration() { + ConfigOption parquetKey = + ConfigOptions.key("parquet.mykey").stringType().noDefaultValue(); + ConfigOption otherKey = ConfigOptions.key("other").stringType().noDefaultValue(); - @Test - public void testPresent() { Options options = new Options(); - options.setString(KEY1.key(), "v1"); - ParquetFileFormat parquet = - new ParquetFileFormatFactory().create(new FormatContext(options, 1024, 1024)); - assertThat(parquet.formatOptions().getString(KEY1)).isEqualTo("v1"); + options.set(parquetKey, "hello"); + options.set(otherKey, "test"); + FormatContext context = new FormatContext(options, 1024, 1024, 2, null); + + Options actual = new ParquetFileFormat(context).getOptions(); + assertThat(actual.get(parquetKey)).isEqualTo("hello"); + assertThat(actual.contains(otherKey)).isFalse(); + assertThat(actual.get("parquet.compression.codec.zstd.level")).isEqualTo("2"); } @Test @@ -68,9 +64,7 @@ public void testFileCompressionHigherPreference() { RowDataParquetBuilder builder = new RowDataParquetBuilder( new RowType(new ArrayList<>()), - getParquetConfiguration( - new FormatContext( - conf.removePrefix(IDENTIFIER + "."), 1024, 1024))); + new ParquetFileFormat(new FormatContext(conf, 1024, 1024)).getOptions()); assertThat(builder.getCompression(null)).isEqualTo(lz4); assertThat(builder.getCompression("SNAPPY")).isEqualTo(lz4); } diff --git a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFormatReadWriteTest.java b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFormatReadWriteTest.java index d5338b1e78be..e0d1d240a9fd 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFormatReadWriteTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetFormatReadWriteTest.java @@ -18,10 +18,27 @@ package org.apache.paimon.format.parquet; +import org.apache.paimon.data.GenericRow; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.FileFormatFactory; import org.apache.paimon.format.FormatReadWriteTest; +import org.apache.paimon.format.FormatWriter; +import org.apache.paimon.fs.PositionOutputStream; import org.apache.paimon.options.Options; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; /** A parquet {@link FormatReadWriteTest}. */ public class ParquetFormatReadWriteTest extends FormatReadWriteTest { @@ -35,4 +52,39 @@ protected FileFormat fileFormat() { return new ParquetFileFormat( new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testEnableBloomFilter(boolean enabled) throws Exception { + Options options = new Options(); + options.set("parquet.bloom.filter.enabled", String.valueOf(enabled)); + ParquetFileFormat format = + new ParquetFileFormat(new FileFormatFactory.FormatContext(options, 1024, 1024)); + + RowType rowType = DataTypes.ROW(DataTypes.INT().notNull(), DataTypes.BIGINT()); + + if (ThreadLocalRandom.current().nextBoolean()) { + rowType = (RowType) rowType.notNull(); + } + + PositionOutputStream out = fileIO.newOutputStream(file, false); + FormatWriter writer = format.createWriterFactory(rowType).create(out, "zstd"); + writer.addElement(GenericRow.of(1, 1L)); + writer.addElement(GenericRow.of(2, 2L)); + writer.addElement(GenericRow.of(3, null)); + writer.close(); + out.close(); + + try (ParquetFileReader reader = ParquetUtil.getParquetReader(fileIO, file, null)) { + ParquetMetadata parquetMetadata = reader.getFooter(); + List blockMetaDataList = parquetMetadata.getBlocks(); + for (BlockMetaData blockMetaData : blockMetaDataList) { + List columnChunkMetaDataList = blockMetaData.getColumns(); + for (ColumnChunkMetaData columnChunkMetaData : columnChunkMetaDataList) { + BloomFilter filter = reader.readBloomFilter(columnChunkMetaData); + Assertions.assertThat(enabled == (filter != null)).isTrue(); + } + } + } + } } diff --git a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java index 0ccf3fe30842..099811d98b52 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/parquet/ParquetReadWriteTest.java @@ -40,6 +40,7 @@ import org.apache.paimon.types.BooleanType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.DecimalType; import org.apache.paimon.types.DoubleType; import org.apache.paimon.types.FloatType; @@ -61,7 +62,13 @@ import org.apache.parquet.hadoop.ParquetWriter; import org.apache.parquet.hadoop.example.ExampleParquetWriter; import org.apache.parquet.hadoop.util.HadoopOutputFile; +import org.apache.parquet.schema.ConversionPatterns; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; @@ -89,6 +96,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -470,6 +478,62 @@ public void testNestedNullMapKey() { .isInstanceOf(RuntimeException.class); } + @Test + public void testConvertToParquetTypeWithId() { + List nestedFields = + Arrays.asList( + new DataField(3, "v1", DataTypes.INT()), + new DataField(4, "v2", DataTypes.STRING())); + List fields = + Arrays.asList( + new DataField(0, "a", DataTypes.INT()), + new DataField(1, "b", DataTypes.ARRAY(DataTypes.STRING())), + new DataField( + 2, "c", DataTypes.MAP(DataTypes.INT(), new RowType(nestedFields)))); + RowType rowType = new RowType(fields); + + int baseId = 536870911; + Type mapValueType = + new GroupType( + Type.Repetition.OPTIONAL, + "value", + Types.primitive(INT32, Type.Repetition.OPTIONAL) + .named("v1") + .withId(3), + Types.primitive( + PrimitiveType.PrimitiveTypeName.BINARY, + Type.Repetition.OPTIONAL) + .as(LogicalTypeAnnotation.stringType()) + .named("v2") + .withId(4)) + .withId(baseId - 2); + Type expected = + new MessageType( + "table", + Types.primitive(INT32, Type.Repetition.OPTIONAL).named("a").withId(0), + ConversionPatterns.listOfElements( + Type.Repetition.OPTIONAL, + "b", + Types.primitive( + PrimitiveType.PrimitiveTypeName.BINARY, + Type.Repetition.OPTIONAL) + .as(LogicalTypeAnnotation.stringType()) + .named("element") + .withId(baseId + 1)) + .withId(1), + ConversionPatterns.mapType( + Type.Repetition.OPTIONAL, + "c", + "key_value", + Types.primitive(INT32, Type.Repetition.REQUIRED) + .named("key") + .withId(baseId + 2), + mapValueType) + .withId(2)); + Type actual = ParquetSchemaConverter.convertToParquetMessageType("table", rowType); + assertThat(actual).isEqualTo(expected); + } + private void innerTestTypes(File folder, List records, int rowGroupSize) throws IOException { List rows = records.stream().map(this::newRow).collect(Collectors.toList()); diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 12307521f882..a1950c374ab3 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -40,17 +40,24 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.CatalogEnvironment; +import org.apache.paimon.table.CatalogTableType; import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.table.FormatTable; -import org.apache.paimon.table.TableType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.Preconditions; +import org.apache.paimon.view.View; +import org.apache.paimon.view.ViewImpl; import org.apache.flink.table.hive.LegacyHiveClasses; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; @@ -84,19 +91,28 @@ import java.util.stream.Collectors; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREWAREHOUSE; +import static org.apache.hadoop.hive.serde.serdeConstants.FIELD_DELIM; +import static org.apache.paimon.CoreOptions.FILE_FORMAT; +import static org.apache.paimon.CoreOptions.PARTITION_EXPIRATION_TIME; +import static org.apache.paimon.CoreOptions.TYPE; +import static org.apache.paimon.TableType.FORMAT_TABLE; import static org.apache.paimon.hive.HiveCatalogLock.acquireTimeout; import static org.apache.paimon.hive.HiveCatalogLock.checkMaxSleep; -import static org.apache.paimon.hive.HiveCatalogOptions.FORMAT_TABLE_ENABLED; import static org.apache.paimon.hive.HiveCatalogOptions.HADOOP_CONF_DIR; import static org.apache.paimon.hive.HiveCatalogOptions.HIVE_CONF_DIR; import static org.apache.paimon.hive.HiveCatalogOptions.IDENTIFIER; import static org.apache.paimon.hive.HiveCatalogOptions.LOCATION_IN_PROPERTIES; +import static org.apache.paimon.hive.HiveTableUtils.convertToFormatTable; import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; +import static org.apache.paimon.options.CatalogOptions.FORMAT_TABLE_ENABLED; import static org.apache.paimon.options.CatalogOptions.SYNC_ALL_PROPERTIES; import static org.apache.paimon.options.CatalogOptions.TABLE_TYPE; import static org.apache.paimon.options.OptionsUtils.convertToPropertiesPrefixKey; +import static org.apache.paimon.table.FormatTableOptions.FIELD_DELIMITER; import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH; +import static org.apache.paimon.utils.HadoopUtils.addHadoopConfIfFound; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.Preconditions.checkNotNull; import static org.apache.paimon.utils.StringUtils.isNullOrWhitespaceOnly; /** A catalog implementation for Hive. */ @@ -158,8 +174,8 @@ public HiveCatalog( this.clients = new CachedClientPool(hiveConf, options, clientClassName); } - private boolean formatTableEnabled() { - return options.get(FORMAT_TABLE_ENABLED); + private boolean formatTableDisabled() { + return !options.get(FORMAT_TABLE_ENABLED); } @Override @@ -175,35 +191,34 @@ public Optional lockContext() { } @Override - public Optional metastoreClientFactory(Identifier identifier) { + public Optional metastoreClientFactory( + Identifier identifier, TableSchema schema) { Identifier tableIdentifier = new Identifier(identifier.getDatabaseName(), identifier.getTableName()); - try { - return Optional.of( - new HiveMetastoreClient.Factory( - tableIdentifier, - getDataTableSchema(tableIdentifier), - hiveConf, - clientClassName, - options)); - } catch (TableNotExistException e) { - throw new RuntimeException( - "Table " + identifier + " does not exist. This is unexpected.", e); - } + return Optional.of( + new HiveMetastoreClient.Factory( + tableIdentifier, schema, hiveConf, clientClassName, options)); } @Override public Path getTableLocation(Identifier identifier) { + Table table = null; + try { + table = getHmsTable(identifier); + } catch (TableNotExistException ignored) { + } + return getTableLocation(identifier, table); + } + + private Path getTableLocation(Identifier identifier, @Nullable Table table) { try { String databaseName = identifier.getDatabaseName(); String tableName = identifier.getTableName(); Optional tablePath = clients.run( client -> { - if (client.tableExists(databaseName, tableName)) { - String location = - locationHelper.getTableLocation( - client.getTable(databaseName, tableName)); + if (table != null) { + String location = locationHelper.getTableLocation(table); if (location != null) { return Optional.of(new Path(location)); } @@ -279,10 +294,18 @@ private Database convertToHiveDatabase(String name, Map properti } @Override - public Map loadDatabasePropertiesImpl(String name) + public org.apache.paimon.catalog.Database getDatabaseImpl(String name) throws DatabaseNotExistException { try { - return convertToProperties(clients.run(client -> client.getDatabase(name))); + Database database = clients.run(client -> client.getDatabase(name)); + Map options = new HashMap<>(database.getParameters()); + if (database.getLocationUri() != null) { + options.put(DB_LOCATION_PROP, database.getLocationUri()); + } + if (database.getDescription() != null) { + options.put(COMMENT_PROP, database.getDescription()); + } + return org.apache.paimon.catalog.Database.of(name, options, database.getDescription()); } catch (NoSuchObjectException e) { throw new DatabaseNotExistException(name); } catch (TException e) { @@ -294,17 +317,6 @@ public Map loadDatabasePropertiesImpl(String name) } } - private Map convertToProperties(Database database) { - Map properties = new HashMap<>(database.getParameters()); - if (database.getLocationUri() != null) { - properties.put(DB_LOCATION_PROP, database.getLocationUri()); - } - if (database.getDescription() != null) { - properties.put(COMMENT_PROP, database.getDescription()); - } - return properties; - } - @Override public void dropPartition(Identifier identifier, Map partitionSpec) throws TableNotExistException { @@ -377,16 +389,16 @@ protected void dropDatabaseImpl(String name) { @Override protected List listTablesImpl(String databaseName) { try { - return clients.run( - client -> - client.getAllTables(databaseName).stream() - .filter( - tableName -> { - Identifier identifier = - new Identifier(databaseName, tableName); - return tableExists(identifier); - }) - .collect(Collectors.toList())); + List allTables = clients.run(client -> client.getAllTables(databaseName)); + List result = new ArrayList<>(allTables.size()); + for (String table : allTables) { + try { + getTable(new Identifier(databaseName, table)); + result.add(table); + } catch (TableNotExistException ignored) { + } + } + return result; } catch (TException e) { throw new RuntimeException("Failed to list all tables in database " + databaseName, e); } catch (InterruptedException e) { @@ -396,106 +408,250 @@ protected List listTablesImpl(String databaseName) { } @Override - public boolean tableExists(Identifier identifier) { - if (isSystemTable(identifier)) { - return super.tableExists(identifier); + public TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { + Table table = getHmsTable(identifier); + return getDataTableSchema(identifier, table); + } + + private TableSchema getDataTableSchema(Identifier identifier, Table table) + throws TableNotExistException { + if (!isPaimonTable(identifier, table)) { + throw new TableNotExistException(identifier); } + return tableSchemaInFileSystem( + getTableLocation(identifier, table), identifier.getBranchNameOrDefault()) + .orElseThrow(() -> new TableNotExistException(identifier)); + } + + @Override + public View getView(Identifier identifier) throws ViewNotExistException { Table table; try { - table = - clients.run( - client -> - client.getTable( - identifier.getDatabaseName(), - identifier.getTableName())); - } catch (NoSuchObjectException e) { - return false; - } catch (TException e) { - throw new RuntimeException( - "Cannot determine if table " + identifier.getFullName() + " is a paimon table.", - e); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException( - "Interrupted in call to tableExists " + identifier.getFullName(), e); + table = getHmsTable(identifier); + } catch (TableNotExistException e) { + throw new ViewNotExistException(identifier); } - boolean isDataTable = - isPaimonTable(table) - && tableSchemaInFileSystem( - getTableLocation(identifier), - identifier.getBranchNameOrDefault()) - .isPresent(); - if (isDataTable) { - return true; + if (!isView(table)) { + throw new ViewNotExistException(identifier); } - if (formatTableEnabled()) { - try { - HiveFormatTableUtils.convertToFormatTable(table); - return true; - } catch (UnsupportedOperationException e) { - return false; + RowType rowType = HiveTableUtils.createRowType(table); + Map options = new HashMap<>(table.getParameters()); + String comment = options.remove(COMMENT_PROP); + return new ViewImpl(identifier, rowType, table.getViewExpandedText(), comment, options); + } + + @Override + public void createView(Identifier identifier, View view, boolean ignoreIfExists) + throws ViewAlreadyExistException, DatabaseNotExistException { + getDatabase(identifier.getDatabaseName()); + + try { + getView(identifier); + if (ignoreIfExists) { + return; } + throw new ViewAlreadyExistException(identifier); + } catch (ViewNotExistException ignored) { } - return false; + Table hiveTable = + org.apache.hadoop.hive.ql.metadata.Table.getEmptyTable( + identifier.getDatabaseName(), identifier.getObjectName()); + hiveTable.setCreateTime((int) (System.currentTimeMillis() / 1000)); + + Map properties = new HashMap<>(view.options()); + // Table comment + if (view.comment().isPresent()) { + properties.put(COMMENT_PROP, view.comment().get()); + } + hiveTable.setParameters(properties); + hiveTable.setPartitionKeys(new ArrayList<>()); + hiveTable.setViewOriginalText(view.query()); + hiveTable.setViewExpandedText(view.query()); + hiveTable.setTableType(TableType.VIRTUAL_VIEW.name()); + + StorageDescriptor sd = hiveTable.getSd(); + List columns = + view.rowType().getFields().stream() + .map(this::convertToFieldSchema) + .collect(Collectors.toList()); + sd.setCols(columns); + + try { + clients.execute(client -> client.createTable(hiveTable)); + } catch (Exception e) { + // we don't need to delete directories since HMS will roll back db and fs if failed. + throw new RuntimeException("Failed to create table " + identifier.getFullName(), e); + } } - private static boolean isPaimonTable(Table table) { - boolean isPaimonTable = - INPUT_FORMAT_CLASS_NAME.equals(table.getSd().getInputFormat()) - && OUTPUT_FORMAT_CLASS_NAME.equals(table.getSd().getOutputFormat()); - return isPaimonTable || LegacyHiveClasses.isPaimonTable(table); + @Override + public void dropView(Identifier identifier, boolean ignoreIfNotExists) + throws ViewNotExistException { + try { + getView(identifier); + } catch (ViewNotExistException e) { + if (ignoreIfNotExists) { + return; + } + throw e; + } + + try { + clients.execute( + client -> + client.dropTable( + identifier.getDatabaseName(), + identifier.getTableName(), + false, + false, + false)); + } catch (TException e) { + throw new RuntimeException("Failed to drop view " + identifier.getFullName(), e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException( + "Interrupted in call to drop view " + identifier.getFullName(), e); + } } @Override - public TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { - if (!tableExists(identifier)) { - throw new TableNotExistException(identifier); + public List listViews(String databaseName) throws DatabaseNotExistException { + if (isSystemDatabase(databaseName)) { + return Collections.emptyList(); } + getDatabase(databaseName); - return tableSchemaInFileSystem( - getTableLocation(identifier), identifier.getBranchNameOrDefault()) - .orElseThrow(() -> new TableNotExistException(identifier)); + try { + List tables = clients.run(client -> client.getAllTables(databaseName)); + List views = new ArrayList<>(); + for (String tableName : tables) { + Table table; + try { + table = getHmsTable(Identifier.create(databaseName, tableName)); + } catch (TableNotExistException e) { + continue; + } + if (isView(table)) { + views.add(tableName); + } + } + return views; + } catch (TException e) { + throw new RuntimeException("Failed to list all tables in database " + databaseName, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted in call to listTables " + databaseName, e); + } } @Override - public FormatTable getFormatTable(Identifier identifier) throws TableNotExistException { - if (!formatTableEnabled()) { - throw new TableNotExistException(identifier); + public void renameView(Identifier fromView, Identifier toView, boolean ignoreIfNotExists) + throws ViewNotExistException, ViewAlreadyExistException { + try { + getView(fromView); + } catch (ViewNotExistException e) { + if (ignoreIfNotExists) { + return; + } + throw new ViewNotExistException(fromView); } - Table table; try { - table = - clients.run( - client -> - client.getTable( - identifier.getDatabaseName(), - identifier.getTableName())); - } catch (NoSuchObjectException e) { - throw new TableNotExistException(identifier); + getView(toView); + throw new ViewAlreadyExistException(toView); + } catch (ViewNotExistException ignored) { + } + + try { + String fromDB = fromView.getDatabaseName(); + String fromViewName = fromView.getTableName(); + Table table = clients.run(client -> client.getTable(fromDB, fromViewName)); + table.setDbName(toView.getDatabaseName()); + table.setTableName(toView.getTableName()); + clients.execute(client -> client.alter_table(fromDB, fromViewName, table)); } catch (TException e) { - throw new RuntimeException(e); + throw new RuntimeException("Failed to rename view " + fromView.getFullName(), e); } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw new RuntimeException(e); + throw new RuntimeException( + "Interrupted in call to rename view " + fromView.getFullName(), e); } + } + + @Override + public org.apache.paimon.table.Table getDataOrFormatTable(Identifier identifier) + throws TableNotExistException { + Preconditions.checkArgument(identifier.getSystemTableName() == null); + Table table = getHmsTable(identifier); try { - return HiveFormatTableUtils.convertToFormatTable(table); + TableSchema tableSchema = getDataTableSchema(identifier, table); + return FileStoreTableFactory.create( + fileIO, + getTableLocation(identifier, table), + tableSchema, + new CatalogEnvironment( + identifier, + Lock.factory( + lockFactory().orElse(null), + lockContext().orElse(null), + identifier), + metastoreClientFactory(identifier, tableSchema).orElse(null), + lineageMetaFactory)); + } catch (TableNotExistException ignore) { + } + + if (formatTableDisabled()) { + throw new TableNotExistException(identifier); + } + + try { + return convertToFormatTable(table); } catch (UnsupportedOperationException e) { throw new TableNotExistException(identifier); } } + @Override + public void createFormatTable(Identifier identifier, Schema schema) { + if (formatTableDisabled()) { + throw new UnsupportedOperationException( + "Format table is not enabled for " + identifier); + } + List fields = schema.fields(); + List partitionKeys = schema.partitionKeys(); + List primaryKeys = schema.primaryKeys(); + Map options = schema.options(); + int highestFieldId = RowType.currentHighestFieldId(fields); + + TableSchema newSchema = + new TableSchema( + 0, + fields, + highestFieldId, + partitionKeys, + primaryKeys, + options, + schema.comment()); + try { + Path location = getTableLocation(identifier, null); + Table hiveTable = createHiveFormatTable(identifier, newSchema, location); + clients.execute(client -> client.createTable(hiveTable)); + } catch (Exception e) { + // we don't need to delete directories since HMS will roll back db and fs if failed. + throw new RuntimeException("Failed to create table " + identifier.getFullName(), e); + } + } + private boolean usingExternalTable() { - TableType tableType = + CatalogTableType tableType = OptionsUtils.convertToEnum( - hiveConf.get(TABLE_TYPE.key(), TableType.MANAGED.toString()), - TableType.class); - return TableType.EXTERNAL.equals(tableType); + hiveConf.get(TABLE_TYPE.key(), CatalogTableType.MANAGED.toString()), + CatalogTableType.class); + return CatalogTableType.EXTERNAL.equals(tableType); } @Override @@ -540,9 +696,11 @@ protected void dropTableImpl(Identifier identifier) { protected void createTableImpl(Identifier identifier, Schema schema) { // first commit changes to underlying files // if changes on Hive fails there is no harm to perform the same changes to files again + Path location = getTableLocation(identifier, null); TableSchema tableSchema; try { - tableSchema = schemaManager(identifier).createTable(schema, usingExternalTable()); + tableSchema = + schemaManager(identifier, location).createTable(schema, usingExternalTable()); } catch (Exception e) { throw new RuntimeException( "Failed to commit changes of table " @@ -552,19 +710,22 @@ protected void createTableImpl(Identifier identifier, Schema schema) { } try { - clients.execute(client -> client.createTable(createHiveTable(identifier, tableSchema))); + clients.execute( + client -> + client.createTable(createHiveTable(identifier, tableSchema, location))); } catch (Exception e) { - Path path = getTableLocation(identifier); try { - fileIO.deleteDirectoryQuietly(path); + fileIO.deleteDirectoryQuietly(location); } catch (Exception ee) { - LOG.error("Delete directory[{}] fail for table {}", path, identifier, ee); + LOG.error("Delete directory[{}] fail for table {}", location, identifier, ee); } throw new RuntimeException("Failed to create table " + identifier.getFullName(), e); } } - private Table createHiveTable(Identifier identifier, TableSchema tableSchema) { + private Table createHiveTable(Identifier identifier, TableSchema tableSchema, Path location) { + checkArgument(Options.fromMap(tableSchema.options()).get(TYPE) != FORMAT_TABLE); + Map tblProperties; if (syncAllProperties()) { tblProperties = new HashMap<>(tableSchema.options()); @@ -573,10 +734,41 @@ private Table createHiveTable(Identifier identifier, TableSchema tableSchema) { tblProperties.putAll(convertToPropertiesTableKey(tableSchema)); } else { tblProperties = convertToPropertiesPrefixKey(tableSchema.options(), HIVE_PREFIX); + if (tableSchema.options().containsKey(PARTITION_EXPIRATION_TIME.key())) { + // This property will be stored in the 'table_params' table of the HMS database for + // querying by other engines or products. + tblProperties.put( + PARTITION_EXPIRATION_TIME.key(), + tableSchema.options().get(PARTITION_EXPIRATION_TIME.key())); + } } - Table table = newHmsTable(identifier, tblProperties); - updateHmsTable(table, identifier, tableSchema); + Table table = newHmsTable(identifier, tblProperties, PAIMON_TABLE_TYPE_VALUE); + updateHmsTable(table, identifier, tableSchema, PAIMON_TABLE_TYPE_VALUE, location); + return table; + } + + private Table createHiveFormatTable( + Identifier identifier, TableSchema tableSchema, Path location) { + Options options = Options.fromMap(tableSchema.options()); + checkArgument(options.get(TYPE) == FORMAT_TABLE); + + String provider = tableSchema.options().get(FILE_FORMAT.key()); + checkNotNull(provider, FILE_FORMAT.key() + " should be configured."); + // valid supported format + FormatTable.Format.valueOf(provider.toUpperCase()); + + Map tblProperties = new HashMap<>(); + + Table table = newHmsTable(identifier, tblProperties, provider); + updateHmsTable(table, identifier, tableSchema, provider, location); + + if (FormatTable.Format.CSV.toString().equalsIgnoreCase(provider)) { + table.getSd() + .getSerdeInfo() + .getParameters() + .put(FIELD_DELIM, options.get(FIELD_DELIMITER)); + } return table; } @@ -623,7 +815,12 @@ protected void renameTableImpl(Identifier fromTable, Identifier toTable) { @Override protected void alterTableImpl(Identifier identifier, List changes) throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { - final SchemaManager schemaManager = schemaManager(identifier); + Table table = getHmsTable(identifier); + if (!isPaimonTable(identifier, table)) { + throw new UnsupportedOperationException("Only data table support alter table."); + } + + final SchemaManager schemaManager = schemaManager(identifier, getTableLocation(identifier)); // first commit changes to underlying files TableSchema schema = schemaManager.commitChanges(changes); @@ -632,12 +829,6 @@ protected void alterTableImpl(Identifier identifier, List changes) return; } try { - Table table = - clients.run( - client -> - client.getTable( - identifier.getDatabaseName(), - identifier.getTableName())); alterTableToHms(table, identifier, schema); } catch (Exception te) { schemaManager.deleteSchema(schema.id()); @@ -648,7 +839,8 @@ protected void alterTableImpl(Identifier identifier, List changes) private void alterTableToHms(Table table, Identifier identifier, TableSchema newSchema) throws TException, InterruptedException { updateHmsTablePars(table, newSchema); - updateHmsTable(table, identifier, newSchema); + Path location = getTableLocation(identifier, table); + updateHmsTable(table, identifier, newSchema, newSchema.options().get("provider"), location); clients.execute( client -> client.alter_table( @@ -685,7 +877,9 @@ public void repairDatabase(String databaseName) { checkNotSystemDatabase(databaseName); // create database if needed - if (!databaseExists(databaseName)) { + try { + getDatabase(databaseName); + } catch (DatabaseNotExistException e) { createDatabaseImpl(databaseName, Collections.emptyMap()); } @@ -713,19 +907,14 @@ public void repairTable(Identifier identifier) throws TableNotExistException { checkNotSystemTable(identifier, "repairTable"); validateIdentifierNameCaseInsensitive(identifier); + Path location = getTableLocation(identifier); TableSchema tableSchema = - tableSchemaInFileSystem( - getTableLocation(identifier), identifier.getBranchNameOrDefault()) + tableSchemaInFileSystem(location, identifier.getBranchNameOrDefault()) .orElseThrow(() -> new TableNotExistException(identifier)); - Table newTable = createHiveTable(identifier, tableSchema); + Table newTable = createHiveTable(identifier, tableSchema, location); try { try { - Table table = - clients.run( - client -> - client.getTable( - identifier.getDatabaseName(), - identifier.getTableName())); + Table table = getHmsTable(identifier); checkArgument( isPaimonTable(table), "Table %s is not a paimon table in hive metastore.", @@ -734,7 +923,7 @@ public void repairTable(Identifier identifier) throws TableNotExistException { || !newTable.getParameters().equals(table.getParameters())) { alterTableToHms(table, identifier, tableSchema); } - } catch (NoSuchObjectException e) { + } catch (TableNotExistException e) { // hive table does not exist. clients.execute(client -> client.createTable(newTable)); } @@ -766,12 +955,63 @@ public String warehouse() { return warehouse; } - private Table newHmsTable(Identifier identifier, Map tableParameters) { + private Table getHmsTable(Identifier identifier) throws TableNotExistException { + try { + return clients.run( + client -> + client.getTable( + identifier.getDatabaseName(), identifier.getTableName())); + } catch (NoSuchObjectException e) { + throw new TableNotExistException(identifier); + } catch (TException e) { + throw new RuntimeException( + "Cannot determine if table " + identifier.getFullName() + " is a paimon table.", + e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException( + "Interrupted in call to tableExists " + identifier.getFullName(), e); + } + } + + private boolean isPaimonTable(Identifier identifier, Table table) { + return isPaimonTable(table) + && tableSchemaInFileSystem( + getTableLocation(identifier, table), + identifier.getBranchNameOrDefault()) + .isPresent(); + } + + private static boolean isPaimonTable(Table table) { + boolean isPaimonTable = + INPUT_FORMAT_CLASS_NAME.equals(table.getSd().getInputFormat()) + && OUTPUT_FORMAT_CLASS_NAME.equals(table.getSd().getOutputFormat()); + return isPaimonTable || LegacyHiveClasses.isPaimonTable(table); + } + + private boolean isFormatTable(Table table) { + try { + convertToFormatTable(table); + return true; + } catch (UnsupportedOperationException e) { + return false; + } + } + + private static boolean isView(Table table) { + return TableType.valueOf(table.getTableType()) == TableType.VIRTUAL_VIEW; + } + + private Table newHmsTable( + Identifier identifier, Map tableParameters, String provider) { long currentTimeMillis = System.currentTimeMillis(); - TableType tableType = + CatalogTableType tableType = OptionsUtils.convertToEnum( - hiveConf.get(TABLE_TYPE.key(), TableType.MANAGED.toString()), - TableType.class); + hiveConf.get(TABLE_TYPE.key(), CatalogTableType.MANAGED.toString()), + CatalogTableType.class); + if (provider == null) { + provider = PAIMON_TABLE_TYPE_VALUE; + } Table table = new Table( identifier.getTableName(), @@ -787,24 +1027,76 @@ private Table newHmsTable(Identifier identifier, Map tableParame null, null, tableType.toString().toUpperCase(Locale.ROOT) + "_TABLE"); - table.getParameters().put(TABLE_TYPE_PROP, PAIMON_TABLE_TYPE_VALUE.toUpperCase()); - table.getParameters() - .put(hive_metastoreConstants.META_TABLE_STORAGE, STORAGE_HANDLER_CLASS_NAME); - if (TableType.EXTERNAL.equals(tableType)) { + table.getParameters().put(TABLE_TYPE_PROP, provider.toUpperCase()); + if (PAIMON_TABLE_TYPE_VALUE.equalsIgnoreCase(provider)) { + table.getParameters() + .put(hive_metastoreConstants.META_TABLE_STORAGE, STORAGE_HANDLER_CLASS_NAME); + } else { + table.getParameters().put(FILE_FORMAT.key(), provider.toLowerCase()); + table.getParameters().put(TYPE.key(), FORMAT_TABLE.toString()); + } + if (CatalogTableType.EXTERNAL.equals(tableType)) { table.getParameters().put("EXTERNAL", "TRUE"); } return table; } - private void updateHmsTable(Table table, Identifier identifier, TableSchema schema) { + private String getSerdeClassName(String provider) { + if (provider == null || provider.equalsIgnoreCase(PAIMON_TABLE_TYPE_VALUE)) { + return SERDE_CLASS_NAME; + } else if (provider.equalsIgnoreCase("csv")) { + return "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"; + } else if (provider.equalsIgnoreCase("parquet")) { + return "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"; + } else if (provider.equalsIgnoreCase("orc")) { + return "org.apache.hadoop.hive.ql.io.orc.OrcSerde"; + } else { + return SERDE_CLASS_NAME; + } + } + + private String getInputFormatName(String provider) { + if (provider == null || provider.equalsIgnoreCase(PAIMON_TABLE_TYPE_VALUE)) { + return INPUT_FORMAT_CLASS_NAME; + } else if (provider.equalsIgnoreCase("csv")) { + return "org.apache.hadoop.mapred.TextInputFormat"; + } else if (provider.equalsIgnoreCase("parquet")) { + return "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; + } else if (provider.equalsIgnoreCase("orc")) { + return "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; + } else { + return INPUT_FORMAT_CLASS_NAME; + } + } + + private String getOutputFormatClassName(String provider) { + if (provider == null || provider.equalsIgnoreCase(PAIMON_TABLE_TYPE_VALUE)) { + return OUTPUT_FORMAT_CLASS_NAME; + } else if (provider.equalsIgnoreCase("csv")) { + return "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"; + } else if (provider.equalsIgnoreCase("parquet")) { + return "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"; + } else if (provider.equalsIgnoreCase("orc")) { + return "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"; + } else { + return OUTPUT_FORMAT_CLASS_NAME; + } + } + + private void updateHmsTable( + Table table, + Identifier identifier, + TableSchema schema, + String provider, + Path location) { StorageDescriptor sd = table.getSd() != null ? table.getSd() : new StorageDescriptor(); - sd.setInputFormat(INPUT_FORMAT_CLASS_NAME); - sd.setOutputFormat(OUTPUT_FORMAT_CLASS_NAME); + sd.setInputFormat(getInputFormatName(provider)); + sd.setOutputFormat(getOutputFormatClassName(provider)); SerDeInfo serDeInfo = sd.getSerdeInfo() != null ? sd.getSerdeInfo() : new SerDeInfo(); - serDeInfo.setParameters(new HashMap<>()); - serDeInfo.setSerializationLib(SERDE_CLASS_NAME); + serDeInfo.setParameters(setSerDeInfoParam(provider)); + serDeInfo.setSerializationLib(getSerdeClassName(provider)); sd.setSerdeInfo(serDeInfo); CoreOptions options = new CoreOptions(schema.options()); @@ -853,7 +1145,18 @@ private void updateHmsTable(Table table, Identifier identifier, TableSchema sche } // update location - locationHelper.specifyTableLocation(table, getTableLocation(identifier).toString()); + if (location == null) { + location = getTableLocation(identifier, table); + } + locationHelper.specifyTableLocation(table, location.toString()); + } + + private Map setSerDeInfoParam(String provider) { + Map param = new HashMap<>(); + if (provider != null && provider.equalsIgnoreCase("csv")) { + param.put(FIELD_DELIM, options.get(FIELD_DELIMITER)); + } + return param; } private void updateHmsTablePars(Table table, TableSchema schema) { @@ -899,9 +1202,8 @@ private FieldSchema convertToFieldSchema(DataField dataField) { dataField.description()); } - private SchemaManager schemaManager(Identifier identifier) { - return new SchemaManager( - fileIO, getTableLocation(identifier), identifier.getBranchNameOrDefault()) + private SchemaManager schemaManager(Identifier identifier, Path location) { + return new SchemaManager(fileIO, location, identifier.getBranchNameOrDefault()) .withLock(lock(identifier)); } @@ -927,8 +1229,7 @@ public static HiveConf createHiveConf( // create HiveConf from hadoop configuration with hadoop conf directory configured. Configuration hadoopConf = defaultHadoopConf; if (!isNullOrWhitespaceOnly(hadoopConfDir)) { - hadoopConf = getHadoopConfiguration(hadoopConfDir); - if (hadoopConf == null) { + if (!addHadoopConfIfFound(hadoopConf, hadoopConfDir, new Options())) { String possiableUsedConfFiles = "core-site.xml | hdfs-site.xml | yarn-site.xml | mapred-site.xml"; throw new RuntimeException( @@ -1032,46 +1333,6 @@ public static HiveConf createHiveConf(CatalogContext context) { return hiveConf; } - /** - * Returns a new Hadoop Configuration object using the path to the hadoop conf configured. - * - * @param hadoopConfDir Hadoop conf directory path. - * @return A Hadoop configuration instance. - */ - public static Configuration getHadoopConfiguration(String hadoopConfDir) { - if (new File(hadoopConfDir).exists()) { - List possiableConfFiles = new ArrayList(); - File coreSite = new File(hadoopConfDir, "core-site.xml"); - if (coreSite.exists()) { - possiableConfFiles.add(coreSite); - } - File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml"); - if (hdfsSite.exists()) { - possiableConfFiles.add(hdfsSite); - } - File yarnSite = new File(hadoopConfDir, "yarn-site.xml"); - if (yarnSite.exists()) { - possiableConfFiles.add(yarnSite); - } - // Add mapred-site.xml. We need to read configurations like compression codec. - File mapredSite = new File(hadoopConfDir, "mapred-site.xml"); - if (mapredSite.exists()) { - possiableConfFiles.add(mapredSite); - } - if (possiableConfFiles.isEmpty()) { - return null; - } else { - Configuration hadoopConfiguration = new Configuration(); - for (File confFile : possiableConfFiles) { - hadoopConfiguration.addResource( - new org.apache.hadoop.fs.Path(confFile.getAbsolutePath())); - } - return hadoopConfiguration; - } - } - return null; - } - public static String possibleHiveConfPath() { return System.getenv("HIVE_CONF_DIR"); } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogOptions.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogOptions.java index c74fa447ea46..38f73bc6bd65 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogOptions.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogOptions.java @@ -85,14 +85,5 @@ public final class HiveCatalogOptions { + "E.g. specifying \"conf:a.b.c\" will add \"a.b.c\" to the key, and so that configurations with different default catalog wouldn't share the same client pool. Multiple conf elements can be specified.")) .build()); - public static final ConfigOption FORMAT_TABLE_ENABLED = - ConfigOptions.key("format-table.enabled") - .booleanType() - .defaultValue(false) - .withDescription( - "Whether to support format tables, format table corresponds to a regular Hive table, allowing read and write operations. " - + "However, during these processes, it does not connect to the metastore; hence, newly added partitions will not be reflected in" - + " the metastore and need to be manually added as separate partition operations."); - private HiveCatalogOptions() {} } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java index b006f09c99a4..cb70e0191145 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveMetastoreClient.java @@ -40,6 +40,8 @@ import java.util.ArrayList; import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; /** {@link MetastoreClient} for Hive tables. */ public class HiveMetastoreClient implements MetastoreClient { @@ -56,11 +58,13 @@ public class HiveMetastoreClient implements MetastoreClient { ClientPool clients) throws TException, InterruptedException { this.identifier = identifier; + CoreOptions options = new CoreOptions(schema.options()); this.partitionComputer = new InternalRowPartitionComputer( - new CoreOptions(schema.options()).partitionDefaultName(), + options.partitionDefaultName(), schema.logicalPartitionType(), - schema.partitionKeys().toArray(new String[0])); + schema.partitionKeys().toArray(new String[0]), + options.legacyPartitionName()); this.clients = clients; this.sd = @@ -78,6 +82,14 @@ public void addPartition(BinaryRow partition) throws Exception { addPartition(partitionComputer.generatePartValues(partition)); } + @Override + public void addPartitions(List partitions) throws Exception { + addPartitionsSpec( + partitions.stream() + .map(partitionComputer::generatePartValues) + .collect(Collectors.toList())); + } + @Override public void addPartition(LinkedHashMap partitionSpec) throws Exception { List partitionValues = new ArrayList<>(partitionSpec.values()); @@ -91,25 +103,49 @@ public void addPartition(LinkedHashMap partitionSpec) throws Exc // do nothing if the partition already exists } catch (NoSuchObjectException e) { // partition not found, create new partition - StorageDescriptor newSd = new StorageDescriptor(sd); - newSd.setLocation( - sd.getLocation() - + "/" - + PartitionPathUtils.generatePartitionPath(partitionSpec)); - - Partition hivePartition = new Partition(); - hivePartition.setDbName(identifier.getDatabaseName()); - hivePartition.setTableName(identifier.getTableName()); - hivePartition.setValues(partitionValues); - hivePartition.setSd(newSd); - int currentTime = (int) (System.currentTimeMillis() / 1000); - hivePartition.setCreateTime(currentTime); - hivePartition.setLastAccessTime(currentTime); - + Partition hivePartition = + toHivePartition(partitionSpec, (int) (System.currentTimeMillis() / 1000)); clients.execute(client -> client.add_partition(hivePartition)); } } + @Override + public void addPartitionsSpec(List> partitionSpecsList) + throws Exception { + int currentTime = (int) (System.currentTimeMillis() / 1000); + List hivePartitions = + partitionSpecsList.stream() + .map(partitionSpec -> toHivePartition(partitionSpec, currentTime)) + .collect(Collectors.toList()); + clients.execute(client -> client.add_partitions(hivePartitions, true, false)); + } + + @Override + public void alterPartition( + LinkedHashMap partitionSpec, + Map parameters, + long modifyTime) + throws Exception { + List partitionValues = new ArrayList<>(partitionSpec.values()); + int currentTime = (int) (modifyTime / 1000); + Partition hivePartition = + clients.run( + client -> + client.getPartition( + identifier.getDatabaseName(), + identifier.getObjectName(), + partitionValues)); + hivePartition.setValues(partitionValues); + hivePartition.setLastAccessTime(currentTime); + hivePartition.getParameters().putAll(parameters); + clients.execute( + client -> + client.alter_partition( + identifier.getDatabaseName(), + identifier.getObjectName(), + hivePartition)); + } + @Override public void deletePartition(LinkedHashMap partitionSpec) throws Exception { List partitionValues = new ArrayList<>(partitionSpec.values()); @@ -150,6 +186,21 @@ public IMetaStoreClient client() throws TException, InterruptedException { return clients.run(client -> client); } + private Partition toHivePartition( + LinkedHashMap partitionSpec, int currentTime) { + Partition hivePartition = new Partition(); + StorageDescriptor newSd = new StorageDescriptor(sd); + newSd.setLocation( + sd.getLocation() + "/" + PartitionPathUtils.generatePartitionPath(partitionSpec)); + hivePartition.setDbName(identifier.getDatabaseName()); + hivePartition.setTableName(identifier.getTableName()); + hivePartition.setValues(new ArrayList<>(partitionSpec.values())); + hivePartition.setSd(newSd); + hivePartition.setCreateTime(currentTime); + hivePartition.setLastAccessTime(currentTime); + return hivePartition; + } + /** Factory to create {@link HiveMetastoreClient}. */ public static class Factory implements MetastoreClient.Factory { diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveFormatTableUtils.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveTableUtils.java similarity index 92% rename from paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveFormatTableUtils.java rename to paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveTableUtils.java index dcb7c7ee74cb..fef2d395298f 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveFormatTableUtils.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveTableUtils.java @@ -40,7 +40,7 @@ import static org.apache.paimon.catalog.Catalog.COMMENT_PROP; import static org.apache.paimon.table.FormatTableOptions.FIELD_DELIMITER; -class HiveFormatTableUtils { +class HiveTableUtils { public static FormatTable convertToFormatTable(Table hiveTable) { if (TableType.valueOf(hiveTable.getTableType()) == TableType.VIRTUAL_VIEW) { @@ -50,9 +50,10 @@ public static FormatTable convertToFormatTable(Table hiveTable) { Identifier identifier = new Identifier(hiveTable.getDbName(), hiveTable.getTableName()); Map options = new HashMap<>(hiveTable.getParameters()); List partitionKeys = getFieldNames(hiveTable.getPartitionKeys()); - RowType rowType = createRowType(hiveTable.getSd().getCols(), hiveTable.getPartitionKeys()); + RowType rowType = createRowType(hiveTable); String comment = options.remove(COMMENT_PROP); String location = hiveTable.getSd().getLocation(); + Format format; SerDeInfo serdeInfo = hiveTable.getSd().getSerdeInfo(); String serLib = serdeInfo.getSerializationLib().toLowerCase(); @@ -70,6 +71,7 @@ public static FormatTable convertToFormatTable(Table hiveTable) { } else { throw new UnsupportedOperationException("Unsupported table: " + hiveTable); } + return FormatTable.builder() .identifier(identifier) .rowType(rowType) @@ -91,10 +93,9 @@ private static List getFieldNames(List fieldSchemas) { } /** Create a Paimon's Schema from Hive table's columns and partition keys. */ - private static RowType createRowType( - List nonPartCols, List partitionKeys) { - List allCols = new ArrayList<>(nonPartCols); - allCols.addAll(partitionKeys); + public static RowType createRowType(Table table) { + List allCols = new ArrayList<>(table.getSd().getCols()); + allCols.addAll(table.getPartitionKeys()); Pair columnInformation = extractColumnInformation(allCols); return RowType.builder() .fields(columnInformation.getRight(), columnInformation.getLeft()) diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/RetryingMetaStoreClientFactory.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/RetryingMetaStoreClientFactory.java index bc5a887057d8..0ac665fa4e11 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/RetryingMetaStoreClientFactory.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/RetryingMetaStoreClientFactory.java @@ -21,6 +21,7 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.common.JavaUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.HiveMetaHookLoader; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; @@ -29,7 +30,9 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.util.Arrays; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; @@ -75,6 +78,16 @@ public class RetryingMetaStoreClientFactory { new ConcurrentHashMap<>(), clientClassName, true)) + .put( + new Class[] { + HiveConf.class, + Class[].class, + Object[].class, + ConcurrentHashMap.class, + String.class + }, + RetryingMetaStoreClientFactory + ::constructorDetectedHiveMetastoreProxySupplier) // for hive 3.x .put( new Class[] { @@ -103,23 +116,8 @@ public class RetryingMetaStoreClientFactory { ConcurrentHashMap.class, String.class }, - (getProxyMethod, hiveConf, clientClassName) -> - (IMetaStoreClient) - getProxyMethod.invoke( - null, - hiveConf, - new Class[] { - HiveConf.class, - HiveMetaHookLoader.class, - Boolean.class - }, - new Object[] { - hiveConf, - (HiveMetaHookLoader) (tbl -> null), - true - }, - new ConcurrentHashMap<>(), - clientClassName)) + RetryingMetaStoreClientFactory + ::constructorDetectedHiveMetastoreProxySupplier) .build(); // If clientClassName is HiveMetaStoreClient, @@ -175,4 +173,50 @@ public interface HiveMetastoreProxySupplier { IMetaStoreClient get(Method getProxyMethod, Configuration conf, String clientClassName) throws IllegalAccessException, IllegalArgumentException, InvocationTargetException; } + + /** Detect the client class whether it has the proper constructor. */ + private static IMetaStoreClient constructorDetectedHiveMetastoreProxySupplier( + Method getProxyMethod, Configuration hiveConf, String clientClassName) + throws IllegalAccessException, IllegalArgumentException, InvocationTargetException { + + try { + Class baseClass = Class.forName(clientClassName, false, JavaUtils.getClassLoader()); + + // Configuration.class or HiveConf.class + List> possibleFirstParamTypes = + Arrays.asList(getProxyMethod.getParameterTypes()[0], hiveConf.getClass()); + + for (Class possibleFirstParamType : possibleFirstParamTypes) { + Class[] fullParams = + new Class[] { + possibleFirstParamType, HiveMetaHookLoader.class, Boolean.TYPE + }; + Object[] fullParamValues = + new Object[] {hiveConf, (HiveMetaHookLoader) (tbl -> null), Boolean.TRUE}; + + for (int i = fullParams.length; i >= 1; i--) { + try { + baseClass.getConstructor(Arrays.copyOfRange(fullParams, 0, i)); + return (IMetaStoreClient) + getProxyMethod.invoke( + null, + hiveConf, + Arrays.copyOfRange(fullParams, 0, i), + Arrays.copyOfRange(fullParamValues, 0, i), + new ConcurrentHashMap<>(), + clientClassName); + } catch (NoSuchMethodException ignored) { + } + } + } + + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } + + throw new IllegalArgumentException( + "Failed to create the desired metastore client with proper constructors (class name: " + + clientClassName + + ")"); + } } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/migrate/HiveMigrator.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/migrate/HiveMigrator.java index b9928ce7311b..d1478830ac6d 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/migrate/HiveMigrator.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/migrate/HiveMigrator.java @@ -19,6 +19,7 @@ package org.apache.paimon.hive.migrate; import org.apache.paimon.CoreOptions; +import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryWriter; @@ -81,8 +82,8 @@ public class HiveMigrator implements Migrator { private final String targetDatabase; private final String targetTable; private final CoreOptions coreOptions; - private Boolean delete = true; - private Integer parallelism; + + private Boolean deleteOriginTable = true; public HiveMigrator( HiveCatalog hiveCatalog, @@ -99,7 +100,6 @@ public HiveMigrator( this.sourceTable = sourceTable; this.targetDatabase = targetDatabase; this.targetTable = targetTable; - this.parallelism = parallelism; this.coreOptions = new CoreOptions(options); this.executor = createCachedThreadPool(parallelism, "HIVE_MIGRATOR"); } @@ -129,8 +129,8 @@ public static List databaseMigrators( } @Override - public void deleteOriginTable(boolean delete) { - this.delete = delete; + public void deleteOriginTable(boolean deleteOriginTable) { + this.deleteOriginTable = deleteOriginTable; } @Override @@ -145,14 +145,18 @@ public void executeMigrate() throws Exception { // create paimon table if not exists Identifier identifier = Identifier.create(targetDatabase, targetTable); - boolean alreadyExist = hiveCatalog.tableExists(identifier); - if (!alreadyExist) { + + boolean deleteIfFail = false; + try { + hiveCatalog.getTable(identifier); + } catch (Catalog.TableNotExistException e) { Schema schema = from( client.getSchema(sourceDatabase, sourceTable), sourceHiveTable.getPartitionKeys(), properties); hiveCatalog.createTable(identifier, schema, false); + deleteIfFail = true; } try { @@ -211,14 +215,14 @@ public void executeMigrate() throws Exception { commit.commit(new ArrayList<>(commitMessages)); } } catch (Exception e) { - if (!alreadyExist) { + if (deleteIfFail) { hiveCatalog.dropTable(identifier, true); } throw new RuntimeException("Migrating failed", e); } // if all success, drop the origin table according the delete field - if (delete) { + if (deleteOriginTable) { client.dropTable(sourceDatabase, sourceTable, true, true); } } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/pool/CachedClientPool.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/pool/CachedClientPool.java index a03098083d3c..f1d09e297b45 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/pool/CachedClientPool.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/pool/CachedClientPool.java @@ -133,24 +133,12 @@ public R run(Action action) return clientPool().run(action); } - @Override - public R run(Action action, boolean retry) - throws TException, InterruptedException { - return clientPool().run(action, retry); - } - @Override public void execute(ExecuteAction action) throws TException, InterruptedException { clientPool().execute(action); } - @Override - public void execute(ExecuteAction action, boolean retry) - throws TException, InterruptedException { - clientPool().execute(action, retry); - } - @VisibleForTesting static Key extractKey(String clientClassName, String cacheKeys, Configuration conf) { // generate key elements in a certain order, so that the Key instances are comparable diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/pool/HiveClientPool.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/pool/HiveClientPool.java index 6dd086950771..687baf37b645 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/pool/HiveClientPool.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/pool/HiveClientPool.java @@ -18,16 +18,15 @@ package org.apache.paimon.hive.pool; -import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.client.ClientPool; import org.apache.paimon.hive.RetryingMetaStoreClientFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.thrift.TException; -import org.apache.thrift.transport.TTransportException; + +import java.util.function.Supplier; /** * Pool of Hive Metastore clients. @@ -36,49 +35,19 @@ */ public class HiveClientPool extends ClientPool.ClientPoolImpl { - private final HiveConf hiveConf; - private final String clientClassName; - public HiveClientPool(int poolSize, Configuration conf, String clientClassName) { - // Do not allow retry by default as we rely on RetryingHiveClient - super(poolSize, TTransportException.class, false); - this.hiveConf = new HiveConf(conf, HiveClientPool.class); - this.hiveConf.addResource(conf); - this.clientClassName = clientClassName; - } - - @Override - protected IMetaStoreClient newClient() { - return new RetryingMetaStoreClientFactory().createClient(hiveConf, clientClassName); + super(poolSize, clientSupplier(conf, clientClassName)); } - @Override - protected IMetaStoreClient reconnect(IMetaStoreClient client) { - try { - client.close(); - client.reconnect(); - } catch (MetaException e) { - throw new RuntimeException("Failed to reconnect to Hive Metastore", e); - } - return client; - } - - @Override - protected boolean isConnectionException(Exception e) { - return super.isConnectionException(e) - || (e instanceof MetaException - && e.getMessage() - .contains( - "Got exception: org.apache.thrift.transport.TTransportException")); + private static Supplier clientSupplier( + Configuration conf, String clientClassName) { + HiveConf hiveConf = new HiveConf(conf, HiveClientPool.class); + hiveConf.addResource(conf); + return () -> new RetryingMetaStoreClientFactory().createClient(hiveConf, clientClassName); } @Override protected void close(IMetaStoreClient client) { client.close(); } - - @VisibleForTesting - HiveConf hiveConf() { - return hiveConf; - } } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitter.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitter.java new file mode 100644 index 000000000000..d913f729e351 --- /dev/null +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitter.java @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.iceberg; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.client.ClientPool; +import org.apache.paimon.fs.Path; +import org.apache.paimon.hive.HiveCatalog; +import org.apache.paimon.hive.HiveCatalogFactory; +import org.apache.paimon.hive.HiveTypeUtils; +import org.apache.paimon.hive.pool.CachedClientPool; +import org.apache.paimon.options.Options; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.types.DataField; +import org.apache.paimon.utils.Preconditions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.Collections; +import java.util.HashMap; +import java.util.stream.Collectors; + +/** + * {@link IcebergMetadataCommitter} to commit Iceberg metadata to Hive metastore, so the table can + * be visited by Iceberg's Hive catalog. + */ +public class IcebergHiveMetadataCommitter implements IcebergMetadataCommitter { + + private static final Logger LOG = LoggerFactory.getLogger(IcebergHiveMetadataCommitter.class); + + private final FileStoreTable table; + private final Identifier identifier; + private final ClientPool clients; + + public IcebergHiveMetadataCommitter(FileStoreTable table) { + this.table = table; + this.identifier = + Preconditions.checkNotNull( + table.catalogEnvironment().identifier(), + "If you want to sync Paimon Iceberg compatible metadata to Hive, " + + "you must use a Paimon table created from a Paimon catalog, " + + "instead of a temporary table."); + Preconditions.checkArgument( + identifier.getBranchName() == null, + "Paimon Iceberg compatibility currently does not support branches."); + + Options options = new Options(table.options()); + String uri = options.get(IcebergOptions.URI); + String hiveConfDir = options.get(IcebergOptions.HIVE_CONF_DIR); + String hadoopConfDir = options.get(IcebergOptions.HADOOP_CONF_DIR); + Configuration hadoopConf = new Configuration(); + hadoopConf.setClassLoader(IcebergHiveMetadataCommitter.class.getClassLoader()); + HiveConf hiveConf = HiveCatalog.createHiveConf(hiveConfDir, hadoopConfDir, hadoopConf); + + table.options().forEach(hiveConf::set); + if (uri != null) { + hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, uri); + } + + if (hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname) == null) { + LOG.error( + "Can't find hive metastore uri to connect: " + + "either set {} for paimon table or set hive.metastore.uris " + + "in hive-site.xml or hadoop configurations. " + + "Will use empty metastore uris, which means we may use a embedded metastore. " + + "This may cause unpredictable consensus problem.", + IcebergOptions.URI.key()); + } + + this.clients = + new CachedClientPool( + hiveConf, + options, + HiveCatalogFactory.METASTORE_CLIENT_CLASS.defaultValue()); + } + + @Override + public void commitMetadata(Path newMetadataPath, @Nullable Path baseMetadataPath) { + try { + commitMetadataImpl(newMetadataPath, baseMetadataPath); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void commitMetadataImpl(Path newMetadataPath, @Nullable Path baseMetadataPath) + throws Exception { + if (!databaseExists(identifier.getDatabaseName())) { + createDatabase(identifier.getDatabaseName()); + } + + Table hiveTable; + if (tableExists(identifier)) { + hiveTable = + clients.run( + client -> + client.getTable( + identifier.getDatabaseName(), + identifier.getTableName())); + } else { + hiveTable = createTable(newMetadataPath); + } + + hiveTable.getParameters().put("metadata_location", newMetadataPath.toString()); + if (baseMetadataPath != null) { + hiveTable + .getParameters() + .put("previous_metadata_location", baseMetadataPath.toString()); + } + + clients.execute( + client -> + client.alter_table( + identifier.getDatabaseName(), + identifier.getTableName(), + hiveTable, + true)); + } + + private boolean databaseExists(String databaseName) throws Exception { + try { + clients.run(client -> client.getDatabase(databaseName)); + return true; + } catch (NoSuchObjectException ignore) { + return false; + } + } + + private void createDatabase(String databaseName) throws Exception { + Database database = new Database(); + database.setName(databaseName); + clients.execute(client -> client.createDatabase(database)); + } + + private boolean tableExists(Identifier identifier) throws Exception { + return clients.run( + client -> + client.tableExists( + identifier.getDatabaseName(), identifier.getTableName())); + } + + private Table createTable(Path metadataPath) throws Exception { + long currentTimeMillis = System.currentTimeMillis(); + Table hiveTable = + new Table( + identifier.getTableName(), + identifier.getDatabaseName(), + // current linux user + System.getProperty("user.name"), + (int) (currentTimeMillis / 1000), + (int) (currentTimeMillis / 1000), + Integer.MAX_VALUE, + new StorageDescriptor(), + Collections.emptyList(), + new HashMap<>(), + null, + null, + "EXTERNAL_TABLE"); + + hiveTable.getParameters().put("DO_NOT_UPDATE_STATS", "true"); + hiveTable.getParameters().put("EXTERNAL", "TRUE"); + hiveTable.getParameters().put("table_type", "ICEBERG"); + + StorageDescriptor sd = hiveTable.getSd(); + sd.setLocation(metadataPath.getParent().getParent().toString()); + sd.setCols( + table.schema().fields().stream() + .map(this::convertToFieldSchema) + .collect(Collectors.toList())); + sd.setInputFormat("org.apache.hadoop.mapred.FileInputFormat"); + sd.setOutputFormat("org.apache.hadoop.mapred.FileOutputFormat"); + + SerDeInfo serDeInfo = new SerDeInfo(); + serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"); + hiveTable.getSd().setSerdeInfo(serDeInfo); + + clients.execute(client -> client.createTable(hiveTable)); + return hiveTable; + } + + private FieldSchema convertToFieldSchema(DataField dataField) { + return new FieldSchema( + dataField.name(), + HiveTypeUtils.toTypeInfo(dataField.type()).getTypeName(), + dataField.description()); + } +} diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitterFactory.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitterFactory.java new file mode 100644 index 000000000000..2ae279d3d7af --- /dev/null +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitterFactory.java @@ -0,0 +1,35 @@ +/* + * 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.iceberg; + +import org.apache.paimon.table.FileStoreTable; + +/** Factory to create {@link IcebergHiveMetadataCommitter}. */ +public class IcebergHiveMetadataCommitterFactory implements IcebergMetadataCommitterFactory { + + @Override + public String identifier() { + return IcebergOptions.StorageType.HIVE_CATALOG.toString(); + } + + @Override + public IcebergMetadataCommitter create(FileStoreTable table) { + return new IcebergHiveMetadataCommitter(table); + } +} diff --git a/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index baab92184129..26f0944d916e 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-hive/paimon-hive-catalog/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -15,3 +15,4 @@ org.apache.paimon.hive.HiveCatalogFactory org.apache.paimon.hive.HiveCatalogLockFactory +org.apache.paimon.iceberg.IcebergHiveMetadataCommitterFactory diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java index 6b13a80e801a..dcd770c57f51 100644 --- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java +++ b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java @@ -18,6 +18,7 @@ package org.apache.paimon.hive; +import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogTestBase; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.client.ClientPool; @@ -39,6 +40,7 @@ import org.junit.jupiter.api.Test; import java.lang.reflect.Field; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -46,6 +48,7 @@ import java.util.Locale; import java.util.Map; import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTORECONNECTURLKEY; import static org.apache.paimon.hive.HiveCatalog.PAIMON_TABLE_TYPE_VALUE; @@ -268,4 +271,92 @@ public void testAlterHiveTableParameters() { fail("Test failed due to exception: " + e.getMessage()); } } + + @Test + public void testListTablesLock() { + try { + String databaseName = "test_db"; + catalog.createDatabase(databaseName, false); + + Map options = new HashMap<>(); + Schema addHiveTableParametersSchema = + new Schema( + Lists.newArrayList( + new DataField(0, "pk", DataTypes.INT()), + new DataField(1, "col1", DataTypes.STRING()), + new DataField(2, "col2", DataTypes.STRING())), + Collections.emptyList(), + Collections.emptyList(), + options, + "this is a hive table"); + + for (int i = 0; i < 100; i++) { + String tableName = "new_table" + i; + catalog.createTable( + Identifier.create(databaseName, tableName), + addHiveTableParametersSchema, + false); + } + List tables1 = new ArrayList<>(); + List tables2 = new ArrayList<>(); + + Thread thread1 = + new Thread( + () -> { + System.out.println( + "First thread started at " + System.currentTimeMillis()); + try { + tables1.addAll(catalog.listTables(databaseName)); + } catch (Catalog.DatabaseNotExistException e) { + throw new RuntimeException(e); + } + }); + Thread thread2 = + new Thread( + () -> { + System.out.println( + "Second thread started at " + System.currentTimeMillis()); + try { + tables2.addAll(catalog.listTables(databaseName)); + } catch (Catalog.DatabaseNotExistException e) { + throw new RuntimeException(e); + } + }); + + thread1.start(); + thread2.start(); + + long timeout = 5000; + long startTime = System.currentTimeMillis(); + + AtomicBoolean deadlockDetected = new AtomicBoolean(false); + while (thread1.isAlive() || thread2.isAlive()) { + if (System.currentTimeMillis() - startTime > timeout) { + deadlockDetected.set(true); + thread1.interrupt(); + thread2.interrupt(); + break; + } + + Thread.sleep(100); + } + + assertThat(deadlockDetected).isFalse(); + assertThat(tables1).size().isEqualTo(100); + assertThat(tables1).containsAll(tables2); + assertThat(tables2).containsAll(tables1); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + protected boolean supportsView() { + return true; + } + + @Override + protected boolean supportsFormatTable() { + return true; + } } diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/pool/TestHiveClientPool.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/pool/TestHiveClientPool.java deleted file mode 100644 index a5daefae7940..000000000000 --- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/pool/TestHiveClientPool.java +++ /dev/null @@ -1,190 +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.hive.pool; - -import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.Function; -import org.apache.hadoop.hive.metastore.api.FunctionType; -import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.PrincipalType; -import org.apache.thrift.transport.TTransportException; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.mockito.Mockito; - -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.nio.charset.StandardCharsets; -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** - * Test Hive Client Pool. - * - *

    Mostly copied from iceberg. - */ -public class TestHiveClientPool { - - private static final String HIVE_SITE_CONTENT = - "\n" - + "\n" - + "\n" - + " \n" - + " hive.metastore.sasl.enabled\n" - + " true\n" - + " \n" - + "\n"; - - HiveClientPool clients; - - @BeforeEach - public void before() { - String metastoreClientClass = "org.apache.hadoop.hive.metastore.HiveMetaStoreClient"; - HiveClientPool clientPool = - new HiveClientPool(2, new Configuration(), metastoreClientClass); - clients = Mockito.spy(clientPool); - } - - @AfterEach - public void after() { - clients.close(); - clients = null; - } - - @Test - public void testConf() { - HiveConf conf = createHiveConf(); - conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "warehouse"); - - String metastoreClientClass = "org.apache.hadoop.hive.metastore.HiveMetaStoreClient"; - HiveClientPool clientPool = new HiveClientPool(10, conf, metastoreClientClass); - HiveConf clientConf = clientPool.hiveConf(); - - assertThat(clientConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)) - .isEqualTo(conf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); - assertThat(clientPool.poolSize()).isEqualTo(10); - - // 'hive.metastore.sasl.enabled' should be 'true' as defined in xml - assertThat(clientConf.get(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname)) - .isEqualTo(conf.get(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL.varname)); - assertThat(clientConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL)).isTrue(); - } - - private HiveConf createHiveConf() { - HiveConf hiveConf = new HiveConf(); - try (InputStream inputStream = - new ByteArrayInputStream(HIVE_SITE_CONTENT.getBytes(StandardCharsets.UTF_8))) { - hiveConf.addResource(inputStream, "for_test"); - } catch (IOException e) { - throw new RuntimeException(e); - } - return hiveConf; - } - - @Test - public void testNewClientFailure() { - Mockito.doThrow(new RuntimeException("Connection exception")).when(clients).newClient(); - assertThatThrownBy(() -> clients.run(Object::toString)) - .isInstanceOf(RuntimeException.class) - .hasMessage("Connection exception"); - } - - @Test - public void testGetTablesFailsForNonReconnectableException() throws Exception { - HiveMetaStoreClient hmsClient = Mockito.mock(HiveMetaStoreClient.class); - Mockito.doReturn(hmsClient).when(clients).newClient(); - Mockito.doThrow(new MetaException("Another meta exception")) - .when(hmsClient) - .getTables(Mockito.anyString(), Mockito.anyString()); - assertThatThrownBy(() -> clients.run(client -> client.getTables("default", "t"))) - .isInstanceOf(MetaException.class) - .hasMessage("Another meta exception"); - } - - @Test - public void testConnectionFailureRestoreForMetaException() throws Exception { - HiveMetaStoreClient hmsClient = newClient(); - - // Throwing an exception may trigger the client to reconnect. - String metaMessage = "Got exception: org.apache.thrift.transport.TTransportException"; - Mockito.doThrow(new MetaException(metaMessage)).when(hmsClient).getAllDatabases(); - - // Create a new client when the reconnect method is called. - HiveMetaStoreClient newClient = reconnect(hmsClient); - - List databases = Lists.newArrayList("db1", "db2"); - - Mockito.doReturn(databases).when(newClient).getAllDatabases(); - // The return is OK when the reconnect method is called. - assertThat((List) clients.run(client -> client.getAllDatabases(), true)) - .isEqualTo(databases); - - // Verify that the method is called. - Mockito.verify(clients).reconnect(hmsClient); - Mockito.verify(clients, Mockito.never()).reconnect(newClient); - } - - @Test - public void testConnectionFailureRestoreForTTransportException() throws Exception { - HiveMetaStoreClient hmsClient = newClient(); - Mockito.doThrow(new TTransportException()).when(hmsClient).getAllFunctions(); - - // Create a new client when getAllFunctions() failed. - HiveMetaStoreClient newClient = reconnect(hmsClient); - - GetAllFunctionsResponse response = new GetAllFunctionsResponse(); - response.addToFunctions( - new Function( - "concat", - "db1", - "classname", - "root", - PrincipalType.USER, - 100, - FunctionType.JAVA, - null)); - Mockito.doReturn(response).when(newClient).getAllFunctions(); - assertThat((GetAllFunctionsResponse) clients.run(client -> client.getAllFunctions(), true)) - .isEqualTo(response); - - Mockito.verify(clients).reconnect(hmsClient); - Mockito.verify(clients, Mockito.never()).reconnect(newClient); - } - - private HiveMetaStoreClient newClient() { - HiveMetaStoreClient hmsClient = Mockito.mock(HiveMetaStoreClient.class); - Mockito.doReturn(hmsClient).when(clients).newClient(); - return hmsClient; - } - - private HiveMetaStoreClient reconnect(HiveMetaStoreClient obsoleteClient) { - HiveMetaStoreClient newClient = Mockito.mock(HiveMetaStoreClient.class); - Mockito.doReturn(newClient).when(clients).reconnect(obsoleteClient); - return newClient; - } -} diff --git a/paimon-hive/paimon-hive-connector-2.3/pom.xml b/paimon-hive/paimon-hive-connector-2.3/pom.xml index e61e493d3a5c..a0f509b53375 100644 --- a/paimon-hive/paimon-hive-connector-2.3/pom.xml +++ b/paimon-hive/paimon-hive-connector-2.3/pom.xml @@ -81,7 +81,7 @@ under the License. org.apache.flink - flink-table-planner_${scala.binary.version} + flink-table-planner_${flink.scala.binary.version} ${test.flink.version} test @@ -562,6 +562,13 @@ under the License. test + + + org.apache.iceberg + iceberg-flink-${iceberg.flink.version} + ${iceberg.version} + test + diff --git a/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/CustomConstructorMetastoreClient.java b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/CustomConstructorMetastoreClient.java new file mode 100644 index 000000000000..c9ddc1be468f --- /dev/null +++ b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/CustomConstructorMetastoreClient.java @@ -0,0 +1,54 @@ +/* + * 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.hive; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaHookLoader; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; + +/** A {@link HiveMetaStoreClient} Factory to test custom Hive metastore client. */ +public class CustomConstructorMetastoreClient { + + /** + * A {@link HiveMetaStoreClient} to test custom Hive metastore client with (HiveConf, + * HiveMetaHookLoader) constructor. + */ + public static class TwoParameterConstructorMetastoreClient extends HiveMetaStoreClient + implements IMetaStoreClient { + + public TwoParameterConstructorMetastoreClient(HiveConf conf, HiveMetaHookLoader hookLoader) + throws MetaException { + super(conf, hookLoader); + } + } + + /** + * A {@link HiveMetaStoreClient} to test custom Hive metastore client with (HiveConf) + * constructor. + */ + public static class OneParameterConstructorMetastoreClient extends HiveMetaStoreClient + implements IMetaStoreClient { + + public OneParameterConstructorMetastoreClient(HiveConf conf) throws MetaException { + super(conf); + } + } +} diff --git a/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/Hive23CatalogITCase.java b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/Hive23CatalogITCase.java index 204f779d71b5..8a4745a09022 100644 --- a/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/Hive23CatalogITCase.java +++ b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/Hive23CatalogITCase.java @@ -85,7 +85,35 @@ public void testCustomMetastoreClient() throws Exception { } @Test - public void testCreateExistTableInHive() throws Exception { + public void testCustomConstructorMetastoreClient() throws Exception { + path = folder.newFolder().toURI().toString(); + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + Class[] customConstructorMetastoreClientClass = { + CustomConstructorMetastoreClient.TwoParameterConstructorMetastoreClient.class, + CustomConstructorMetastoreClient.OneParameterConstructorMetastoreClient.class + }; + for (Class clazz : customConstructorMetastoreClientClass) { + tEnv = TableEnvironmentImpl.create(settings); + tEnv.executeSql( + String.join( + "\n", + "CREATE CATALOG my_hive WITH (", + " 'type' = 'paimon',", + " 'metastore' = 'hive',", + " 'uri' = '',", + " 'default-database' = 'test_db',", + " 'warehouse' = '" + path + "',", + " 'metastore.client.class' = '" + clazz.getName() + "'", + ")")) + .await(); + tEnv.executeSql("USE CATALOG my_hive").await(); + assertThat(collect("SHOW DATABASES")) + .isEqualTo(Arrays.asList(Row.of("default"), Row.of("test_db"))); + } + } + + @Test + public void testCreateExistTableInHive() { tEnv.executeSql( String.join( "\n", @@ -105,7 +133,6 @@ public void testCreateExistTableInHive() throws Exception { tEnv.executeSql( "CREATE TABLE hive_table(a INT, b INT, c INT, d INT)") .await()) - .isInstanceOf(TableException.class) .hasMessage( "Could not execute CreateTable in path `my_hive_custom_client`.`test_db`.`hive_table`"); assertThat( diff --git a/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/iceberg/IcebergHive23MetadataCommitterITCase.java b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/iceberg/IcebergHive23MetadataCommitterITCase.java new file mode 100644 index 000000000000..a9e4ba945440 --- /dev/null +++ b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/iceberg/IcebergHive23MetadataCommitterITCase.java @@ -0,0 +1,22 @@ +/* + * 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.iceberg; + +/** IT cases for {@link IcebergHiveMetadataCommitter} in Hive 2.3. */ +public class IcebergHive23MetadataCommitterITCase extends IcebergHiveMetadataCommitterITCaseBase {} diff --git a/paimon-hive/paimon-hive-connector-3.1/pom.xml b/paimon-hive/paimon-hive-connector-3.1/pom.xml index 3e08196a76af..5383af90c3e5 100644 --- a/paimon-hive/paimon-hive-connector-3.1/pom.xml +++ b/paimon-hive/paimon-hive-connector-3.1/pom.xml @@ -88,7 +88,7 @@ under the License. org.apache.flink - flink-table-planner_${scala.binary.version} + flink-table-planner_${flink.scala.binary.version} ${test.flink.version} test @@ -592,6 +592,13 @@ under the License. test + + + org.apache.iceberg + iceberg-flink-${iceberg.flink.version} + ${iceberg.version} + test + diff --git a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/CustomConstructorMetastoreClient.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/CustomConstructorMetastoreClient.java new file mode 100644 index 000000000000..6b08f16d4ab6 --- /dev/null +++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/CustomConstructorMetastoreClient.java @@ -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. + */ + +package org.apache.paimon.hive; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaHookLoader; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; + +/** A {@link HiveMetaStoreClient} Factory to test custom Hive metastore client. */ +public class CustomConstructorMetastoreClient { + + /** + * A {@link HiveMetaStoreClient} to test custom Hive metastore client with (Configuration, + * HiveMetaHookLoader) constructor. + */ + public static class TwoParameterConstructorMetastoreClient extends HiveMetaStoreClient + implements IMetaStoreClient { + + public TwoParameterConstructorMetastoreClient( + Configuration conf, HiveMetaHookLoader hookLoader) throws MetaException { + super(conf, hookLoader); + } + } + + /** + * A {@link HiveMetaStoreClient} to test custom Hive metastore client with (Configuration) + * constructor. + */ + public static class OneParameterConstructorMetastoreClient extends HiveMetaStoreClient + implements IMetaStoreClient { + + public OneParameterConstructorMetastoreClient(Configuration conf) throws MetaException { + super(conf); + } + } + + /** + * A {@link HiveMetaStoreClient} to test custom Hive metastore client with (HiveConf) + * constructor. + */ + public static class OtherParameterConstructorMetastoreClient extends HiveMetaStoreClient + implements IMetaStoreClient { + + public OtherParameterConstructorMetastoreClient(HiveConf conf) throws MetaException { + super(conf); + } + } +} diff --git a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/Hive31CatalogITCase.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/Hive31CatalogITCase.java index 1cd17553fd3f..48d41d27e8d8 100644 --- a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/Hive31CatalogITCase.java +++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/Hive31CatalogITCase.java @@ -83,6 +83,35 @@ public void testCustomMetastoreClient() throws Exception { Row.of(TestHiveMetaStoreClient.MOCK_DATABASE))); } + @Test + public void testCustomConstructorMetastoreClient() throws Exception { + path = folder.newFolder().toURI().toString(); + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + Class[] customConstructorMetastoreClientClass = { + CustomConstructorMetastoreClient.TwoParameterConstructorMetastoreClient.class, + CustomConstructorMetastoreClient.OneParameterConstructorMetastoreClient.class, + CustomConstructorMetastoreClient.OtherParameterConstructorMetastoreClient.class + }; + + for (Class clazz : customConstructorMetastoreClientClass) { + tEnv = TableEnvironmentImpl.create(settings); + tEnv.executeSql( + String.join( + "\n", + "CREATE CATALOG my_hive WITH (", + " 'type' = 'paimon',", + " 'metastore' = 'hive',", + " 'uri' = '',", + " 'warehouse' = '" + path + "',", + " 'metastore.client.class' = '" + clazz.getName() + "'", + ")")) + .await(); + tEnv.executeSql("USE CATALOG my_hive").await(); + assertThat(collect("SHOW DATABASES")) + .isEqualTo(Arrays.asList(Row.of("default"), Row.of("test_db"))); + } + } + @Test public void testCreateExistTableInHive() throws Exception { tEnv.executeSql( @@ -104,7 +133,6 @@ public void testCreateExistTableInHive() throws Exception { tEnv.executeSql( "CREATE TABLE hive_table(a INT, b INT, c INT, d INT)") .await()) - .isInstanceOf(TableException.class) .hasMessage( "Could not execute CreateTable in path `my_hive_custom_client`.`test_db`.`hive_table`"); assertThat( diff --git a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/iceberg/IcebergHive31MetadataCommitterITCase.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/iceberg/IcebergHive31MetadataCommitterITCase.java new file mode 100644 index 000000000000..6f4b0afd1ae1 --- /dev/null +++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/iceberg/IcebergHive31MetadataCommitterITCase.java @@ -0,0 +1,22 @@ +/* + * 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.iceberg; + +/** IT cases for {@link IcebergHiveMetadataCommitter} in Hive 3.1. */ +public class IcebergHive31MetadataCommitterITCase extends IcebergHiveMetadataCommitterITCaseBase {} diff --git a/paimon-hive/paimon-hive-connector-common/pom.xml b/paimon-hive/paimon-hive-connector-common/pom.xml index 0f7da151b884..f22b73dea71e 100644 --- a/paimon-hive/paimon-hive-connector-common/pom.xml +++ b/paimon-hive/paimon-hive-connector-common/pom.xml @@ -111,14 +111,14 @@ under the License. org.apache.flink - flink-table-planner_${scala.binary.version} + flink-table-planner_${flink.scala.binary.version} ${test.flink.version} test org.apache.flink - flink-connector-hive_${scala.binary.version} + flink-connector-hive_${flink.scala.binary.version} ${test.flink.version} test diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/FlinkGenericCatalogITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/FlinkGenericCatalogITCase.java index f2ae892059b8..8507c4c21aee 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/FlinkGenericCatalogITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/FlinkGenericCatalogITCase.java @@ -201,4 +201,20 @@ public void testReadPaimonAllProcedures() { assertThat(result) .contains(Row.of("compact"), Row.of("merge_into"), Row.of("migrate_table")); } + + @Test + public void testCreateTag() { + sql( + "CREATE TABLE paimon_t ( " + + "f0 INT, " + + "f1 INT " + + ") WITH ('connector'='paimon', 'file.format' = 'avro' )"); + sql("INSERT INTO paimon_t VALUES (1, 1), (2, 2)"); + assertThat(sql("SELECT * FROM paimon_t")) + .containsExactlyInAnyOrder(Row.of(1, 1), Row.of(2, 2)); + sql("CALL sys.create_tag('test_db.paimon_t', 'tag_1')"); + + List result = sql("SELECT tag_name FROM paimon_t$tags"); + assertThat(result).contains(Row.of("tag_1")); + } } diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogFormatTableITCaseBase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogFormatTableITCaseBase.java index 9a7ff1e586a1..fc58ad59527c 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogFormatTableITCaseBase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogFormatTableITCaseBase.java @@ -44,7 +44,7 @@ import java.util.Map; import java.util.stream.Collectors; -import static org.apache.paimon.hive.HiveCatalogOptions.FORMAT_TABLE_ENABLED; +import static org.apache.paimon.options.CatalogOptions.FORMAT_TABLE_ENABLED; import static org.assertj.core.api.Assertions.assertThat; /** IT cases for using Paimon {@link HiveCatalog}. */ @@ -136,6 +136,28 @@ public void testPartitionTable() throws Exception { doTestFormatTable("partition_table"); } + @Test + public void testFlinkCreateCsvFormatTable() throws Exception { + tEnv.executeSql( + "CREATE TABLE flink_csv_table (a INT, b STRING) with ('type'='format-table', 'file.format'='csv')") + .await(); + doTestFormatTable("flink_csv_table"); + } + + @Test + public void testFlinkCreateFormatTableWithDelimiter() throws Exception { + tEnv.executeSql( + "CREATE TABLE flink_csv_table_delimiter (a INT, b STRING) with ('type'='format-table', 'file.format'='csv', 'field-delimiter'=';')"); + doTestFormatTable("flink_csv_table_delimiter"); + } + + @Test + public void testFlinkCreatePartitionTable() throws Exception { + tEnv.executeSql( + "CREATE TABLE flink_partition_table (a INT,b STRING) PARTITIONED BY (b) with ('type'='format-table', 'file.format'='csv')"); + doTestFormatTable("flink_partition_table"); + } + private void doTestFormatTable(String tableName) throws Exception { hiveShell.execute( String.format("INSERT INTO %s VALUES (100, 'Hive'), (200, 'Table')", tableName)); diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java index aa3062ec7db3..74d2d7e1c343 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java @@ -19,15 +19,15 @@ package org.apache.paimon.hive; import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.catalog.CatalogLock; -import org.apache.paimon.catalog.CatalogLockFactory; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkCatalog; import org.apache.paimon.hive.annotation.Minio; import org.apache.paimon.hive.runner.PaimonEmbeddedHiveRunner; import org.apache.paimon.metastore.MetastoreClient; +import org.apache.paimon.operation.Lock; import org.apache.paimon.privilege.NoPrivilegeException; import org.apache.paimon.s3.MinioTestContainer; +import org.apache.paimon.table.CatalogEnvironment; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.utils.IOUtils; @@ -39,7 +39,6 @@ import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.config.ExecutionConfigOptions; @@ -203,7 +202,7 @@ public void testDbLocation() { @Test @LocationInProperties public void testDbLocationWithMetastoreLocationInProperties() - throws Catalog.DatabaseAlreadyExistException { + throws Catalog.DatabaseAlreadyExistException, Catalog.DatabaseNotExistException { String dbLocation = minioTestContainer.getS3UriForDefaultBucket() + "/" + UUID.randomUUID(); Catalog catalog = ((FlinkCatalog) tEnv.getCatalog(tEnv.getCurrentCatalog()).get()).catalog(); @@ -211,7 +210,7 @@ public void testDbLocationWithMetastoreLocationInProperties() properties.put("location", dbLocation); catalog.createDatabase("location_test_db", false, properties); - assertThat(catalog.databaseExists("location_test_db")); + catalog.getDatabase("location_test_db"); hiveShell.execute("USE location_test_db"); hiveShell.execute("CREATE TABLE location_test_db ( a INT, b INT )"); @@ -275,7 +274,8 @@ public void testTableOperations() throws Exception { .await(); tEnv.executeSql("CREATE TABLE s ( a INT, b STRING ) WITH ( 'file.format' = 'avro' )") .await(); - assertThat(collect("SHOW TABLES")).isEqualTo(Arrays.asList(Row.of("s"), Row.of("t"))); + assertThat(collect("SHOW TABLES")) + .containsExactlyInAnyOrder(Row.of("s"), Row.of("t"), Row.of("hive_table")); tEnv.executeSql( "CREATE TABLE IF NOT EXISTS s ( a INT, b STRING ) WITH ( 'file.format' = 'avro' )") @@ -294,17 +294,14 @@ public void testTableOperations() throws Exception { Path tablePath = new Path(path, "test_db.db/s"); assertThat(tablePath.getFileSystem().exists(tablePath)).isTrue(); tEnv.executeSql("DROP TABLE s").await(); - assertThat(collect("SHOW TABLES")).isEqualTo(Collections.singletonList(Row.of("t"))); + assertThat(collect("SHOW TABLES")) + .containsExactlyInAnyOrder(Row.of("t"), Row.of("hive_table")); assertThat(tablePath.getFileSystem().exists(tablePath)).isFalse(); tEnv.executeSql("DROP TABLE IF EXISTS s").await(); assertThatThrownBy(() -> tEnv.executeSql("DROP TABLE s").await()) .isInstanceOf(ValidationException.class) .hasMessage("Table with identifier 'my_hive.test_db.s' does not exist."); - assertThatThrownBy(() -> tEnv.executeSql("DROP TABLE hive_table").await()) - .isInstanceOf(ValidationException.class) - .hasMessage("Table with identifier 'my_hive.test_db.hive_table' does not exist."); - // alter table tEnv.executeSql("ALTER TABLE t SET ( 'manifest.target-file-size' = '16MB' )").await(); List actual = collect("SHOW CREATE TABLE t"); @@ -329,9 +326,9 @@ public void testTableOperations() throws Exception { tEnv.executeSql( "ALTER TABLE hive_table SET ( 'manifest.target-file-size' = '16MB' )") .await()) - .isInstanceOf(RuntimeException.class) + .rootCause() .hasMessage( - "Table `my_hive`.`test_db`.`hive_table` doesn't exist or is a temporary table."); + "Only support alter data table, but is: class org.apache.paimon.table.FormatTable$FormatTableImpl"); } @Test @@ -656,15 +653,6 @@ public void testFlinkWriteAndHiveRead() throws Exception { Arrays.asList( "true\t1\t1\t1\t1234567890123456789\t1.23\t3.14159\t1234.56\tABC\tv1\tHello, World!\t01\t010203\t2023-01-01\t2023-01-01 12:00:00.123\t[\"value1\",\"value2\",\"value3\"]\tvalue1\tvalue1\tvalue2\t{\"f0\":\"v1\",\"f1\":1}\tv1\t1", "false\t2\t2\t2\t234567890123456789\t2.34\t2.111111\t2345.67\tDEF\tv2\tApache Paimon\t04\t040506\t2023-02-01\t2023-02-01 12:00:00.456\t[\"value4\",\"value5\",\"value6\"]\tvalue4\tvalue11\tvalue22\t{\"f0\":\"v2\",\"f1\":2}\tv2\t2")); - - assertThatThrownBy( - () -> - tEnv.executeSql( - "INSERT INTO hive_table VALUES (1, 'Hi'), (2, 'Hello')") - .await()) - .isInstanceOf(TableException.class) - .hasMessage( - "Cannot find table '`my_hive`.`test_db`.`hive_table`' in any of the catalogs [default_catalog, my_hive], nor as a temporary table."); } @Test @@ -1128,11 +1116,12 @@ public void testAlterTable() throws Exception { } @Test - public void testHiveLock() throws InterruptedException { + public void testHiveLock() throws InterruptedException, Catalog.TableNotExistException { tEnv.executeSql("CREATE TABLE t (a INT)"); Catalog catalog = ((FlinkCatalog) tEnv.getCatalog(tEnv.getCurrentCatalog()).get()).catalog(); - CatalogLockFactory lockFactory = catalog.lockFactory().get(); + FileStoreTable table = (FileStoreTable) catalog.getTable(new Identifier("test_db", "t")); + CatalogEnvironment catalogEnv = table.catalogEnvironment(); AtomicInteger count = new AtomicInteger(0); List threads = new ArrayList<>(); @@ -1147,11 +1136,10 @@ public void testHiveLock() throws InterruptedException { Thread thread = new Thread( () -> { - CatalogLock lock = - lockFactory.createLock(catalog.lockContext().get()); + Lock lock = catalogEnv.lockFactory().create(); for (int j = 0; j < 10; j++) { try { - lock.runWithLock("test_db", "t", unsafeIncrement); + lock.runWithLock(unsafeIncrement); } catch (Exception e) { throw new RuntimeException(e); } @@ -1348,6 +1336,26 @@ public void testDropPartitionsToMetastore() throws Exception { "ptb=2a/pta=2", "ptb=2b/pta=2", "ptb=3a/pta=3", "ptb=3b/pta=3"); } + @Test + public void testCreatePartitionsToMetastore() throws Exception { + prepareTestAddPartitionsToMetastore(); + + // add partition + tEnv.executeSql( + "ALTER TABLE t ADD PARTITION (ptb = '1c', pta = 1) PARTITION (ptb = '1d', pta = 6)") + .await(); + assertThat(hiveShell.executeQuery("show partitions t")) + .containsExactlyInAnyOrder( + "ptb=1a/pta=1", + "ptb=1b/pta=1", + "ptb=1c/pta=1", + "ptb=1d/pta=6", + "ptb=2a/pta=2", + "ptb=2b/pta=2", + "ptb=3a/pta=3", + "ptb=3b/pta=3"); + } + @Test public void testAddPartitionsForTag() throws Exception { tEnv.executeSql( @@ -1891,6 +1899,49 @@ public void testExpiredPartitionsSyncToMetastore() throws Exception { .containsExactlyInAnyOrder("dt=9998-06-15", "dt=9999-06-15"); } + @Test + public void testView() throws Exception { + tEnv.executeSql("CREATE TABLE t ( a INT, b STRING ) WITH ( 'file.format' = 'avro' )") + .await(); + tEnv.executeSql("INSERT INTO t VALUES (1, 'Hi'), (2, 'Hello')").await(); + + // test flink view + tEnv.executeSql("CREATE VIEW flink_v AS SELECT a + 1, b FROM t").await(); + assertThat(collect("SELECT * FROM flink_v")) + .containsExactlyInAnyOrder(Row.of(2, "Hi"), Row.of(3, "Hello")); + assertThat(hiveShell.executeQuery("SELECT * FROM flink_v")) + .containsExactlyInAnyOrder("2\tHi", "3\tHello"); + + // test hive view + hiveShell.executeQuery("CREATE VIEW hive_v AS SELECT a + 1, b FROM t"); + assertThat(collect("SELECT * FROM hive_v")) + .containsExactlyInAnyOrder(Row.of(2, "Hi"), Row.of(3, "Hello")); + assertThat(hiveShell.executeQuery("SELECT * FROM hive_v")) + .containsExactlyInAnyOrder("2\tHi", "3\tHello"); + + assertThat(collect("SHOW VIEWS")) + .containsExactlyInAnyOrder(Row.of("flink_v"), Row.of("hive_v")); + + collect("DROP VIEW flink_v"); + collect("DROP VIEW hive_v"); + } + + @Test + public void renameView() throws Exception { + tEnv.executeSql("CREATE TABLE t ( a INT, b STRING ) WITH ( 'file.format' = 'avro' )") + .await(); + tEnv.executeSql("INSERT INTO t VALUES (1, 'Hi'), (2, 'Hello')").await(); + + tEnv.executeSql("CREATE VIEW flink_v AS SELECT a + 1, b FROM t").await(); + tEnv.executeSql("ALTER VIEW flink_v rename to flink_v_rename").await(); + assertThat(collect("SHOW VIEWS")).containsExactlyInAnyOrder(Row.of("flink_v_rename")); + + hiveShell.executeQuery("CREATE VIEW hive_v AS SELECT a + 1, b FROM t"); + tEnv.executeSql("ALTER VIEW hive_v rename to hive_v_rename").await(); + assertThat(collect("SHOW VIEWS")) + .containsExactlyInAnyOrder(Row.of("flink_v_rename"), Row.of("hive_v_rename")); + } + /** Prepare to update a paimon table with a custom path in the paimon file system. */ private void alterTableInFileSystem(TableEnvironment tEnv) throws Exception { tEnv.executeSql( @@ -1954,14 +2005,4 @@ protected List collect(String sql) throws Exception { } return result; } - - private List collectString(String sql) throws Exception { - List result = new ArrayList<>(); - try (CloseableIterator it = tEnv.executeSql(sql).collect()) { - while (it.hasNext()) { - result.add(it.next().toString()); - } - } - return result; - } } diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitterITCaseBase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitterITCaseBase.java new file mode 100644 index 000000000000..fab22775751b --- /dev/null +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/iceberg/IcebergHiveMetadataCommitterITCaseBase.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.iceberg; + +import org.apache.paimon.hive.runner.PaimonEmbeddedHiveRunner; + +import com.klarna.hiverunner.HiveShell; +import com.klarna.hiverunner.annotations.HiveSQL; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** IT cases for {@link IcebergHiveMetadataCommitter}. */ +@RunWith(PaimonEmbeddedHiveRunner.class) +public abstract class IcebergHiveMetadataCommitterITCaseBase { + + @Rule public TemporaryFolder folder = new TemporaryFolder(); + + @HiveSQL(files = {}) + protected static HiveShell hiveShell; + + private String path; + + @Before + public void before() throws Exception { + path = folder.newFolder().toURI().toString(); + } + + @After + public void after() { + hiveShell.execute("DROP DATABASE IF EXISTS test_db CASCADE"); + } + + @Test + public void testPrimaryKeyTable() throws Exception { + TableEnvironment tEnv = + TableEnvironmentImpl.create( + EnvironmentSettings.newInstance().inBatchMode().build()); + tEnv.executeSql( + "CREATE CATALOG my_paimon WITH ( 'type' = 'paimon', 'warehouse' = '" + + path + + "' )"); + tEnv.executeSql("CREATE DATABASE my_paimon.test_db"); + tEnv.executeSql( + "CREATE TABLE my_paimon.test_db.t ( pt INT, id INT, data STRING, PRIMARY KEY (pt, id) NOT ENFORCED ) " + + "PARTITIONED BY (pt) WITH " + + "( 'metadata.iceberg.storage' = 'hive-catalog', 'metadata.iceberg.uri' = '', 'file.format' = 'avro', " + // make sure all changes are visible in iceberg metadata + + " 'full-compaction.delta-commits' = '1' )"); + tEnv.executeSql( + "INSERT INTO my_paimon.test_db.t VALUES " + + "(1, 1, 'apple'), (1, 2, 'pear'), (2, 1, 'cat'), (2, 2, 'dog')") + .await(); + + tEnv.executeSql( + "CREATE CATALOG my_iceberg WITH " + + "( 'type' = 'iceberg', 'catalog-type' = 'hive', 'uri' = '', 'warehouse' = '" + + path + + "', 'cache-enabled' = 'false' )"); + Assert.assertEquals( + Arrays.asList(Row.of("pear", 2, 1), Row.of("dog", 2, 2)), + collect( + tEnv.executeSql( + "SELECT data, id, pt FROM my_iceberg.test_db.t WHERE id = 2 ORDER BY pt, id"))); + + tEnv.executeSql( + "INSERT INTO my_paimon.test_db.t VALUES " + + "(1, 1, 'cherry'), (2, 2, 'elephant')") + .await(); + Assert.assertEquals( + Arrays.asList( + Row.of(1, 1, "cherry"), + Row.of(1, 2, "pear"), + Row.of(2, 1, "cat"), + Row.of(2, 2, "elephant")), + collect(tEnv.executeSql("SELECT * FROM my_iceberg.test_db.t ORDER BY pt, id"))); + } + + @Test + public void testAppendOnlyTable() throws Exception { + TableEnvironment tEnv = + TableEnvironmentImpl.create( + EnvironmentSettings.newInstance().inBatchMode().build()); + tEnv.executeSql( + "CREATE CATALOG my_paimon WITH ( 'type' = 'paimon', 'warehouse' = '" + + path + + "' )"); + tEnv.executeSql("CREATE DATABASE my_paimon.test_db"); + tEnv.executeSql( + "CREATE TABLE my_paimon.test_db.t ( pt INT, id INT, data STRING ) PARTITIONED BY (pt) WITH " + + "( 'metadata.iceberg.storage' = 'hive-catalog', 'metadata.iceberg.uri' = '', 'file.format' = 'avro' )"); + tEnv.executeSql( + "INSERT INTO my_paimon.test_db.t VALUES " + + "(1, 1, 'apple'), (1, 2, 'pear'), (2, 1, 'cat'), (2, 2, 'dog')") + .await(); + + tEnv.executeSql( + "CREATE CATALOG my_iceberg WITH " + + "( 'type' = 'iceberg', 'catalog-type' = 'hive', 'uri' = '', 'warehouse' = '" + + path + + "', 'cache-enabled' = 'false' )"); + Assert.assertEquals( + Arrays.asList(Row.of("pear", 2, 1), Row.of("dog", 2, 2)), + collect( + tEnv.executeSql( + "SELECT data, id, pt FROM my_iceberg.test_db.t WHERE id = 2 ORDER BY pt, id"))); + + tEnv.executeSql( + "INSERT INTO my_paimon.test_db.t VALUES " + + "(1, 3, 'cherry'), (2, 3, 'elephant')") + .await(); + Assert.assertEquals( + Arrays.asList( + Row.of("pear", 2, 1), + Row.of("cherry", 3, 1), + Row.of("dog", 2, 2), + Row.of("elephant", 3, 2)), + collect( + tEnv.executeSql( + "SELECT data, id, pt FROM my_iceberg.test_db.t WHERE id > 1 ORDER BY pt, id"))); + } + + private List collect(TableResult result) throws Exception { + List rows = new ArrayList<>(); + try (CloseableIterator it = result.collect()) { + while (it.hasNext()) { + rows.add(it.next()); + } + } + return rows; + } +} diff --git a/paimon-hive/pom.xml b/paimon-hive/pom.xml index 5c306f4749b2..7d1d0f2c499c 100644 --- a/paimon-hive/pom.xml +++ b/paimon-hive/pom.xml @@ -49,6 +49,7 @@ under the License. 4.0.0 0.9.8 1.12.319 + 1.19 diff --git a/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NetworkClient.java b/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NetworkClient.java index 2a16d53b97de..748d1d2f055e 100644 --- a/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NetworkClient.java +++ b/paimon-service/paimon-service-client/src/main/java/org/apache/paimon/service/network/NetworkClient.java @@ -143,30 +143,23 @@ public CompletableFuture sendRequest( new IllegalStateException(clientName + " is already shut down.")); } - final ServerConnection connection = - connections.computeIfAbsent( - serverAddress, - ignored -> { - final ServerConnection newConnection = - ServerConnection.createPendingConnection( - clientName, messageSerializer, stats); - bootstrap - .connect(serverAddress.getAddress(), serverAddress.getPort()) - .addListener( - (ChannelFutureListener) - newConnection::establishConnection); - - newConnection - .getCloseFuture() - .handle( - (ignoredA, ignoredB) -> - connections.remove( - serverAddress, newConnection)); - - return newConnection; - }); - - return connection.sendRequest(request); + ServerConnection serverConnection = connections.get(serverAddress); + if (serverConnection == null) { + final ServerConnection newConnection = + ServerConnection.createPendingConnection(clientName, messageSerializer, stats); + serverConnection = newConnection; + connections.put(serverAddress, newConnection); + bootstrap + .connect(serverAddress.getAddress(), serverAddress.getPort()) + .addListener((ChannelFutureListener) newConnection::establishConnection); + + newConnection + .getCloseFuture() + .handle( + (ignoredA, ignoredB) -> + connections.remove(serverAddress, newConnection)); + } + return serverConnection.sendRequest(request); } /** diff --git a/paimon-spark/paimon-spark-3.1/pom.xml b/paimon-spark/paimon-spark-3.1/pom.xml deleted file mode 100644 index b74c66684be0..000000000000 --- a/paimon-spark/paimon-spark-3.1/pom.xml +++ /dev/null @@ -1,274 +0,0 @@ - - - - 4.0.0 - - - org.apache.paimon - paimon-spark - 1.0-SNAPSHOT - - - paimon-spark-3.1 - Paimon : Spark : 3.1 - - - 3.1.3 - - - - - org.apache.paimon - paimon-spark-common - ${project.version} - - - - org.scala-lang - scala-library - ${scala.version} - - - org.scala-lang - scala-reflect - ${scala.version} - - - org.scala-lang - scala-compiler - ${scala.version} - - - - org.apache.spark - spark-sql_2.12 - ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-hadoop - - - - - - org.apache.spark - spark-core_2.12 - ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.orc - orc-mapreduce - - - org.apache.parquet - parquet-column - - - com.google.protobuf - protobuf-java - - - - - - org.apache.spark - spark-sql_2.12 - ${spark.version} - tests - test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-hadoop - - - - - org.apache.spark - spark-core_2.12 - ${spark.version} - tests - test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - - - org.apache.spark - spark-catalyst_2.12 - ${spark.version} - tests - test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - - - - org.scalatest - scalatest_${scala.binary.version} - 3.1.0 - test - - - - org.apache.avro - avro - ${avro.version} - test - - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-paimon - package - - shade - - - - - - org.apache.paimon:paimon-spark-common - - - - - - - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - - scala-compile-first - process-resources - - add-source - compile - - - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - - diff --git a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeInto.scala b/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeInto.scala deleted file mode 100644 index b16fbb727e02..000000000000 --- a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeInto.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.spark.catalyst.analysis - -import org.apache.paimon.spark.SparkTable -import org.apache.paimon.spark.commands.MergeIntoPaimonTable - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.catalyst.plans.logical.{MergeAction, MergeIntoTable} - -/** A post-hoc resolution rule for MergeInto. */ -case class PaimonMergeInto(spark: SparkSession) extends PaimonMergeIntoBase { - - override def resolveNotMatchedBySourceActions( - merge: MergeIntoTable, - targetOutput: Seq[AttributeReference]): Seq[MergeAction] = { - Seq.empty - } - - override def buildMergeIntoPaimonTable( - v2Table: SparkTable, - merge: MergeIntoTable, - alignedMatchedActions: Seq[MergeAction], - alignedNotMatchedActions: Seq[MergeAction], - alignedNotMatchedBySourceActions: Seq[MergeAction]): MergeIntoPaimonTable = { - if (alignedNotMatchedBySourceActions.nonEmpty) { - throw new RuntimeException("WHEN NOT MATCHED BY SOURCE is not supported here.") - } - - MergeIntoPaimonTable( - v2Table, - merge.targetTable, - merge.sourceTable, - merge.mergeCondition, - alignedMatchedActions, - alignedNotMatchedActions, - alignedNotMatchedBySourceActions - ) - } -} diff --git a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala b/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala deleted file mode 100644 index 031ea0a182f0..000000000000 --- a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala +++ /dev/null @@ -1,50 +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.catalyst.analysis - -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, MergeAction, MergeIntoTable} - -object PaimonMergeIntoResolver extends PaimonMergeIntoResolverBase { - - def resolveNotMatchedBySourceActions( - merge: MergeIntoTable, - target: LogicalPlan, - source: LogicalPlan, - resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] = { - Seq.empty - } - - def build( - merge: MergeIntoTable, - resolvedCond: Expression, - resolvedMatched: Seq[MergeAction], - resolvedNotMatched: Seq[MergeAction], - resolvedNotMatchedBySource: Seq[MergeAction]): MergeIntoTable = { - if (resolvedNotMatchedBySource.nonEmpty) { - throw new RuntimeException("WHEN NOT MATCHED BY SOURCE is not supported here.") - } - - merge.copy( - mergeCondition = resolvedCond, - matchedActions = resolvedMatched, - notMatchedActions = resolvedNotMatched) - } - -} diff --git a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/spark/sql/PaimonStatsUtils.scala b/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/spark/sql/PaimonStatsUtils.scala deleted file mode 100644 index f31195f66dd9..000000000000 --- a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/spark/sql/PaimonStatsUtils.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql - -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} -import org.apache.spark.sql.execution.command.CommandUtils -import org.apache.spark.sql.types._ - -object PaimonStatsUtils { - - def computeColumnStats( - sparkSession: SparkSession, - relation: LogicalPlan, - columns: Seq[Attribute]): (Long, Map[Attribute, ColumnStat]) = { - CommandUtils.computeColumnStats(sparkSession, relation, columns) - } - - /** DatetimeType was added after spark33, overwrite it for compatibility. */ - def analyzeSupportsType(dataType: DataType): Boolean = dataType match { - case _: IntegralType => true - case _: DecimalType => true - case DoubleType | FloatType => true - case BooleanType => true - case DateType => true - case TimestampType => true - case BinaryType | StringType => true - case _ => false - } - - /** DatetimeType was added after spark33, overwrite it for compatibility. */ - def hasMinMax(dataType: DataType): Boolean = dataType match { - case _: IntegralType => true - case _: DecimalType => true - case DoubleType | FloatType => true - case BooleanType => true - case DateType => true - case TimestampType => true - case _ => false - } -} diff --git a/paimon-spark/paimon-spark-3.1/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java b/paimon-spark/paimon-spark-3.1/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java deleted file mode 100644 index bb3e81c9dd36..000000000000 --- a/paimon-spark/paimon-spark-3.1/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java +++ /dev/null @@ -1,123 +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; - -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.spark.extensions.PaimonSparkSessionExtensions; -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.table.sink.BatchWriteBuilder; - -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import java.util.List; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Base tests for spark read. */ -public class SparkGenericCatalogTest { - - protected static SparkSession spark = null; - - protected static Path warehousePath = null; - - @BeforeAll - public static void startMetastoreAndSpark(@TempDir java.nio.file.Path tempDir) { - warehousePath = new Path("file:" + tempDir.toString()); - spark = - SparkSession.builder() - .config("spark.sql.warehouse.dir", warehousePath.toString()) - .config( - "spark.sql.extensions", - PaimonSparkSessionExtensions.class.getName()) - .master("local[2]") - .getOrCreate(); - spark.conf().set("spark.sql.catalog.spark_catalog", SparkGenericCatalog.class.getName()); - } - - @AfterAll - public static void stopMetastoreAndSpark() { - if (spark != null) { - spark.stop(); - spark = null; - } - } - - @Test - public void testPaimonTable() throws Exception { - spark.sql( - "CREATE TABLE PT (a INT, b INT, c STRING) USING paimon TBLPROPERTIES" - + " ('file.format'='avro')"); - writeTable( - "PT", - GenericRow.of(1, 2, BinaryString.fromString("3")), - GenericRow.of(4, 5, BinaryString.fromString("6"))); - assertThat(spark.sql("SELECT * FROM PT").collectAsList().stream().map(Object::toString)) - .containsExactlyInAnyOrder("[1,2,3]", "[4,5,6]"); - - spark.sql("CREATE DATABASE my_db"); - spark.sql( - "CREATE TABLE DB_PT (a INT, b INT, c STRING) USING paimon TBLPROPERTIES" - + " ('file.format'='avro')"); - writeTable( - "DB_PT", - GenericRow.of(1, 2, BinaryString.fromString("3")), - GenericRow.of(4, 5, BinaryString.fromString("6"))); - assertThat(spark.sql("SELECT * FROM DB_PT").collectAsList().stream().map(Object::toString)) - .containsExactlyInAnyOrder("[1,2,3]", "[4,5,6]"); - - assertThat(spark.sql("SHOW NAMESPACES").collectAsList().stream().map(Object::toString)) - .containsExactlyInAnyOrder("[default]", "[my_db]"); - } - - @Test - public void testCsvTable() { - spark.sql("CREATE TABLE CT (a INT, b INT, c STRING) USING csv"); - spark.sql("INSERT INTO CT VALUES (1, 2, '3'), (4, 5, '6')").collectAsList(); - List rows = spark.sql("SELECT * FROM CT").collectAsList(); - assertThat(rows.stream().map(Object::toString)) - .containsExactlyInAnyOrder("[1,2,3]", "[4,5,6]"); - } - - private static void writeTable(String tableName, GenericRow... rows) throws Exception { - FileStoreTable fileStoreTable = - FileStoreTableFactory.create( - LocalFileIO.create(), - new Path(warehousePath, String.format("default.db/%s", tableName))); - BatchWriteBuilder writeBuilder = fileStoreTable.newBatchWriteBuilder(); - BatchTableWrite writer = writeBuilder.newWrite(); - BatchTableCommit commit = writeBuilder.newCommit(); - for (GenericRow row : rows) { - writer.write(row); - } - commit.commit(writer.prepareCommit()); - writer.close(); - commit.close(); - } -} diff --git a/paimon-spark/paimon-spark-3.2/pom.xml b/paimon-spark/paimon-spark-3.2/pom.xml index 24e5198e9b2e..626bb5bae833 100644 --- a/paimon-spark/paimon-spark-3.2/pom.xml +++ b/paimon-spark/paimon-spark-3.2/pom.xml @@ -38,250 +38,70 @@ under the License. org.apache.paimon - paimon-spark-common + paimon-spark-common_${scala.binary.version} ${project.version} - org.scala-lang - scala-library - ${scala.version} - - - org.scala-lang - scala-reflect - ${scala.version} - - - org.scala-lang - scala-compiler - ${scala.version} + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} org.apache.spark - spark-sql_2.12 + spark-core_${scala.binary.version} ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.orc - orc-mapreduce - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-hadoop - - - org.apache.spark - spark-core_2.12 - ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.orc - orc-mapreduce - - - org.apache.parquet - parquet-column - - - com.google.protobuf - protobuf-java - - + org.apache.paimon + paimon-bundle + + org.apache.paimon - paimon-spark-common + paimon-spark-common_${scala.binary.version} ${project.version} - test-jar + tests test + org.apache.spark - spark-sql_2.12 + spark-sql_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-hadoop - - + org.apache.spark - spark-core_2.12 + spark-catalyst_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - + org.apache.spark - spark-catalyst_2.12 + spark-core_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - - org.scalatest - scalatest_${scala.binary.version} - 3.1.0 - test - org.apache.spark - spark-hive_2.12 + spark-hive_${scala.binary.version} ${spark.version} test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - com.google.protobuf - protobuf-java - - - - - org.apache.paimon - paimon-hive-common - ${project.version} - test - - - org.apache.paimon - paimon-hive-common - ${project.version} - tests - test-jar - test - - - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - -nobootcp - -target:jvm-${target.java.version} - - false - - - - org.apache.maven.plugins - maven-compiler-plugin - - - org.apache.maven.plugins @@ -299,69 +119,20 @@ under the License. * com/github/luben/zstd/** + **/*libzstd-jni-*.so + **/*libzstd-jni-*.dll - org.apache.paimon:paimon-spark-common + org.apache.paimon:paimon-spark-common_${scala.binary.version} - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - - scala-compile-first - process-resources - - add-source - compile - - - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - org.scalatest - scalatest-maven-plugin - ${scalatest-maven-plugin.version} - - ${project.build.directory}/surefire-reports - . - -ea -Xmx4g -Xss4m -XX:MaxMetaspaceSize=2g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true - PaimonTestSuite.txt - - once - - - - test - - test - - - - diff --git a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/PaimonScan.scala b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/PaimonScan.scala index 361b1e7a77d6..92a719b3bc47 100644 --- a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/PaimonScan.scala +++ b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/PaimonScan.scala @@ -34,7 +34,7 @@ case class PaimonScan( requiredSchema: StructType, filters: Seq[Predicate], reservedFilters: Seq[Filter], - pushDownLimit: Option[Int], + override val pushDownLimit: Option[Int], disableBucketedScan: Boolean = false) extends PaimonBaseScan(table, requiredSchema, filters, reservedFilters, pushDownLimit) with SupportsRuntimeFiltering { diff --git a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala index 031ea0a182f0..e0869a6089e9 100644 --- a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala +++ b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala @@ -25,8 +25,6 @@ object PaimonMergeIntoResolver extends PaimonMergeIntoResolverBase { def resolveNotMatchedBySourceActions( merge: MergeIntoTable, - target: LogicalPlan, - source: LogicalPlan, resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] = { Seq.empty } diff --git a/paimon-spark/paimon-spark-3.1/src/main/java/org/apache/paimon/spark/catalog/SupportFunction.java b/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala similarity index 86% rename from paimon-spark/paimon-spark-3.1/src/main/java/org/apache/paimon/spark/catalog/SupportFunction.java rename to paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala index 4171ea38dcbc..2d199491dc0a 100644 --- a/paimon-spark/paimon-spark-3.1/src/main/java/org/apache/paimon/spark/catalog/SupportFunction.java +++ b/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala @@ -16,7 +16,6 @@ * limitations under the License. */ -package org.apache.paimon.spark.catalog; +package org.apache.paimon.spark.sql -/** Catalog methods for working with Functions. */ -public interface SupportFunction {} +class TagDdlTest extends PaimonTagDdlTestBase diff --git a/paimon-spark/paimon-spark-3.3/pom.xml b/paimon-spark/paimon-spark-3.3/pom.xml index 3950ff9f8aca..689e4131ccd9 100644 --- a/paimon-spark/paimon-spark-3.3/pom.xml +++ b/paimon-spark/paimon-spark-3.3/pom.xml @@ -38,245 +38,70 @@ under the License. org.apache.paimon - paimon-spark-common + paimon-spark-common_${scala.binary.version} ${project.version} - org.scala-lang - scala-library - ${scala.version} - - - org.scala-lang - scala-reflect - ${scala.version} - - - org.scala-lang - scala-compiler - ${scala.version} + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} org.apache.spark - spark-sql_2.12 + spark-core_${scala.binary.version} ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-hadoop - - - org.apache.spark - spark-core_2.12 - ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.orc - orc-mapreduce - - - org.apache.parquet - parquet-column - - - com.google.protobuf - protobuf-java - - + org.apache.paimon + paimon-bundle + + org.apache.paimon - paimon-spark-common + paimon-spark-common_${scala.binary.version} ${project.version} tests test + org.apache.spark - spark-sql_2.12 + spark-sql_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-hadoop - - + org.apache.spark - spark-core_2.12 + spark-catalyst_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - + org.apache.spark - spark-catalyst_2.12 + spark-core_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - - - org.scalatest - scalatest_${scala.binary.version} - 3.1.0 - test + org.apache.spark - spark-hive_2.12 + spark-hive_${scala.binary.version} ${spark.version} test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - com.google.protobuf - protobuf-java - - - - - org.apache.paimon - paimon-hive-common - ${project.version} - test - - - org.apache.paimon - paimon-hive-common - ${project.version} - tests - test-jar - test - - - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - -nobootcp - -target:jvm-${target.java.version} - - false - - - - org.apache.maven.plugins - maven-compiler-plugin - - - org.apache.maven.plugins @@ -294,69 +119,20 @@ under the License. * com/github/luben/zstd/** + **/*libzstd-jni-*.so + **/*libzstd-jni-*.dll - org.apache.paimon:paimon-spark-common + org.apache.paimon:paimon-spark-common_${scala.binary.version} - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - - scala-compile-first - process-resources - - add-source - compile - - - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - org.scalatest - scalatest-maven-plugin - ${scalatest-maven-plugin.version} - - ${project.build.directory}/surefire-reports - . - -ea -Xmx4g -Xss4m -XX:MaxMetaspaceSize=2g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true - PaimonTestSuite.txt - - once - - - - test - - test - - - - diff --git a/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala b/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala index 031ea0a182f0..e0869a6089e9 100644 --- a/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala +++ b/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala @@ -25,8 +25,6 @@ object PaimonMergeIntoResolver extends PaimonMergeIntoResolverBase { def resolveNotMatchedBySourceActions( merge: MergeIntoTable, - target: LogicalPlan, - source: LogicalPlan, resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] = { Seq.empty } diff --git a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala similarity index 84% rename from paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala rename to paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala index 10b83ccf08b1..2d199491dc0a 100644 --- a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala +++ b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala @@ -16,8 +16,6 @@ * limitations under the License. */ -package org.apache.paimon.spark +package org.apache.paimon.spark.sql -import org.apache.paimon.table.Table - -class PaimonScanBuilder(table: Table) extends PaimonBaseScanBuilder(table) +class TagDdlTest extends PaimonTagDdlTestBase diff --git a/paimon-spark/paimon-spark-3.4/pom.xml b/paimon-spark/paimon-spark-3.4/pom.xml index 000acdc26c3c..d1ded508a927 100644 --- a/paimon-spark/paimon-spark-3.4/pom.xml +++ b/paimon-spark/paimon-spark-3.4/pom.xml @@ -38,245 +38,70 @@ under the License. org.apache.paimon - paimon-spark-common + paimon-spark-common_${scala.binary.version} ${project.version} - org.scala-lang - scala-library - ${scala.version} - - - org.scala-lang - scala-reflect - ${scala.version} - - - org.scala-lang - scala-compiler - ${scala.version} + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} org.apache.spark - spark-sql_2.12 + spark-core_${scala.binary.version} ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-hadoop - - - org.apache.spark - spark-core_2.12 - ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.orc - orc-mapreduce - - - org.apache.parquet - parquet-column - - - com.google.protobuf - protobuf-java - - + org.apache.paimon + paimon-bundle + + org.apache.paimon - paimon-spark-common + paimon-spark-common_${scala.binary.version} ${project.version} tests test + org.apache.spark - spark-sql_2.12 + spark-sql_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - org.apache.parquet - parquet-column - - - org.apache.parquet - parquet-hadoop - - + org.apache.spark - spark-core_2.12 + spark-catalyst_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - + org.apache.spark - spark-catalyst_2.12 + spark-core_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - - - org.scalatest - scalatest_${scala.binary.version} - 3.1.0 - test + org.apache.spark - spark-hive_2.12 + spark-hive_${scala.binary.version} ${spark.version} test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - com.google.protobuf - protobuf-java - - - - - org.apache.paimon - paimon-hive-common - ${project.version} - test - - - org.apache.paimon - paimon-hive-common - ${project.version} - tests - test-jar - test - - - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - -nobootcp - -target:jvm-${target.java.version} - - false - - - - org.apache.maven.plugins - maven-compiler-plugin - - - org.apache.maven.plugins @@ -294,69 +119,20 @@ under the License. * com/github/luben/zstd/** + **/*libzstd-jni-*.so + **/*libzstd-jni-*.dll - org.apache.paimon:paimon-spark-common + org.apache.paimon:paimon-spark-common_${scala.binary.version} - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - - scala-compile-first - process-resources - - add-source - compile - - - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - org.scalatest - scalatest-maven-plugin - ${scalatest-maven-plugin.version} - - ${project.build.directory}/surefire-reports - . - -ea -Xmx4g -Xss4m -XX:MaxMetaspaceSize=2g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true - PaimonTestSuite.txt - - once - - - - test - - test - - - - diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/CreateAndDeleteTagProcedureTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/CreateAndDeleteTagProcedureTest.scala index c4b861b29752..3f59e897ec6c 100644 --- a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/CreateAndDeleteTagProcedureTest.scala +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/CreateAndDeleteTagProcedureTest.scala @@ -75,8 +75,16 @@ class CreateAndDeleteTagProcedureTest extends PaimonSparkTestBase with StreamTes checkAnswer( spark.sql("SELECT tag_name FROM paimon.test.`T$tags`"), Row("test_tag") :: Nil) + // test rename_tag checkAnswer( - spark.sql("CALL paimon.sys.delete_tag(table => 'test.T', tag => 'test_tag')"), + spark.sql( + "CALL paimon.sys.rename_tag(table => 'test.T', tag => 'test_tag', target_tag => 'test_tag_1')"), + Row(true) :: Nil) + checkAnswer( + spark.sql("SELECT tag_name FROM paimon.test.`T$tags`"), + Row("test_tag_1") :: Nil) + checkAnswer( + spark.sql("CALL paimon.sys.delete_tag(table => 'test.T', tag => 'test_tag_1')"), Row(true) :: Nil) checkAnswer(spark.sql("SELECT tag_name FROM paimon.test.`T$tags`"), Nil) checkAnswer( diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala new file mode 100644 index 000000000000..2d199491dc0a --- /dev/null +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.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 TagDdlTest extends PaimonTagDdlTestBase diff --git a/paimon-spark/paimon-spark-3.5/pom.xml b/paimon-spark/paimon-spark-3.5/pom.xml index 0ba453068c07..92803cda540e 100644 --- a/paimon-spark/paimon-spark-3.5/pom.xml +++ b/paimon-spark/paimon-spark-3.5/pom.xml @@ -38,237 +38,70 @@ under the License. org.apache.paimon - paimon-spark-common + paimon-spark-common_${scala.binary.version} ${project.version} - org.scala-lang - scala-library - ${scala.version} - - - org.scala-lang - scala-reflect - ${scala.version} - - - org.scala-lang - scala-compiler - ${scala.version} + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} org.apache.spark - spark-sql_2.12 + spark-core_${scala.binary.version} ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.parquet - parquet-column - - - org.apache.spark - spark-core_2.12 - ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.orc - orc-mapreduce - - - org.apache.parquet - parquet-column - - - com.google.protobuf - protobuf-java - - + org.apache.paimon + paimon-bundle + + org.apache.paimon - paimon-spark-common + paimon-spark-common_${scala.binary.version} ${project.version} tests test + org.apache.spark - spark-sql_2.12 + spark-sql_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - org.apache.parquet - parquet-column - - + org.apache.spark - spark-core_2.12 + spark-catalyst_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - + org.apache.spark - spark-catalyst_2.12 + spark-core_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - - - org.scalatest - scalatest_${scala.binary.version} - 3.1.0 - test + org.apache.spark - spark-hive_2.12 + spark-hive_${scala.binary.version} ${spark.version} test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - com.google.protobuf - protobuf-java - - - - - org.apache.paimon - paimon-hive-common - ${project.version} - test - - - org.apache.paimon - paimon-hive-common - ${project.version} - tests - test-jar - test - - - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - -nobootcp - -target:jvm-${target.java.version} - - false - - - - org.apache.maven.plugins - maven-compiler-plugin - - - org.apache.maven.plugins @@ -286,69 +119,20 @@ under the License. * com/github/luben/zstd/** + **/*libzstd-jni-*.so + **/*libzstd-jni-*.dll - org.apache.paimon:paimon-spark-common + org.apache.paimon:paimon-spark-common_${scala.binary.version} - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - - scala-compile-first - process-resources - - add-source - compile - - - - - - scala-test-compile - process-test-resources - - testCompile - - - - - - org.scalatest - scalatest-maven-plugin - ${scalatest-maven-plugin.version} - - ${project.build.directory}/surefire-reports - . - -ea -Xmx4g -Xss4m -XX:MaxMetaspaceSize=2g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true - PaimonTestSuite.txt - - once - - - - test - - test - - - - diff --git a/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala new file mode 100644 index 000000000000..92309d54167b --- /dev/null +++ b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.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 TagDdlTest extends PaimonTagDdlTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/pom.xml b/paimon-spark/paimon-spark-4.0/pom.xml new file mode 100644 index 000000000000..9f819f820ce2 --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/pom.xml @@ -0,0 +1,138 @@ + + + + 4.0.0 + + + org.apache.paimon + paimon-spark + 1.0-SNAPSHOT + + + paimon-spark-4.0 + Paimon : Spark : 4.0 + + + 4.0.0-preview2 + + + + + org.apache.paimon + paimon-spark-common_${scala.binary.version} + ${project.version} + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + + + + org.apache.paimon + paimon-bundle + + + + + + org.apache.paimon + paimon-spark-common_${scala.binary.version} + ${project.version} + tests + test + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + tests + test + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${spark.version} + tests + test + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + tests + test + + + + org.apache.spark + spark-hive_${scala.binary.version} + ${spark.version} + test + + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-paimon + package + + shade + + + + + * + + com/github/luben/zstd/** + **/*libzstd-jni-*.so + **/*libzstd-jni-*.dll + + + + + + org.apache.paimon:paimon-spark-common_${scala.binary.version} + + + + + + + + + diff --git a/paimon-spark/paimon-spark-4.0/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueries.scala b/paimon-spark/paimon-spark-4.0/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueries.scala new file mode 100644 index 000000000000..2144f77f3a6c --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueries.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.catalyst.optimizer + +import org.apache.paimon.spark.PaimonScan + +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, ExprId, ScalarSubquery, SortOrder} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation + +object MergePaimonScalarSubqueries extends MergePaimonScalarSubqueriesBase { + + override def tryMergeDataSourceV2ScanRelation( + newV2ScanRelation: DataSourceV2ScanRelation, + cachedV2ScanRelation: DataSourceV2ScanRelation) + : Option[(LogicalPlan, AttributeMap[Attribute])] = { + (newV2ScanRelation, cachedV2ScanRelation) match { + case ( + DataSourceV2ScanRelation( + newRelation, + newScan: PaimonScan, + newOutput, + newPartitioning, + newOrdering), + DataSourceV2ScanRelation( + cachedRelation, + cachedScan: PaimonScan, + _, + cachedPartitioning, + cacheOrdering)) => + checkIdenticalPlans(newRelation, cachedRelation).flatMap { + outputMap => + if ( + samePartitioning(newPartitioning, cachedPartitioning, outputMap) && sameOrdering( + newOrdering, + cacheOrdering, + outputMap) + ) { + mergePaimonScan(newScan, cachedScan).map { + mergedScan => + val mergedAttributes = mergedScan + .readSchema() + .map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) + val cachedOutputNameMap = cachedRelation.output.map(a => a.name -> a).toMap + val mergedOutput = + mergedAttributes.map(a => cachedOutputNameMap.getOrElse(a.name, a)) + val newV2ScanRelation = DataSourceV2ScanRelation( + cachedRelation, + mergedScan, + mergedOutput, + cachedPartitioning) + + val mergedOutputNameMap = mergedOutput.map(a => a.name -> a).toMap + val newOutputMap = + AttributeMap(newOutput.map(a => a -> mergedOutputNameMap(a.name).toAttribute)) + + newV2ScanRelation -> newOutputMap + } + } else { + None + } + } + + case _ => None + } + } + + private def sameOrdering( + newOrdering: Option[Seq[SortOrder]], + cachedOrdering: Option[Seq[SortOrder]], + outputAttrMap: AttributeMap[Attribute]): Boolean = { + val mappedNewOrdering = newOrdering.map(_.map(mapAttributes(_, outputAttrMap))) + mappedNewOrdering.map(_.map(_.canonicalized)) == cachedOrdering.map(_.map(_.canonicalized)) + + } + + override protected def createScalarSubquery(plan: LogicalPlan, exprId: ExprId): ScalarSubquery = { + ScalarSubquery(plan, exprId = exprId) + } +} diff --git a/paimon-spark/paimon-spark-4.0/src/test/resources/hive-site.xml b/paimon-spark/paimon-spark-4.0/src/test/resources/hive-site.xml new file mode 100644 index 000000000000..bdf2bb090760 --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/resources/hive-site.xml @@ -0,0 +1,56 @@ + + + + + hive.metastore.integral.jdo.pushdown + true + + + + hive.metastore.schema.verification + false + + + + hive.metastore.client.capability.check + false + + + + datanucleus.schema.autoCreateTables + true + + + + datanucleus.schema.autoCreateAll + true + + + + + datanucleus.connectionPoolingType + DBCP + + + + hive.metastore.uris + thrift://localhost:9090 + Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore. + + \ No newline at end of file diff --git a/paimon-spark/paimon-spark-3.1/src/test/resources/log4j2-test.properties b/paimon-spark/paimon-spark-4.0/src/test/resources/log4j2-test.properties similarity index 84% rename from paimon-spark/paimon-spark-3.1/src/test/resources/log4j2-test.properties rename to paimon-spark/paimon-spark-4.0/src/test/resources/log4j2-test.properties index 1b3980d15104..6f324f5863ac 100644 --- a/paimon-spark/paimon-spark-3.1/src/test/resources/log4j2-test.properties +++ b/paimon-spark/paimon-spark-4.0/src/test/resources/log4j2-test.properties @@ -26,3 +26,13 @@ appender.testlogger.type = CONSOLE appender.testlogger.target = SYSTEM_ERR appender.testlogger.layout.type = PatternLayout appender.testlogger.layout.pattern = %-4r [%tid %t] %-5p %c %x - %m%n + +logger.kafka.name = kafka +logger.kafka.level = OFF +logger.kafka2.name = state.change +logger.kafka2.level = OFF + +logger.zookeeper.name = org.apache.zookeeper +logger.zookeeper.level = OFF +logger.I0Itec.name = org.I0Itec +logger.I0Itec.level = OFF diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTest.scala new file mode 100644 index 000000000000..322d50a62127 --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTest.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.procedure + +class CompactProcedureTest extends CompactProcedureTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTest.scala new file mode 100644 index 000000000000..d57846709877 --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTest.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.procedure + +class ProcedureTest extends ProcedureTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTest.scala new file mode 100644 index 000000000000..255906d04bf2 --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTest.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 AnalyzeTableTest extends AnalyzeTableTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/DDLTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/DDLTest.scala new file mode 100644 index 000000000000..b729f57b33e7 --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/DDLTest.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 DDLTest extends DDLTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTest.scala new file mode 100644 index 000000000000..a9ea3efc89ba --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTest.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 DDLWithHiveCatalogTest extends DDLWithHiveCatalogTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTest.scala new file mode 100644 index 000000000000..09554a1dbf8d --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTest.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 DeleteFromTableTest extends DeleteFromTableTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTest.scala new file mode 100644 index 000000000000..4f66584c303b --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTest.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 InsertOverwriteTableTest extends InsertOverwriteTableTestBase {} 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 new file mode 100644 index 000000000000..e1cfe3a3960f --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala @@ -0,0 +1,43 @@ +/* + * 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 + +import org.apache.paimon.spark.{PaimonAppendBucketedTableTest, PaimonAppendNonBucketTableTest, PaimonPrimaryKeyBucketedTableTest, PaimonPrimaryKeyNonBucketTableTest} + +class MergeIntoPrimaryKeyBucketedTableTest + extends MergeIntoTableTestBase + with MergeIntoPrimaryKeyTableTest + with MergeIntoNotMatchedBySourceTest + with PaimonPrimaryKeyBucketedTableTest {} + +class MergeIntoPrimaryKeyNonBucketTableTest + extends MergeIntoTableTestBase + with MergeIntoPrimaryKeyTableTest + with MergeIntoNotMatchedBySourceTest + with PaimonPrimaryKeyNonBucketTableTest {} + +class MergeIntoAppendBucketedTableTest + extends MergeIntoTableTestBase + with MergeIntoNotMatchedBySourceTest + with PaimonAppendBucketedTableTest {} + +class MergeIntoAppendNonBucketedTableTest + extends MergeIntoTableTestBase + with MergeIntoNotMatchedBySourceTest + with PaimonAppendNonBucketTableTest {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTest.scala new file mode 100644 index 000000000000..635185a9ed0e --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTest.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 PaimonCompositePartitionKeyTest extends PaimonCompositePartitionKeyTestBase {} diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SystemColumns.java b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala similarity index 56% rename from paimon-core/src/main/java/org/apache/paimon/schema/SystemColumns.java rename to paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala index f6350f44ace8..0a4dfb76959c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SystemColumns.java +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala @@ -16,21 +16,22 @@ * limitations under the License. */ -package org.apache.paimon.schema; +package org.apache.paimon.spark.sql -import java.util.Arrays; -import java.util.List; +import org.apache.paimon.spark.util.CTERelationRefUtils -/** System columns for key value store. */ -public class SystemColumns { +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, GetStructField, NamedExpression, ScalarSubquery} - /** System field names. */ - public static final String KEY_FIELD_PREFIX = "_KEY_"; +class PaimonOptimizationTest extends PaimonOptimizationTestBase { - public static final String VALUE_COUNT = "_VALUE_COUNT"; - public static final String SEQUENCE_NUMBER = "_SEQUENCE_NUMBER"; - public static final String VALUE_KIND = "_VALUE_KIND"; - public static final String LEVEL = "_LEVEL"; - public static final List SYSTEM_FIELD_NAMES = - Arrays.asList(VALUE_COUNT, SEQUENCE_NUMBER, VALUE_KIND); + override def extractorExpression( + cteIndex: Int, + output: Seq[Attribute], + fieldIndex: Int): NamedExpression = { + GetStructField( + ScalarSubquery(CTERelationRefUtils.createCTERelationRef(cteIndex, _resolved = true, output)), + fieldIndex) + .as("scalarsubquery()") + } } diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/ShowColumnsTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/ShowColumnsTest.scala new file mode 100644 index 000000000000..6601dc2fca37 --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/ShowColumnsTest.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 ShowColumnsTest extends PaimonShowColumnsTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala new file mode 100644 index 000000000000..92309d54167b --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.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 TagDdlTest extends PaimonTagDdlTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTest.scala new file mode 100644 index 000000000000..194aab278c0e --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTest.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 UpdateTableTest extends UpdateTableTestBase {} diff --git a/paimon-spark/paimon-spark-common/pom.xml b/paimon-spark/paimon-spark-common/pom.xml index e9ca8b0e1bee..1cfc53f42d48 100644 --- a/paimon-spark/paimon-spark-common/pom.xml +++ b/paimon-spark/paimon-spark-common/pom.xml @@ -23,284 +23,99 @@ under the License. 4.0.0 - paimon-spark org.apache.paimon + paimon-spark 1.0-SNAPSHOT jar - paimon-spark-common - Paimon : Spark : Common + paimon-spark-common_${scala.binary.version} + Paimon : Spark : Common : ${scala.binary.version} - 3.5.3 + ${paimon-spark-common.spark.version} - org.scala-lang - scala-library - ${scala.version} - - - org.scala-lang - scala-reflect - ${scala.version} - - - org.scala-lang - scala-compiler - ${scala.version} + org.apache.paimon + ${paimon-sparkx-common} + ${project.version} + + + * + * + + org.apache.spark - spark-avro_2.12 + spark-sql_${scala.binary.version} ${spark.version} - test org.apache.spark - spark-sql_2.12 + spark-core_${scala.binary.version} ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.orc - orc-mapreduce - - - org.apache.parquet - parquet-column - - - org.apache.spark - spark-core_2.12 - ${spark.version} - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - org.apache.orc - orc-mapreduce - - - org.apache.parquet - parquet-column - - - com.google.protobuf - protobuf-java - - + org.antlr + antlr4-runtime + ${antlr4.version} - org.apache.spark - spark-hive_2.12 - ${spark.version} - test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - com.google.protobuf - protobuf-java - - + org.apache.paimon + paimon-bundle + + org.apache.spark - spark-sql_2.12 + spark-sql_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - org.apache.parquet - parquet-column - - + org.apache.spark - spark-core_2.12 + spark-catalyst_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.logging.log4j - log4j-slf4j2-impl - - - org.apache.orc - orc-core - - - com.google.protobuf - protobuf-java - - + org.apache.spark - spark-catalyst_2.12 + spark-core_${scala.binary.version} ${spark.version} tests test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - org.apache.orc - orc-core - - - - - org.antlr - antlr4-runtime - ${antlr4.version} - - - org.scalatest - scalatest_${scala.binary.version} - 3.1.0 - test - - - org.apache.paimon - paimon-common - ${project.version} - test-jar - test + - org.apache.paimon - paimon-hive-common - ${project.version} + org.apache.spark + spark-hive_${scala.binary.version} + ${spark.version} test + - org.apache.paimon - paimon-hive-common - ${project.version} - tests - test-jar + org.apache.spark + spark-avro_${scala.binary.version} + ${spark.version} test - - - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - -nobootcp - -target:jvm-${target.java.version} - - false - - - - org.apache.maven.plugins - maven-compiler-plugin - - - - - - org.apache.maven.plugins - maven-jar-plugin - - - prepare-test-jar - test-compile - - test-jar - - - - org.apache.maven.plugins maven-shade-plugin @@ -315,39 +130,14 @@ under the License. org.apache.paimon:paimon-bundle + org.apache.paimon:${paimon-sparkx-common} - - net.alchim31.maven - scala-maven-plugin - ${scala-maven-plugin.version} - - - - scala-compile-first - process-resources - - add-source - compile - - - - - scala-test-compile - process-test-resources - - testCompile - - - - org.antlr antlr4-maven-plugin @@ -365,26 +155,17 @@ under the License. src/main/antlr4 + + - org.scalatest - scalatest-maven-plugin - ${scalatest-maven-plugin.version} - - ${project.build.directory}/surefire-reports - . - -ea -Xmx4g -Xss4m -XX:MaxMetaspaceSize=2g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true - PaimonTestSuite.txt - - once - + org.apache.maven.plugins + maven-jar-plugin - test + prepare-test-jar + test-compile - test + test-jar diff --git a/paimon-spark/paimon-spark-common/src/main/antlr4/org.apache.spark.sql.catalyst.parser.extensions/PaimonSqlExtensions.g4 b/paimon-spark/paimon-spark-common/src/main/antlr4/org.apache.spark.sql.catalyst.parser.extensions/PaimonSqlExtensions.g4 index 54e71b362fc3..207d9732160f 100644 --- a/paimon-spark/paimon-spark-common/src/main/antlr4/org.apache.spark.sql.catalyst.parser.extensions/PaimonSqlExtensions.g4 +++ b/paimon-spark/paimon-spark-common/src/main/antlr4/org.apache.spark.sql.catalyst.parser.extensions/PaimonSqlExtensions.g4 @@ -70,13 +70,40 @@ singleStatement statement : CALL multipartIdentifier '(' (callArgument (',' callArgument)*)? ')' #call - ; + | SHOW TAGS multipartIdentifier #showTags + | ALTER TABLE multipartIdentifier createReplaceTagClause #createOrReplaceTag + | ALTER TABLE multipartIdentifier DELETE TAG (IF EXISTS)? identifier #deleteTag + | ALTER TABLE multipartIdentifier RENAME TAG identifier TO identifier #renameTag + ; callArgument : expression #positionalArgument | identifier '=>' expression #namedArgument ; +createReplaceTagClause + : CREATE TAG (IF NOT EXISTS)? identifier tagOptions + | (CREATE OR)? REPLACE TAG identifier tagOptions + ; + +tagOptions + : (AS OF VERSION snapshotId)? (timeRetain)? + ; + +snapshotId + : number + ; + +timeRetain + : RETAIN number timeUnit + ; + +timeUnit + : DAYS + | HOURS + | MINUTES + ; + expression : constant | stringMap @@ -124,12 +151,34 @@ quotedIdentifier ; nonReserved - : CALL + : ALTER | AS | CALL | CREATE | DAYS | DELETE | EXISTS | HOURS | IF | NOT | OF | OR | TABLE + | REPLACE | RETAIN | VERSION | TAG | TRUE | FALSE | MAP ; +ALTER: 'ALTER'; +AS: 'AS'; CALL: 'CALL'; +CREATE: 'CREATE'; +DAYS: 'DAYS'; +DELETE: 'DELETE'; +EXISTS: 'EXISTS'; +HOURS: 'HOURS'; +IF : 'IF'; +MINUTES: 'MINUTES'; +NOT: 'NOT'; +OF: 'OF'; +OR: 'OR'; +RENAME: 'RENAME'; +REPLACE: 'REPLACE'; +RETAIN: 'RETAIN'; +SHOW: 'SHOW'; +TABLE: 'TABLE'; +TAG: 'TAG'; +TAGS: 'TAGS'; +TO: 'TO'; +VERSION: 'VERSION'; TRUE: 'TRUE'; FALSE: 'FALSE'; diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkArrayData.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkArrayData.java index 0e7428eabde7..9934047a1825 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkArrayData.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkArrayData.java @@ -38,7 +38,7 @@ import static org.apache.paimon.utils.InternalRowUtils.copyArray; /** Spark {@link ArrayData} to wrap Paimon {@link InternalArray}. */ -public class SparkArrayData extends ArrayData { +public class SparkArrayData extends org.apache.spark.sql.paimon.shims.ArrayData { private final DataType elementType; 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 9e05ee6493c9..82c8939eab87 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 @@ -27,6 +27,8 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.spark.catalog.SparkBaseCatalog; import org.apache.paimon.spark.catalog.SupportFunction; +import org.apache.paimon.table.FormatTable; +import org.apache.paimon.table.FormatTableOptions; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; @@ -41,6 +43,13 @@ import org.apache.spark.sql.connector.expressions.IdentityTransform; import org.apache.spark.sql.connector.expressions.NamedReference; import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat; +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat; +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat; +import org.apache.spark.sql.execution.datasources.v2.FileTable; +import org.apache.spark.sql.execution.datasources.v2.csv.CSVTable; +import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable; +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable; import org.apache.spark.sql.internal.SessionState; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; @@ -56,6 +65,9 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.paimon.CoreOptions.FILE_FORMAT; +import static org.apache.paimon.CoreOptions.TYPE; +import static org.apache.paimon.TableType.FORMAT_TABLE; import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; import static org.apache.paimon.spark.SparkCatalogOptions.DEFAULT_DATABASE; import static org.apache.paimon.spark.SparkTypeUtils.toPaimonType; @@ -91,7 +103,9 @@ public void initialize(String name, CaseInsensitiveStringMap options) { this.catalog = CatalogFactory.createCatalog(catalogContext); this.defaultDatabase = options.getOrDefault(DEFAULT_DATABASE.key(), DEFAULT_DATABASE.defaultValue()); - if (!catalog.databaseExists(defaultNamespace()[0])) { + try { + catalog.getDatabase(defaultNamespace()[0]); + } catch (Catalog.DatabaseNotExistException e) { try { createNamespace(defaultNamespace(), new HashMap<>()); } catch (NamespaceAlreadyExistsException ignored) { @@ -141,10 +155,12 @@ public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceExcep if (!isValidateNamespace(namespace)) { throw new NoSuchNamespaceException(namespace); } - if (catalog.databaseExists(namespace[0])) { + try { + catalog.getDatabase(namespace[0]); return new String[0][]; + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchNamespaceException(namespace); } - throw new NoSuchNamespaceException(namespace); } @Override @@ -156,7 +172,7 @@ public Map loadNamespaceMetadata(String[] namespace) Arrays.toString(namespace)); String dataBaseName = namespace[0]; try { - return catalog.loadDatabaseProperties(dataBaseName); + return catalog.getDatabase(dataBaseName).options(); } catch (Catalog.DatabaseNotExistException e) { throw new NoSuchNamespaceException(namespace); } @@ -230,7 +246,8 @@ public void invalidateTable(Identifier ident) { } @Override - public SparkTable loadTable(Identifier ident) throws NoSuchTableException { + public org.apache.spark.sql.connector.catalog.Table loadTable(Identifier ident) + throws NoSuchTableException { return loadSparkTable(ident, Collections.emptyMap()); } @@ -239,8 +256,14 @@ public SparkTable loadTable(Identifier ident) throws NoSuchTableException { */ public SparkTable loadTable(Identifier ident, String version) throws NoSuchTableException { LOG.info("Time travel to version '{}'.", version); - return loadSparkTable( - ident, Collections.singletonMap(CoreOptions.SCAN_VERSION.key(), version)); + org.apache.spark.sql.connector.catalog.Table table = + loadSparkTable( + ident, Collections.singletonMap(CoreOptions.SCAN_VERSION.key(), version)); + if (table instanceof SparkTable) { + return (SparkTable) table; + } else { + throw new NoSuchTableException(ident); + } } /** @@ -253,18 +276,16 @@ public SparkTable loadTable(Identifier ident, long timestamp) throws NoSuchTable // Paimon's timestamp use millisecond timestamp = timestamp / 1000; LOG.info("Time travel target timestamp is {} milliseconds.", timestamp); - return loadSparkTable( - ident, - Collections.singletonMap( - CoreOptions.SCAN_TIMESTAMP_MILLIS.key(), String.valueOf(timestamp))); - } - - @Override - public boolean tableExists(Identifier ident) { - try { - return catalog.tableExists(toIdentifier(ident)); - } catch (NoSuchTableException e) { - return false; + org.apache.spark.sql.connector.catalog.Table table = + loadSparkTable( + ident, + Collections.singletonMap( + CoreOptions.SCAN_TIMESTAMP_MILLIS.key(), + String.valueOf(timestamp))); + if (table instanceof SparkTable) { + return (SparkTable) table; + } else { + throw new NoSuchTableException(ident); } } @@ -284,7 +305,7 @@ public org.apache.spark.sql.connector.catalog.Table alterTable( } @Override - public SparkTable createTable( + public org.apache.spark.sql.connector.catalog.Table createTable( Identifier ident, StructType schema, Transform[] partitions, @@ -292,12 +313,25 @@ public SparkTable createTable( throws TableAlreadyExistsException, NoSuchNamespaceException { try { String provider = properties.get(TableCatalog.PROP_PROVIDER); - checkArgument( - usePaimon(provider), - "SparkCatalog can only create paimon table, but current provider is %s", - provider); - catalog.createTable( - toIdentifier(ident), toInitialSchema(schema, partitions, properties), false); + if ((!usePaimon(provider)) + && SparkSource.FORMAT_NAMES().contains(provider.toLowerCase())) { + Map newProperties = new HashMap<>(properties); + newProperties.put(TYPE.key(), FORMAT_TABLE.toString()); + newProperties.put(FILE_FORMAT.key(), provider.toLowerCase()); + catalog.createTable( + toIdentifier(ident), + toInitialSchema(schema, partitions, newProperties), + false); + } else { + checkArgument( + usePaimon(provider), + "SparkCatalog can only create paimon table, but current provider is %s", + provider); + catalog.createTable( + toIdentifier(ident), + toInitialSchema(schema, partitions, properties), + false); + } return loadTable(ident); } catch (Catalog.TableAlreadyExistException e) { throw new TableAlreadyExistsException(ident); @@ -337,21 +371,18 @@ private SchemaChange toSchemaChange(TableChange change) { } } else if (change instanceof TableChange.AddColumn) { TableChange.AddColumn add = (TableChange.AddColumn) change; - validateAlterNestedField(add.fieldNames()); SchemaChange.Move move = getMove(add.position(), add.fieldNames()); return SchemaChange.addColumn( - add.fieldNames()[0], + Arrays.asList(add.fieldNames()), toPaimonType(add.dataType()).copy(add.isNullable()), add.comment(), move); } else if (change instanceof TableChange.RenameColumn) { TableChange.RenameColumn rename = (TableChange.RenameColumn) change; - validateAlterNestedField(rename.fieldNames()); - return SchemaChange.renameColumn(rename.fieldNames()[0], rename.newName()); + return SchemaChange.renameColumn(Arrays.asList(rename.fieldNames()), rename.newName()); } else if (change instanceof TableChange.DeleteColumn) { TableChange.DeleteColumn delete = (TableChange.DeleteColumn) change; - validateAlterNestedField(delete.fieldNames()); - return SchemaChange.dropColumn(delete.fieldNames()[0]); + return SchemaChange.dropColumn(Arrays.asList(delete.fieldNames())); } else if (change instanceof TableChange.UpdateColumnType) { TableChange.UpdateColumnType update = (TableChange.UpdateColumnType) change; validateAlterNestedField(update.fieldNames()); @@ -458,16 +489,63 @@ protected org.apache.paimon.catalog.Identifier toIdentifier(Identifier ident) return new org.apache.paimon.catalog.Identifier(ident.namespace()[0], ident.name()); } - protected SparkTable loadSparkTable(Identifier ident, Map extraOptions) - throws NoSuchTableException { + protected org.apache.spark.sql.connector.catalog.Table loadSparkTable( + Identifier ident, Map extraOptions) throws NoSuchTableException { try { - return new SparkTable( - copyWithSQLConf(catalog.getTable(toIdentifier(ident)), extraOptions)); + org.apache.paimon.table.Table paimonTable = catalog.getTable(toIdentifier(ident)); + if (paimonTable instanceof FormatTable) { + return convertToFileTable(ident, (FormatTable) paimonTable); + } else { + return new SparkTable( + copyWithSQLConf( + paimonTable, catalogName, toIdentifier(ident), extraOptions)); + } } catch (Catalog.TableNotExistException e) { throw new NoSuchTableException(ident); } } + private static FileTable convertToFileTable(Identifier ident, FormatTable formatTable) { + StructType schema = SparkTypeUtils.fromPaimonRowType(formatTable.rowType()); + List pathList = new ArrayList<>(); + pathList.add(formatTable.location()); + Options options = Options.fromMap(formatTable.options()); + CaseInsensitiveStringMap dsOptions = new CaseInsensitiveStringMap(options.toMap()); + if (formatTable.format() == FormatTable.Format.CSV) { + options.set("sep", options.get(FormatTableOptions.FIELD_DELIMITER)); + dsOptions = new CaseInsensitiveStringMap(options.toMap()); + return new CSVTable( + ident.name(), + SparkSession.active(), + dsOptions, + scala.collection.JavaConverters.asScalaBuffer(pathList).toSeq(), + scala.Option.apply(schema), + CSVFileFormat.class); + } else if (formatTable.format() == FormatTable.Format.ORC) { + return new OrcTable( + ident.name(), + SparkSession.active(), + dsOptions, + scala.collection.JavaConverters.asScalaBuffer(pathList).toSeq(), + scala.Option.apply(schema), + OrcFileFormat.class); + } else if (formatTable.format() == FormatTable.Format.PARQUET) { + return new ParquetTable( + ident.name(), + SparkSession.active(), + dsOptions, + scala.collection.JavaConverters.asScalaBuffer(pathList).toSeq(), + scala.Option.apply(schema), + ParquetFileFormat.class); + } else { + throw new UnsupportedOperationException( + "Unsupported format table " + + ident.name() + + " format " + + formatTable.format().name()); + } + } + protected List convertPartitionTransforms(Transform[] transforms) { List partitionColNames = new ArrayList<>(transforms.length); for (Transform transform : transforms) { diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java index 12407f2614ff..d4b712fcb8ee 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkGenericCatalog.java @@ -52,6 +52,7 @@ import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.internal.SessionState; import org.apache.spark.sql.internal.StaticSQLConf; +import org.apache.spark.sql.paimon.shims; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.slf4j.Logger; @@ -202,7 +203,7 @@ public Table createTable( return sparkCatalog.createTable(ident, schema, partitions, properties); } else { // delegate to the session catalog - return asTableCatalog().createTable(ident, schema, partitions, properties); + return shims.createTable(asTableCatalog(), ident, schema, partitions, properties); } } @@ -317,6 +318,7 @@ private void fillCommonConfigurations(Map options, SQLConf sqlCo options.put(METASTORE.key(), metastore); } } + options.put(CatalogOptions.FORMAT_TABLE_ENABLED.key(), "false"); String sessionCatalogDefaultDatabase = SQLConfUtils.defaultDatabase(sqlConf); if (options.containsKey(DEFAULT_DATABASE.key())) { String userDefineDefaultDatabase = options.get(DEFAULT_DATABASE.key()); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRow.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRow.java index 9dd6c7b68519..147c6c2d77c8 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRow.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRow.java @@ -59,10 +59,12 @@ import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; +import java.util.Objects; + import static org.apache.paimon.utils.InternalRowUtils.copyInternalRow; /** Spark {@link org.apache.spark.sql.catalyst.InternalRow} to wrap {@link InternalRow}. */ -public class SparkInternalRow extends org.apache.spark.sql.catalyst.InternalRow { +public class SparkInternalRow extends org.apache.spark.sql.paimon.shims.InternalRow { private final RowType rowType; @@ -245,6 +247,25 @@ public Object get(int ordinal, org.apache.spark.sql.types.DataType dataType) { throw new UnsupportedOperationException("Unsupported data type " + dataType.simpleString()); } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SparkInternalRow that = (SparkInternalRow) o; + return Objects.equals(rowType, that.rowType) && Objects.equals(row, that.row); + } + + @Override + public int hashCode() { + return Objects.hash(rowType, row); + } + + // ================== static methods ========================================= + public static Object fromPaimon(Object o, DataType type) { if (o == null) { return null; diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkProcedures.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkProcedures.java index 0a9076247360..c93aad41a732 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkProcedures.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkProcedures.java @@ -18,6 +18,7 @@ package org.apache.paimon.spark; +import org.apache.paimon.spark.procedure.CompactManifestProcedure; import org.apache.paimon.spark.procedure.CompactProcedure; import org.apache.paimon.spark.procedure.CreateBranchProcedure; import org.apache.paimon.spark.procedure.CreateTagFromTimestampProcedure; @@ -26,16 +27,21 @@ import org.apache.paimon.spark.procedure.DeleteTagProcedure; import org.apache.paimon.spark.procedure.ExpirePartitionsProcedure; import org.apache.paimon.spark.procedure.ExpireSnapshotsProcedure; +import org.apache.paimon.spark.procedure.ExpireTagsProcedure; import org.apache.paimon.spark.procedure.FastForwardProcedure; import org.apache.paimon.spark.procedure.MarkPartitionDoneProcedure; +import org.apache.paimon.spark.procedure.MigrateDatabaseProcedure; import org.apache.paimon.spark.procedure.MigrateFileProcedure; import org.apache.paimon.spark.procedure.MigrateTableProcedure; import org.apache.paimon.spark.procedure.Procedure; import org.apache.paimon.spark.procedure.ProcedureBuilder; import org.apache.paimon.spark.procedure.RemoveOrphanFilesProcedure; +import org.apache.paimon.spark.procedure.RenameTagProcedure; import org.apache.paimon.spark.procedure.RepairProcedure; +import org.apache.paimon.spark.procedure.ReplaceTagProcedure; import org.apache.paimon.spark.procedure.ResetConsumerProcedure; import org.apache.paimon.spark.procedure.RollbackProcedure; +import org.apache.paimon.spark.procedure.RollbackToTimestampProcedure; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; @@ -59,13 +65,18 @@ private static Map> initProcedureBuilders() { ImmutableMap.Builder> procedureBuilders = ImmutableMap.builder(); procedureBuilders.put("rollback", RollbackProcedure::builder); + procedureBuilders.put("rollback_to_timestamp", RollbackToTimestampProcedure::builder); procedureBuilders.put("create_tag", CreateTagProcedure::builder); + procedureBuilders.put("replace_tag", ReplaceTagProcedure::builder); + procedureBuilders.put("rename_tag", RenameTagProcedure::builder); procedureBuilders.put( "create_tag_from_timestamp", CreateTagFromTimestampProcedure::builder); procedureBuilders.put("delete_tag", DeleteTagProcedure::builder); + procedureBuilders.put("expire_tags", ExpireTagsProcedure::builder); procedureBuilders.put("create_branch", CreateBranchProcedure::builder); procedureBuilders.put("delete_branch", DeleteBranchProcedure::builder); procedureBuilders.put("compact", CompactProcedure::builder); + procedureBuilders.put("migrate_database", MigrateDatabaseProcedure::builder); procedureBuilders.put("migrate_table", MigrateTableProcedure::builder); procedureBuilders.put("migrate_file", MigrateFileProcedure::builder); procedureBuilders.put("remove_orphan_files", RemoveOrphanFilesProcedure::builder); @@ -75,6 +86,7 @@ private static Map> initProcedureBuilders() { procedureBuilders.put("fast_forward", FastForwardProcedure::builder); procedureBuilders.put("reset_consumer", ResetConsumerProcedure::builder); procedureBuilders.put("mark_partition_done", MarkPartitionDoneProcedure::builder); + procedureBuilders.put("compact_manifest", CompactManifestProcedure::builder); return procedureBuilders.build(); } } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java index b343be247824..60648a8385dc 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkRow.java @@ -33,8 +33,6 @@ import org.apache.paimon.types.RowType; import org.apache.paimon.utils.DateTimeUtils; -import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; - import org.apache.spark.sql.Row; import java.io.Serializable; @@ -48,7 +46,6 @@ import java.util.Map; import scala.collection.JavaConverters; -import scala.collection.mutable.WrappedArray; /** A {@link InternalRow} wraps spark {@link Row}. */ public class SparkRow implements InternalRow, Serializable { @@ -305,26 +302,22 @@ public byte[] getBinary(int i) { @Override public InternalArray getArray(int i) { - Object array = getAs(i); - if (array instanceof WrappedArray) { - List result = Lists.newArrayList(); - ((WrappedArray) array).iterator().foreach(x -> result.add(x)); - return new PaimonArray(((ArrayType) elementType).getElementType(), result); - } - return new PaimonArray( - ((ArrayType) elementType).getElementType(), (List) array); + Object o = getAs(i); + List array = + o instanceof scala.collection.Seq + ? JavaConverters.seqAsJavaList((scala.collection.Seq) o) + : (List) o; + return new PaimonArray(((ArrayType) elementType).getElementType(), array); } @Override public InternalMap getMap(int i) { - Object map = getAs(i); - if (map instanceof scala.collection.immutable.Map) { - return toPaimonMap( - (MapType) elementType, - JavaConverters.mapAsJavaMap( - (scala.collection.immutable.Map) map)); - } - return toPaimonMap((MapType) elementType, (Map) map); + Object o = getAs(i); + Map map = + o instanceof scala.collection.Map + ? JavaConverters.mapAsJavaMap((scala.collection.Map) o) + : (Map) o; + return toPaimonMap((MapType) elementType, map); } @Override diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java index bc5eed00e98b..8bba676200ce 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkTypeUtils.java @@ -60,13 +60,17 @@ public class SparkTypeUtils { private SparkTypeUtils() {} - public static StructType toSparkPartitionType(Table table) { + public static RowType toPartitionType(Table table) { int[] projections = table.rowType().getFieldIndices(table.partitionKeys()); List partitionTypes = new ArrayList<>(); for (int i : projections) { partitionTypes.add(table.rowType().getFields().get(i)); } - return (StructType) SparkTypeUtils.fromPaimonType(new RowType(false, partitionTypes)); + return new RowType(false, partitionTypes); + } + + public static StructType toSparkPartitionType(Table table) { + return (StructType) SparkTypeUtils.fromPaimonType(toPartitionType(table)); } public static StructType fromPaimonRowType(RowType type) { @@ -81,6 +85,42 @@ public static org.apache.paimon.types.DataType toPaimonType(DataType dataType) { return SparkToPaimonTypeVisitor.visit(dataType); } + /** + * Prune Paimon `RowType` by required Spark `StructType`, use this method instead of {@link + * #toPaimonType(DataType)} when need to retain the field id. + */ + public static RowType prunePaimonRowType(StructType requiredStructType, RowType rowType) { + return (RowType) prunePaimonType(requiredStructType, rowType); + } + + private static org.apache.paimon.types.DataType prunePaimonType( + DataType sparkDataType, org.apache.paimon.types.DataType paimonDataType) { + if (sparkDataType instanceof StructType) { + StructType s = (StructType) sparkDataType; + RowType p = (RowType) paimonDataType; + List newFields = new ArrayList<>(); + for (StructField field : s.fields()) { + DataField f = p.getField(field.name()); + newFields.add(f.newType(prunePaimonType(field.dataType(), f.type()))); + } + return p.copy(newFields); + } else if (sparkDataType instanceof org.apache.spark.sql.types.MapType) { + org.apache.spark.sql.types.MapType s = + (org.apache.spark.sql.types.MapType) sparkDataType; + MapType p = (MapType) paimonDataType; + return p.newKeyValueType( + prunePaimonType(s.keyType(), p.getKeyType()), + prunePaimonType(s.valueType(), p.getValueType())); + } else if (sparkDataType instanceof org.apache.spark.sql.types.ArrayType) { + org.apache.spark.sql.types.ArrayType s = + (org.apache.spark.sql.types.ArrayType) sparkDataType; + ArrayType r = (ArrayType) paimonDataType; + return r.newElementType(prunePaimonType(s.elementType(), r.getElementType())); + } else { + return paimonDataType; + } + } + private static class PaimonToSparkTypeVisitor extends DataTypeDefaultVisitor { private static final PaimonToSparkTypeVisitor INSTANCE = new PaimonToSparkTypeVisitor(); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportFunction.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportFunction.java index 91a6d7b4a2e6..772a2f4ed53d 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportFunction.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportFunction.java @@ -29,8 +29,6 @@ import java.util.Arrays; -import scala.Option; - import static org.apache.paimon.catalog.Catalog.SYSTEM_DATABASE_NAME; /** Catalog methods for working with Functions. */ @@ -56,8 +54,7 @@ default Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceExc return new Identifier[0]; } - throw new NoSuchNamespaceException( - "Namespace " + Arrays.toString(namespace) + " is not valid", Option.empty()); + throw new RuntimeException("Namespace " + Arrays.toString(namespace) + " is not valid"); } @Override @@ -69,7 +66,6 @@ default UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionExce } } - throw new NoSuchFunctionException( - "Function " + ident + " is not a paimon function", Option.empty()); + throw new RuntimeException("Function " + ident + " is not a paimon function"); } } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactManifestProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactManifestProcedure.java new file mode 100644 index 000000000000..dd064d892c3d --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactManifestProcedure.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.procedure; + +import org.apache.paimon.operation.FileStoreCommit; +import org.apache.paimon.table.FileStoreTable; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import static org.apache.spark.sql.types.DataTypes.StringType; + +/** + * Compact manifest procedure. Usage: + * + *
    
    + *  CALL sys.compact_manifest(table => 'tableId')
    + * 
    + */ +public class CompactManifestProcedure extends BaseProcedure { + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {ProcedureParameter.required("table", StringType)}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("result", DataTypes.BooleanType, true, Metadata.empty()) + }); + + protected CompactManifestProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public StructType outputType() { + return OUTPUT_TYPE; + } + + @Override + public InternalRow[] call(InternalRow args) { + + Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); + FileStoreTable table = (FileStoreTable) loadSparkTable(tableIdent).getTable(); + + try (FileStoreCommit commit = + table.store() + .newCommit(table.coreOptions().createCommitUser()) + .ignoreEmptyCommit(false)) { + commit.compactManifest(); + } + + return new InternalRow[] {newInternalRow(true)}; + } + + @Override + public String description() { + return "This procedure execute compact action on paimon table."; + } + + public static ProcedureBuilder builder() { + return new Builder() { + @Override + public CompactManifestProcedure doBuild() { + return new CompactManifestProcedure(tableCatalog()); + } + }; + } +} 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 5b5f4e33c2ee..71cf04cf5ef5 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 @@ -43,6 +43,7 @@ import org.apache.paimon.table.sink.CompactionTaskSerializer; import org.apache.paimon.table.sink.TableCommitImpl; import org.apache.paimon.table.source.DataSplit; +import org.apache.paimon.table.source.EndOfScanException; import org.apache.paimon.table.source.snapshot.SnapshotReader; import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.ParameterUtils; @@ -340,8 +341,13 @@ private void compactUnAwareBucketTable( @Nullable Predicate filter, @Nullable Duration partitionIdleTime, JavaSparkContext javaSparkContext) { - List compactionTasks = - new UnawareAppendTableCompactionCoordinator(table, false, filter).run(); + List compactionTasks; + try { + compactionTasks = + new UnawareAppendTableCompactionCoordinator(table, false, filter).run(); + } catch (EndOfScanException e) { + compactionTasks = new ArrayList<>(); + } if (partitionIdleTime != null) { Map partitionInfo = table.newSnapshotReader().partitionEntries().stream() diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateOrReplaceTagBaseProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateOrReplaceTagBaseProcedure.java new file mode 100644 index 000000000000..ed264140b797 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateOrReplaceTagBaseProcedure.java @@ -0,0 +1,87 @@ +/* + * 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.procedure; + +import org.apache.paimon.table.Table; +import org.apache.paimon.utils.TimeUtils; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import java.time.Duration; + +import static org.apache.spark.sql.types.DataTypes.LongType; +import static org.apache.spark.sql.types.DataTypes.StringType; + +/** A base procedure to create or replace a tag. */ +public abstract class CreateOrReplaceTagBaseProcedure extends BaseProcedure { + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + ProcedureParameter.required("table", StringType), + ProcedureParameter.required("tag", StringType), + ProcedureParameter.optional("snapshot", LongType), + ProcedureParameter.optional("time_retained", StringType) + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("result", DataTypes.BooleanType, true, Metadata.empty()) + }); + + protected CreateOrReplaceTagBaseProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public StructType outputType() { + return OUTPUT_TYPE; + } + + @Override + public InternalRow[] call(InternalRow args) { + Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); + String tag = args.getString(1); + Long snapshot = args.isNullAt(2) ? null : args.getLong(2); + Duration timeRetained = + args.isNullAt(3) ? null : TimeUtils.parseDuration(args.getString(3)); + + return modifyPaimonTable( + tableIdent, + table -> { + createOrReplaceTag(table, tag, snapshot, timeRetained); + InternalRow outputRow = newInternalRow(true); + return new InternalRow[] {outputRow}; + }); + } + + abstract void createOrReplaceTag( + Table table, String tagName, Long snapshotId, Duration timeRetained); +} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateTagProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateTagProcedure.java index b3f863c5e305..157743f9892e 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateTagProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CreateTagProcedure.java @@ -18,71 +18,26 @@ package org.apache.paimon.spark.procedure; -import org.apache.paimon.utils.TimeUtils; +import org.apache.paimon.table.Table; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; import java.time.Duration; -import static org.apache.spark.sql.types.DataTypes.LongType; -import static org.apache.spark.sql.types.DataTypes.StringType; - /** A procedure to create a tag. */ -public class CreateTagProcedure extends BaseProcedure { - - private static final ProcedureParameter[] PARAMETERS = - new ProcedureParameter[] { - ProcedureParameter.required("table", StringType), - ProcedureParameter.required("tag", StringType), - ProcedureParameter.optional("snapshot", LongType), - ProcedureParameter.optional("time_retained", StringType) - }; - - private static final StructType OUTPUT_TYPE = - new StructType( - new StructField[] { - new StructField("result", DataTypes.BooleanType, true, Metadata.empty()) - }); +public class CreateTagProcedure extends CreateOrReplaceTagBaseProcedure { - protected CreateTagProcedure(TableCatalog tableCatalog) { + private CreateTagProcedure(TableCatalog tableCatalog) { super(tableCatalog); } @Override - public ProcedureParameter[] parameters() { - return PARAMETERS; - } - - @Override - public StructType outputType() { - return OUTPUT_TYPE; - } - - @Override - public InternalRow[] call(InternalRow args) { - Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); - String tag = args.getString(1); - Long snapshot = args.isNullAt(2) ? null : args.getLong(2); - Duration timeRetained = - args.isNullAt(3) ? null : TimeUtils.parseDuration(args.getString(3)); - - return modifyPaimonTable( - tableIdent, - table -> { - if (snapshot == null) { - table.createTag(tag, timeRetained); - } else { - table.createTag(tag, snapshot, timeRetained); - } - InternalRow outputRow = newInternalRow(true); - return new InternalRow[] {outputRow}; - }); + void createOrReplaceTag(Table table, String tagName, Long snapshotId, Duration timeRetained) { + if (snapshotId == null) { + table.createTag(tagName, timeRetained); + } else { + table.createTag(tagName, snapshotId, timeRetained); + } } public static ProcedureBuilder builder() { diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpirePartitionsProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpirePartitionsProcedure.java index 81012abc9e0b..7b388227e5a4 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpirePartitionsProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpirePartitionsProcedure.java @@ -40,6 +40,7 @@ import java.util.Optional; import static org.apache.paimon.partition.PartitionExpireStrategy.createPartitionExpireStrategy; +import static org.apache.spark.sql.types.DataTypes.IntegerType; import static org.apache.spark.sql.types.DataTypes.StringType; /** A procedure to expire partitions. */ @@ -51,7 +52,8 @@ public class ExpirePartitionsProcedure extends BaseProcedure { ProcedureParameter.required("expiration_time", StringType), ProcedureParameter.optional("timestamp_formatter", StringType), ProcedureParameter.optional("timestamp_pattern", StringType), - ProcedureParameter.optional("expire_strategy", StringType) + ProcedureParameter.optional("expire_strategy", StringType), + ProcedureParameter.optional("max_expires", IntegerType) }; private static final StructType OUTPUT_TYPE = @@ -81,6 +83,7 @@ public InternalRow[] call(InternalRow args) { String timestampFormatter = args.isNullAt(2) ? null : args.getString(2); String timestampPattern = args.isNullAt(3) ? null : args.getString(3); String expireStrategy = args.isNullAt(4) ? null : args.getString(4); + Integer maxExpires = args.isNullAt(5) ? null : args.getInt(5); return modifyPaimonTable( tableIdent, table -> { @@ -105,6 +108,9 @@ public InternalRow[] call(InternalRow args) { .metastoreClientFactory()) .map(MetastoreClient.Factory::create) .orElse(null)); + if (maxExpires != null) { + partitionExpire.withMaxExpires(maxExpires); + } List> expired = partitionExpire.expire(Long.MAX_VALUE); return expired == null || expired.isEmpty() ? new InternalRow[] { diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedure.java index 4716f6add5c5..f24f18067a39 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedure.java @@ -20,6 +20,8 @@ import org.apache.paimon.options.ExpireConfig; import org.apache.paimon.table.ExpireSnapshots; +import org.apache.paimon.utils.DateTimeUtils; +import org.apache.paimon.utils.StringUtils; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.Identifier; @@ -29,10 +31,10 @@ import org.apache.spark.sql.types.StructType; import java.time.Duration; +import java.util.TimeZone; import static org.apache.spark.sql.types.DataTypes.IntegerType; import static org.apache.spark.sql.types.DataTypes.StringType; -import static org.apache.spark.sql.types.DataTypes.TimestampType; /** A procedure to expire snapshots. */ public class ExpireSnapshotsProcedure extends BaseProcedure { @@ -42,7 +44,7 @@ public class ExpireSnapshotsProcedure extends BaseProcedure { ProcedureParameter.required("table", StringType), ProcedureParameter.optional("retain_max", IntegerType), ProcedureParameter.optional("retain_min", IntegerType), - ProcedureParameter.optional("older_than", TimestampType), + ProcedureParameter.optional("older_than", StringType), ProcedureParameter.optional("max_deletes", IntegerType) }; @@ -72,8 +74,9 @@ public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); Integer retainMax = args.isNullAt(1) ? null : args.getInt(1); Integer retainMin = args.isNullAt(2) ? null : args.getInt(2); - Long olderThanMills = args.isNullAt(3) ? null : args.getLong(3) / 1000; + String olderThanStr = args.isNullAt(3) ? null : args.getString(3); Integer maxDeletes = args.isNullAt(4) ? null : args.getInt(4); + return modifyPaimonTable( tableIdent, table -> { @@ -85,7 +88,11 @@ public InternalRow[] call(InternalRow args) { if (retainMin != null) { builder.snapshotRetainMin(retainMin); } - if (olderThanMills != null) { + if (!StringUtils.isNullOrWhitespaceOnly(olderThanStr)) { + long olderThanMills = + DateTimeUtils.parseTimestampData( + olderThanStr, 3, TimeZone.getDefault()) + .getMillisecond(); builder.snapshotTimeRetain( Duration.ofMillis(System.currentTimeMillis() - olderThanMills)); } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpireTagsProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpireTagsProcedure.java new file mode 100644 index 000000000000..d75ca5ee0aac --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ExpireTagsProcedure.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.procedure; + +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.tag.TagTimeExpire; +import org.apache.paimon.utils.DateTimeUtils; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +import java.time.LocalDateTime; +import java.util.List; +import java.util.TimeZone; + +import static org.apache.spark.sql.types.DataTypes.StringType; + +/** A procedure to expire tags by time. */ +public class ExpireTagsProcedure extends BaseProcedure { + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + ProcedureParameter.required("table", StringType), + ProcedureParameter.optional("older_than", StringType) + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("expired_tags", StringType, false, Metadata.empty()) + }); + + protected ExpireTagsProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public StructType outputType() { + return OUTPUT_TYPE; + } + + @Override + public InternalRow[] call(InternalRow args) { + Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); + String olderThanStr = args.isNullAt(1) ? null : args.getString(1); + return modifyPaimonTable( + tableIdent, + table -> { + FileStoreTable fileStoreTable = (FileStoreTable) table; + TagTimeExpire tagTimeExpire = + fileStoreTable.store().newTagCreationManager().getTagTimeExpire(); + if (olderThanStr != null) { + LocalDateTime olderThanTime = + DateTimeUtils.parseTimestampData( + olderThanStr, 3, TimeZone.getDefault()) + .toLocalDateTime(); + tagTimeExpire.withOlderThanTime(olderThanTime); + } + List expired = tagTimeExpire.expire(); + return expired.isEmpty() + ? new InternalRow[] { + newInternalRow(UTF8String.fromString("No expired tags.")) + } + : expired.stream() + .map(x -> newInternalRow(UTF8String.fromString(x))) + .toArray(InternalRow[]::new); + }); + } + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + public ExpireTagsProcedure doBuild() { + return new ExpireTagsProcedure(tableCatalog()); + } + }; + } + + @Override + public String description() { + return "ExpireTagsProcedure"; + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/MigrateDatabaseProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/MigrateDatabaseProcedure.java new file mode 100644 index 000000000000..db0c7cebb0a3 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/MigrateDatabaseProcedure.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.procedure; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.migrate.Migrator; +import org.apache.paimon.spark.catalog.WithPaimonCatalog; +import org.apache.paimon.spark.utils.TableMigrationUtils; +import org.apache.paimon.utils.ParameterUtils; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.spark.sql.types.DataTypes.IntegerType; +import static org.apache.spark.sql.types.DataTypes.StringType; + +/** + * Migrate database procedure. Usage: + * + *
    
    + *  CALL sys.migrate_database(source_type => 'hive', database => 'db01', options => 'x1=y1,x2=y2')
    + * 
    + */ +public class MigrateDatabaseProcedure extends BaseProcedure { + + private static final Logger LOG = LoggerFactory.getLogger(MigrateDatabaseProcedure.class); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + ProcedureParameter.required("source_type", StringType), + ProcedureParameter.required("database", StringType), + ProcedureParameter.optional("options", StringType), + ProcedureParameter.optional( + "options_map", DataTypes.createMapType(StringType, StringType)), + ProcedureParameter.optional("parallelism", IntegerType), + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("result", StringType, true, Metadata.empty()) + }); + + protected MigrateDatabaseProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public StructType outputType() { + return OUTPUT_TYPE; + } + + @Override + public InternalRow[] call(InternalRow args) { + String format = args.getString(0); + String database = args.getString(1); + String properties = args.isNullAt(2) ? null : args.getString(2); + MapData mapData = args.isNullAt(3) ? null : args.getMap(3); + Map optionMap = mapDataToHashMap(mapData); + int parallelism = + args.isNullAt(4) ? Runtime.getRuntime().availableProcessors() : args.getInt(4); + + Catalog paimonCatalog = ((WithPaimonCatalog) tableCatalog()).paimonCatalog(); + + Map options = ParameterUtils.parseCommaSeparatedKeyValues(properties); + options.putAll(optionMap); + + List migrators = + TableMigrationUtils.getImporters( + format, paimonCatalog, database, parallelism, options); + + int errorCount = 0; + int successCount = 0; + + for (Migrator migrator : migrators) { + try { + migrator.executeMigrate(); + migrator.renameTable(false); + successCount++; + } catch (Exception e) { + errorCount++; + LOG.error("Call migrate_database error:" + e.getMessage()); + } + } + String retStr = + String.format( + "migrate database is finished, success cnt: %s , failed cnt: %s", + String.valueOf(successCount), String.valueOf(errorCount)); + + return new InternalRow[] {newInternalRow(retStr)}; + } + + public static Map mapDataToHashMap(MapData mapData) { + HashMap map = new HashMap<>(); + if (mapData != null) { + for (int index = 0; index < mapData.numElements(); index++) { + map.put( + mapData.keyArray().getUTF8String(index).toString(), + mapData.valueArray().getUTF8String(index).toString()); + } + } + return map; + } + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + public MigrateDatabaseProcedure doBuild() { + return new MigrateDatabaseProcedure(tableCatalog()); + } + }; + } + + @Override + public String description() { + return "MigrateDatabaseProcedure"; + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/MigrateFileProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/MigrateFileProcedure.java index 32f89d47b1a0..95d55df01178 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/MigrateFileProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/MigrateFileProcedure.java @@ -88,7 +88,9 @@ public InternalRow[] call(InternalRow args) { Catalog paimonCatalog = ((WithPaimonCatalog) tableCatalog()).paimonCatalog(); - if (!(paimonCatalog.tableExists(targetTableId))) { + try { + paimonCatalog.getTable(targetTableId); + } catch (Catalog.TableNotExistException e) { throw new IllegalArgumentException( "Target paimon table does not exist: " + targetTable); } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedure.java index 4f442fbae361..293e84ca14bd 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedure.java @@ -19,6 +19,7 @@ package org.apache.paimon.spark.procedure; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.operation.LocalOrphanFilesClean; import org.apache.paimon.operation.OrphanFilesClean; import org.apache.paimon.spark.catalog.WithPaimonCatalog; import org.apache.paimon.spark.orphan.SparkOrphanFilesClean; @@ -32,6 +33,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Locale; + import static org.apache.spark.sql.types.DataTypes.BooleanType; import static org.apache.spark.sql.types.DataTypes.IntegerType; import static org.apache.spark.sql.types.DataTypes.LongType; @@ -56,7 +59,8 @@ public class RemoveOrphanFilesProcedure extends BaseProcedure { ProcedureParameter.required("table", StringType), ProcedureParameter.optional("older_than", StringType), ProcedureParameter.optional("dry_run", BooleanType), - ProcedureParameter.optional("parallelism", IntegerType) + ProcedureParameter.optional("parallelism", IntegerType), + ProcedureParameter.optional("mode", StringType) }; private static final StructType OUTPUT_TYPE = @@ -98,18 +102,45 @@ public InternalRow[] call(InternalRow args) { LOG.info("identifier is {}.", identifier); Catalog catalog = ((WithPaimonCatalog) tableCatalog()).paimonCatalog(); - long deletedFiles = - SparkOrphanFilesClean.executeDatabaseOrphanFiles( - catalog, - identifier.getDatabaseName(), - identifier.getTableName(), - OrphanFilesClean.olderThanMillis( - args.isNullAt(1) ? null : args.getString(1)), - OrphanFilesClean.createFileCleaner( - catalog, !args.isNullAt(2) && args.getBoolean(2)), - args.isNullAt(3) ? null : args.getInt(3)); - - return new InternalRow[] {newInternalRow(deletedFiles)}; + String mode = args.isNullAt(4) ? "DISTRIBUTED" : args.getString(4); + + long deletedFiles; + try { + switch (mode.toUpperCase(Locale.ROOT)) { + case "LOCAL": + deletedFiles = + LocalOrphanFilesClean.executeDatabaseOrphanFiles( + catalog, + identifier.getDatabaseName(), + identifier.getTableName(), + OrphanFilesClean.olderThanMillis( + args.isNullAt(1) ? null : args.getString(1)), + OrphanFilesClean.createFileCleaner( + catalog, !args.isNullAt(2) && args.getBoolean(2)), + args.isNullAt(3) ? null : args.getInt(3)); + break; + case "DISTRIBUTED": + deletedFiles = + SparkOrphanFilesClean.executeDatabaseOrphanFiles( + catalog, + identifier.getDatabaseName(), + identifier.getTableName(), + OrphanFilesClean.olderThanMillis( + args.isNullAt(1) ? null : args.getString(1)), + OrphanFilesClean.createFileCleaner( + catalog, !args.isNullAt(2) && args.getBoolean(2)), + args.isNullAt(3) ? null : args.getInt(3)); + break; + default: + throw new IllegalArgumentException( + "Unknown mode: " + + mode + + ". Only 'DISTRIBUTED' and 'LOCAL' are supported."); + } + return new InternalRow[] {newInternalRow(deletedFiles)}; + } catch (Exception e) { + throw new RuntimeException(e); + } } public static ProcedureBuilder builder() { diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RenameTagProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RenameTagProcedure.java new file mode 100644 index 000000000000..73c0dc35374d --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RenameTagProcedure.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.procedure; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +import static org.apache.spark.sql.types.DataTypes.StringType; + +/** + * Rename Tag Procedure. Usage: + * + *
    
    + *  CALL sys.rename_tag(tag => 'tag0', target_tag => 'tag1')
    + * 
    + */ +public class RenameTagProcedure extends BaseProcedure { + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + ProcedureParameter.required("table", StringType), + ProcedureParameter.required("tag", StringType), + ProcedureParameter.required("target_tag", StringType) + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("result", DataTypes.BooleanType, true, Metadata.empty()) + }); + + protected RenameTagProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public StructType outputType() { + return OUTPUT_TYPE; + } + + @Override + public InternalRow[] call(InternalRow args) { + Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); + String tag = args.getString(1); + String targetTag = args.getString(2); + + return modifyPaimonTable( + tableIdent, + table -> { + table.renameTag(tag, targetTag); + InternalRow outputRow = newInternalRow(true); + return new InternalRow[] {outputRow}; + }); + } + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + public RenameTagProcedure doBuild() { + return new RenameTagProcedure(tableCatalog()); + } + }; + } + + @Override + public String description() { + return "RenameTagProcedure"; + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ReplaceTagProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ReplaceTagProcedure.java new file mode 100644 index 000000000000..205fca5ee69e --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ReplaceTagProcedure.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.procedure; + +import org.apache.paimon.table.Table; + +import org.apache.spark.sql.connector.catalog.TableCatalog; + +import java.time.Duration; + +/** A procedure to replace a tag. */ +public class ReplaceTagProcedure extends CreateOrReplaceTagBaseProcedure { + + private ReplaceTagProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + void createOrReplaceTag(Table table, String tagName, Long snapshotId, Duration timeRetained) { + table.replaceTag(tagName, snapshotId, timeRetained); + } + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + public ReplaceTagProcedure doBuild() { + return new ReplaceTagProcedure(tableCatalog()); + } + }; + } + + @Override + public String description() { + return "ReplaceTagProcedure"; + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ResetConsumerProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ResetConsumerProcedure.java index a13227e95dc7..0f7fabd05d13 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ResetConsumerProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ResetConsumerProcedure.java @@ -90,6 +90,7 @@ public InternalRow[] call(InternalRow args) { if (nextSnapshotId == null) { consumerManager.deleteConsumer(consumerId); } else { + fileStoreTable.snapshotManager().snapshot(nextSnapshotId); consumerManager.resetConsumer(consumerId, new Consumer(nextSnapshotId)); } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RollbackToTimestampProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RollbackToTimestampProcedure.java new file mode 100644 index 000000000000..a01f08b3fc7d --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RollbackToTimestampProcedure.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.procedure; + +import org.apache.paimon.Snapshot; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.Preconditions; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +import static org.apache.spark.sql.types.DataTypes.LongType; +import static org.apache.spark.sql.types.DataTypes.StringType; + +/** A procedure to rollback to a timestamp. */ +public class RollbackToTimestampProcedure extends BaseProcedure { + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + ProcedureParameter.required("table", StringType), + // timestamp value + ProcedureParameter.required("timestamp", LongType) + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("result", StringType, true, Metadata.empty()) + }); + + private RollbackToTimestampProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public StructType outputType() { + return OUTPUT_TYPE; + } + + @Override + public InternalRow[] call(InternalRow args) { + Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); + Long timestamp = args.getLong(1); + + return modifyPaimonTable( + tableIdent, + table -> { + FileStoreTable fileStoreTable = (FileStoreTable) table; + Snapshot snapshot = + fileStoreTable.snapshotManager().earlierOrEqualTimeMills(timestamp); + Preconditions.checkNotNull( + snapshot, + String.format("count not find snapshot earlier than %s", timestamp)); + long snapshotId = snapshot.id(); + fileStoreTable.rollbackTo(snapshotId); + InternalRow outputRow = + newInternalRow( + UTF8String.fromString( + String.format( + "Success roll back to snapshot: %s .", + snapshotId))); + return new InternalRow[] {outputRow}; + }); + } + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + public RollbackToTimestampProcedure doBuild() { + return new RollbackToTimestampProcedure(tableCatalog()); + } + }; + } + + @Override + public String description() { + return "RollbackToTimestampProcedure"; + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/TableMigrationUtils.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/TableMigrationUtils.java index 73702bf35c2b..3499a5e409b6 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/TableMigrationUtils.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/TableMigrationUtils.java @@ -24,6 +24,7 @@ import org.apache.paimon.hive.migrate.HiveMigrator; import org.apache.paimon.migrate.Migrator; +import java.util.List; import java.util.Map; /** Migration util to choose importer according to connector. */ @@ -58,4 +59,25 @@ public static Migrator getImporter( throw new UnsupportedOperationException("Unsupported connector " + connector); } } + + public static List getImporters( + String connector, + Catalog catalog, + String sourceDatabase, + Integer parallelism, + Map options) { + switch (connector) { + case "hive": + if (catalog instanceof CachingCatalog) { + catalog = ((CachingCatalog) catalog).wrapped(); + } + if (!(catalog instanceof HiveCatalog)) { + throw new IllegalArgumentException("Only support Hive Catalog"); + } + return HiveMigrator.databaseMigrators( + (HiveCatalog) catalog, sourceDatabase, options, parallelism); + default: + throw new UnsupportedOperationException("Don't support connector " + connector); + } + } } 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 new file mode 100644 index 000000000000..95c8f4b3a9a8 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ColumnPruningAndPushDown.scala @@ -0,0 +1,81 @@ +/* + * 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 + +import org.apache.paimon.predicate.{Predicate, PredicateBuilder} +import org.apache.paimon.spark.schema.PaimonMetadataColumn +import org.apache.paimon.table.Table +import org.apache.paimon.table.source.ReadBuilder +import org.apache.paimon.types.RowType +import org.apache.paimon.utils.Preconditions.checkState + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.connector.read.Scan +import org.apache.spark.sql.types.StructType + +trait ColumnPruningAndPushDown extends Scan with Logging { + def table: Table + def requiredSchema: StructType + def filters: Seq[Predicate] + def pushDownLimit: Option[Int] = None + + lazy val tableRowType: RowType = table.rowType + lazy val tableSchema: StructType = SparkTypeUtils.fromPaimonRowType(tableRowType) + + final def partitionType: StructType = { + SparkTypeUtils.toSparkPartitionType(table) + } + + private[paimon] val (readTableRowType, metadataFields) = { + checkState( + requiredSchema.fields.forall( + field => + tableRowType.containsField(field.name) || + PaimonMetadataColumn.SUPPORTED_METADATA_COLUMNS.contains(field.name))) + val (_requiredTableFields, _metadataFields) = + requiredSchema.fields.partition(field => tableRowType.containsField(field.name)) + val _readTableRowType = + SparkTypeUtils.prunePaimonRowType(StructType(_requiredTableFields), tableRowType) + (_readTableRowType, _metadataFields) + } + + lazy val readBuilder: ReadBuilder = { + val _readBuilder = table.newReadBuilder().withReadType(readTableRowType) + if (filters.nonEmpty) { + val pushedPredicate = PredicateBuilder.and(filters: _*) + _readBuilder.withFilter(pushedPredicate) + } + pushDownLimit.foreach(_readBuilder.withLimit) + _readBuilder + } + + final def metadataColumns: Seq[PaimonMetadataColumn] = { + metadataFields.map(field => PaimonMetadataColumn.get(field.name, partitionType)) + } + + override def readSchema(): StructType = { + val _readSchema = StructType( + SparkTypeUtils.fromPaimonRowType(readTableRowType).fields ++ metadataFields) + if (!_readSchema.equals(requiredSchema)) { + logInfo( + s"Actual readSchema: ${_readSchema} is not equal to spark pushed requiredSchema: $requiredSchema") + } + _readSchema + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScan.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScan.scala index fb494a492ce7..b9d235a9de1d 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScan.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScan.scala @@ -20,14 +20,12 @@ package org.apache.paimon.spark import org.apache.paimon.{stats, CoreOptions} import org.apache.paimon.annotation.VisibleForTesting -import org.apache.paimon.predicate.{Predicate, PredicateBuilder} +import org.apache.paimon.predicate.Predicate import org.apache.paimon.spark.metric.SparkMetricRegistry -import org.apache.paimon.spark.schema.PaimonMetadataColumn import org.apache.paimon.spark.sources.PaimonMicroBatchStream import org.apache.paimon.spark.statistics.StatisticsHelper import org.apache.paimon.table.{DataTable, FileStoreTable, Table} -import org.apache.paimon.table.source.{InnerTableScan, ReadBuilder, Split} -import org.apache.paimon.types.RowType +import org.apache.paimon.table.source.{InnerTableScan, Split} import org.apache.spark.sql.connector.metric.{CustomMetric, CustomTaskMetric} import org.apache.spark.sql.connector.read.{Batch, Scan, Statistics, SupportsReportStatistics} @@ -48,22 +46,9 @@ abstract class PaimonBaseScan( extends Scan with SupportsReportStatistics with ScanHelper + with ColumnPruningAndPushDown with StatisticsHelper { - val tableRowType: RowType = table.rowType - - private lazy val tableSchema = SparkTypeUtils.fromPaimonRowType(tableRowType) - - private[paimon] val (requiredTableFields, metadataFields) = { - val nameToField = tableSchema.map(field => (field.name, field)).toMap - val _tableFields = requiredSchema.flatMap(field => nameToField.get(field.name)) - val _metadataFields = - requiredSchema - .filterNot(field => tableSchema.fieldNames.contains(field.name)) - .filter(field => PaimonMetadataColumn.SUPPORTED_METADATA_COLUMNS.contains(field.name)) - (_tableFields, _metadataFields) - } - protected var runtimeFilters: Array[Filter] = Array.empty protected var inputPartitions: Seq[PaimonInputPartition] = _ @@ -75,25 +60,11 @@ abstract class PaimonBaseScan( private lazy val paimonMetricsRegistry: SparkMetricRegistry = SparkMetricRegistry() lazy val requiredStatsSchema: StructType = { - val fieldNames = requiredTableFields.map(_.name) ++ reservedFilters.flatMap(_.references) + val fieldNames = + readTableRowType.getFields.asScala.map(_.name) ++ reservedFilters.flatMap(_.references) StructType(tableSchema.filter(field => fieldNames.contains(field.name))) } - lazy val readBuilder: ReadBuilder = { - val _readBuilder = table.newReadBuilder() - - val projection = - requiredTableFields.map(field => tableSchema.fieldNames.indexOf(field.name)).toArray - _readBuilder.withProjection(projection) - if (filters.nonEmpty) { - val pushedPredicate = PredicateBuilder.and(filters: _*) - _readBuilder.withFilter(pushedPredicate) - } - pushDownLimit.foreach(_readBuilder.withLimit) - - _readBuilder - } - @VisibleForTesting def getOriginSplits: Array[Split] = { readBuilder @@ -113,17 +84,7 @@ abstract class PaimonBaseScan( inputPartitions } - final def partitionType: StructType = { - SparkTypeUtils.toSparkPartitionType(table) - } - - override def readSchema(): StructType = { - StructType(requiredTableFields ++ metadataFields) - } - override def toBatch: Batch = { - val metadataColumns = - metadataFields.map(field => PaimonMetadataColumn.get(field.name, partitionType)) PaimonBatch(lazyInputPartitions, readBuilder, metadataColumns) } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScanBuilder.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScanBuilder.scala index 0efe14552afe..a265ee78f5b9 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScanBuilder.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonBaseScanBuilder.scala @@ -36,14 +36,16 @@ abstract class PaimonBaseScanBuilder(table: Table) protected var requiredSchema: StructType = SparkTypeUtils.fromPaimonRowType(table.rowType()) - protected var pushed: Array[(Filter, Predicate)] = Array.empty + protected var pushedPredicates: Array[(Filter, Predicate)] = Array.empty - protected var reservedFilters: Array[Filter] = Array.empty + protected var partitionFilters: Array[Filter] = Array.empty + + protected var postScanFilters: Array[Filter] = Array.empty protected var pushDownLimit: Option[Int] = None override def build(): Scan = { - PaimonScan(table, requiredSchema, pushed.map(_._2), reservedFilters, pushDownLimit) + PaimonScan(table, requiredSchema, pushedPredicates.map(_._2), partitionFilters, pushDownLimit) } /** @@ -54,7 +56,7 @@ abstract class PaimonBaseScanBuilder(table: Table) override def pushFilters(filters: Array[Filter]): Array[Filter] = { val pushable = mutable.ArrayBuffer.empty[(Filter, Predicate)] val postScan = mutable.ArrayBuffer.empty[Filter] - val reserved = mutable.ArrayBuffer.empty[Filter] + val partitionFilter = mutable.ArrayBuffer.empty[Filter] val converter = new SparkFilterConverter(table.rowType) val visitor = new PartitionPredicateVisitor(table.partitionKeys()) @@ -66,7 +68,7 @@ abstract class PaimonBaseScanBuilder(table: Table) } else { pushable.append((filter, predicate)) if (predicate.visit(visitor)) { - reserved.append(filter) + partitionFilter.append(filter) } else { postScan.append(filter) } @@ -74,16 +76,19 @@ abstract class PaimonBaseScanBuilder(table: Table) } if (pushable.nonEmpty) { - this.pushed = pushable.toArray + this.pushedPredicates = pushable.toArray + } + if (partitionFilter.nonEmpty) { + this.partitionFilters = partitionFilter.toArray } - if (reserved.nonEmpty) { - this.reservedFilters = reserved.toArray + if (postScan.nonEmpty) { + this.postScanFilters = postScan.toArray } postScan.toArray } override def pushedFilters(): Array[Filter] = { - pushed.map(_._1) + pushedPredicates.map(_._1) } override def pruneColumns(requiredSchema: StructType): Unit = { diff --git a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/PaimonScan.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonLocalScan.scala similarity index 64% rename from paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/PaimonScan.scala rename to paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonLocalScan.scala index 254c63679bca..490a1b133f6f 100644 --- a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/PaimonScan.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonLocalScan.scala @@ -18,17 +18,27 @@ package org.apache.paimon.spark -import org.apache.paimon.predicate.Predicate import org.apache.paimon.table.Table +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.LocalScan import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType -case class PaimonScan( +/** A scan does not require [[RDD]] to execute */ +case class PaimonLocalScan( + rows: Array[InternalRow], + readSchema: StructType, table: Table, - requiredSchema: StructType, - filters: Seq[Predicate], - reservedFilters: Seq[Filter], - pushDownLimit: Option[Int], - disableBucketedScan: Boolean = false) - extends PaimonBaseScan(table, requiredSchema, filters, reservedFilters, pushDownLimit) + filters: Array[Filter]) + extends LocalScan { + + override def description(): String = { + val pushedFiltersStr = if (filters.nonEmpty) { + ", PushedFilters: [" + filters.mkString(",") + "]" + } else { + "" + } + s"PaimonLocalScan: [${table.name}]" + pushedFiltersStr + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala index 113ad4c3b7a8..54970bfe3cb2 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionManagement.scala @@ -18,6 +18,8 @@ package org.apache.paimon.spark +import org.apache.paimon.CoreOptions +import org.apache.paimon.metastore.MetastoreClient import org.apache.paimon.operation.FileStoreCommit import org.apache.paimon.table.FileStoreTable import org.apache.paimon.table.sink.BatchWriteBuilder @@ -41,7 +43,8 @@ trait PaimonPartitionManagement extends SupportsAtomicPartitionManagement { override lazy val partitionSchema: StructType = SparkTypeUtils.fromPaimonRowType(partitionRowType) - override def dropPartitions(internalRows: Array[InternalRow]): Boolean = { + private def toPaimonPartitions( + rows: Array[InternalRow]): Array[java.util.LinkedHashMap[String, String]] = { table match { case fileStoreTable: FileStoreTable => val rowConverter = CatalystTypeConverters @@ -49,19 +52,38 @@ trait PaimonPartitionManagement extends SupportsAtomicPartitionManagement { val rowDataPartitionComputer = new InternalRowPartitionComputer( fileStoreTable.coreOptions().partitionDefaultName(), partitionRowType, - table.partitionKeys().asScala.toArray) + table.partitionKeys().asScala.toArray, + CoreOptions.fromMap(table.options()).legacyPartitionName) - val partitions = internalRows.map { + rows.map { r => rowDataPartitionComputer .generatePartValues(new SparkRow(partitionRowType, rowConverter(r).asInstanceOf[Row])) - .asInstanceOf[JMap[String, String]] } + case _ => + throw new UnsupportedOperationException("Only FileStoreTable supports partitions.") + } + } + + override def dropPartitions(rows: Array[InternalRow]): Boolean = { + table match { + case fileStoreTable: FileStoreTable => + val partitions = toPaimonPartitions(rows).map(_.asInstanceOf[JMap[String, String]]) val commit: FileStoreCommit = fileStoreTable.store.newCommit(UUID.randomUUID.toString) + var metastoreClient: MetastoreClient = null + val clientFactory = fileStoreTable.catalogEnvironment().metastoreClientFactory try { commit.dropPartitions(partitions.toSeq.asJava, BatchWriteBuilder.COMMIT_IDENTIFIER) + // sync to metastore with delete partitions + if (clientFactory != null && fileStoreTable.coreOptions().partitionedTableInMetastore()) { + metastoreClient = clientFactory.create() + toPaimonPartitions(rows).foreach(metastoreClient.deletePartition) + } } finally { commit.close() + if (metastoreClient != null) { + metastoreClient.close() + } } true @@ -112,8 +134,26 @@ trait PaimonPartitionManagement extends SupportsAtomicPartitionManagement { } override def createPartitions( - internalRows: Array[InternalRow], + rows: Array[InternalRow], maps: Array[JMap[String, String]]): Unit = { - throw new UnsupportedOperationException("Create partition is not supported") + table match { + case fileStoreTable: FileStoreTable => + val partitions = toPaimonPartitions(rows) + val metastoreFactory = fileStoreTable.catalogEnvironment().metastoreClientFactory() + if (metastoreFactory == null) { + throw new UnsupportedOperationException( + "The table must have metastore to create partition.") + } + val metastoreClient: MetastoreClient = metastoreFactory.create + try { + if (fileStoreTable.coreOptions().partitionedTableInMetastore()) { + partitions.foreach(metastoreClient.addPartition) + } + } finally { + metastoreClient.close() + } + case _ => + throw new UnsupportedOperationException("Only FileStoreTable supports create partitions.") + } } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScan.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScan.scala index 5f73ae553064..2f1e6c53ab0a 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScan.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScan.scala @@ -36,7 +36,7 @@ case class PaimonScan( requiredSchema: StructType, filters: Seq[Predicate], reservedFilters: Seq[Filter], - pushDownLimit: Option[Int], + override val pushDownLimit: Option[Int], bucketedScanDisabled: Boolean = false) extends PaimonBaseScan(table, requiredSchema, filters, reservedFilters, pushDownLimit) with SupportsRuntimeFiltering diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala index 1ffdd9bde9e2..d8b66e1cd1e0 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonScanBuilder.scala @@ -18,19 +18,70 @@ package org.apache.paimon.spark +import org.apache.paimon.predicate.PredicateBuilder +import org.apache.paimon.spark.aggregate.LocalAggregator import org.apache.paimon.table.Table -import org.apache.spark.sql.connector.read.SupportsPushDownLimit +import org.apache.spark.sql.connector.expressions.aggregate.Aggregation +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownAggregates, SupportsPushDownLimit} + +import scala.collection.JavaConverters._ class PaimonScanBuilder(table: Table) extends PaimonBaseScanBuilder(table) - with SupportsPushDownLimit { + with SupportsPushDownLimit + with SupportsPushDownAggregates { + private var localScan: Option[Scan] = None override def pushLimit(limit: Int): Boolean = { - if (table.primaryKeys().isEmpty) { - pushDownLimit = Some(limit) - } + // It is safe, since we will do nothing if it is the primary table and the split is not `rawConvertible` + pushDownLimit = Some(limit) // just make a best effort to push down limit false } + + override def supportCompletePushDown(aggregation: Aggregation): Boolean = { + // for now we only support complete push down, so there is no difference with `pushAggregation` + pushAggregation(aggregation) + } + + // Spark does not support push down aggregation for streaming scan. + override def pushAggregation(aggregation: Aggregation): Boolean = { + if (localScan.isDefined) { + return true + } + + // Only support with push down partition filter + if (postScanFilters.nonEmpty) { + return false + } + + val aggregator = new LocalAggregator(table) + if (!aggregator.pushAggregation(aggregation)) { + return false + } + + val readBuilder = table.newReadBuilder + if (pushedPredicates.nonEmpty) { + val pushedPartitionPredicate = PredicateBuilder.and(pushedPredicates.map(_._2): _*) + readBuilder.withFilter(pushedPartitionPredicate) + } + val scan = readBuilder.newScan() + scan.listPartitionEntries.asScala.foreach(aggregator.update) + localScan = Some( + PaimonLocalScan( + aggregator.result(), + aggregator.resultSchema(), + table, + pushedPredicates.map(_._1))) + true + } + + override def build(): Scan = { + if (localScan.isDefined) { + localScan.get + } else { + super.build() + } + } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonSplitScan.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonSplitScan.scala index 9f792de4804d..8d9e643f9485 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonSplitScan.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonSplitScan.scala @@ -19,35 +19,50 @@ package org.apache.paimon.spark import org.apache.paimon.CoreOptions -import org.apache.paimon.spark.schema.PaimonMetadataColumn -import org.apache.paimon.table.Table +import org.apache.paimon.predicate.Predicate +import org.apache.paimon.table.{KnownSplitsTable, Table} import org.apache.paimon.table.source.{DataSplit, Split} import org.apache.spark.sql.connector.read.{Batch, Scan} import org.apache.spark.sql.types.StructType +class PaimonSplitScanBuilder(table: KnownSplitsTable) extends PaimonBaseScanBuilder(table) { + override def build(): Scan = { + PaimonSplitScan(table, table.splits(), requiredSchema, pushedPredicates.map(_._2)) + } +} + /** For internal use only. */ case class PaimonSplitScan( table: Table, dataSplits: Array[DataSplit], - metadataColumns: Seq[PaimonMetadataColumn] = Seq.empty) - extends Scan + requiredSchema: StructType, + filters: Seq[Predicate]) + extends ColumnPruningAndPushDown with ScanHelper { override val coreOptions: CoreOptions = CoreOptions.fromMap(table.options()) - override def readSchema(): StructType = SparkTypeUtils.fromPaimonRowType(table.rowType()) - override def toBatch: Batch = { PaimonBatch( getInputPartitions(dataSplits.asInstanceOf[Array[Split]]), - table.newReadBuilder, + readBuilder, metadataColumns) } + + override def description(): String = { + val pushedFiltersStr = if (filters.nonEmpty) { + ", PushedFilters: [" + filters.mkString(",") + "]" + } else { + "" + } + s"PaimonSplitScan: [${table.name}]" + pushedFiltersStr + } } object PaimonSplitScan { def apply(table: Table, dataSplits: Array[DataSplit]): PaimonSplitScan = { - new PaimonSplitScan(table, dataSplits) + val requiredSchema = SparkTypeUtils.fromPaimonRowType(table.rowType) + new PaimonSplitScan(table, dataSplits, requiredSchema, Seq.empty) } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonStatistics.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonStatistics.scala index 963d9fadd297..28af4ac0a4fd 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonStatistics.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonStatistics.scala @@ -18,7 +18,6 @@ package org.apache.paimon.spark -import org.apache.paimon.spark.schema.PaimonMetadataColumn import org.apache.paimon.stats.ColStats import org.apache.paimon.types.{DataField, DataType, RowType} @@ -65,19 +64,13 @@ case class PaimonStatistics[T <: PaimonBaseScan](scan: T) extends Statistics { val wholeSchemaSize = getSizeForRow(scan.tableRowType) - val requiredDataSchemaSize = scan.requiredTableFields.map { - field => - val dataField = scan.tableRowType.getField(field.name) - getSizeForField(dataField) - }.sum + val requiredDataSchemaSize = + scan.readTableRowType.getFields.asScala.map(field => getSizeForField(field)).sum val requiredDataSizeInBytes = paimonStats.mergedRecordSize().getAsLong * (requiredDataSchemaSize.toDouble / wholeSchemaSize) - val metadataSchemaSize = scan.metadataFields.map { - field => - val dataField = PaimonMetadataColumn.get(field.name, scan.partitionType).toPaimonDataField - getSizeForField(dataField) - }.sum + val metadataSchemaSize = + scan.metadataColumns.map(field => getSizeForField(field.toPaimonDataField)).sum val metadataSizeInBytes = paimonStats.mergedRecordCount().getAsLong * metadataSchemaSize val sizeInBytes = (requiredDataSizeInBytes + metadataSizeInBytes).toLong @@ -136,12 +129,12 @@ object PaimonColumnStats { def apply(v1ColStats: ColumnStat): PaimonColumnStats = { import PaimonImplicits._ PaimonColumnStats( - if (v1ColStats.nullCount.isDefined) OptionalLong.of(v1ColStats.nullCount.get.longValue()) + if (v1ColStats.nullCount.isDefined) OptionalLong.of(v1ColStats.nullCount.get.longValue) else OptionalLong.empty(), v1ColStats.min, v1ColStats.max, if (v1ColStats.distinctCount.isDefined) - OptionalLong.of(v1ColStats.distinctCount.get.longValue()) + OptionalLong.of(v1ColStats.distinctCount.get.longValue) else OptionalLong.empty(), if (v1ColStats.avgLen.isDefined) OptionalLong.of(v1ColStats.avgLen.get.longValue()) else OptionalLong.empty(), diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SaveMode.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SaveMode.scala index c4dae5e3ddd8..bcd6f68ab1a1 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SaveMode.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SaveMode.scala @@ -23,15 +23,15 @@ import org.apache.spark.sql.sources.{AlwaysTrue, Filter} sealed private[spark] trait SaveMode extends Serializable -object InsertInto extends SaveMode +case object InsertInto extends SaveMode case class Overwrite(filters: Option[Filter]) extends SaveMode -object DynamicOverWrite extends SaveMode +case object DynamicOverWrite extends SaveMode -object ErrorIfExists extends SaveMode +case object ErrorIfExists extends SaveMode -object Ignore extends SaveMode +case object Ignore extends SaveMode object SaveMode { def transform(saveMode: SparkSaveMode): SaveMode = { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ScanHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ScanHelper.scala index a5d4f10bac33..b5b56ba1d509 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ScanHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ScanHelper.scala @@ -76,7 +76,7 @@ trait ScanHelper extends Logging { def closeDataSplit(): Unit = { if (currentSplit.nonEmpty && currentDataFiles.nonEmpty) { val newSplit = - copyDataSplit(currentSplit.get, currentDataFiles, currentDeletionFiles) + copyDataSplit(currentSplit.get, currentDataFiles.toSeq, currentDeletionFiles.toSeq) currentSplits += newSplit } currentDataFiles.clear() diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala index 8ea2c31bc8f6..0170a29f68d3 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala @@ -18,11 +18,13 @@ package org.apache.paimon.spark -import org.apache.paimon.catalog.CatalogContext +import org.apache.paimon.CoreOptions +import org.apache.paimon.catalog.{CatalogContext, CatalogUtils, Identifier} import org.apache.paimon.options.Options +import org.apache.paimon.spark.SparkSource.NAME import org.apache.paimon.spark.commands.WriteIntoPaimonTable import org.apache.paimon.spark.sources.PaimonSink -import org.apache.paimon.spark.util.OptionUtils.mergeSQLConf +import org.apache.paimon.spark.util.OptionUtils.{extractCatalogName, mergeSQLConfWithIdentifier} import org.apache.paimon.table.{DataTable, FileStoreTable, FileStoreTableFactory} import org.apache.paimon.table.system.AuditLogTable @@ -80,9 +82,15 @@ class SparkSource } private def loadTable(options: JMap[String, String]): DataTable = { + val path = CoreOptions.path(options) val catalogContext = CatalogContext.create( - Options.fromMap(mergeSQLConf(options)), - SparkSession.active.sessionState.newHadoopConf()) + Options.fromMap( + mergeSQLConfWithIdentifier( + options, + extractCatalogName().getOrElse(NAME), + Identifier.create(CatalogUtils.database(path), CatalogUtils.table(path)))), + SparkSession.active.sessionState.newHadoopConf() + ) val table = FileStoreTableFactory.create(catalogContext) if (Options.fromMap(options).get(SparkConnectorOptions.READ_CHANGELOG)) { new AuditLogTable(table) @@ -110,6 +118,8 @@ object SparkSource { val NAME = "paimon" + val FORMAT_NAMES = Seq("csv", "orc", "parquet") + def toBaseRelation(table: FileStoreTable, _sqlContext: SQLContext): BaseRelation = { new BaseRelation { override def sqlContext: SQLContext = _sqlContext diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala index d06c125d24f4..39b1947e4f37 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala @@ -21,7 +21,7 @@ package org.apache.paimon.spark import org.apache.paimon.CoreOptions import org.apache.paimon.options.Options import org.apache.paimon.spark.schema.PaimonMetadataColumn -import org.apache.paimon.table.{DataTable, FileStoreTable, Table} +import org.apache.paimon.table.{DataTable, FileStoreTable, KnownSplitsTable, Table} import org.apache.paimon.utils.StringUtils import org.apache.spark.sql.connector.catalog.{MetadataColumn, SupportsMetadataColumns, SupportsRead, SupportsWrite, TableCapability, TableCatalog} @@ -91,7 +91,12 @@ case class SparkTable(table: Table) } override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { - new PaimonScanBuilder(table.copy(options.asCaseSensitiveMap)) + table match { + case t: KnownSplitsTable => + new PaimonSplitScanBuilder(t) + case _ => + new PaimonScanBuilder(table.copy(options.asCaseSensitiveMap)) + } } override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { @@ -102,4 +107,8 @@ case class SparkTable(table: Table) throw new RuntimeException("Only FileStoreTable can be written.") } } + + override def toString: String = { + s"${table.getClass.getSimpleName}[${table.fullName()}]" + } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkWrite.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkWrite.scala index 7e7919592eb7..fd43decfc6ad 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkWrite.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkWrite.scala @@ -35,4 +35,8 @@ class SparkWrite(val table: FileStoreTable, saveMode: SaveMode, options: Options WriteIntoPaimonTable(table, saveMode, data, options).run(data.sparkSession) } } + + override def toString: String = { + s"table: ${table.fullName()}, saveMode: $saveMode, options: ${options.toMap}" + } } 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 new file mode 100644 index 000000000000..cd9718cf44eb --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala @@ -0,0 +1,167 @@ +/* + * 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.aggregate + +import org.apache.paimon.data.BinaryRow +import org.apache.paimon.manifest.PartitionEntry +import org.apache.paimon.spark.{SparkInternalRow, SparkTypeUtils} +import org.apache.paimon.table.{DataTable, Table} +import org.apache.paimon.utils.{InternalRowUtils, ProjectedRow} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.sql.connector.expressions.{Expression, NamedReference} +import org.apache.spark.sql.connector.expressions.aggregate.{AggregateFunc, Aggregation, CountStar} +import org.apache.spark.sql.types.{DataType, LongType, StructField, StructType} + +import scala.collection.mutable + +class LocalAggregator(table: Table) { + 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 aggFuncEvaluatorGetter: () => Seq[AggFuncEvaluator[_]] = _ + private var isInitialized = false + + private def initialize(aggregation: Aggregation): Unit = { + aggFuncEvaluatorGetter = () => + aggregation.aggregateExpressions().map { + case _: CountStar => new CountStarEvaluator() + case _ => throw new UnsupportedOperationException() + } + + requiredGroupByType = aggregation.groupByExpressions().map { + case r: NamedReference => + SparkTypeUtils.fromPaimonType(partitionType.getField(r.fieldNames().head).`type`()) + } + + requiredGroupByIndexMapping = aggregation.groupByExpressions().map { + case r: NamedReference => + partitionType.getFieldIndex(r.fieldNames().head) + } + + isInitialized = true + } + + private def supportAggregateFunction(func: AggregateFunc): Boolean = { + func match { + case _: CountStar => true + case _ => false + } + } + + private def supportGroupByExpressions(exprs: Array[Expression]): Boolean = { + // Support empty group by keys or group by partition column + exprs.forall { + case r: NamedReference => + r.fieldNames.length == 1 && table.partitionKeys().contains(r.fieldNames().head) + case _ => false + } + } + + def pushAggregation(aggregation: Aggregation): Boolean = { + if ( + !table.isInstanceOf[DataTable] || + !table.primaryKeys.isEmpty + ) { + return false + } + if (table.asInstanceOf[DataTable].coreOptions.deletionVectorsEnabled) { + return false + } + + if ( + !supportGroupByExpressions(aggregation.groupByExpressions()) || + aggregation.aggregateExpressions().isEmpty || + aggregation.aggregateExpressions().exists(!supportAggregateFunction(_)) + ) { + return false + } + + initialize(aggregation) + 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) + new SparkInternalRow(partitionType).replace(genericRow) + } + + def update(partitionEntry: PartitionEntry): Unit = { + assert(isInitialized) + val groupByRow = requiredGroupByRow(partitionEntry.partition()) + val aggFuncEvaluator = + groupByEvaluatorMap.getOrElseUpdate(groupByRow, aggFuncEvaluatorGetter()) + aggFuncEvaluator.foreach(_.update(partitionEntry)) + } + + def result(): Array[InternalRow] = { + assert(isInitialized) + if (groupByEvaluatorMap.isEmpty && requiredGroupByType.isEmpty) { + // Always return one row for global aggregate + Array(InternalRow.fromSeq(aggFuncEvaluatorGetter().map(_.result()))) + } else { + groupByEvaluatorMap.map { + case (partitionRow, aggFuncEvaluator) => + new JoinedRow(partitionRow, InternalRow.fromSeq(aggFuncEvaluator.map(_.result()))) + }.toArray + } + } + + def resultSchema(): StructType = { + assert(isInitialized) + // Always put the group by keys before the aggregate function result + val groupByFields = requiredGroupByType.zipWithIndex.map { + case (dt, i) => + StructField(s"groupby_$i", dt) + } + val aggResultFields = aggFuncEvaluatorGetter().zipWithIndex.map { + case (evaluator, i) => + // Note that, Spark will re-assign the attribute name to original name, + // so here we just return an arbitrary name + StructField(s"${evaluator.prettyName}_$i", evaluator.resultType) + } + StructType.apply(groupByFields ++ aggResultFields) + } +} + +trait AggFuncEvaluator[T] { + def update(partitionEntry: PartitionEntry): Unit + def result(): T + def resultType: DataType + def prettyName: String +} + +class CountStarEvaluator extends AggFuncEvaluator[Long] { + private var _result: Long = 0L + + override def update(partitionEntry: PartitionEntry): Unit = { + _result += partitionEntry.recordCount() + } + + override def result(): Long = _result + + override def resultType: DataType = LongType + + override def prettyName: String = "count_star" +} 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 98d3c03aacbb..f567d925ea57 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 @@ -148,7 +148,8 @@ class PaimonAnalysis(session: SparkSession) extends Rule[LogicalPlan] { case (s1: StructType, s2: StructType) => s1.zip(s2).forall { case (d1, d2) => schemaCompatible(d1.dataType, d2.dataType) } case (a1: ArrayType, a2: ArrayType) => - a1.containsNull == a2.containsNull && schemaCompatible(a1.elementType, a2.elementType) + // todo: support array type nullable evaluation + schemaCompatible(a1.elementType, a2.elementType) case (m1: MapType, m2: MapType) => m1.valueContainsNull == m2.valueContainsNull && schemaCompatible(m1.keyType, m2.keyType) && diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoBase.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoBase.scala index ba6108395a7c..f2530b50c04c 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoBase.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoBase.scala @@ -53,7 +53,7 @@ trait PaimonMergeIntoBase merge.notMatchedActions.flatMap(_.condition).foreach(checkCondition) val updateActions = merge.matchedActions.collect { case a: UpdateAction => a } - val primaryKeys = v2Table.getTable.primaryKeys().asScala + val primaryKeys = v2Table.getTable.primaryKeys().asScala.toSeq if (primaryKeys.nonEmpty) { checkUpdateActionValidity( AttributeSet(targetOutput), diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala index bb66e38266dc..4525393bd2de 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala @@ -27,36 +27,20 @@ object PaimonMergeIntoResolver extends PaimonMergeIntoResolverBase { def resolveNotMatchedBySourceActions( merge: MergeIntoTable, - target: LogicalPlan, - source: LogicalPlan, resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] = { - val fakeSource = Project(source.output, source) - - def resolveMergeAction(action: MergeAction): MergeAction = { - action match { - case DeleteAction(condition) => - val resolvedCond = condition.map(resolve(_, target)) - DeleteAction(resolvedCond) - case UpdateAction(condition, assignments) => - val resolvedCond = condition.map(resolve(_, target)) - val resolvedAssignments = assignments.map { - assignment => - assignment.copy( - key = resolve(assignment.key, target), - value = resolve(assignment.value, target)) - } - UpdateAction(resolvedCond, resolvedAssignments) - case UpdateStarAction(condition) => - val resolvedCond = condition.map(resolve(_, target)) - val resolvedAssignments = target.output.map { - attr => - Assignment(attr, resolve(UnresolvedAttribute.quotedString(attr.name), fakeSource)) - } - UpdateAction(resolvedCond, resolvedAssignments) - } + merge.notMatchedBySourceActions.map { + case DeleteAction(condition) => + // The condition must be from the target table + val resolvedCond = condition.map(resolveCondition(resolve, _, merge, TARGET_ONLY)) + DeleteAction(resolvedCond) + case UpdateAction(condition, assignments) => + // The condition and value must be from the target table + val resolvedCond = condition.map(resolveCondition(resolve, _, merge, TARGET_ONLY)) + val resolvedAssignments = resolveAssignments(resolve, assignments, merge, TARGET_ONLY) + UpdateAction(resolvedCond, resolvedAssignments) + case action => + throw new RuntimeException(s"Can't recognize this action: $action") } - - merge.notMatchedBySourceActions.map(resolveMergeAction) } def build( diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala index cbd6b52c01af..218fc9c0f3ef 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala @@ -23,7 +23,7 @@ import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, LogicalPlan, MergeAction, MergeIntoTable, Project, UpdateAction, UpdateStarAction} +import org.apache.spark.sql.catalyst.plans.logical._ trait PaimonMergeIntoResolverBase extends ExpressionHelper { @@ -33,73 +33,111 @@ trait PaimonMergeIntoResolverBase extends ExpressionHelper { assert(target.resolved, "Target should have been resolved here.") assert(source.resolved, "Source should have been resolved here.") - val condition = merge.mergeCondition - val matched = merge.matchedActions - val notMatched = merge.notMatchedActions - val fakeSource = Project(source.output, source) - val resolve: (Expression, LogicalPlan) => Expression = resolveExpression(spark) - def resolveMergeAction(action: MergeAction): MergeAction = { - action match { - case DeleteAction(condition) => - val resolvedCond = condition.map(resolve(_, merge)) - DeleteAction(resolvedCond) - case UpdateAction(condition, assignments) => - val resolvedCond = condition.map(resolve(_, merge)) - val resolvedAssignments = assignments.map { - assignment => - assignment.copy( - key = resolve(assignment.key, target), - value = resolve(assignment.value, merge)) - } - UpdateAction(resolvedCond, resolvedAssignments) - case UpdateStarAction(condition) => - val resolvedCond = condition.map(resolve(_, merge)) - val resolvedAssignments = target.output.map { - attr => Assignment(attr, resolve(UnresolvedAttribute.quotedString(attr.name), source)) - } - UpdateAction(resolvedCond, resolvedAssignments) - case InsertAction(condition, assignments) => - val resolvedCond = condition.map(resolve(_, fakeSource)) - val resolvedAssignments = assignments.map { - assignment => - assignment.copy( - key = resolve(assignment.key, fakeSource), - value = resolve(assignment.value, fakeSource)) - } - InsertAction(resolvedCond, resolvedAssignments) - case InsertStarAction(condition) => - val resolvedCond = condition.map(resolve(_, fakeSource)) - val resolvedAssignments = target.output.map { - attr => - Assignment(attr, resolve(UnresolvedAttribute.quotedString(attr.name), fakeSource)) - } - InsertAction(resolvedCond, resolvedAssignments) - case _ => - throw new RuntimeException(s"Can't recognize this action: $action") - } - } - - val resolvedCond = resolve(condition, merge) - val resolvedMatched: Seq[MergeAction] = matched.map(resolveMergeAction) - val resolvedNotMatched: Seq[MergeAction] = notMatched.map(resolveMergeAction) - val resolvedNotMatchedBySource: Seq[MergeAction] = - resolveNotMatchedBySourceActions(merge, target, source, resolve) + val resolvedCond = resolveCondition(resolve, merge.mergeCondition, merge, ALL) + val resolvedMatched = resolveMatchedByTargetActions(merge, resolve) + val resolvedNotMatched = resolveNotMatchedByTargetActions(merge, resolve) + val resolvedNotMatchedBySource = resolveNotMatchedBySourceActions(merge, resolve) build(merge, resolvedCond, resolvedMatched, resolvedNotMatched, resolvedNotMatchedBySource) } - def resolveNotMatchedBySourceActions( - merge: MergeIntoTable, - target: LogicalPlan, - source: LogicalPlan, - resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] - def build( merge: MergeIntoTable, resolvedCond: Expression, resolvedMatched: Seq[MergeAction], resolvedNotMatched: Seq[MergeAction], resolvedNotMatchedBySource: Seq[MergeAction]): MergeIntoTable + + private def resolveMatchedByTargetActions( + merge: MergeIntoTable, + resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] = { + merge.matchedActions.map { + case DeleteAction(condition) => + // The condition can be from both target and source tables + val resolvedCond = condition.map(resolveCondition(resolve, _, merge, ALL)) + DeleteAction(resolvedCond) + case UpdateAction(condition, assignments) => + // The condition and value can be from both target and source tables + val resolvedCond = condition.map(resolveCondition(resolve, _, merge, ALL)) + val resolvedAssignments = resolveAssignments(resolve, assignments, merge, ALL) + UpdateAction(resolvedCond, resolvedAssignments) + case UpdateStarAction(condition) => + // The condition can be from both target and source tables, but the value must be from the source table + val resolvedCond = condition.map(resolveCondition(resolve, _, merge, ALL)) + val assignments = merge.targetTable.output.map { + attr => Assignment(attr, UnresolvedAttribute(Seq(attr.name))) + } + val resolvedAssignments = + resolveAssignments(resolve, assignments, merge, SOURCE_ONLY) + UpdateAction(resolvedCond, resolvedAssignments) + case action => + throw new RuntimeException(s"Can't recognize this action: $action") + } + } + + private def resolveNotMatchedByTargetActions( + merge: MergeIntoTable, + resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] = { + merge.notMatchedActions.map { + case InsertAction(condition, assignments) => + // The condition and value must be from the source table + val resolvedCond = + condition.map(resolveCondition(resolve, _, merge, SOURCE_ONLY)) + val resolvedAssignments = + resolveAssignments(resolve, assignments, merge, SOURCE_ONLY) + InsertAction(resolvedCond, resolvedAssignments) + case InsertStarAction(condition) => + // The condition and value must be from the source table + val resolvedCond = + condition.map(resolveCondition(resolve, _, merge, SOURCE_ONLY)) + val assignments = merge.targetTable.output.map { + attr => Assignment(attr, UnresolvedAttribute(Seq(attr.name))) + } + val resolvedAssignments = + resolveAssignments(resolve, assignments, merge, SOURCE_ONLY) + InsertAction(resolvedCond, resolvedAssignments) + case action => + throw new RuntimeException(s"Can't recognize this action: $action") + } + } + + def resolveNotMatchedBySourceActions( + merge: MergeIntoTable, + resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] + + sealed trait ResolvedWith + case object ALL extends ResolvedWith + case object SOURCE_ONLY extends ResolvedWith + case object TARGET_ONLY extends ResolvedWith + + def resolveCondition( + resolve: (Expression, LogicalPlan) => Expression, + condition: Expression, + mergeInto: MergeIntoTable, + resolvedWith: ResolvedWith): Expression = { + resolvedWith match { + case ALL => resolve(condition, mergeInto) + case SOURCE_ONLY => resolve(condition, Project(Nil, mergeInto.sourceTable)) + case TARGET_ONLY => resolve(condition, Project(Nil, mergeInto.targetTable)) + } + } + + def resolveAssignments( + resolve: (Expression, LogicalPlan) => Expression, + assignments: Seq[Assignment], + mergeInto: MergeIntoTable, + resolvedWith: ResolvedWith): Seq[Assignment] = { + assignments.map { + assign => + val resolvedKey = resolve(assign.key, Project(Nil, mergeInto.targetTable)) + val resolvedValue = resolvedWith match { + case ALL => resolve(assign.value, mergeInto) + case SOURCE_ONLY => resolve(assign.value, Project(Nil, mergeInto.sourceTable)) + case TARGET_ONLY => resolve(assign.value, Project(Nil, mergeInto.targetTable)) + } + Assignment(resolvedKey, resolvedValue) + } + } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonUpdateTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonUpdateTable.scala index 123c67a2fc20..ad3912ddb70d 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonUpdateTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonUpdateTable.scala @@ -41,7 +41,7 @@ object PaimonUpdateTable table.getTable match { case paimonTable: FileStoreTable => - val primaryKeys = paimonTable.primaryKeys().asScala + val primaryKeys = paimonTable.primaryKeys().asScala.toSeq if (!validUpdateAssignment(u.table.outputSet, primaryKeys, assignments)) { throw new RuntimeException("Can't update the primary key column.") } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/EvalSubqueriesForDeleteTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/EvalSubqueriesForDeleteTable.scala index 5d264370adcd..4cf9284f97f6 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/EvalSubqueriesForDeleteTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/EvalSubqueriesForDeleteTable.scala @@ -49,7 +49,10 @@ object EvalSubqueriesForDeleteTable extends Rule[LogicalPlan] with ExpressionHel plan.transformDown { case d @ DeleteFromPaimonTableCommand(_, table, condition) if SubqueryExpression.hasSubquery(condition) && - isPredicatePartitionColumnsOnly(condition, table.partitionKeys().asScala, resolver) => + isPredicatePartitionColumnsOnly( + condition, + table.partitionKeys().asScala.toSeq, + resolver) => try { d.copy(condition = evalSubquery(condition)) } catch { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala index eca8c9cdfced..b0b1a76e7a1f 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.paimon.shims import org.apache.spark.sql.types.{DataType, StructType} import scala.collection.mutable.ArrayBuffer @@ -95,7 +96,7 @@ trait MergePaimonScalarSubqueriesBase extends Rule[LogicalPlan] with PredicateHe val newPlan = removeReferences(planWithReferences, cache) val subqueryCTEs = cache.filter(_.merged).map(_.plan.asInstanceOf[CTERelationDef]) if (subqueryCTEs.nonEmpty) { - WithCTE(newPlan, subqueryCTEs) + WithCTE(newPlan, subqueryCTEs.toSeq) } else { newPlan } @@ -335,22 +336,24 @@ trait MergePaimonScalarSubqueriesBase extends Rule[LogicalPlan] with PredicateHe // Only allow aggregates of the same implementation because merging different implementations // could cause performance regression. private def supportedAggregateMerge(newPlan: Aggregate, cachedPlan: Aggregate) = { - val newPlanAggregateExpressions = newPlan.aggregateExpressions.flatMap(_.collect { - case a: AggregateExpression => a - }) - val cachedPlanAggregateExpressions = cachedPlan.aggregateExpressions.flatMap(_.collect { - case a: AggregateExpression => a - }) - val newPlanSupportsHashAggregate = Aggregate.supportsHashAggregate( - newPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)) - val cachedPlanSupportsHashAggregate = Aggregate.supportsHashAggregate( - cachedPlanAggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes)) + val aggregateExpressionsSeq = Seq(newPlan, cachedPlan).map { + plan => plan.aggregateExpressions.flatMap(_.collect { case a: AggregateExpression => a }) + } + val groupByExpressionSeq = Seq(newPlan, cachedPlan).map(_.groupingExpressions) + + val Seq(newPlanSupportsHashAggregate, cachedPlanSupportsHashAggregate) = + aggregateExpressionsSeq.zip(groupByExpressionSeq).map { + case (aggregateExpressions, groupByExpressions) => + shims.Aggregate.supportsHashAggregate( + aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes), + groupByExpressions) + } + newPlanSupportsHashAggregate && cachedPlanSupportsHashAggregate || newPlanSupportsHashAggregate == cachedPlanSupportsHashAggregate && { - val newPlanSupportsObjectHashAggregate = - Aggregate.supportsObjectHashAggregate(newPlanAggregateExpressions) - val cachedPlanSupportsObjectHashAggregate = - Aggregate.supportsObjectHashAggregate(cachedPlanAggregateExpressions) + val Seq(newPlanSupportsObjectHashAggregate, cachedPlanSupportsObjectHashAggregate) = + aggregateExpressionsSeq.map( + aggregateExpressions => Aggregate.supportsObjectHashAggregate(aggregateExpressions)) newPlanSupportsObjectHashAggregate && cachedPlanSupportsObjectHashAggregate || newPlanSupportsObjectHashAggregate == cachedPlanSupportsObjectHashAggregate } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/CreateOrReplaceTagCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/CreateOrReplaceTagCommand.scala new file mode 100644 index 000000000000..0830fc9ed3d6 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/CreateOrReplaceTagCommand.scala @@ -0,0 +1,39 @@ +/* + * 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.catalyst.plans.logical + +import org.apache.paimon.spark.leafnode.PaimonLeafCommand + +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class CreateOrReplaceTagCommand( + table: Seq[String], + tagName: String, + tagOptions: TagOptions, + create: Boolean, + replace: Boolean, + ifNotExists: Boolean) + extends PaimonLeafCommand { + + override def output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"Create tag: $tagName for table: $table" + } +} diff --git a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/statistics/StatisticsHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/DeleteTagCommand.scala similarity index 65% rename from paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/statistics/StatisticsHelper.scala rename to paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/DeleteTagCommand.scala index e64785ddee12..072ed6b09f39 100644 --- a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/statistics/StatisticsHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/DeleteTagCommand.scala @@ -16,15 +16,18 @@ * limitations under the License. */ -package org.apache.paimon.spark.statistics +package org.apache.paimon.spark.catalyst.plans.logical + +import org.apache.paimon.spark.leafnode.PaimonLeafCommand import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.connector.read.Statistics -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -trait StatisticsHelper extends StatisticsHelperBase { - protected def toV1Stats(v2Stats: Statistics, attrs: Seq[Attribute]): logical.Statistics = { - DataSourceV2Relation.transformV2Stats(v2Stats, None, conf.defaultSizeInBytes) +case class DeleteTagCommand(table: Seq[String], tagStr: String, ifExists: Boolean) + extends PaimonLeafCommand { + + override def output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"Delete tag: $tagStr for table: $table" } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/RenameTagCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/RenameTagCommand.scala new file mode 100644 index 000000000000..df68c40382e7 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/RenameTagCommand.scala @@ -0,0 +1,33 @@ +/* + * 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.catalyst.plans.logical + +import org.apache.paimon.spark.leafnode.PaimonLeafCommand + +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} + +case class RenameTagCommand(table: Seq[String], sourceTag: String, targetTag: String) + extends PaimonLeafCommand { + + override def output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"Rename tag from $sourceTag to $targetTag for table: $table" + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/ShowTagsCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/ShowTagsCommand.scala new file mode 100644 index 000000000000..f5b62d333861 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/ShowTagsCommand.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.catalyst.plans.logical + +import org.apache.paimon.spark.leafnode.PaimonLeafCommand + +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.types.StringType + +case class ShowTagsCommand(table: Seq[String]) extends PaimonLeafCommand { + + override def output: Seq[Attribute] = + Seq(AttributeReference("tag", StringType, nullable = false)()) + + override def simpleString(maxFields: Int): String = { + s"Show Tags for table: $table" + } +} diff --git a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/PaimonInputPartition.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/TagOptions.scala similarity index 80% rename from paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/PaimonInputPartition.scala rename to paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/TagOptions.scala index 49bc71e937de..242e9dac15a6 100644 --- a/paimon-spark/paimon-spark-3.1/src/main/scala/org/apache/paimon/spark/PaimonInputPartition.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/TagOptions.scala @@ -16,10 +16,8 @@ * limitations under the License. */ -package org.apache.paimon.spark +package org.apache.paimon.spark.catalyst.plans.logical -import org.apache.paimon.table.source.Split +import java.time.Duration -// never be used -case class PaimonBucketedInputPartition(splits: Seq[Split], bucket: Int) - extends PaimonInputPartition +case class TagOptions(snapshotId: Option[Long], timeRetained: Option[Duration]) 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 2b3888911226..097823d730ce 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 @@ -47,8 +47,7 @@ case class DeleteFromPaimonTableCommand( extends PaimonLeafRunnableCommand with PaimonCommand with ExpressionHelper - with SupportsSubquery - with SQLHelper { + with SupportsSubquery { private lazy val writer = PaimonSparkWriter(table) @@ -60,7 +59,7 @@ case class DeleteFromPaimonTableCommand( } else { val (partitionCondition, otherCondition) = splitPruePartitionAndOtherPredicates( condition, - table.partitionKeys().asScala, + table.partitionKeys().asScala.toSeq, sparkSession.sessionState.conf.resolver) val partitionPredicate = if (partitionCondition.isEmpty) { @@ -83,7 +82,8 @@ case class DeleteFromPaimonTableCommand( val rowDataPartitionComputer = new InternalRowPartitionComputer( table.coreOptions().partitionDefaultName(), table.schema().logicalPartitionType(), - table.partitionKeys.asScala.toArray + table.partitionKeys.asScala.toArray, + table.coreOptions().legacyPartitionName() ) val dropPartitions = matchedPartitions.map { partition => rowDataPartitionComputer.generatePartValues(partition).asScala.asJava 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 f4185806777a..f557a0cf38ee 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 @@ -28,16 +28,17 @@ import org.apache.paimon.table.FileStoreTable import org.apache.paimon.table.sink.CommitMessage import org.apache.paimon.types.RowKind -import org.apache.spark.sql.{Column, Dataset, Row, SparkSession} +import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.PaimonUtils._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, BasePredicate, EqualTo, Expression, Literal, Or, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, BasePredicate, Expression, Literal, UnsafeProjection} import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.functions.{col, lit, monotonically_increasing_id, sum} +import org.apache.spark.sql.paimon.shims.ExpressionUtils.{column, convertToExpression} import org.apache.spark.sql.types.{ByteType, StructField, StructType} import scala.collection.mutable @@ -152,12 +153,12 @@ case class MergeIntoPaimonTable( } if (hasUpdate(matchedActions)) { touchedFilePathsSet ++= findTouchedFiles( - targetDS.join(sourceDS, new Column(mergeCondition), "inner"), + targetDS.join(sourceDS, column(mergeCondition), "inner"), sparkSession) } if (hasUpdate(notMatchedBySourceActions)) { touchedFilePathsSet ++= findTouchedFiles( - targetDS.join(sourceDS, new Column(mergeCondition), "left_anti"), + targetDS.join(sourceDS, column(mergeCondition), "left_anti"), sparkSession) } @@ -199,7 +200,7 @@ case class MergeIntoPaimonTable( val sourceDS = createDataset(sparkSession, sourceTable) .withColumn(SOURCE_ROW_COL, lit(true)) - val joinedDS = sourceDS.join(targetDS, new Column(mergeCondition), "fullOuter") + val joinedDS = sourceDS.join(targetDS, column(mergeCondition), "fullOuter") val joinedPlan = joinedDS.queryExecution.analyzed def resolveOnJoinedPlan(exprs: Seq[Expression]): Seq[Expression] = { @@ -207,8 +208,10 @@ case class MergeIntoPaimonTable( } val targetOutput = filteredTargetPlan.output - val targetRowNotMatched = resolveOnJoinedPlan(Seq(col(SOURCE_ROW_COL).isNull.expr)).head - val sourceRowNotMatched = resolveOnJoinedPlan(Seq(col(TARGET_ROW_COL).isNull.expr)).head + val targetRowNotMatched = resolveOnJoinedPlan( + Seq(convertToExpression(sparkSession, col(SOURCE_ROW_COL).isNull))).head + val sourceRowNotMatched = resolveOnJoinedPlan( + Seq(convertToExpression(sparkSession, col(TARGET_ROW_COL).isNull))).head val matchedExprs = matchedActions.map(_.condition.getOrElse(TrueLiteral)) val notMatchedExprs = notMatchedActions.map(_.condition.getOrElse(TrueLiteral)) val notMatchedBySourceExprs = notMatchedBySourceActions.map(_.condition.getOrElse(TrueLiteral)) @@ -243,7 +246,7 @@ case class MergeIntoPaimonTable( val outputFields = mutable.ArrayBuffer(tableSchema.fields: _*) outputFields += StructField(ROW_KIND_COL, ByteType) outputFields ++= metadataCols.map(_.toStructField) - val outputSchema = StructType(outputFields) + val outputSchema = StructType(outputFields.toSeq) val joinedRowEncoder = EncoderUtils.encode(joinedPlan.schema) val outputEncoder = EncoderUtils.encode(outputSchema).resolveAndBind() @@ -272,7 +275,7 @@ case class MergeIntoPaimonTable( .withColumn(ROW_ID_COL, monotonically_increasing_id()) val sourceDS = createDataset(sparkSession, sourceTable) val count = sourceDS - .join(targetDS, new Column(mergeCondition), "inner") + .join(targetDS, column(mergeCondition), "inner") .select(col(ROW_ID_COL), lit(1).as("one")) .groupBy(ROW_ID_COL) .agg(sum("one").as("count")) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonAnalyzeTableColumnCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonAnalyzeTableColumnCommand.scala index b13e5add01d3..19f73cb6cc68 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonAnalyzeTableColumnCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonAnalyzeTableColumnCommand.scala @@ -25,8 +25,8 @@ import org.apache.paimon.stats.{ColStats, Statistics} import org.apache.paimon.table.FileStoreTable import org.apache.paimon.table.sink.BatchWriteBuilder import org.apache.paimon.table.source.DataSplit +import org.apache.paimon.utils.Preconditions.checkState -import org.apache.parquet.Preconditions import org.apache.spark.sql.{PaimonStatsUtils, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.ColumnStat @@ -74,9 +74,7 @@ case class PaimonAnalyzeTableColumnCommand( PaimonStatsUtils.computeColumnStats(sparkSession, relation, attributes) val totalRecordCount = currentSnapshot.totalRecordCount() - Preconditions.checkState( - totalRecordCount >= mergedRecordCount, - s"totalRecordCount: $totalRecordCount should be greater or equal than mergedRecordCount: $mergedRecordCount.") + checkState(totalRecordCount >= mergedRecordCount) val mergedRecordSize = totalSize * (mergedRecordCount.toDouble / totalRecordCount).toLong // convert to paimon stats @@ -97,6 +95,7 @@ case class PaimonAnalyzeTableColumnCommand( // commit stats val commit = table.store.newCommit(UUID.randomUUID.toString) commit.commitStatistics(stats, BatchWriteBuilder.COMMIT_IDENTIFIER) + commit.close() Seq.empty[Row] } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala index e8caea3cdd34..191d7a766b71 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala @@ -22,12 +22,12 @@ import org.apache.paimon.deletionvectors.BitmapDeletionVector 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.spark.{PaimonSplitScan, SparkFilterConverter} -import org.apache.paimon.spark.catalyst.Compatibility +import org.apache.paimon.spark.{SparkFilterConverter, SparkTable} import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper import org.apache.paimon.spark.commands.SparkDataFileMeta.convertToSparkDataFileMeta import org.apache.paimon.spark.schema.PaimonMetadataColumn import org.apache.paimon.spark.schema.PaimonMetadataColumn._ +import org.apache.paimon.table.{FileStoreTable, KnownSplitsTable} import org.apache.paimon.table.sink.{CommitMessage, CommitMessageImpl} import org.apache.paimon.table.source.DataSplit import org.apache.paimon.types.RowType @@ -35,10 +35,11 @@ import org.apache.paimon.utils.SerializationUtils import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.PaimonUtils.createDataset +import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral -import org.apache.spark.sql.catalyst.plans.logical.{Filter => FilterLogicalNode, LogicalPlan} -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} +import org.apache.spark.sql.catalyst.plans.logical.{Filter => FilterLogicalNode, LogicalPlan, Project} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.sources.{AlwaysTrue, And, EqualNullSafe, EqualTo, Filter} import java.net.URI @@ -47,7 +48,7 @@ import java.util.Collections import scala.collection.JavaConverters._ /** Helper trait for all paimon commands. */ -trait PaimonCommand extends WithFileStoreTable with ExpressionHelper { +trait PaimonCommand extends WithFileStoreTable with ExpressionHelper with SQLConfHelper { /** * For the 'INSERT OVERWRITE' semantics of SQL, Spark DataSourceV2 will call the `truncate` @@ -100,7 +101,7 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper { filter.foreach(snapshotReader.withFilter) } - snapshotReader.read().splits().asScala.collect { case s: DataSplit => s } + snapshotReader.read().splits().asScala.collect { case s: DataSplit => s }.toSeq } protected def findTouchedFiles( @@ -115,8 +116,7 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper { } } - val metadataCols = Seq(FILE_PATH) - val filteredRelation = createNewScanPlan(candidateDataSplits, condition, relation, metadataCols) + val filteredRelation = createNewScanPlan(candidateDataSplits, condition, relation) findTouchedFiles(createDataset(sparkSession, filteredRelation), sparkSession) } @@ -132,21 +132,58 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper { .map(relativePath) } + protected def createNewScanPlan( + candidateDataSplits: Seq[DataSplit], + condition: Expression, + relation: DataSourceV2Relation, + metadataColumns: Seq[PaimonMetadataColumn]): LogicalPlan = { + val newRelation = createNewScanPlan(candidateDataSplits, condition, relation) + val resolvedMetadataColumns = metadataColumns.map { + col => + val attr = newRelation.resolve(col.name :: Nil, conf.resolver) + assert(attr.isDefined) + attr.get + } + Project(relation.output ++ resolvedMetadataColumns, newRelation) + } + + protected def createNewScanPlan( + candidateDataSplits: Seq[DataSplit], + condition: Expression, + relation: DataSourceV2Relation): LogicalPlan = { + val newRelation = createNewRelation(candidateDataSplits, relation) + FilterLogicalNode(condition, newRelation) + } + protected def createNewRelation( filePaths: Array[String], filePathToMeta: Map[String, SparkDataFileMeta], - relation: DataSourceV2Relation): (Array[SparkDataFileMeta], DataSourceV2ScanRelation) = { + relation: DataSourceV2Relation): (Array[SparkDataFileMeta], DataSourceV2Relation) = { val files = filePaths.map( file => filePathToMeta.getOrElse(file, throw new RuntimeException(s"Missing file: $file"))) val touchedDataSplits = SparkDataFileMeta.convertToDataSplits(files, rawConvertible = true, fileStore.pathFactory()) - val newRelation = Compatibility.createDataSourceV2ScanRelation( - relation, - PaimonSplitScan(table, touchedDataSplits), - relation.output) + val newRelation = createNewRelation(touchedDataSplits, relation) (files, newRelation) } + protected def createNewRelation( + splits: Seq[DataSplit], + relation: DataSourceV2Relation): DataSourceV2Relation = { + assert(relation.table.isInstanceOf[SparkTable]) + val sparkTable = relation.table.asInstanceOf[SparkTable] + assert(sparkTable.table.isInstanceOf[FileStoreTable]) + val knownSplitsTable = + KnownSplitsTable.create(sparkTable.table.asInstanceOf[FileStoreTable], splits.toArray) + // We re-plan the relation to skip analyze phase, so we should append all + // metadata columns manually and let Spark do column pruning during optimization. + relation.copy( + table = relation.table.asInstanceOf[SparkTable].copy(table = knownSplitsTable), + output = relation.output ++ sparkTable.metadataColumns.map( + _.asInstanceOf[PaimonMetadataColumn].toAttribute) + ) + } + /** Notice that, the key is a relative path, not just the file name. */ protected def candidateFileMap( candidateDataSplits: Seq[DataSplit]): Map[String, SparkDataFileMeta] = { @@ -165,8 +202,7 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper { condition: Expression, relation: DataSourceV2Relation, sparkSession: SparkSession): Dataset[SparkDeletionVectors] = { - val metadataCols = Seq(FILE_PATH, ROW_INDEX) - val filteredRelation = createNewScanPlan(candidateDataSplits, condition, relation, metadataCols) + val filteredRelation = createNewScanPlan(candidateDataSplits, condition, relation) val dataWithMetadataColumns = createDataset(sparkSession, filteredRelation) collectDeletionVectors(dataFilePathToMeta, dataWithMetadataColumns, sparkSession) } @@ -196,7 +232,7 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper { .as[(String, Long)] .groupByKey(_._1) .mapGroups { - case (filePath, iter) => + (filePath, iter) => val dv = new BitmapDeletionVector() while (iter.hasNext) { dv.delete(iter.next()._2) @@ -218,20 +254,6 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper { } } - protected def createNewScanPlan( - candidateDataSplits: Seq[DataSplit], - condition: Expression, - relation: DataSourceV2Relation, - metadataCols: Seq[PaimonMetadataColumn]): LogicalPlan = { - val metadataProj = metadataCols.map(_.toAttribute) - val newRelation = relation.copy(output = relation.output ++ metadataProj) - val scan = PaimonSplitScan(table, candidateDataSplits.toArray, metadataCols) - FilterLogicalNode( - condition, - Compatibility.createDataSourceV2ScanRelation(newRelation, scan, newRelation.output)) - - } - protected def buildDeletedCommitMessage( deletedFiles: Array[SparkDataFileMeta]): Seq[CommitMessage] = { deletedFiles 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 391ba2b87c93..7d56fe867a1b 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 @@ -250,7 +250,7 @@ case class PaimonSparkWriter(table: FileStoreTable) { val serializedCommits = deletionVectors .groupByKey(_.partitionAndBucket) .mapGroups { - case (_, iter: Iterator[SparkDeletionVectors]) => + (_, iter: Iterator[SparkDeletionVectors]) => val indexHandler = table.store().newIndexFileHandler() var dvIndexFileMaintainer: AppendDeletionFileMaintainer = null while (iter.hasNext) { @@ -397,7 +397,7 @@ case class PaimonSparkWriter(table: FileStoreTable) { } private def repartitionByPartitionsAndBucket(df: DataFrame): DataFrame = { - val partitionCols = tableSchema.partitionKeys().asScala.map(col) + val partitionCols = tableSchema.partitionKeys().asScala.map(col).toSeq df.repartition(partitionCols ++ Seq(col(BUCKET_COL)): _*) } 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 b380d36c3f81..9c377b47c4fd 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 @@ -58,7 +58,7 @@ object SparkDataFileMeta { file, dvFactory.create(file.fileName())) } - } + }.toSeq def convertToDataSplits( sparkDataFiles: Array[SparkDataFileMeta], 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 dd88f388cb63..f2ea965d1407 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 @@ -18,8 +18,6 @@ package org.apache.paimon.spark.commands -import org.apache.paimon.spark.PaimonSplitScan -import org.apache.paimon.spark.catalyst.Compatibility import org.apache.paimon.spark.catalyst.analysis.AssignmentAlignmentHelper import org.apache.paimon.spark.leafnode.PaimonLeafRunnableCommand import org.apache.paimon.spark.schema.SparkSystemColumns.ROW_KIND_COL @@ -28,13 +26,14 @@ import org.apache.paimon.table.sink.CommitMessage import org.apache.paimon.table.source.DataSplit import org.apache.paimon.types.RowKind -import org.apache.spark.sql.{Column, Row, SparkSession} +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.PaimonUtils.createDataset import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, If} import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral import org.apache.spark.sql.catalyst.plans.logical.{Assignment, Filter, Project, SupportsSubquery} -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.paimon.shims.ExpressionUtils.column case class UpdatePaimonTableCommand( relation: DataSourceV2Relation, @@ -134,14 +133,10 @@ case class UpdatePaimonTableCommand( sparkSession: SparkSession, touchedDataSplits: Array[DataSplit]): Seq[CommitMessage] = { val updateColumns = updateExpressions.zip(relation.output).map { - case (update, origin) => - new Column(update).as(origin.name, origin.metadata) + case (update, origin) => column(update).as(origin.name, origin.metadata) } - val toUpdateScanRelation = Compatibility.createDataSourceV2ScanRelation( - relation, - PaimonSplitScan(table, touchedDataSplits), - relation.output) + val toUpdateScanRelation = createNewRelation(touchedDataSplits, relation) val newPlan = if (condition == TrueLiteral) { toUpdateScanRelation } else { @@ -153,7 +148,7 @@ case class UpdatePaimonTableCommand( private def writeUpdatedAndUnchangedData( sparkSession: SparkSession, - toUpdateScanRelation: DataSourceV2ScanRelation): Seq[CommitMessage] = { + toUpdateScanRelation: DataSourceV2Relation): Seq[CommitMessage] = { val updateColumns = updateExpressions.zip(relation.output).map { case (update, origin) => val updated = if (condition == TrueLiteral) { @@ -161,7 +156,7 @@ case class UpdatePaimonTableCommand( } else { If(condition, update, origin) } - new Column(updated).as(origin.name, origin.metadata) + column(updated).as(origin.name, origin.metadata) } val data = createDataset(sparkSession, toUpdateScanRelation).select(updateColumns: _*) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WriteIntoPaimonTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WriteIntoPaimonTable.scala index fe740ea8ca11..27d9a0786a56 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WriteIntoPaimonTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WriteIntoPaimonTable.scala @@ -18,11 +18,15 @@ package org.apache.paimon.spark.commands +import org.apache.paimon.CoreOptions import org.apache.paimon.CoreOptions.DYNAMIC_PARTITION_OVERWRITE import org.apache.paimon.options.Options +import org.apache.paimon.partition.actions.PartitionMarkDoneAction import org.apache.paimon.spark._ import org.apache.paimon.spark.schema.SparkSystemColumns import org.apache.paimon.table.FileStoreTable +import org.apache.paimon.table.sink.CommitMessage +import org.apache.paimon.utils.{InternalRowPartitionComputer, PartitionPathUtils, TypeUtils} import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row, SparkSession} @@ -63,9 +67,29 @@ case class WriteIntoPaimonTable( val commitMessages = writer.write(data) writer.commit(commitMessages) + markDoneIfNeeded(commitMessages) Seq.empty } + private def markDoneIfNeeded(commitMessages: Seq[CommitMessage]): Unit = { + val coreOptions = table.coreOptions() + if (coreOptions.toConfiguration.get(CoreOptions.PARTITION_MARK_DONE_WHEN_END_INPUT)) { + val actions = PartitionMarkDoneAction.createActions(table, table.coreOptions()) + val partitionComputer = new InternalRowPartitionComputer( + coreOptions.partitionDefaultName, + TypeUtils.project(table.rowType(), table.partitionKeys()), + table.partitionKeys().asScala.toArray, + coreOptions.legacyPartitionName() + ) + val partitions = commitMessages + .map(c => c.partition()) + .map(p => PartitionPathUtils.generatePartitionPath(partitionComputer.generatePartValues(p))) + for (partition <- partitions) { + actions.forEach(a => a.markDone(partition)) + } + } + } + private def parseSaveMode(): (Boolean, Map[String, String]) = { var dynamicPartitionOverwriteMode = false val overwritePartition = saveMode match { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/CreateOrReplaceTagExec.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/CreateOrReplaceTagExec.scala new file mode 100644 index 000000000000..0506ed42f1f4 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/CreateOrReplaceTagExec.scala @@ -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.spark.execution + +import org.apache.paimon.spark.SparkTable +import org.apache.paimon.spark.catalyst.plans.logical.TagOptions +import org.apache.paimon.spark.leafnode.PaimonLeafV2CommandExec +import org.apache.paimon.table.FileStoreTable + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} + +case class CreateOrReplaceTagExec( + catalog: TableCatalog, + ident: Identifier, + tagName: String, + tagOptions: TagOptions, + create: Boolean, + replace: Boolean, + ifNotExists: Boolean) + extends PaimonLeafV2CommandExec { + + override protected def run(): Seq[InternalRow] = { + val table = catalog.loadTable(ident) + assert(table.isInstanceOf[SparkTable]) + + table.asInstanceOf[SparkTable].getTable match { + case paimonTable: FileStoreTable => + val tagIsExists = paimonTable.tagManager().tagExists(tagName) + val timeRetained = tagOptions.timeRetained.orNull + val snapshotId = tagOptions.snapshotId + + if (create && replace && !tagIsExists) { + if (snapshotId.isEmpty) { + paimonTable.createTag(tagName, timeRetained) + } else { + paimonTable.createTag(tagName, snapshotId.get, timeRetained) + } + } else if (replace) { + paimonTable.replaceTag(tagName, snapshotId.get, timeRetained) + } else { + if (tagIsExists && ifNotExists) { + return Nil + } + + if (snapshotId.isEmpty) { + paimonTable.createTag(tagName, timeRetained) + } else { + paimonTable.createTag(tagName, snapshotId.get, timeRetained) + } + } + case t => + throw new UnsupportedOperationException( + s"Can not create tag for non-paimon FileStoreTable: $t") + } + Nil + } + + override def output: Seq[Attribute] = Nil +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/DeleteTagExec.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/DeleteTagExec.scala new file mode 100644 index 000000000000..d27839bdc6b6 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/DeleteTagExec.scala @@ -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.spark.execution + +import org.apache.paimon.spark.SparkTable +import org.apache.paimon.spark.leafnode.PaimonLeafV2CommandExec +import org.apache.paimon.table.FileStoreTable + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} + +case class DeleteTagExec( + catalog: TableCatalog, + ident: Identifier, + tagStr: String, + ifExists: Boolean) + extends PaimonLeafV2CommandExec { + + private val DELIMITER = "," + + override protected def run(): Seq[InternalRow] = { + val table = catalog.loadTable(ident) + assert(table.isInstanceOf[SparkTable]) + + table.asInstanceOf[SparkTable].getTable match { + case paimonTable: FileStoreTable => + val tagNames = tagStr.split(DELIMITER).map(_.trim) + for (tagName <- tagNames) { + val tagIsExists = paimonTable.tagManager().tagExists(tagName) + if (tagIsExists || !ifExists) { + paimonTable.deleteTag(tagName) + } + } + case t => + throw new UnsupportedOperationException( + s"Can not delete tag for non-paimon FileStoreTable: $t") + } + Nil + } + + override def output: Seq[Attribute] = Nil +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala index c6c6fc8759c0..0c3d3e6b6dc6 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonStrategy.scala @@ -18,16 +18,25 @@ package org.apache.paimon.spark.execution -import org.apache.paimon.spark.catalyst.plans.logical.PaimonCallCommand +import org.apache.paimon.spark.{SparkCatalog, SparkUtils} +import org.apache.paimon.spark.catalyst.plans.logical.{CreateOrReplaceTagCommand, DeleteTagCommand, PaimonCallCommand, RenameTagCommand, ShowTagsCommand} import org.apache.spark.sql.{SparkSession, Strategy} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow, PredicateHelper} import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LogicalPlan} +import org.apache.spark.sql.connector.catalog.{Identifier, PaimonLookupCatalog, TableCatalog} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.shim.PaimonCreateTableAsSelectStrategy -case class PaimonStrategy(spark: SparkSession) extends Strategy with PredicateHelper { +import scala.collection.JavaConverters._ + +case class PaimonStrategy(spark: SparkSession) + extends Strategy + with PredicateHelper + with PaimonLookupCatalog { + + protected lazy val catalogManager = spark.sessionState.catalogManager override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { @@ -37,6 +46,25 @@ case class PaimonStrategy(spark: SparkSession) extends Strategy with PredicateHe case c @ PaimonCallCommand(procedure, args) => val input = buildInternalRow(args) PaimonCallExec(c.output, procedure, input) :: Nil + + case t @ ShowTagsCommand(PaimonCatalogAndIdentifier(catalog, ident)) => + ShowTagsExec(catalog, ident, t.output) :: Nil + + case CreateOrReplaceTagCommand( + PaimonCatalogAndIdentifier(table, ident), + tagName, + tagOptions, + create, + replace, + ifNotExists) => + CreateOrReplaceTagExec(table, ident, tagName, tagOptions, create, replace, ifNotExists) :: Nil + + case DeleteTagCommand(PaimonCatalogAndIdentifier(catalog, ident), tagStr, ifExists) => + DeleteTagExec(catalog, ident, tagStr, ifExists) :: Nil + + case RenameTagCommand(PaimonCatalogAndIdentifier(catalog, ident), sourceTag, targetTag) => + RenameTagExec(catalog, ident, sourceTag, targetTag) :: Nil + case _ => Nil } @@ -48,4 +76,16 @@ case class PaimonStrategy(spark: SparkSession) extends Strategy with PredicateHe new GenericInternalRow(values) } + private object PaimonCatalogAndIdentifier { + def unapply(identifier: Seq[String]): Option[(TableCatalog, Identifier)] = { + val catalogAndIdentifier = + SparkUtils.catalogAndIdentifier(spark, identifier.asJava, catalogManager.currentCatalog) + catalogAndIdentifier.catalog match { + case paimonCatalog: SparkCatalog => + Some((paimonCatalog, catalogAndIdentifier.identifier())) + case _ => + None + } + } + } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/RenameTagExec.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/RenameTagExec.scala new file mode 100644 index 000000000000..655cf70b9cc1 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/RenameTagExec.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.execution + +import org.apache.paimon.spark.SparkTable +import org.apache.paimon.spark.leafnode.PaimonLeafV2CommandExec +import org.apache.paimon.table.FileStoreTable + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} + +case class RenameTagExec( + catalog: TableCatalog, + ident: Identifier, + sourceTag: String, + targetTag: String) + extends PaimonLeafV2CommandExec { + + override protected def run(): Seq[InternalRow] = { + val table = catalog.loadTable(ident) + assert(table.isInstanceOf[SparkTable]) + + table.asInstanceOf[SparkTable].getTable match { + case paimonTable: FileStoreTable => + paimonTable.renameTag(sourceTag, targetTag) + case t => + throw new UnsupportedOperationException( + s"Can not rename tag for non-paimon FileStoreTable: $t") + } + Nil + } + + override def output: Seq[Attribute] = Nil +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/ShowTagsExec.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/ShowTagsExec.scala new file mode 100644 index 000000000000..8f8d2b4665a5 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/ShowTagsExec.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.execution + +import org.apache.paimon.spark.SparkTable +import org.apache.paimon.spark.leafnode.PaimonLeafV2CommandExec +import org.apache.paimon.table.FileStoreTable + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.unsafe.types.UTF8String + +import scala.collection.JavaConverters._ + +case class ShowTagsExec(catalog: TableCatalog, ident: Identifier, out: Seq[Attribute]) + extends PaimonLeafV2CommandExec { + + override protected def run(): Seq[InternalRow] = { + val table = catalog.loadTable(ident) + assert(table.isInstanceOf[SparkTable]) + + var tags: Seq[InternalRow] = Nil + table.asInstanceOf[SparkTable].getTable match { + case paimonTable: FileStoreTable => + val tagNames = paimonTable.tagManager().allTagNames() + tags = tagNames.asScala.toList.sorted.map(t => InternalRow(UTF8String.fromString(t))) + case t => + throw new UnsupportedOperationException( + s"Can not show tags for non-paimon FileStoreTable: $t") + } + tags + } + + override def output: Seq[Attribute] = out +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/orphan/SparkOrphanFilesClean.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/orphan/SparkOrphanFilesClean.scala index d79105e24eec..488d70e34935 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/orphan/SparkOrphanFilesClean.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/orphan/SparkOrphanFilesClean.scala @@ -62,7 +62,7 @@ case class SparkOrphanFilesClean( val maxBranchParallelism = Math.min(branches.size(), parallelism) // find snapshots using branch and find manifests(manifest, index, statistics) using snapshot val usedManifestFiles = spark.sparkContext - .parallelize(branches.asScala, maxBranchParallelism) + .parallelize(branches.asScala.toSeq, maxBranchParallelism) .mapPartitions(_.flatMap { branch => safelyGetAllSnapshots(branch).asScala.map(snapshot => (branch, snapshot.toJson)) }) @@ -114,7 +114,7 @@ case class SparkOrphanFilesClean( .toDF("used_name") // find candidate files which can be removed - val fileDirs = listPaimonFileDirs.asScala.map(_.toUri.toString) + val fileDirs = listPaimonFileDirs.asScala.map(_.toUri.toString).toSeq val maxFileDirsParallelism = Math.min(fileDirs.size, parallelism) val candidates = spark.sparkContext .parallelize(fileDirs, maxFileDirsParallelism) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/sources/StreamHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/sources/StreamHelper.scala index 91df04e6dc47..b44a66fce3ff 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/sources/StreamHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/sources/StreamHelper.scala @@ -49,12 +49,15 @@ private[spark] trait StreamHelper { private lazy val partitionSchema: StructType = SparkTypeUtils.fromPaimonRowType(TypeUtils.project(table.rowType(), table.partitionKeys())) - private lazy val partitionComputer: InternalRowPartitionComputer = + private lazy val partitionComputer: InternalRowPartitionComputer = { + val options = new CoreOptions(table.options) new InternalRowPartitionComputer( - new CoreOptions(table.options).partitionDefaultName, + options.partitionDefaultName, TypeUtils.project(table.rowType(), table.partitionKeys()), - table.partitionKeys().asScala.toArray + table.partitionKeys().asScala.toArray, + options.legacyPartitionName() ) + } // Used to get the initial offset. lazy val streamScanStartingContext: StartingContext = streamScan.startingContext() diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/statistics/StatisticsHelperBase.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/statistics/StatisticsHelperBase.scala index 32fa48210c7f..627c6a168819 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/statistics/StatisticsHelperBase.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/statistics/StatisticsHelperBase.scala @@ -77,11 +77,11 @@ trait StatisticsHelperBase extends SQLConfHelper { private def toV2Stats(v1Stats: logical.Statistics): Statistics = { new Statistics() { override def sizeInBytes(): OptionalLong = if (v1Stats.sizeInBytes != null) - OptionalLong.of(v1Stats.sizeInBytes.longValue()) + OptionalLong.of(v1Stats.sizeInBytes.longValue) else OptionalLong.empty() override def numRows(): OptionalLong = if (v1Stats.rowCount.isDefined) - OptionalLong.of(v1Stats.rowCount.get.longValue()) + OptionalLong.of(v1Stats.rowCount.get.longValue) else OptionalLong.empty() override def columnStats(): java.util.Map[NamedReference, ColumnStatistics] = { 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 af7ff7204cda..b60dd1fb2173 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 @@ -18,33 +18,61 @@ package org.apache.paimon.spark.util +import org.apache.paimon.catalog.Identifier import org.apache.paimon.table.Table import org.apache.spark.sql.catalyst.SQLConfHelper -import java.util.{HashMap => JHashMap, Map => JMap} +import java.util.{Map => JMap} +import java.util.regex.Pattern import scala.collection.JavaConverters._ object OptionUtils extends SQLConfHelper { private val PAIMON_OPTION_PREFIX = "spark.paimon." + private val SPARK_CATALOG_PREFIX = "spark.sql.catalog." - def mergeSQLConf(extraOptions: JMap[String, String]): JMap[String, String] = { - val mergedOptions = new JHashMap[String, String]( - conf.getAllConfs - .filterKeys(_.startsWith(PAIMON_OPTION_PREFIX)) - .map { - case (key, value) => - key.stripPrefix(PAIMON_OPTION_PREFIX) -> value - } - .asJava) + def extractCatalogName(): Option[String] = { + val sparkCatalogTemplate = String.format("%s([^.]*)$", SPARK_CATALOG_PREFIX) + val sparkCatalogPattern = Pattern.compile(sparkCatalogTemplate) + conf.getAllConfs.filterKeys(_.startsWith(SPARK_CATALOG_PREFIX)).foreach { + case (key, _) => + val matcher = sparkCatalogPattern.matcher(key) + if (matcher.find()) + return Option(matcher.group(1)) + } + Option.empty + } + + def mergeSQLConfWithIdentifier( + extraOptions: JMap[String, String], + catalogName: String, + ident: Identifier): JMap[String, String] = { + val tableOptionsTemplate = String.format( + "(%s)(%s|\\*)\\.(%s|\\*)\\.(%s|\\*)\\.(.+)", + PAIMON_OPTION_PREFIX, + catalogName, + ident.getDatabaseName, + ident.getObjectName) + val tableOptionsPattern = Pattern.compile(tableOptionsTemplate) + val mergedOptions = org.apache.paimon.options.OptionsUtils + .convertToDynamicTableProperties( + conf.getAllConfs.asJava, + PAIMON_OPTION_PREFIX, + tableOptionsPattern, + 5) mergedOptions.putAll(extraOptions) mergedOptions } - def copyWithSQLConf[T <: Table](table: T, extraOptions: JMap[String, String]): T = { - val mergedOptions = mergeSQLConf(extraOptions) + def copyWithSQLConf[T <: Table]( + table: T, + catalogName: String, + ident: Identifier, + extraOptions: JMap[String, String]): T = { + val mergedOptions: JMap[String, String] = + mergeSQLConfWithIdentifier(extraOptions, catalogName, ident) if (mergedOptions.isEmpty) { table } else { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala index 26a351bc673a..dd0a48159b2a 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala @@ -47,8 +47,8 @@ import java.util.Locale * @param delegate * The extension parser. */ -class PaimonSparkSqlExtensionsParser(delegate: ParserInterface) - extends ParserInterface +class PaimonSparkSqlExtensionsParser(val delegate: ParserInterface) + extends org.apache.spark.sql.paimon.shims.ParserInterface with Logging { private lazy val substitutor = new VariableSubstitution() @@ -57,7 +57,7 @@ class PaimonSparkSqlExtensionsParser(delegate: ParserInterface) /** Parses a string to a LogicalPlan. */ override def parsePlan(sqlText: String): LogicalPlan = { val sqlTextAfterSubstitution = substitutor.substitute(sqlText) - if (isCommand(sqlTextAfterSubstitution)) { + if (isPaimonCommand(sqlTextAfterSubstitution)) { parse(sqlTextAfterSubstitution)(parser => astBuilder.visit(parser.singleStatement())) .asInstanceOf[LogicalPlan] } else { @@ -93,7 +93,7 @@ class PaimonSparkSqlExtensionsParser(delegate: ParserInterface) delegate.parseMultipartIdentifier(sqlText) /** Returns whether SQL text is command. */ - private def isCommand(sqlText: String): Boolean = { + private def isPaimonCommand(sqlText: String): Boolean = { val normalized = sqlText .toLowerCase(Locale.ROOT) .trim() @@ -101,7 +101,16 @@ class PaimonSparkSqlExtensionsParser(delegate: ParserInterface) .replaceAll("\\s+", " ") .replaceAll("/\\*.*?\\*/", " ") .trim() - normalized.startsWith("call") + normalized.startsWith("call") || isTagRefDdl(normalized) + } + + private def isTagRefDdl(normalized: String): Boolean = { + normalized.startsWith("show tags") || + (normalized.startsWith("alter table") && + (normalized.contains("create tag") || + normalized.contains("replace tag") || + normalized.contains("rename tag") || + normalized.contains("delete tag"))) } protected def parse[T](command: String)(toResult: PaimonSqlExtensionsParser => T): T = { diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSqlExtensionsAstBuilder.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSqlExtensionsAstBuilder.scala index 397424c636d1..b864894e7498 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSqlExtensionsAstBuilder.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSqlExtensionsAstBuilder.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.catalyst.parser.extensions import org.apache.paimon.spark.catalyst.plans.logical -import org.apache.paimon.spark.catalyst.plans.logical.{PaimonCallArgument, PaimonCallStatement, PaimonNamedArgument, PaimonPositionalArgument} +import org.apache.paimon.spark.catalyst.plans.logical.{CreateOrReplaceTagCommand, DeleteTagCommand, PaimonCallArgument, PaimonCallStatement, PaimonNamedArgument, PaimonPositionalArgument, RenameTagCommand, ShowTagsCommand, TagOptions} +import org.apache.paimon.utils.TimeUtils import org.antlr.v4.runtime._ import org.antlr.v4.runtime.misc.Interval @@ -57,8 +58,8 @@ class PaimonSqlExtensionsAstBuilder(delegate: ParserInterface) /** Creates a [[PaimonCallStatement]] for a stored procedure call. */ override def visitCall(ctx: CallContext): PaimonCallStatement = withOrigin(ctx) { - val name = toSeq(ctx.multipartIdentifier.parts).map(_.getText) - val args = toSeq(ctx.callArgument).map(typedVisit[PaimonCallArgument]) + val name = ctx.multipartIdentifier.parts.asScala.map(_.getText).toSeq + val args = ctx.callArgument.asScala.map(typedVisit[PaimonCallArgument]).toSeq logical.PaimonCallStatement(name, args) } @@ -89,9 +90,69 @@ class PaimonSqlExtensionsAstBuilder(delegate: ParserInterface) /** Returns a multi-part identifier as Seq[String]. */ override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) { - ctx.parts.asScala.map(_.getText) + ctx.parts.asScala.map(_.getText).toSeq } + /** Create a SHOW TAGS logical command. */ + override def visitShowTags(ctx: ShowTagsContext): ShowTagsCommand = withOrigin(ctx) { + ShowTagsCommand(typedVisit[Seq[String]](ctx.multipartIdentifier)) + } + + /** Create a CREATE OR REPLACE TAG logical command. */ + override def visitCreateOrReplaceTag(ctx: CreateOrReplaceTagContext): CreateOrReplaceTagCommand = + withOrigin(ctx) { + val createTagClause = ctx.createReplaceTagClause() + + val tagName = createTagClause.identifier().getText + val tagOptionsContext = Option(createTagClause.tagOptions()) + val snapshotId = + tagOptionsContext + .flatMap(tagOptions => Option(tagOptions.snapshotId())) + .map(_.getText.toLong) + val timeRetainCtx = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.timeRetain())) + val timeRetained = if (timeRetainCtx.nonEmpty) { + val (number, timeUnit) = + timeRetainCtx + .map(retain => (retain.number().getText.toLong, retain.timeUnit().getText)) + .get + Option(TimeUtils.parseDuration(number, timeUnit)) + } else { + None + } + val tagOptions = TagOptions( + snapshotId, + timeRetained + ) + + val create = createTagClause.CREATE() != null + val replace = createTagClause.REPLACE() != null + val ifNotExists = createTagClause.EXISTS() != null + + CreateOrReplaceTagCommand( + typedVisit[Seq[String]](ctx.multipartIdentifier), + tagName, + tagOptions, + create, + replace, + ifNotExists) + } + + /** Create a DELETE TAG logical command. */ + override def visitDeleteTag(ctx: DeleteTagContext): DeleteTagCommand = withOrigin(ctx) { + DeleteTagCommand( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.identifier().getText, + ctx.EXISTS() != null) + } + + /** Create a RENAME TAG logical command. */ + override def visitRenameTag(ctx: RenameTagContext): RenameTagCommand = withOrigin(ctx) { + RenameTagCommand( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.identifier(0).getText, + ctx.identifier(1).getText) + } + private def toBuffer[T](list: java.util.List[T]) = list.asScala private def toSeq[T](list: java.util.List[T]) = toBuffer(list) diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java new file mode 100644 index 000000000000..68cf91b8ec7b --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java @@ -0,0 +1,139 @@ +/* + * 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; + +import org.apache.paimon.fs.Path; +import org.apache.paimon.hive.TestHiveMetastore; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.FileNotFoundException; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Base tests for spark read. */ +public class SparkCatalogWithHiveTest { + + private static TestHiveMetastore testHiveMetastore; + + private static final int PORT = 9087; + + @BeforeAll + public static void startMetastore() { + testHiveMetastore = new TestHiveMetastore(); + testHiveMetastore.start(PORT); + } + + @AfterAll + public static void closeMetastore() throws Exception { + testHiveMetastore.stop(); + } + + @Test + public void testCreateFormatTable(@TempDir java.nio.file.Path tempDir) { + Path warehousePath = new Path("file:" + tempDir.toString()); + SparkSession spark = + SparkSession.builder() + .config("spark.sql.warehouse.dir", warehousePath.toString()) + // with hive metastore + .config("spark.sql.catalogImplementation", "hive") + .config("hive.metastore.uris", "thrift://localhost:" + PORT) + .config("spark.sql.catalog.spark_catalog", SparkCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.metastore", "hive") + .config( + "spark.sql.catalog.spark_catalog.hive.metastore.uris", + "thrift://localhost:" + PORT) + .config("spark.sql.catalog.spark_catalog.format-table.enabled", "true") + .config( + "spark.sql.catalog.spark_catalog.warehouse", + warehousePath.toString()) + .master("local[2]") + .getOrCreate(); + + spark.sql("CREATE DATABASE IF NOT EXISTS my_db1"); + spark.sql("USE spark_catalog.my_db1"); + + // test orc table + + spark.sql("CREATE TABLE IF NOT EXISTS table_orc (a INT, bb INT, c STRING) USING orc"); + + assertThat( + spark.sql("SHOW TABLES").collectAsList().stream() + .map(s -> s.get(1)) + .map(Object::toString)) + .containsExactlyInAnyOrder("table_orc"); + + assertThat( + spark.sql("EXPLAIN EXTENDED SELECT * from table_orc").collectAsList() + .stream() + .map(s -> s.get(0)) + .map(Object::toString) + .filter(s -> s.contains("OrcScan")) + .count()) + .isGreaterThan(0); + + // todo: There are some bugs with Spark CSV table's options. In Spark 3.x, both reading and + // writing using the default delimiter value ',' even if we specific it. In Spark 4.x, + // reading is correct, but writing is still incorrect, just skip setting it for now. + // test csv table + + spark.sql( + "CREATE TABLE IF NOT EXISTS table_csv (a INT, bb INT, c STRING) USING csv OPTIONS ('field-delimiter' ',')"); + spark.sql("INSERT INTO table_csv VALUES (1, 1, '1'), (2, 2, '2')").collect(); + assertThat(spark.sql("DESCRIBE FORMATTED table_csv").collectAsList().toString()) + .contains("sep=,"); + assertThat( + spark.sql("SELECT * FROM table_csv").collectAsList().stream() + .map(Row::toString) + .collect(Collectors.toList())) + .containsExactlyInAnyOrder("[1,1,1]", "[2,2,2]"); + + spark.close(); + } + + @Test + public void testSpecifyHiveConfDir(@TempDir java.nio.file.Path tempDir) { + Path warehousePath = new Path("file:" + tempDir.toString()); + SparkSession spark = + SparkSession.builder() + .config("spark.sql.catalog.spark_catalog.hive-conf-dir", "nonExistentPath") + .config("spark.sql.warehouse.dir", warehousePath.toString()) + // with hive metastore + .config("spark.sql.catalogImplementation", "hive") + .config( + "spark.sql.catalog.spark_catalog", + SparkGenericCatalog.class.getName()) + .master("local[2]") + .getOrCreate(); + + assertThatThrownBy(() -> spark.sql("CREATE DATABASE my_db")) + .rootCause() + .isInstanceOf(FileNotFoundException.class) + .hasMessageContaining("nonExistentPath"); + + spark.close(); + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java index 7af83186ef6e..7da8f2e6b7fe 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java @@ -27,7 +27,7 @@ import org.apache.paimon.fileindex.FileIndexFormat; import org.apache.paimon.fileindex.FileIndexReader; import org.apache.paimon.fileindex.FileIndexResult; -import org.apache.paimon.fileindex.bitmap.BitmapIndexResultLazy; +import org.apache.paimon.fileindex.bitmap.BitmapIndexResult; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; @@ -103,22 +103,49 @@ public void testReadWriteTableWithBitmapIndex() throws Catalog.TableNotExistExce + "'file-index.in-manifest-threshold'='1B');"); spark.sql("INSERT INTO T VALUES (0),(1),(2),(3),(4),(5);"); + List rows1 = spark.sql("SELECT a FROM T where a>3;").collectAsList(); + assertThat(rows1.toString()).isEqualTo("[[4], [5]]"); + // check query result - List rows = spark.sql("SELECT a FROM T where a='3';").collectAsList(); - assertThat(rows.toString()).isEqualTo("[[3]]"); + List rows2 = spark.sql("SELECT a FROM T where a=3;").collectAsList(); + assertThat(rows2.toString()).isEqualTo("[[3]]"); // check index reader foreachIndexReader( fileIndexReader -> { FileIndexResult fileIndexResult = fileIndexReader.visitEqual(new FieldRef(0, "", new IntType()), 3); - assert fileIndexResult instanceof BitmapIndexResultLazy; - RoaringBitmap32 roaringBitmap32 = - ((BitmapIndexResultLazy) fileIndexResult).get(); + assert fileIndexResult instanceof BitmapIndexResult; + RoaringBitmap32 roaringBitmap32 = ((BitmapIndexResult) fileIndexResult).get(); assert roaringBitmap32.equals(RoaringBitmap32.bitmapOf(3)); }); } + @Test + public void testReadWriteTableWithBitSliceIndex() throws Catalog.TableNotExistException { + + spark.sql( + "CREATE TABLE T(a int) TBLPROPERTIES (" + + "'file-index.bsi.columns'='a'," + + "'file-index.in-manifest-threshold'='1B');"); + spark.sql("INSERT INTO T VALUES (0),(1),(2),(3),(4),(5);"); + + // check query result + List rows = spark.sql("SELECT a FROM T where a>=3;").collectAsList(); + assertThat(rows.toString()).isEqualTo("[[3], [4], [5]]"); + + // check index reader + foreachIndexReader( + fileIndexReader -> { + FileIndexResult fileIndexResult = + fileIndexReader.visitGreaterOrEqual( + new FieldRef(0, "", new IntType()), 3); + assertThat(fileIndexResult).isInstanceOf(BitmapIndexResult.class); + RoaringBitmap32 roaringBitmap32 = ((BitmapIndexResult) fileIndexResult).get(); + assertThat(roaringBitmap32).isEqualTo(RoaringBitmap32.bitmapOf(3, 4, 5)); + }); + } + protected void foreachIndexReader(Consumer consumer) throws Catalog.TableNotExistException { Path tableRoot = fileSystemCatalog.getTableLocation(Identifier.create("db", "T")); @@ -128,9 +155,12 @@ protected void foreachIndexReader(Consumer consumer) tableRoot, RowType.of(), new CoreOptions(new Options()).partitionDefaultName(), - CoreOptions.FILE_FORMAT.defaultValue().toString(), + CoreOptions.FILE_FORMAT.defaultValue(), CoreOptions.DATA_FILE_PREFIX.defaultValue(), - CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue()); + CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), + CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), + CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), + CoreOptions.FILE_COMPRESSION.defaultValue()); Table table = fileSystemCatalog.getTable(Identifier.create("db", "T")); ReadBuilder readBuilder = table.newReadBuilder(); @@ -151,12 +181,11 @@ protected void foreachIndexReader(Consumer consumer) .collect(Collectors.toList()); // assert index file exist and only one index file assert indexFiles.size() == 1; - try { - FileIndexFormat.Reader reader = - FileIndexFormat.createReader( - fileIO.newInputStream( - dataFilePathFactory.toPath(indexFiles.get(0))), - tableSchema.logicalRowType()); + try (FileIndexFormat.Reader reader = + FileIndexFormat.createReader( + fileIO.newInputStream( + dataFilePathFactory.toPath(indexFiles.get(0))), + tableSchema.logicalRowType())) { Optional fileIndexReader = reader.readColumnIndex("a").stream().findFirst(); // assert index reader exist diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java index b0f1749dfeb3..84ea1ab5cba2 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java @@ -27,7 +27,10 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import java.io.FileNotFoundException; + import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.Assertions.assertThrows; /** Base tests for spark read. */ @@ -100,7 +103,7 @@ public void testCreateTableCaseSensitive(@TempDir java.nio.file.Path tempDir) { @Test public void testBuildWithHive(@TempDir java.nio.file.Path tempDir) { - // firstly, we use hive metastore to creata table, and check the result. + // firstly, we use hive metastore to create table, and check the result. Path warehousePath = new Path("file:" + tempDir.toString()); SparkSession spark = SparkSession.builder() @@ -148,4 +151,27 @@ public void testBuildWithHive(@TempDir java.nio.file.Path tempDir) { .map(Object::toString)) .containsExactlyInAnyOrder("t1"); } + + @Test + public void testHiveCatalogOptions(@TempDir java.nio.file.Path tempDir) { + Path warehousePath = new Path("file:" + tempDir.toString()); + SparkSession spark = + SparkSession.builder() + .config("spark.sql.catalog.spark_catalog.hive-conf-dir", "nonExistentPath") + .config("spark.sql.warehouse.dir", warehousePath.toString()) + // with hive metastore + .config("spark.sql.catalogImplementation", "hive") + .config( + "spark.sql.catalog.spark_catalog", + SparkGenericCatalog.class.getName()) + .master("local[2]") + .getOrCreate(); + + assertThatThrownBy(() -> spark.sql("CREATE DATABASE my_db")) + .rootCause() + .isInstanceOf(FileNotFoundException.class) + .hasMessageContaining("nonExistentPath"); + + spark.close(); + } } diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java index 9af886d8369f..b98213c0e662 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java @@ -104,8 +104,8 @@ public void test() { + "paimon," + "22.2," + "Map(key2 -> [2.4,3.5], key1 -> [1.2,2.3])," - + "WrappedArray(v1, v5)," - + "WrappedArray(10, 30)," + + "[v1, v5]," + + "[10, 30]," + "true," + "22," + "356," @@ -129,6 +129,12 @@ public void test() { private String sparkRowToString(org.apache.spark.sql.Row row) { return JavaConverters.seqAsJavaList(row.toSeq()).stream() + .map( + x -> + (x instanceof scala.collection.Seq) + ? JavaConverters.seqAsJavaList( + (scala.collection.Seq) x) + : x) .map(Object::toString) // Since the toString result of Spark's binary col is unstable, replace it .map(x -> x.startsWith("[B@") ? "[B@" : x) diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java index be6264f7b2d0..b4565447c6fc 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java @@ -440,6 +440,38 @@ public void testCreateAndDropTable() { innerTest("MyTable6", false, true); } + @Test + public void testReadNestedColumnTable() { + String tableName = "testAddNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v STRUCT>) " + + "TBLPROPERTIES ('bucket' = '1', 'primary-key' = 'k', 'file.format' = 'parquet')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, STRUCT(10, STRUCT('apple', 100)))"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (2, STRUCT(20, STRUCT('banana', 200)))"); + assertThat( + spark.sql("SELECT v.f2.f1, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[banana,2]"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, STRUCT(30, STRUCT('cat', 100)))"); + assertThat( + spark.sql("SELECT v.f2.f1, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[cat,1]", "[banana,2]"); + } + private void innerTest(String tableName, boolean hasPk, boolean partitioned) { String ddlTemplate = "CREATE TABLE default.%s (\n" diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java index 7d94e7d5df73..ccae59e88675 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java @@ -23,6 +23,8 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.util.HashMap; import java.util.List; @@ -203,7 +205,7 @@ public void testRenamePartitionKey() { .satisfies( anyCauseMatches( UnsupportedOperationException.class, - "Cannot drop/rename partition key[a]")); + "Cannot rename partition column: [a]")); } @Test @@ -254,7 +256,7 @@ public void testDropPartitionKey() { .satisfies( anyCauseMatches( UnsupportedOperationException.class, - "Cannot drop/rename partition key[a]")); + "Cannot drop partition key or primary key: [a]")); } @Test @@ -276,7 +278,72 @@ public void testDropPrimaryKey() { .satisfies( anyCauseMatches( UnsupportedOperationException.class, - "Cannot drop/rename primary key[b]")); + "Cannot drop partition key or primary key: [b]")); + } + + @Test + public void testRenamePrimaryKey() { + spark.sql( + "CREATE TABLE test_rename_primary_key_table (\n" + + "a BIGINT NOT NULL,\n" + + "b STRING)\n" + + "TBLPROPERTIES ('primary-key' = 'a')"); + + spark.sql("INSERT INTO test_rename_primary_key_table VALUES(1, 'aaa'), (2, 'bbb')"); + + spark.sql("ALTER TABLE test_rename_primary_key_table RENAME COLUMN a to a_"); + + List result = + spark.sql("SHOW CREATE TABLE test_rename_primary_key_table").collectAsList(); + assertThat(result.toString()) + .contains( + showCreateString( + "test_rename_primary_key_table", "a_ BIGINT NOT NULL", "b STRING")) + .contains("'primary-key' = 'a_'"); + + List actual = + spark.sql("SELECT * FROM test_rename_primary_key_table").collectAsList().stream() + .map(Row::toString) + .collect(Collectors.toList()); + + assertThat(actual).containsExactlyInAnyOrder("[1,aaa]", "[2,bbb]"); + + spark.sql("INSERT INTO test_rename_primary_key_table VALUES(1, 'AAA'), (2, 'BBB')"); + + actual = + spark.sql("SELECT * FROM test_rename_primary_key_table").collectAsList().stream() + .map(Row::toString) + .collect(Collectors.toList()); + assertThat(actual).containsExactlyInAnyOrder("[1,AAA]", "[2,BBB]"); + } + + @Test + public void testRenameBucketKey() { + spark.sql( + "CREATE TABLE test_rename_bucket_key_table (\n" + + "a BIGINT NOT NULL,\n" + + "b STRING)\n" + + "TBLPROPERTIES ('bucket-key' = 'a,b', 'bucket'='16')"); + + spark.sql("INSERT INTO test_rename_bucket_key_table VALUES(1, 'aaa'), (2, 'bbb')"); + + spark.sql("ALTER TABLE test_rename_bucket_key_table RENAME COLUMN b to b_"); + + List result = + spark.sql("SHOW CREATE TABLE test_rename_bucket_key_table").collectAsList(); + assertThat(result.toString()) + .contains( + showCreateString( + "test_rename_bucket_key_table", "a BIGINT NOT NULL", "b_ STRING")) + .contains("'bucket-key' = 'a,b_'"); + + List actual = + spark.sql("SELECT * FROM test_rename_bucket_key_table where b_ = 'bbb'") + .collectAsList().stream() + .map(Row::toString) + .collect(Collectors.toList()); + + assertThat(actual).containsExactlyInAnyOrder("[2,bbb]"); } @Test @@ -640,4 +707,114 @@ private List getFieldStatsList(List fieldStatsRows) { ",")) .collect(Collectors.toList()); } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testAddAndDropNestedColumn(String formatType) { + String tableName = "testAddNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v STRUCT>) " + + "TBLPROPERTIES ('bucket' = '1', 'primary-key' = 'k', 'file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, STRUCT(10, STRUCT('apple', 100))), (2, STRUCT(20, STRUCT('banana', 200)))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[1,[10,[apple,100]]]", "[2,[20,[banana,200]]]"); + assertThat( + spark.sql("SELECT v.f2.f1, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[banana,2]"); + + spark.sql("ALTER TABLE paimon.default." + tableName + " ADD COLUMN v.f3 STRING"); + spark.sql("ALTER TABLE paimon.default." + tableName + " ADD COLUMN v.f2.f3 BIGINT"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, STRUCT(11, STRUCT('APPLE', 101, 1001), 'one')), (3, STRUCT(31, STRUCT('CHERRY', 301, 3001), 'three'))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,[11,[APPLE,101,1001],one]]", + "[2,[20,[banana,200,null],null]]", + "[3,[31,[CHERRY,301,3001],three]]"); + assertThat( + spark.sql("SELECT v.f2.f2, v.f3, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[101,one,1]", "[200,null,2]", "[301,three,3]"); + + spark.sql("ALTER TABLE paimon.default." + tableName + " DROP COLUMN v.f2.f1"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, STRUCT(12, STRUCT(102, 1002), 'one')), (4, STRUCT(42, STRUCT(402, 4002), 'four'))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,[12,[102,1002],one]]", + "[2,[20,[200,null],null]]", + "[3,[31,[301,3001],three]]", + "[4,[42,[402,4002],four]]"); + + spark.sql( + "ALTER TABLE paimon.default." + + tableName + + " ADD COLUMN v.f2.f1 DECIMAL(5, 2) AFTER f2"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, STRUCT(13, STRUCT(103, 100.03, 1003), 'one')), (5, STRUCT(53, STRUCT(503, 500.03, 5003), 'five'))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,[13,[103,100.03,1003],one]]", + "[2,[20,[200,null,null],null]]", + "[3,[31,[301,null,3001],three]]", + "[4,[42,[402,null,4002],four]]", + "[5,[53,[503,500.03,5003],five]]"); + } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testRenameNestedColumn(String formatType) { + String tableName = "testRenameNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v STRUCT>) " + + "TBLPROPERTIES ('file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, STRUCT(10, STRUCT('apple', 100))), (2, STRUCT(20, STRUCT('banana', 200)))"); + assertThat( + spark.sql("SELECT v.f2.f1, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[banana,2]"); + + spark.sql("ALTER TABLE paimon.default." + tableName + " RENAME COLUMN v.f2.f1 to f100"); + assertThat( + spark.sql("SELECT v.f2.f100, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[banana,2]"); + } } diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java index 0cc17639fd80..b0d5b380c1f2 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java +++ b/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java @@ -36,6 +36,7 @@ import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.io.TempDir; +import java.io.IOException; import java.util.Arrays; import java.util.Comparator; import java.util.List; @@ -295,6 +296,9 @@ public void testDataFilePrefixForPKTable() { for (String fileName : fileNames) { Assertions.assertTrue(fileName.startsWith("test-")); } + + // reset config, it will affect other tests + spark.conf().unset("spark.paimon.data-file.prefix"); } @Test @@ -316,6 +320,114 @@ public void testChangelogFilePrefixForPkTable() throws Exception { spark.sql("INSERT INTO T VALUES (2, 2, 'bb')"); FileStatus[] files2 = fileIO.listStatus(new Path(tabLocation, "bucket-0")); Assertions.assertEquals(1, dataFileCount(files2, "test-changelog-")); + + // reset config, it will affect other tests + spark.conf().unset("spark.paimon.changelog-file.prefix"); + } + + @Test + public void testMarkDone() throws IOException { + spark.sql( + "CREATE TABLE T (a INT, b INT, c STRING) PARTITIONED BY (c) TBLPROPERTIES (" + + "'partition.end-input-to-done' = 'true', 'partition.mark-done-action' = 'success-file')"); + spark.sql("INSERT INTO T VALUES (1, 1, 'aa')"); + + FileStoreTable table = getTable("T"); + FileIO fileIO = table.fileIO(); + Path tabLocation = table.location(); + + Assertions.assertTrue(fileIO.exists(new Path(tabLocation, "c=aa/_SUCCESS"))); + } + + @Test + public void testDataFileSuffixName() { + spark.sql( + "CREATE TABLE T (a INT, b INT, c STRING)" + + " TBLPROPERTIES (" + + "'bucket' = '1', " + + "'primary-key'='a', " + + "'write-only' = 'true', " + + "'file.format' = 'parquet', " + + "'file.compression' = 'zstd')"); + + spark.sql("INSERT INTO T VALUES (1, 1, 'aa')"); + spark.sql("INSERT INTO T VALUES (2, 2, 'bb')"); + + // enable file suffix + spark.conf().set("spark.paimon.file.suffix.include.compression", true); + spark.sql("INSERT INTO T VALUES (3, 3, 'cc')"); + spark.sql("INSERT INTO T VALUES (4, 4, 'dd')"); + + List data2 = spark.sql("SELECT * FROM T order by a").collectAsList(); + assertThat(data2.toString()).isEqualTo("[[1,1,aa], [2,2,bb], [3,3,cc], [4,4,dd]]"); + + // check files suffix name + List files = + spark.sql("select file_path from `T$files`").collectAsList().stream() + .map(x -> x.getString(0)) + .collect(Collectors.toList()); + Assertions.assertEquals(4, files.size()); + + String defaultExtension = "." + "parquet"; + String newExtension = "." + "zstd" + "." + "parquet"; + // two data files end with ".parquet", two data file end with ".zstd.parquet" + Assertions.assertEquals( + 2, + files.stream() + .filter( + name -> + name.endsWith(defaultExtension) + && !name.endsWith(newExtension)) + .count()); + Assertions.assertEquals( + 2, files.stream().filter(name -> name.endsWith(newExtension)).count()); + + // reset config + spark.conf().unset("spark.paimon.file.suffix.include.compression"); + } + + @Test + public void testChangelogFileSuffixName() throws Exception { + spark.sql( + "CREATE TABLE T (a INT, b INT, c STRING) " + + "TBLPROPERTIES (" + + "'primary-key'='a', " + + "'bucket' = '1', " + + "'changelog-producer' = 'lookup', " + + "'file.format' = 'parquet', " + + "'file.compression' = 'zstd')"); + + FileStoreTable table = getTable("T"); + Path tabLocation = table.location(); + FileIO fileIO = table.fileIO(); + + spark.sql("INSERT INTO T VALUES (1, 1, 'aa')"); + + spark.conf().set("spark.paimon.file.suffix.include.compression", true); + spark.sql("INSERT INTO T VALUES (2, 2, 'bb')"); + + // collect changelog files + List files = + Arrays.stream(fileIO.listStatus(new Path(tabLocation, "bucket-0"))) + .map(name -> name.getPath().getName()) + .filter(name -> name.startsWith("changelog-")) + .collect(Collectors.toList()); + String defaultExtension = "." + "parquet"; + String newExtension = "." + "zstd" + "." + "parquet"; + // one changelog file end with ".parquet", one changelog file end with ".zstd.parquet" + Assertions.assertEquals( + 1, + files.stream() + .filter( + name -> + name.endsWith(defaultExtension) + && !name.endsWith(newExtension)) + .count()); + Assertions.assertEquals( + 1, files.stream().filter(name -> name.endsWith(newExtension)).count()); + + // reset config + spark.conf().unset("spark.paimon.file.suffix.include.compression"); } protected static FileStoreTable getTable(String tableName) { diff --git a/paimon-spark/paimon-spark-common/src/test/resources/hive-site.xml b/paimon-spark/paimon-spark-common/src/test/resources/hive-site.xml index 4972efc5900e..c4a016d51d04 100644 --- a/paimon-spark/paimon-spark-common/src/test/resources/hive-site.xml +++ b/paimon-spark/paimon-spark-common/src/test/resources/hive-site.xml @@ -42,6 +42,12 @@ true + + + datanucleus.connectionPoolingType + DBCP + + hive.metastore.uris thrift://localhost:9083 diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala index 3deb91cbcba7..9b4a344259a9 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala @@ -18,9 +18,8 @@ package org.apache.paimon.spark -import org.apache.paimon.catalog.{Catalog, CatalogContext, CatalogFactory, Identifier} -import org.apache.paimon.options.{CatalogOptions, Options} -import org.apache.paimon.spark.catalog.Catalogs +import org.apache.paimon.catalog.{Catalog, Identifier} +import org.apache.paimon.spark.catalog.WithPaimonCatalog import org.apache.paimon.spark.extensions.PaimonSparkSessionExtensions import org.apache.paimon.spark.sql.{SparkVersionSupport, WithTableOptions} import org.apache.paimon.table.FileStoreTable @@ -36,7 +35,6 @@ import org.scalactic.source.Position import org.scalatest.Tag import java.io.File -import java.util.{HashMap => JHashMap} import java.util.TimeZone import scala.util.Random @@ -49,7 +47,9 @@ class PaimonSparkTestBase protected lazy val tempDBDir: File = Utils.createTempDir - protected lazy val catalog: Catalog = initCatalog() + protected def paimonCatalog: Catalog = { + spark.sessionState.catalogManager.currentCatalog.asInstanceOf[WithPaimonCatalog].paimonCatalog() + } protected val dbName0: String = "test" @@ -122,18 +122,12 @@ class PaimonSparkTestBase super.test(testName, testTags: _*)(testFun)(pos) } - private def initCatalog(): Catalog = { - val currentCatalog = spark.sessionState.catalogManager.currentCatalog.name() - val options = - new JHashMap[String, String](Catalogs.catalogOptions(currentCatalog, spark.sessionState.conf)) - options.put(CatalogOptions.CACHE_ENABLED.key(), "false") - val catalogContext = - CatalogContext.create(Options.fromMap(options), spark.sessionState.newHadoopConf()) - CatalogFactory.createCatalog(catalogContext) + def loadTable(tableName: String): FileStoreTable = { + loadTable(dbName0, tableName) } - def loadTable(tableName: String): FileStoreTable = { - catalog.getTable(Identifier.create(dbName0, tableName)).asInstanceOf[FileStoreTable] + def loadTable(dbName: String, tableName: String): FileStoreTable = { + paimonCatalog.getTable(Identifier.create(dbName, tableName)).asInstanceOf[FileStoreTable] } protected def createRelationV2(tableName: String): LogicalPlan = { diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala index 63711393039b..fc787246f9f1 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala @@ -26,6 +26,7 @@ import org.apache.paimon.table.source.{DataSplit, Split} import org.junit.jupiter.api.Assertions +import java.util import java.util.{HashMap => JHashMap} import scala.collection.JavaConverters._ @@ -42,7 +43,18 @@ class ScanHelperTest extends PaimonSparkTestBase { 0.until(fileNum).foreach { i => val path = s"f$i.parquet" - files += DataFileMeta.forAppend(path, 750000, 30000, null, 0, 29999, 1, FileSource.APPEND) + files += DataFileMeta.forAppend( + path, + 750000, + 30000, + null, + 0, + 29999, + 1, + new java.util.ArrayList[String](), + null, + FileSource.APPEND, + null) } val dataSplits = mutable.ArrayBuffer.empty[Split] @@ -67,7 +79,18 @@ class ScanHelperTest extends PaimonSparkTestBase { test("Paimon: reshuffle one split") { val files = List( - DataFileMeta.forAppend("f1.parquet", 750000, 30000, null, 0, 29999, 1, FileSource.APPEND) + DataFileMeta.forAppend( + "f1.parquet", + 750000, + 30000, + null, + 0, + 29999, + 1, + new java.util.ArrayList[String](), + null, + FileSource.APPEND, + null) ).asJava val dataSplits: Array[Split] = Array( diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala new file mode 100644 index 000000000000..316c36c40c56 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala @@ -0,0 +1,97 @@ +/* + * 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.procedure + +import org.apache.paimon.spark.PaimonSparkTestBase + +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.streaming.StreamTest + +class AlterBranchProcedureTest extends PaimonSparkTestBase with StreamTest { + + import testImplicits._ + test("Paimon Procedure: alter schema structure and test $branch syntax.") { + withTempDir { + checkpointDir => + // define a change-log table and test `forEachBatch` api + spark.sql(s""" + |CREATE TABLE T (a INT, b STRING) + |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') + |""".stripMargin) + val location = loadTable("T").location().toString + + val inputData = MemoryStream[(Int, String)] + val stream = inputData + .toDS() + .toDF("a", "b") + .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 a") + try { + // snapshot-1 + inputData.addData((1, "a")) + stream.processAllAvailable() + checkAnswer(query(), Row(1, "a") :: Nil) + + // snapshot-2 + inputData.addData((2, "b")) + stream.processAllAvailable() + checkAnswer(query(), Row(1, "a") :: Row(2, "b") :: Nil) + + // snapshot-3 + inputData.addData((2, "b2")) + stream.processAllAvailable() + checkAnswer(query(), Row(1, "a") :: Row(2, "b2") :: Nil) + + val table = loadTable("T") + val branchManager = table.branchManager() + + // create branch with tag + checkAnswer( + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 's_2', snapshot => 2)"), + Row(true) :: Nil) + checkAnswer( + spark.sql( + "CALL paimon.sys.create_branch(table => 'test.T', branch => 'snapshot_branch', tag => 's_2')"), + Row(true) :: Nil) + assert(branchManager.branchExists("snapshot_branch")) + + spark.sql("INSERT INTO T VALUES (1, 'APPLE'), (2,'DOG'), (2, 'horse')") + spark.sql("ALTER TABLE `T$branch_snapshot_branch` ADD COLUMNS(c INT)") + spark.sql( + "INSERT INTO `T$branch_snapshot_branch` VALUES " + "(1,'cherry', 100), (2,'bird', 200), (3, 'wolf', 400)") + + checkAnswer( + spark.sql("SELECT * FROM T ORDER BY a, b"), + Row(1, "APPLE") :: Row(2, "horse") :: Nil) + checkAnswer( + spark.sql("SELECT * FROM `T$branch_snapshot_branch` ORDER BY a, b,c"), + Row(1, "cherry", 100) :: Row(2, "bird", 200) :: Row(3, "wolf", 400) :: Nil) + assert(branchManager.branchExists("snapshot_branch")) + } + } + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala new file mode 100644 index 000000000000..c1c90251338f --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.procedure + +import org.apache.paimon.spark.PaimonSparkTestBase + +import org.apache.spark.sql.streaming.StreamTest +import org.assertj.core.api.Assertions + +/** Test compact manifest procedure. See [[CompactManifestProcedure]]. */ +class CompactManifestProcedureTest extends PaimonSparkTestBase with StreamTest { + + test("Paimon Procedure: compact manifest") { + spark.sql( + s""" + |CREATE TABLE T (id INT, value STRING, dt STRING, hh INT) + |TBLPROPERTIES ('bucket'='-1', 'write-only'='true', 'compaction.min.file-num'='2', 'compaction.max.file-num'='2') + |PARTITIONED BY (dt, hh) + |""".stripMargin) + + spark.sql(s"INSERT INTO T VALUES (5, '5', '2024-01-02', 0), (6, '6', '2024-01-02', 1)") + spark.sql(s"INSERT OVERWRITE T VALUES (5, '5', '2024-01-02', 0), (6, '6', '2024-01-02', 1)") + spark.sql(s"INSERT OVERWRITE T VALUES (5, '5', '2024-01-02', 0), (6, '6', '2024-01-02', 1)") + spark.sql(s"INSERT OVERWRITE T VALUES (5, '5', '2024-01-02', 0), (6, '6', '2024-01-02', 1)") + + Thread.sleep(10000); + + var rows = spark.sql("SELECT sum(num_deleted_files) FROM `T$manifests`").collectAsList() + Assertions.assertThat(rows.get(0).getLong(0)).isEqualTo(6L) + spark.sql("CALL sys.compact_manifest(table => 'T')") + rows = spark.sql("SELECT sum(num_deleted_files) FROM `T$manifests`").collectAsList() + Assertions.assertThat(rows.get(0).getLong(0)).isEqualTo(0L) + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala index 296680919264..4a4c7ae215df 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala @@ -110,10 +110,20 @@ class CreateAndDeleteTagProcedureTest extends PaimonSparkTestBase with StreamTes spark.sql("SELECT tag_name FROM paimon.test.`T$tags`"), Row("test_tag_1") :: Row("test_tag_2") :: Nil) - // delete test_tag_1 and test_tag_2 + // test rename_tag checkAnswer( spark.sql( - "CALL paimon.sys.delete_tag(table => 'test.T', tag => 'test_tag_1,test_tag_2')"), + "CALL paimon.sys.rename_tag(table => 'test.T', tag => 'test_tag_1', target_tag => 'test_tag_3')"), + Row(true) :: Nil + ) + checkAnswer( + spark.sql("SELECT tag_name FROM paimon.test.`T$tags`"), + Row("test_tag_2") :: Row("test_tag_3") :: Nil) + + // delete test_tag_2 and test_tag_3 + checkAnswer( + spark.sql( + "CALL paimon.sys.delete_tag(table => 'test.T', tag => 'test_tag_2,test_tag_3')"), Row(true) :: Nil) checkAnswer(spark.sql("SELECT tag_name FROM paimon.test.`T$tags`"), Nil) @@ -162,23 +172,53 @@ class CreateAndDeleteTagProcedureTest extends PaimonSparkTestBase with StreamTes "table => 'test.T', tag => 'test_tag', snapshot => 1)"), Row(true) :: Nil) checkAnswer( - spark.sql( - "SELECT count(time_retained) FROM paimon.test.`T$tags` where tag_name = 'test_tag'"), - Row(0) :: Nil) + spark.sql("SELECT count(*) FROM paimon.test.`T$tags` where tag_name = 'test_tag'"), + Row(1) :: Nil) - checkAnswer( + // throw exception "Tag test_tag already exists" + assertThrows[IllegalArgumentException] { spark.sql( "CALL paimon.sys.create_tag(" + - "table => 'test.T', tag => 'test_tag', time_retained => '5 d', snapshot => 1)"), - Row(true) :: Nil) - checkAnswer( - spark.sql( - "SELECT count(time_retained) FROM paimon.test.`T$tags` where tag_name = 'test_tag'"), - Row(1) :: Nil) + "table => 'test.T', tag => 'test_tag', time_retained => '5 d', snapshot => 1)") + } } finally { stream.stop() } } } } + + test("Paimon Procedure: delete tag not failed if tag not exists") { + spark.sql("CREATE TABLE T (id STRING, name STRING) USING PAIMON") + + checkAnswer( + spark.sql("CALL paimon.sys.delete_tag(table => 'test.T', tag => 'test_tag')"), + Row(true) :: Nil) + } + + test("Paimon Procedure: delete multiple tags") { + spark.sql("CREATE TABLE T (id INT, name STRING) USING PAIMON") + spark.sql("insert into T values (1, 'a')") + + // create four tags + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-1')") + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-2')") + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-3')") + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-4')") + checkAnswer(spark.sql("SELECT count(*) FROM paimon.test.`T$tags`"), Row(4) :: Nil) + + // multiple tags with no space + checkAnswer( + spark.sql("CALL paimon.sys.delete_tag(table => 'test.T', tag => 'tag-1,tag-2')"), + Row(true) :: Nil) + checkAnswer( + spark.sql("SELECT tag_name FROM paimon.test.`T$tags`"), + Row("tag-3") :: Row("tag-4") :: Nil) + + // multiple tags with space + checkAnswer( + spark.sql("CALL paimon.sys.delete_tag(table => 'test.T', tag => 'tag-3, tag-4')"), + Row(true) :: Nil) + checkAnswer(spark.sql("SELECT tag_name FROM paimon.test.`T$tags`"), Nil) + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala index db4696047f9d..4561e532f538 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala @@ -474,4 +474,81 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest } } } + + test("Paimon procedure : sorted the expired partitions with max_expires.") { + failAfter(streamingTimeout) { + withTempDir { + checkpointDir => + spark.sql(s""" + |CREATE TABLE T (k STRING, pt STRING, hm STRING) + |TBLPROPERTIES ('primary-key'='k,pt,hm', 'bucket'='1') + | PARTITIONED BY (pt,hm) + |""".stripMargin) + val location = loadTable("T").location().toString + + val inputData = MemoryStream[(String, String, String)] + val stream = inputData + .toDS() + .toDF("k", "pt", "hm") + .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 { + // Show results : There are no expired partitions. + checkAnswer( + spark.sql( + "CALL paimon.sys.expire_partitions(table => 'test.T', expiration_time => '1 d'" + + ", timestamp_formatter => 'yyyy-MM-dd')"), + Row("No expired partitions.") :: Nil + ) + + inputData.addData(("a", "2024-06-02", "02:00")) + stream.processAllAvailable() + inputData.addData(("b", "2024-06-02", "01:00")) + stream.processAllAvailable() + inputData.addData(("d", "2024-06-03", "01:00")) + stream.processAllAvailable() + inputData.addData(("c", "2024-06-01", "01:00")) + stream.processAllAvailable() + // this snapshot never expires. + inputData.addData(("Never-expire", "9999-09-09", "99:99")) + stream.processAllAvailable() + + checkAnswer( + query(), + Row("a", "2024-06-02", "02:00") :: Row("b", "2024-06-02", "01:00") :: Row( + "d", + "2024-06-03", + "01:00") :: Row("c", "2024-06-01", "01:00") :: Row( + "Never-expire", + "9999-09-09", + "99:99") :: Nil + ) + + // sorted result of limited expired partitions. + checkAnswer( + spark.sql( + "CALL paimon.sys.expire_partitions(table => 'test.T'" + + ", expiration_time => '1 d'" + + ", timestamp_formatter => 'yyyy-MM-dd', max_expires => 3)"), + Row("pt=2024-06-01, hm=01:00") :: Row("pt=2024-06-02, hm=01:00") :: Row( + "pt=2024-06-02, hm=02:00") :: Nil + ) + + checkAnswer( + query(), + Row("d", "2024-06-03", "01:00") :: Row("Never-expire", "9999-09-09", "99:99") :: Nil) + } finally { + stream.stop() + } + } + } + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala index 6fcb48d68400..da7be423108a 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala @@ -19,10 +19,14 @@ package org.apache.paimon.spark.procedure import org.apache.paimon.spark.PaimonSparkTestBase +import org.apache.paimon.utils.SnapshotManager 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.{assertThat, assertThatIllegalArgumentException} + +import java.sql.Timestamp class ExpireSnapshotsProcedureTest extends PaimonSparkTestBase with StreamTest { @@ -136,4 +140,29 @@ class ExpireSnapshotsProcedureTest extends PaimonSparkTestBase with StreamTest { } } } + + test("Paimon Procedure: test parameter order_than with string type") { + sql( + "CREATE TABLE T (a INT, b STRING) " + + "TBLPROPERTIES ( 'num-sorted-run.compaction-trigger' = '999' )") + val table = loadTable("T") + val snapshotManager = table.snapshotManager + + // generate 5 snapshot + for (i <- 1 to 5) { + sql(s"INSERT INTO T VALUES ($i, '$i')") + } + checkSnapshots(snapshotManager, 1, 5) + + val timestamp = new Timestamp(snapshotManager.latestSnapshot().timeMillis) + spark.sql( + s"CALL paimon.sys.expire_snapshots(table => 'test.T', older_than => '${timestamp.toString}', max_deletes => 2)") + checkSnapshots(snapshotManager, 3, 5) + } + + def checkSnapshots(sm: SnapshotManager, earliest: Int, latest: Int): Unit = { + assertThat(sm.snapshotCount).isEqualTo(latest - earliest + 1) + assertThat(sm.earliestSnapshotId).isEqualTo(earliest) + assertThat(sm.latestSnapshotId).isEqualTo(latest) + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala new file mode 100644 index 000000000000..65c0f2b9a203 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala @@ -0,0 +1,132 @@ +/* + * 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.procedure; + +import org.apache.paimon.data.Timestamp +import org.apache.paimon.spark.PaimonSparkTestBase +import org.apache.paimon.utils.SnapshotManager + +import org.apache.spark.sql.Row +import org.assertj.core.api.Assertions.assertThat + +class ExpireTagsProcedureTest extends PaimonSparkTestBase { + + test("Paimon procedure: expire tags that reached its timeRetained") { + spark.sql(s""" + |CREATE TABLE T (id STRING, name STRING) + |USING PAIMON + |""".stripMargin) + + val table = loadTable("T") + val snapshotManager = table.snapshotManager() + + // generate 5 snapshots + for (i <- 1 to 5) { + spark.sql(s"INSERT INTO T VALUES($i, '$i')") + } + checkSnapshots(snapshotManager, 1, 5) + + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-1', snapshot => 1)") + spark.sql( + "CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-2', snapshot => 2, time_retained => '1h')") + + // no tags expired + checkAnswer( + spark.sql("CALL paimon.sys.expire_tags(table => 'test.T')"), + Row("No expired tags.") :: Nil) + + spark.sql( + "CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-3', snapshot => 3, time_retained => '1s')") + spark.sql( + "CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-4', snapshot => 4, time_retained => '1s')") + checkAnswer(spark.sql("select count(tag_name) from `T$tags`"), Row(4) :: Nil) + + Thread.sleep(2000) + // tag-3,tag-4 expired + checkAnswer( + spark.sql("CALL paimon.sys.expire_tags(table => 'test.T')"), + Row("tag-3") :: Row("tag-4") :: Nil) + + checkAnswer(spark.sql("select tag_name from `T$tags`"), Row("tag-1") :: Row("tag-2") :: Nil) + } + + test("Paimon procedure: expire tags that createTime less than specified older_than") { + spark.sql(s""" + |CREATE TABLE T (id STRING, name STRING) + |USING PAIMON + |""".stripMargin) + + val table = loadTable("T") + val snapshotManager = table.snapshotManager() + + // generate 5 snapshots + for (i <- 1 to 5) { + spark.sql(s"INSERT INTO T VALUES($i, '$i')") + } + checkSnapshots(snapshotManager, 1, 5) + + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-1', snapshot => 1)") + spark.sql( + "CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-2', snapshot => 2, time_retained => '1d')") + spark.sql( + "CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-3', snapshot => 3, time_retained => '1d')") + spark.sql( + "CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-4', snapshot => 4, time_retained => '1d')") + checkAnswer(spark.sql("select count(tag_name) from `T$tags`"), Row(4) :: Nil) + + // no tags expired + checkAnswer( + spark.sql("CALL paimon.sys.expire_tags(table => 'test.T')"), + Row("No expired tags.") :: Nil) + + // tag-2 as the base older_than time. + // tag-1 expired by its file creation time. + val olderThanTime1 = table.tagManager().tag("tag-2").getTagCreateTime + val timestamp1 = + new java.sql.Timestamp(Timestamp.fromLocalDateTime(olderThanTime1).getMillisecond) + checkAnswer( + spark.sql( + s"CALL paimon.sys.expire_tags(table => 'test.T', older_than => '${timestamp1.toString}')"), + Row("tag-1") :: Nil + ) + + spark.sql( + "CALL paimon.sys.create_tag(table => 'test.T', tag => 'tag-5', snapshot => 5, time_retained => '1s')") + Thread.sleep(1000) + + // tag-4 as the base older_than time. + // tag-2,tag-3,tag-5 expired, tag-5 reached its tagTimeRetained. + val olderThanTime2 = table.tagManager().tag("tag-4").getTagCreateTime + val timestamp2 = + new java.sql.Timestamp(Timestamp.fromLocalDateTime(olderThanTime2).getMillisecond) + checkAnswer( + spark.sql( + s"CALL paimon.sys.expire_tags(table => 'test.T', older_than => '${timestamp2.toString}')"), + Row("tag-2") :: Row("tag-3") :: Row("tag-5") :: Nil + ) + + checkAnswer(spark.sql("select tag_name from `T$tags`"), Row("tag-4") :: Nil) + } + + private def checkSnapshots(sm: SnapshotManager, earliest: Int, latest: Int): Unit = { + assertThat(sm.snapshotCount).isEqualTo(latest - earliest + 1) + assertThat(sm.earliestSnapshotId).isEqualTo(earliest) + assertThat(sm.latestSnapshotId).isEqualTo(latest) + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala new file mode 100644 index 000000000000..332a52d36c4d --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala @@ -0,0 +1,104 @@ +/* + * 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.procedure + +import org.apache.paimon.spark.PaimonHiveTestBase + +import org.apache.spark.sql.Row +class MigrateDatabaseProcedureTest extends PaimonHiveTestBase { + Seq("parquet", "orc", "avro").foreach( + format => { + test(s"Paimon migrate database procedure: migrate $format non-partitioned database") { + withTable("hive_tbl", "hive_tbl1") { + // create hive table + spark.sql(s""" + |CREATE TABLE hive_tbl (id STRING, name STRING, pt STRING) + |USING $format + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE hive_tbl1 (id STRING, name STRING, pt STRING) + |USING $format + |""".stripMargin) + + var rows0 = spark.sql("SHOW CREATE TABLE hive_tbl").collect() + assert(!rows0.apply(0).toString().contains("USING paimon")) + + rows0 = spark.sql("SHOW CREATE TABLE hive_tbl1").collect() + assert(!rows0.apply(0).toString().contains("USING paimon")) + + spark.sql(s"INSERT INTO hive_tbl VALUES ('1', 'a', 'p1'), ('2', 'b', 'p2')") + + spark.sql( + s"CALL sys.migrate_database(source_type => 'hive', database => '$hiveDbName', options => 'file.format=$format')") + + checkAnswer( + spark.sql(s"SELECT * FROM hive_tbl ORDER BY id"), + Row("1", "a", "p1") :: Row("2", "b", "p2") :: Nil) + + var rows1 = spark.sql("SHOW CREATE TABLE hive_tbl").collect() + assert(rows1.apply(0).toString().contains("USING paimon")) + + rows1 = spark.sql("SHOW CREATE TABLE hive_tbl1").collect() + assert(rows1.apply(0).toString().contains("USING paimon")) + + } + } + }) + + Seq("parquet", "orc", "avro").foreach( + format => { + test( + s"Paimon migrate database procedure: migrate $format database with setting parallelism") { + withTable("hive_tbl_01", "hive_tbl_02") { + // create hive table + spark.sql(s""" + |CREATE TABLE hive_tbl_01 (id STRING, name STRING, pt STRING) + |USING $format + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE hive_tbl_02 (id STRING, name STRING, pt STRING) + |USING $format + |""".stripMargin) + + var rows0 = spark.sql("SHOW CREATE TABLE hive_tbl_01").collect() + assert(!rows0.apply(0).toString().contains("USING paimon")) + + rows0 = spark.sql("SHOW CREATE TABLE hive_tbl_02").collect() + assert(!rows0.apply(0).toString().contains("USING paimon")) + + spark.sql(s"INSERT INTO hive_tbl_01 VALUES ('1', 'a', 'p1'), ('2', 'b', 'p2')") + + spark.sql( + s"CALL sys.migrate_database(source_type => 'hive', database => '$hiveDbName', options => 'file.format=$format', parallelism => 6)") + + checkAnswer( + spark.sql(s"SELECT * FROM hive_tbl_01 ORDER BY id"), + Row("1", "a", "p1") :: Row("2", "b", "p2") :: Nil) + + var rows1 = spark.sql("SHOW CREATE TABLE hive_tbl_01").collect() + assert(rows1.apply(0).toString().contains("USING paimon")) + + rows1 = spark.sql("SHOW CREATE TABLE hive_tbl_02").collect() + assert(rows1.apply(0).toString().contains("USING paimon")) + } + } + }) +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala index c414515f1885..d9d73811266d 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala @@ -172,4 +172,51 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*')"), Row(0) :: Nil) } + test("Paimon procedure: remove orphan files with mode") { + spark.sql(s""" + |CREATE TABLE T (id STRING, name STRING) + |USING PAIMON + |TBLPROPERTIES ('primary-key'='id') + |""".stripMargin) + + spark.sql(s"INSERT INTO T VALUES ('1', 'a'), ('2', 'b')") + + val table = loadTable("T") + val fileIO = table.fileIO() + val tablePath = table.location() + + val orphanFile1 = new Path(tablePath, ORPHAN_FILE_1) + val orphanFile2 = new Path(tablePath, ORPHAN_FILE_2) + + fileIO.tryToWriteAtomic(orphanFile1, "a") + Thread.sleep(2000) + fileIO.tryToWriteAtomic(orphanFile2, "b") + + // by default, no file deleted + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + + val orphanFile2ModTime = fileIO.getFileStatus(orphanFile2).getModificationTime + val older_than1 = DateTimeUtils.formatLocalDateTime( + DateTimeUtils.toLocalDateTime( + orphanFile2ModTime - + TimeUnit.SECONDS.toMillis(1)), + 3) + + checkAnswer( + spark.sql( + s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than1', mode => 'diSTributed')"), + Row(1) :: Nil) + + val older_than2 = DateTimeUtils.formatLocalDateTime( + DateTimeUtils.toLocalDateTime(System.currentTimeMillis()), + 3) + + checkAnswer( + spark.sql( + s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than2', mode => 'local')"), + Row(1) :: Nil) + + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + } + } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala new file mode 100644 index 000000000000..5a9280887031 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala @@ -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.spark.procedure + +import org.apache.paimon.spark.PaimonSparkTestBase + +import org.apache.spark.sql.Row + +class ReplaceTagProcedureTest extends PaimonSparkTestBase { + test("Paimon Procedure: replace tag to update tag meta") { + spark.sql(s""" + |CREATE TABLE T (a INT, b STRING) + |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') + |""".stripMargin) + spark.sql("insert into T values(1, 'a')") + spark.sql("insert into T values(2, 'b')") + assertResult(2)(loadTable("T").snapshotManager().snapshotCount()) + + // throw exception "Tag test_tag does not exist" + assertThrows[IllegalArgumentException] { + spark.sql("CALL paimon.sys.replace_tag(table => 'test.T', tag => 'test_tag')") + } + + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 'test_tag')") + checkAnswer( + spark.sql("select tag_name,snapshot_id,time_retained from `T$tags`"), + Row("test_tag", 2, null) :: Nil) + + // replace tag with new time_retained + spark.sql( + "CALL paimon.sys.replace_tag(table => 'test.T', tag => 'test_tag', time_retained => '1 d')") + checkAnswer( + spark.sql("select tag_name,snapshot_id,time_retained from `T$tags`"), + Row("test_tag", 2, "PT24H") :: Nil) + + // replace tag with new snapshot and time_retained + spark.sql( + "CALL paimon.sys.replace_tag(table => 'test.T', tag => 'test_tag', snapshot => 1, time_retained => '2 d')") + checkAnswer( + spark.sql("select tag_name,snapshot_id,time_retained from `T$tags`"), + Row("test_tag", 1, "PT48H") :: Nil) + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala index 945f70ce0e63..457c5ba513ec 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala @@ -93,4 +93,62 @@ class RollbackProcedureTest extends PaimonSparkTestBase with StreamTest { } } } + + test("Paimon Procedure: rollback to timestamp") { + failAfter(streamingTimeout) { + withTempDir { + checkpointDir => + // define a change-log table and test `forEachBatch` api + spark.sql(s""" + |CREATE TABLE T (a INT, b STRING) + |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') + |""".stripMargin) + val location = loadTable("T").location().toString + + val inputData = MemoryStream[(Int, String)] + val stream = inputData + .toDS() + .toDF("a", "b") + .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 a") + + try { + // snapshot-1 + inputData.addData((1, "a")) + stream.processAllAvailable() + checkAnswer(query(), Row(1, "a") :: Nil) + + // snapshot-2 + inputData.addData((2, "b")) + stream.processAllAvailable() + checkAnswer(query(), Row(1, "a") :: Row(2, "b") :: Nil) + + val timestamp = System.currentTimeMillis() + + // snapshot-3 + inputData.addData((2, "b2")) + stream.processAllAvailable() + checkAnswer(query(), Row(1, "a") :: Row(2, "b2") :: Nil) + + // rollback to timestamp + checkAnswer( + spark.sql( + s"CALL paimon.sys.rollback_to_timestamp(table => 'test.T', timestamp => $timestamp)"), + Row("Success roll back to snapshot: 2 .") :: Nil) + checkAnswer(query(), Row(1, "a") :: Row(2, "b") :: Nil) + + } finally { + stream.stop() + } + } + } + } + } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala index be7579b20e8d..238dd039969a 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala @@ -73,6 +73,75 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { Row(2, 0, 2, "{ }")) } + test("Paimon analyze: test statistic system table with snapshot") { + spark.sql(s""" + |CREATE TABLE T (id STRING, name STRING, i INT, l LONG) + |USING PAIMON + |TBLPROPERTIES ('primary-key'='id') + |""".stripMargin) + + spark.sql(s"INSERT INTO T VALUES ('1', 'a', 1, 1)") + spark.sql(s"INSERT INTO T VALUES ('2', 'aaa', 1, 2)") + Assertions.assertEquals(0, spark.sql("select * from `T$statistics`").count()) + + spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS") + + withSQLConf("spark.paimon.scan.timestamp-millis" -> System.currentTimeMillis.toString) { + checkAnswer( + sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), + Row(2, 0, 2, "{ }")) + } + + spark.sql(s"INSERT INTO T VALUES ('3', 'b', 2, 1)") + spark.sql(s"INSERT INTO T VALUES ('4', 'bbb', 3, 2)") + + spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS") + + withSQLConf("spark.paimon.scan.timestamp-millis" -> System.currentTimeMillis.toString) { + checkAnswer( + sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), + Row(5, 0, 4, "{ }")) + } + // create tag + checkAnswer( + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 'test_tag5', snapshot => 5)"), + Row(true) :: Nil) + + checkAnswer( + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => 'test_tag6', snapshot => 6)"), + Row(true) :: Nil) + + withSQLConf("spark.paimon.scan.tag-name" -> "test_tag5") { + checkAnswer( + sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), + Row(2, 0, 2, "{ }")) + } + + withSQLConf("spark.paimon.scan.tag-name" -> "test_tag6") { + checkAnswer( + sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), + Row(5, 0, 4, "{ }")) + } + + withSQLConf("spark.paimon.scan.snapshot-id" -> "3") { + checkAnswer( + sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), + Row(2, 0, 2, "{ }")) + } + + withSQLConf("spark.paimon.scan.snapshot-id" -> "4") { + checkAnswer( + sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), + Row(2, 0, 2, "{ }")) + } + + withSQLConf("spark.paimon.scan.snapshot-id" -> "6") { + checkAnswer( + sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), + Row(5, 0, 4, "{ }")) + } + } + test("Paimon analyze: analyze table without snapshot") { spark.sql(s"CREATE TABLE T (id STRING, name STRING)") spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS") @@ -338,7 +407,7 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS") val stats = getScanStatistic("SELECT * FROM T") - Assertions.assertEquals(2L, stats.rowCount.get.longValue()) + Assertions.assertEquals(2L, stats.rowCount.get.longValue) } test("Paimon analyze: spark use col stats") { @@ -353,7 +422,7 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS FOR ALL COLUMNS") val stats = getScanStatistic("SELECT * FROM T") - Assertions.assertEquals(2L, stats.rowCount.get.longValue()) + Assertions.assertEquals(2L, stats.rowCount.get.longValue) Assertions.assertEquals(if (gteqSpark3_4) 4 else 0, stats.attributeStats.size) } @@ -372,19 +441,19 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { var sql = "SELECT * FROM T WHERE pt < 1" Assertions.assertEquals( if (gteqSpark3_4) 0L else 4L, - getScanStatistic(sql).rowCount.get.longValue()) + getScanStatistic(sql).rowCount.get.longValue) checkAnswer(spark.sql(sql), Nil) // partition push down hit and select without it sql = "SELECT id FROM T WHERE pt < 1" Assertions.assertEquals( if (gteqSpark3_4) 0L else 4L, - getScanStatistic(sql).rowCount.get.longValue()) + getScanStatistic(sql).rowCount.get.longValue) checkAnswer(spark.sql(sql), Nil) // partition push down not hit sql = "SELECT * FROM T WHERE id < 1" - Assertions.assertEquals(4L, getScanStatistic(sql).rowCount.get.longValue()) + Assertions.assertEquals(4L, getScanStatistic(sql).rowCount.get.longValue) checkAnswer(spark.sql(sql), Nil) } @@ -403,10 +472,10 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { // For col type such as char, varchar that don't have min and max, filter estimation on stats has no effect. var sqlText = "SELECT * FROM T WHERE pt < '1'" - Assertions.assertEquals(4L, getScanStatistic(sqlText).rowCount.get.longValue()) + Assertions.assertEquals(4L, getScanStatistic(sqlText).rowCount.get.longValue) sqlText = "SELECT id FROM T WHERE pt < '1'" - Assertions.assertEquals(4L, getScanStatistic(sqlText).rowCount.get.longValue()) + Assertions.assertEquals(4L, getScanStatistic(sqlText).rowCount.get.longValue) } }) } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala index b8009ea8136a..afc70bf9105d 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala @@ -122,7 +122,11 @@ class BucketedTableQueryTest extends PaimonSparkTestBase with AdaptiveSparkPlanH spark.sql( "CREATE TABLE t5 (id INT, c STRING) TBLPROPERTIES ('primary-key' = 'id', 'bucket'='10')") spark.sql("INSERT INTO t5 VALUES (1, 'x1')") - checkAnswerAndShuffleSorts("SELECT * FROM t1 JOIN t5 on t1.id = t5.id", 2, 2) + if (gteqSpark4_0) { + checkAnswerAndShuffleSorts("SELECT * FROM t1 JOIN t5 on t1.id = t5.id", 0, 0) + } else { + checkAnswerAndShuffleSorts("SELECT * FROM t1 JOIN t5 on t1.id = t5.id", 2, 2) + } // one more bucket keys spark.sql( @@ -152,10 +156,10 @@ class BucketedTableQueryTest extends PaimonSparkTestBase with AdaptiveSparkPlanH checkAnswerAndShuffleSorts("SELECT id, max(c) FROM t1 GROUP BY id", 0, 0) checkAnswerAndShuffleSorts("SELECT c, count(*) FROM t1 GROUP BY c", 1, 0) checkAnswerAndShuffleSorts("SELECT c, max(c) FROM t1 GROUP BY c", 1, 2) - checkAnswerAndShuffleSorts("select sum(c) OVER (PARTITION BY id ORDER BY c) from t1", 0, 1) + checkAnswerAndShuffleSorts("select max(c) OVER (PARTITION BY id ORDER BY c) from t1", 0, 1) // TODO: it is a Spark issue for `WindowExec` which would required partition-by + and order-by // without do distinct.. - checkAnswerAndShuffleSorts("select sum(c) OVER (PARTITION BY id ORDER BY id) from t1", 0, 1) + checkAnswerAndShuffleSorts("select max(c) OVER (PARTITION BY id ORDER BY id) from t1", 0, 1) checkAnswerAndShuffleSorts("select sum(id) OVER (PARTITION BY c ORDER BY id) from t1", 1, 1) withSQLConf("spark.sql.requireAllClusterKeysForDistribution" -> "false") { diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala index f99bff2af3ad..cf1a71d51fcc 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala @@ -27,7 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.Row import org.junit.jupiter.api.Assertions -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.time.LocalDateTime abstract class DDLTestBase extends PaimonSparkTestBase { @@ -154,9 +154,7 @@ abstract class DDLTestBase extends PaimonSparkTestBase { val error = intercept[Exception] { spark.sql(s"CREATE TABLE parquet_tbl (id int) USING parquet") }.getMessage - assert( - error.contains( - "SparkCatalog can only create paimon table, but current provider is parquet")) + assert(error.contains("does not support format table")) } } @@ -354,7 +352,7 @@ abstract class DDLTestBase extends PaimonSparkTestBase { .column("ts", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) .column("ts_ntz", DataTypes.TIMESTAMP()) .build - catalog.createTable(identifier, schema, false) + paimonCatalog.createTable(identifier, schema, false) sql( s"INSERT INTO paimon_tbl VALUES (timestamp'2024-01-01 00:00:00', timestamp_ntz'2024-01-01 00:00:00')") @@ -372,7 +370,7 @@ abstract class DDLTestBase extends PaimonSparkTestBase { // Due to previous design, read timestamp ltz type with spark 3.3 and below will cause problems, // skip testing it if (gteqSpark3_4) { - val table = catalog.getTable(identifier) + val table = paimonCatalog.getTable(identifier) val builder = table.newReadBuilder.withProjection(Array[Int](0, 1)) val splits = builder.newScan().plan().splits() builder.newRead @@ -407,7 +405,7 @@ abstract class DDLTestBase extends PaimonSparkTestBase { // Due to previous design, read timestamp ltz type with spark 3.3 and below will cause problems, // skip testing it if (gteqSpark3_4) { - val table = catalog.getTable(identifier) + val table = paimonCatalog.getTable(identifier) val builder = table.newReadBuilder.withProjection(Array[Int](0, 1)) val splits = builder.newScan().plan().splits() builder.newRead @@ -425,7 +423,7 @@ abstract class DDLTestBase extends PaimonSparkTestBase { } } } finally { - catalog.dropTable(identifier, true) + paimonCatalog.dropTable(identifier, true) } } @@ -497,4 +495,55 @@ abstract class DDLTestBase extends PaimonSparkTestBase { }.getMessage assert(error.contains("Unsupported partition transform")) } + + test("Fix partition column generate wrong partition spec") { + Seq(true, false).foreach { + legacyPartName => + withTable("p_t") { + spark.sql(s""" + |CREATE TABLE p_t ( + | id BIGINT, + | c1 STRING + |) using paimon + |PARTITIONED BY (day binary) + |tblproperties('partition.legacy-name'='$legacyPartName'); + |""".stripMargin) + + if (legacyPartName) { + spark.sql("insert into table p_t values(1, 'a', cast('2021' as binary))") + intercept[Exception] { + spark.sql("SELECT * FROM p_t").collect() + } + } else { + spark.sql("insert into table p_t values(1, 'a', cast('2021' as binary))") + checkAnswer(spark.sql("SELECT * FROM p_t"), Row(1, "a", "2021".getBytes)) + val path = spark.sql("SELECT __paimon_file_path FROM p_t").collect() + assert(path.length == 1) + assert(path.head.getString(0).contains("/day=2021/")) + } + } + + withTable("p_t") { + spark.sql(s""" + |CREATE TABLE p_t ( + | id BIGINT, + | c1 STRING + |) using paimon + |PARTITIONED BY (day date) + |tblproperties('partition.legacy-name'='$legacyPartName'); + |""".stripMargin) + + spark.sql("insert into table p_t values(1, 'a', cast('2021-01-01' as date))") + checkAnswer(spark.sql("SELECT * FROM p_t"), Row(1, "a", Date.valueOf("2021-01-01"))) + + val path = spark.sql("SELECT __paimon_file_path FROM p_t").collect() + assert(path.length == 1) + if (legacyPartName) { + assert(path.head.getString(0).contains("/day=18628/")) + } else { + assert(path.head.getString(0).contains("/day=2021-01-01/")) + } + } + } + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index a371f3e31be0..7478f9628cfb 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -18,6 +18,7 @@ package org.apache.paimon.spark.sql +import org.apache.paimon.hive.HiveMetastoreClient import org.apache.paimon.spark.PaimonHiveTestBase import org.apache.paimon.table.FileStoreTable @@ -59,6 +60,126 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } } + test("Paimon DDL with hive catalog: drop partition for paimon table sparkCatalogName") { + Seq(paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + withTempDir { + dBLocation => + withDatabase("paimon_db") { + val comment = "this is a test comment" + spark.sql( + s"CREATE DATABASE paimon_db LOCATION '${dBLocation.getCanonicalPath}' COMMENT '$comment'") + Assertions.assertEquals(getDatabaseLocation("paimon_db"), dBLocation.getCanonicalPath) + Assertions.assertEquals(getDatabaseComment("paimon_db"), comment) + + withTable("paimon_db.paimon_tbl") { + spark.sql(s""" + |CREATE TABLE paimon_db.paimon_tbl (id STRING, name STRING, pt STRING) + |USING PAIMON + |PARTITIONED BY (name, pt) + |TBLPROPERTIES('metastore.partitioned-table' = 'true') + |""".stripMargin) + Assertions.assertEquals( + getTableLocation("paimon_db.paimon_tbl"), + s"${dBLocation.getCanonicalPath}/paimon_tbl") + spark.sql("insert into paimon_db.paimon_tbl select '1', 'n', 'cc'") + spark.sql("insert into paimon_db.paimon_tbl select '1', 'n1', 'aa'") + spark.sql("insert into paimon_db.paimon_tbl select '1', 'n2', 'bb'") + + spark.sql("show partitions paimon_db.paimon_tbl") + checkAnswer( + spark.sql("show partitions paimon_db.paimon_tbl"), + Row("name=n/pt=cc") :: Row("name=n1/pt=aa") :: Row("name=n2/pt=bb") :: Nil) + spark.sql( + "alter table paimon_db.paimon_tbl drop partition (name='n1', `pt`='aa'), partition (name='n2', `pt`='bb')") + spark.sql("show partitions paimon_db.paimon_tbl") + checkAnswer( + spark.sql("show partitions paimon_db.paimon_tbl"), + Row("name=n/pt=cc") :: Nil) + + } + + // disable metastore.partitioned-table + withTable("paimon_db.paimon_tbl2") { + spark.sql(s""" + |CREATE TABLE paimon_db.paimon_tbl2 (id STRING, name STRING, pt STRING) + |USING PAIMON + |PARTITIONED BY (name, pt) + |TBLPROPERTIES('metastore.partitioned-table' = 'false') + |""".stripMargin) + Assertions.assertEquals( + getTableLocation("paimon_db.paimon_tbl2"), + s"${dBLocation.getCanonicalPath}/paimon_tbl2") + spark.sql("insert into paimon_db.paimon_tbl2 select '1', 'n', 'cc'") + spark.sql("insert into paimon_db.paimon_tbl2 select '1', 'n1', 'aa'") + spark.sql("insert into paimon_db.paimon_tbl2 select '1', 'n2', 'bb'") + + spark.sql("show partitions paimon_db.paimon_tbl2") + checkAnswer( + spark.sql("show partitions paimon_db.paimon_tbl2"), + Row("name=n/pt=cc") :: Row("name=n1/pt=aa") :: Row("name=n2/pt=bb") :: Nil) + spark.sql( + "alter table paimon_db.paimon_tbl2 drop partition (name='n1', `pt`='aa'), partition (name='n2', `pt`='bb')") + spark.sql("show partitions paimon_db.paimon_tbl2") + checkAnswer( + spark.sql("show partitions paimon_db.paimon_tbl2"), + Row("name=n/pt=cc") :: Nil) + + } + } + } + } + } + + test("Paimon DDL with hive catalog: create partition for paimon table sparkCatalogName") { + Seq(paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + withTempDir { + dBLocation => + withDatabase("paimon_db") { + val comment = "this is a test comment" + spark.sql( + s"CREATE DATABASE paimon_db LOCATION '${dBLocation.getCanonicalPath}' COMMENT '$comment'") + Assertions.assertEquals(getDatabaseLocation("paimon_db"), dBLocation.getCanonicalPath) + Assertions.assertEquals(getDatabaseComment("paimon_db"), comment) + + withTable("paimon_db.paimon_tbl") { + spark.sql(s""" + |CREATE TABLE paimon_db.paimon_tbl (id STRING, name STRING, pt STRING) + |USING PAIMON + |PARTITIONED BY (name, pt) + |TBLPROPERTIES('metastore.partitioned-table' = 'true') + |""".stripMargin) + Assertions.assertEquals( + getTableLocation("paimon_db.paimon_tbl"), + s"${dBLocation.getCanonicalPath}/paimon_tbl") + spark.sql("insert into paimon_db.paimon_tbl select '1', 'n', 'cc'") + + spark.sql("alter table paimon_db.paimon_tbl add partition(name='cc', `pt`='aa') ") + } + + // disable metastore.partitioned-table + withTable("paimon_db.paimon_tbl2") { + spark.sql(s""" + |CREATE TABLE paimon_db.paimon_tbl2 (id STRING, name STRING, pt STRING) + |USING PAIMON + |PARTITIONED BY (name, pt) + |TBLPROPERTIES('metastore.partitioned-table' = 'false') + |""".stripMargin) + Assertions.assertEquals( + getTableLocation("paimon_db.paimon_tbl2"), + s"${dBLocation.getCanonicalPath}/paimon_tbl2") + spark.sql("insert into paimon_db.paimon_tbl2 select '1', 'n', 'cc'") + + spark.sql("alter table paimon_db.paimon_tbl2 add partition(name='cc', `pt`='aa') ") + } + } + } + } + } + test("Paimon DDL with hive catalog: create database with props") { Seq(sparkCatalogName, paimonHiveCatalogName).foreach { catalogName => @@ -131,6 +252,50 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } } + test("Paimon DDL with hive catalog: sync partitions to HMS") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + val dbName = "default" + val tblName = "t" + spark.sql(s"USE $catalogName.$dbName") + withTable(tblName) { + spark.sql(s""" + |CREATE TABLE $tblName (id INT, pt INT) + |USING PAIMON + |TBLPROPERTIES ('metastore.partitioned-table' = 'true') + |PARTITIONED BY (pt) + |""".stripMargin) + + val metastoreClient = loadTable(dbName, tblName) + .catalogEnvironment() + .metastoreClientFactory() + .create() + .asInstanceOf[HiveMetastoreClient] + .client() + + spark.sql(s"INSERT INTO $tblName VALUES (1, 1), (2, 2), (3, 3)") + // check partitions in paimon + checkAnswer( + spark.sql(s"show partitions $tblName"), + Seq(Row("pt=1"), Row("pt=2"), Row("pt=3"))) + // check partitions in HMS + assert(metastoreClient.listPartitions(dbName, tblName, 100).size() == 3) + + spark.sql(s"INSERT INTO $tblName VALUES (4, 3), (5, 4)") + checkAnswer( + spark.sql(s"show partitions $tblName"), + Seq(Row("pt=1"), Row("pt=2"), Row("pt=3"), Row("pt=4"))) + assert(metastoreClient.listPartitions(dbName, tblName, 100).size() == 4) + + spark.sql(s"ALTER TABLE $tblName DROP PARTITION (pt=1)") + checkAnswer( + spark.sql(s"show partitions $tblName"), + Seq(Row("pt=2"), Row("pt=3"), Row("pt=4"))) + assert(metastoreClient.listPartitions(dbName, tblName, 100).size() == 3) + } + } + } + def getDatabaseLocation(dbName: String): String = { spark .sql(s"DESC DATABASE $dbName") diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala index 3f6e81da018c..a0a94afacfb9 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala @@ -473,7 +473,11 @@ class DataFrameWriteTest extends PaimonSparkTestBase { .writeTo("t") .overwrite($"c1" === ($"c2" + 1)) }.getMessage - assert(msg3.contains("cannot translate expression to source filter")) + if (gteqSpark4_0) { + assert(msg3.contains("Table does not support overwrite by expression")) + } else { + assert(msg3.contains("cannot translate expression to source filter")) + } val msg4 = intercept[Exception] { spark diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala index 3adccaa7c157..e944429e4218 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala @@ -21,20 +21,78 @@ package org.apache.paimon.spark.sql import org.apache.paimon.data.BinaryRow import org.apache.paimon.deletionvectors.{DeletionVector, DeletionVectorsMaintainer} import org.apache.paimon.fs.Path -import org.apache.paimon.spark.PaimonSparkTestBase +import org.apache.paimon.spark.{PaimonSparkTestBase, PaimonSplitScan} +import org.apache.paimon.spark.schema.PaimonMetadataColumn import org.apache.paimon.table.FileStoreTable +import org.apache.spark.paimon.Utils import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation} import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.util.QueryExecutionListener import org.junit.jupiter.api.Assertions import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.util.Random -class DeletionVectorTest extends PaimonSparkTestBase { +class DeletionVectorTest extends PaimonSparkTestBase with AdaptiveSparkPlanHelper { import testImplicits._ + private def runAndCheckSplitScan(query: String): Unit = { + val batchScans = new ArrayBuffer[(DataSourceV2Relation, BatchScanExec)]() + val listener = new QueryExecutionListener { + override def onFailure(f: String, qe: QueryExecution, e: Exception): Unit = {} + + private def isValidSplitScan(scan: BatchScanExec): Boolean = { + if (!scan.scan.isInstanceOf[PaimonSplitScan]) { + return false + } + val splitScan = scan.scan.asInstanceOf[PaimonSplitScan] + assert(splitScan.table.primaryKeys().isEmpty) + splitScan.coreOptions.deletionVectorsEnabled() && + scan.output.exists( + attr => PaimonMetadataColumn.SUPPORTED_METADATA_COLUMNS.contains(attr.name)) + } + + private def appendScan(qe: QueryExecution, plan: SparkPlan): Unit = { + plan match { + case memory: InMemoryTableScanExec => + foreach(memory.relation.cachedPlan)(p => appendScan(qe, p)) + case scan: BatchScanExec if isValidSplitScan(scan) => + val logicalScan = qe.analyzed.find(_.isInstanceOf[DataSourceV2Relation]) + assert(logicalScan.isDefined) + batchScans.append((logicalScan.get.asInstanceOf[DataSourceV2Relation], scan)) + case _ => + } + } + + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + foreach(qe.executedPlan)(p => appendScan(qe, p)) + } + } + spark.listenerManager.register(listener) + + try { + val df = spark.sql(query) + df.collect() + Utils.waitUntilEventEmpty(df.sparkSession) + assert(batchScans.nonEmpty, query) + assert( + batchScans.forall { + case (logicalScan, scan) => + logicalScan.output.size > scan.output.size + }, + batchScans) + } finally { + spark.listenerManager.unregister(listener) + } + } + bucketModes.foreach { bucket => test(s"Paimon DeletionVector: merge into with bucket = $bucket") { @@ -59,21 +117,21 @@ class DeletionVectorTest extends PaimonSparkTestBase { val table = loadTable("target") val dvMaintainerFactory = new DeletionVectorsMaintainer.Factory(table.store().newIndexFileHandler()) - spark.sql(s""" - |MERGE INTO target - |USING source - |ON target.a = source.a - |WHEN MATCHED AND target.a = 5 THEN - |UPDATE SET b = source.b + target.b - |WHEN MATCHED AND source.c > 'c2' THEN - |UPDATE SET * - |WHEN MATCHED THEN - |DELETE - |WHEN NOT MATCHED AND c > 'c9' THEN - |INSERT (a, b, c) VALUES (a, b * 1.1, c) - |WHEN NOT MATCHED THEN - |INSERT * - |""".stripMargin) + runAndCheckSplitScan(s""" + |MERGE INTO target + |USING source + |ON target.a = source.a + |WHEN MATCHED AND target.a = 5 THEN + |UPDATE SET b = source.b + target.b + |WHEN MATCHED AND source.c > 'c2' THEN + |UPDATE SET * + |WHEN MATCHED THEN + |DELETE + |WHEN NOT MATCHED AND c > 'c9' THEN + |INSERT (a, b, c) VALUES (a, b * 1.1, c) + |WHEN NOT MATCHED THEN + |INSERT * + |""".stripMargin) checkAnswer( spark.sql("SELECT * FROM target ORDER BY a, b"), @@ -116,7 +174,7 @@ class DeletionVectorTest extends PaimonSparkTestBase { val cond1 = "id = 2" val rowMetaInfo1 = getFilePathAndRowIndex(cond1) - spark.sql(s"UPDATE T SET name = 'b_2' WHERE $cond1") + runAndCheckSplitScan(s"UPDATE T SET name = 'b_2' WHERE $cond1") checkAnswer( spark.sql(s"SELECT * from T ORDER BY id"), Row(1, "a") :: Row(2, "b_2") :: Row(3, "c") :: Nil) @@ -136,12 +194,12 @@ class DeletionVectorTest extends PaimonSparkTestBase { Assertions.assertTrue(deletionVectors2 == deletionVectors3) val cond2 = "id % 2 = 1" - spark.sql(s"UPDATE T SET name = concat(name, '_2') WHERE $cond2") + runAndCheckSplitScan(s"UPDATE T SET name = concat(name, '_2') WHERE $cond2") checkAnswer( spark.sql(s"SELECT * from T ORDER BY id"), Row(1, "a_2") :: Row(2, "b_2") :: Row(3, "c_2") :: Row(4, "d") :: Row(5, "e_2") :: Nil) - spark.sql(s"UPDATE T SET name = '_all'") + runAndCheckSplitScan("UPDATE T SET name = '_all'") checkAnswer( spark.sql(s"SELECT * from T ORDER BY id"), Row(1, "_all") :: Row(2, "_all") :: Row(3, "_all") :: Row(4, "_all") :: Row( @@ -188,7 +246,7 @@ class DeletionVectorTest extends PaimonSparkTestBase { val cond1 = "id = 2" val rowMetaInfo1 = getFilePathAndRowIndex(cond1) - spark.sql(s"UPDATE T SET name = 'b_2' WHERE $cond1") + runAndCheckSplitScan(s"UPDATE T SET name = 'b_2' WHERE $cond1") checkAnswer( spark.sql(s"SELECT * from T ORDER BY id"), Row(1, "a", "2024") :: Row(2, "b_2", "2024") :: Row(3, "c", "2025") :: Row( @@ -209,7 +267,7 @@ class DeletionVectorTest extends PaimonSparkTestBase { val cond2 = "pt = '2025'" val rowMetaInfo2 = rowMetaInfo1 ++ getFilePathAndRowIndex(cond2) - spark.sql(s"UPDATE T SET name = concat(name, '_2') WHERE $cond2") + runAndCheckSplitScan(s"UPDATE T SET name = concat(name, '_2') WHERE $cond2") checkAnswer( spark.sql(s"SELECT * from T ORDER BY id"), Row(1, "a", "2024") :: Row(2, "b_2", "2024") :: Row(3, "c_2", "2025") :: Row( @@ -273,7 +331,7 @@ class DeletionVectorTest extends PaimonSparkTestBase { val cond1 = "id = 2" val rowMetaInfo1 = getFilePathAndRowIndex(cond1) - spark.sql(s"DELETE FROM T WHERE $cond1") + runAndCheckSplitScan(s"DELETE FROM T WHERE $cond1") checkAnswer(spark.sql(s"SELECT * from T ORDER BY id"), Row(1, "a") :: Nil) val deletionVectors2 = getAllLatestDeletionVectors(table, dvMaintainerFactory) Assertions.assertEquals(1, deletionVectors2.size) @@ -291,7 +349,7 @@ class DeletionVectorTest extends PaimonSparkTestBase { Assertions.assertTrue(deletionVectors2 == deletionVectors3) val cond2 = "id % 2 = 1" - spark.sql(s"DELETE FROM T WHERE $cond2") + runAndCheckSplitScan(s"DELETE FROM T WHERE $cond2") checkAnswer(spark.sql(s"SELECT * from T ORDER BY id"), Row(2, "bb") :: Row(4, "d") :: Nil) spark.sql("CALL sys.compact('T')") @@ -337,7 +395,7 @@ class DeletionVectorTest extends PaimonSparkTestBase { val cond1 = "id = 2" val rowMetaInfo1 = getFilePathAndRowIndex(cond1) - spark.sql(s"DELETE FROM T WHERE $cond1") + runAndCheckSplitScan(s"DELETE FROM T WHERE $cond1") checkAnswer( spark.sql(s"SELECT * from T ORDER BY id"), Row(1, "a", "2024") :: Row(3, "c", "2025") :: Row(4, "d", "2025") :: Nil) @@ -351,7 +409,7 @@ class DeletionVectorTest extends PaimonSparkTestBase { val cond2 = "id = 3" val rowMetaInfo2 = rowMetaInfo1 ++ getFilePathAndRowIndex(cond2) - spark.sql(s"DELETE FROM T WHERE $cond2") + runAndCheckSplitScan(s"DELETE FROM T WHERE $cond2") checkAnswer( spark.sql(s"SELECT * from T ORDER BY id"), Row(1, "a", "2024") :: Row(4, "d", "2025") :: Nil) diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala index 6a29181659e0..2c46af0aed2f 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala @@ -142,4 +142,45 @@ trait MergeIntoNotMatchedBySourceTest extends PaimonSparkTestBase with PaimonTab ) } } + + test(s"Paimon MergeInto: multiple clauses with not matched by source with alias") { + withTable("source", "target") { + + Seq((1, 100, "c11"), (3, 300, "c33"), (5, 500, "c55"), (7, 700, "c77"), (9, 900, "c99")) + .toDF("a", "b", "c") + .createOrReplaceTempView("source") + + createTable("target", "a INT, b INT, c STRING", Seq("a")) + spark.sql( + "INSERT INTO target values (1, 10, 'c1'), (2, 20, 'c2'), (3, 30, 'c3'), (4, 40, 'c4'), (5, 50, 'c5')") + + spark.sql(s""" + |MERGE INTO target t + |USING source s + |ON t.a = s.a + |WHEN MATCHED AND t.a = 5 THEN + |UPDATE SET t.b = s.b + t.b + |WHEN MATCHED AND s.c > 'c2' THEN + |UPDATE SET * + |WHEN MATCHED THEN + |DELETE + |WHEN NOT MATCHED AND s.c > 'c9' THEN + |INSERT (t.a, t.b, t.c) VALUES (s.a, s.b * 1.1, s.c) + |WHEN NOT MATCHED THEN + |INSERT * + |WHEN NOT MATCHED BY SOURCE AND t.a = 2 THEN + |UPDATE SET t.b = t.b * 10 + |WHEN NOT MATCHED BY SOURCE THEN + |DELETE + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM target ORDER BY a, b"), + Row(2, 200, "c2") :: Row(3, 300, "c33") :: Row(5, 550, "c5") :: Row(7, 700, "c77") :: Row( + 9, + 990, + "c99") :: Nil + ) + } + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala index 1a4eae51d007..8973ea93d8a0 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala @@ -497,6 +497,7 @@ abstract class MergeIntoTableTestBase extends PaimonSparkTestBase with PaimonTab Row(1, 10, Row("x1", "y")) :: Row(2, 20, Row("x", "y")) :: Nil) } } + test(s"Paimon MergeInto: update on source eq target condition") { withTable("source", "target") { Seq((1, 100, "c11"), (3, 300, "c33")).toDF("a", "b", "c").createOrReplaceTempView("source") @@ -517,6 +518,43 @@ abstract class MergeIntoTableTestBase extends PaimonSparkTestBase with PaimonTab Row(1, 100, "c11") :: Row(2, 20, "c2") :: Nil) } } + + test(s"Paimon MergeInto: merge into with alias") { + withTable("source", "target") { + + Seq((1, 100, "c11"), (3, 300, "c33"), (5, 500, "c55"), (7, 700, "c77"), (9, 900, "c99")) + .toDF("a", "b", "c") + .createOrReplaceTempView("source") + + createTable("target", "a INT, b INT, c STRING", Seq("a")) + spark.sql( + "INSERT INTO target values (1, 10, 'c1'), (2, 20, 'c2'), (3, 30, 'c3'), (4, 40, 'c4'), (5, 50, 'c5')") + + spark.sql(s""" + |MERGE INTO target t + |USING source s + |ON t.a = s.a + |WHEN MATCHED AND t.a = 5 THEN + |UPDATE SET t.b = s.b + t.b + |WHEN MATCHED AND s.c > 'c2' THEN + |UPDATE SET * + |WHEN MATCHED THEN + |DELETE + |WHEN NOT MATCHED AND s.c > 'c9' THEN + |INSERT (t.a, t.b, t.c) VALUES (s.a, s.b * 1.1, s.c) + |WHEN NOT MATCHED THEN + |INSERT * + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM target ORDER BY a, b"), + Row(2, 20, "c2") :: Row(3, 300, "c33") :: Row(4, 40, "c4") :: Row(5, 550, "c5") :: Row( + 7, + 700, + "c77") :: Row(9, 990, "c99") :: Nil + ) + } + } } trait MergeIntoPrimaryKeyTableTest extends PaimonSparkTestBase with PaimonPrimaryKeyTable { diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala index d35ac1d709c3..f74d6959b9f1 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala @@ -76,4 +76,130 @@ class PaimonOptionTest extends PaimonSparkTestBase { } } } + + test("Paimon Table Options: query one table with sql conf and table options") { + sql("CREATE TABLE T (id INT)") + sql("INSERT INTO T VALUES 1") + sql("INSERT INTO T VALUES 2") + checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1) :: Row(2) :: Nil) + val table = loadTable("T") + + // query with global options + withSQLConf("spark.paimon.scan.snapshot-id" -> "1") { + checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1)) + checkAnswer(spark.read.format("paimon").load(table.location().toString), Row(1)) + } + + // query with table options + withSQLConf("spark.paimon.*.*.T.scan.snapshot-id" -> "1") { + checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1)) + checkAnswer(spark.read.format("paimon").load(table.location().toString), Row(1)) + } + + // query with both global and table options + withSQLConf( + "spark.paimon.scan.snapshot-id" -> "1", + "spark.paimon.*.*.T.scan.snapshot-id" -> "2") { + checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1) :: Row(2) :: Nil) + checkAnswer( + spark.read.format("paimon").load(table.location().toString), + Row(1) :: Row(2) :: Nil) + } + } + + test("Paimon Table Options: query multiple tables with sql conf and table options") { + sql("CREATE TABLE T1 (id INT)") + sql("INSERT INTO T1 VALUES 1") + sql("INSERT INTO T1 VALUES 2") + + sql("CREATE TABLE T2 (id INT)") + sql("INSERT INTO T2 VALUES 1") + sql("INSERT INTO T2 VALUES 2") + checkAnswer( + sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), + Row(1, 1) :: Row(2, 2) :: Nil) + val table1 = loadTable("T1") + val table2 = loadTable("T1") + + // query with global options + withSQLConf("spark.paimon.scan.snapshot-id" -> "1") { + checkAnswer(sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), Row(1, 1)) + checkAnswer( + spark.read + .format("paimon") + .load(table1.location().toString) + .join(spark.read.format("paimon").load(table2.location().toString), "id"), + Row(1) + ) + } + + // query with table options + withSQLConf("spark.paimon.*.*.*.scan.snapshot-id" -> "1") { + checkAnswer(sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), Row(1, 1)) + checkAnswer( + spark.read + .format("paimon") + .load(table1.location().toString) + .join(spark.read.format("paimon").load(table2.location().toString), "id"), + Row(1) + ) + } + + // query with both global and table options + withSQLConf( + "spark.paimon.scan.snapshot-id" -> "1", + "spark.paimon.*.*.*.scan.snapshot-id" -> "2") { + checkAnswer( + sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), + Row(1, 1) :: Row(2, 2) :: Nil) + checkAnswer( + spark.read + .format("paimon") + .load(table1.location().toString) + .join(spark.read.format("paimon").load(table2.location().toString), "id"), + Row(1) :: Row(2) :: Nil + ) + } + + withSQLConf( + "spark.paimon.*.*.T1.scan.snapshot-id" -> "1", + "spark.paimon.*.*.T2.scan.snapshot-id" -> "1") { + checkAnswer(sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), Row(1, 1)) + checkAnswer( + spark.read + .format("paimon") + .load(table1.location().toString) + .join(spark.read.format("paimon").load(table2.location().toString), "id"), + Row(1) + ) + } + + withSQLConf( + "spark.paimon.*.*.T1.scan.snapshot-id" -> "1", + "spark.paimon.*.*.T2.scan.snapshot-id" -> "2") { + checkAnswer(sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), Row(1, 1)) + checkAnswer( + spark.read + .format("paimon") + .load(table1.location().toString) + .join(spark.read.format("paimon").load(table2.location().toString), "id"), + Row(1) + ) + } + + withSQLConf( + "spark.paimon.*.*.T1.scan.snapshot-id" -> "2", + "spark.paimon.*.*.T2.scan.snapshot-id" -> "2") { + checkAnswer( + sql("SELECT * FROM T1 join T2 on T1.id = T2.id ORDER BY T1.id"), + Row(1, 1) :: Row(2, 2) :: Nil) + checkAnswer( + spark.read + .format("paimon") + .load(table1.location().toString) + .join(spark.read.format("paimon").load(table2.location().toString), "id"), + Row(1) :: Row(2) :: Nil + ) + } + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala index c55ed876d6b1..ba314e3afa81 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala @@ -86,6 +86,22 @@ class PaimonPushDownTest extends PaimonSparkTestBase { checkAnswer(spark.sql(q), Row(1, "a", "p1") :: Row(2, "b", "p1") :: Row(3, "c", "p2") :: Nil) } + test("Paimon pushDown: limit for append-only tables with deletion vector") { + withTable("dv_test") { + spark.sql( + """ + |CREATE TABLE dv_test (c1 INT, c2 STRING) + |TBLPROPERTIES ('deletion-vectors.enabled' = 'true', 'source.split.target-size' = '1') + |""".stripMargin) + + spark.sql("insert into table dv_test values(1, 'a'),(2, 'b'),(3, 'c')") + assert(spark.sql("select * from dv_test limit 2").count() == 2) + + spark.sql("delete from dv_test where c1 = 1") + assert(spark.sql("select * from dv_test limit 2").count() == 2) + } + } + test("Paimon pushDown: limit for append-only tables") { spark.sql(s""" |CREATE TABLE T (a INT, b STRING, c STRING) @@ -113,7 +129,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { Assertions.assertEquals(1, spark.sql("SELECT * FROM T LIMIT 1").count()) } - test("Paimon pushDown: limit for change-log tables") { + test("Paimon pushDown: limit for primary key table") { spark.sql(s""" |CREATE TABLE T (a INT, b STRING, c STRING) |TBLPROPERTIES ('primary-key'='a') @@ -125,8 +141,72 @@ class PaimonPushDownTest extends PaimonSparkTestBase { val scanBuilder = getScanBuilder() Assertions.assertTrue(scanBuilder.isInstanceOf[SupportsPushDownLimit]) - // Tables with primary keys can't support the push-down limit. + // Case 1: All dataSplits is rawConvertible. + val dataSplitsWithoutLimit = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits + Assertions.assertEquals(4, dataSplitsWithoutLimit.length) + // All dataSplits is rawConvertible. + dataSplitsWithoutLimit.foreach( + splits => { + Assertions.assertTrue(splits.asInstanceOf[DataSplit].rawConvertible()) + }) + + // It still returns false even it can push down limit. Assertions.assertFalse(scanBuilder.asInstanceOf[SupportsPushDownLimit].pushLimit(1)) + val dataSplitsWithLimit = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits + Assertions.assertEquals(1, dataSplitsWithLimit.length) + Assertions.assertEquals(1, spark.sql("SELECT * FROM T LIMIT 1").count()) + + Assertions.assertFalse(scanBuilder.asInstanceOf[SupportsPushDownLimit].pushLimit(2)) + val dataSplitsWithLimit1 = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits + Assertions.assertEquals(2, dataSplitsWithLimit1.length) + Assertions.assertEquals(2, spark.sql("SELECT * FROM T LIMIT 2").count()) + + // Case 2: Update 2 rawConvertible dataSplits to convert to nonRawConvertible. + spark.sql("INSERT INTO T VALUES (1, 'a2', '11'), (2, 'b2', '22')") + val scanBuilder2 = getScanBuilder() + val dataSplitsWithoutLimit2 = scanBuilder2.build().asInstanceOf[PaimonScan].getOriginSplits + Assertions.assertEquals(4, dataSplitsWithoutLimit2.length) + // Now, we have 4 dataSplits, and 2 dataSplit is nonRawConvertible, 2 dataSplit is rawConvertible. + Assertions.assertEquals( + 2, + dataSplitsWithoutLimit2 + .filter( + split => { + split.asInstanceOf[DataSplit].rawConvertible() + }) + .length) + + // Return 2 dataSplits. + Assertions.assertFalse(scanBuilder2.asInstanceOf[SupportsPushDownLimit].pushLimit(2)) + val dataSplitsWithLimit2 = scanBuilder2.build().asInstanceOf[PaimonScan].getOriginSplits + Assertions.assertEquals(2, dataSplitsWithLimit2.length) + Assertions.assertEquals(2, spark.sql("SELECT * FROM T LIMIT 2").count()) + + // 2 dataSplits cannot meet the limit requirement, so need to scan all dataSplits. + Assertions.assertFalse(scanBuilder2.asInstanceOf[SupportsPushDownLimit].pushLimit(3)) + val dataSplitsWithLimit22 = scanBuilder2.build().asInstanceOf[PaimonScan].getOriginSplits + // Need to scan all dataSplits. + Assertions.assertEquals(4, dataSplitsWithLimit22.length) + Assertions.assertEquals(3, spark.sql("SELECT * FROM T LIMIT 3").count()) + + // Case 3: Update the remaining 2 rawConvertible dataSplits to make all dataSplits is nonRawConvertible. + spark.sql("INSERT INTO T VALUES (3, 'c', '11'), (4, 'd', '22')") + val scanBuilder3 = getScanBuilder() + val dataSplitsWithoutLimit3 = scanBuilder3.build().asInstanceOf[PaimonScan].getOriginSplits + Assertions.assertEquals(4, dataSplitsWithoutLimit3.length) + + // All dataSplits is nonRawConvertible. + dataSplitsWithoutLimit3.foreach( + splits => { + Assertions.assertFalse(splits.asInstanceOf[DataSplit].rawConvertible()) + }) + + Assertions.assertFalse(scanBuilder3.asInstanceOf[SupportsPushDownLimit].pushLimit(1)) + val dataSplitsWithLimit3 = scanBuilder3.build().asInstanceOf[PaimonScan].getOriginSplits + // Need to scan all dataSplits. + Assertions.assertEquals(4, dataSplitsWithLimit3.length) + Assertions.assertEquals(1, spark.sql("SELECT * FROM T LIMIT 1").count()) + } test("Paimon pushDown: runtime filter") { diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala index c1814096fb7d..beea19c35e92 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala @@ -218,6 +218,156 @@ class PaimonQueryTest extends PaimonSparkTestBase { } } + test("Paimon Query: query nested cols") { + withPk.foreach { + hasPk => + fileFormats.foreach { + fileFormat => + bucketModes.foreach { + bucketMode => + val key = if (hasPk) "primary-key" else "bucket-key" + val props = if (bucketMode != -1) { + s", '$key'='name', 'bucket' = '$bucketMode' " + } else { + "" + } + withTable("students") { + sql(s""" + |CREATE TABLE students ( + | name STRING, + | course STRUCT, + | teacher STRUCT>, + | m MAP>, + | l ARRAY>, + | s STRUCT>>>>, + | m2 MAP, STRUCT> + |) USING paimon + |TBLPROPERTIES ('file.format'='$fileFormat' $props) + |""".stripMargin) + + sql(s""" + |INSERT INTO students VALUES ( + | 'Alice', + | STRUCT('Math', 85.0), + | STRUCT('John', STRUCT('Street 1', 'City 1')), + | MAP('k1', STRUCT('s1', 1, 1.0), 'k2', STRUCT('s11', 11, 11.0)), + | ARRAY(STRUCT('s1', 1, 1.0), STRUCT('s11', 11, 11.0)), + | STRUCT('a', MAP('k1', STRUCT('s1', 1, ARRAY(STRUCT('s1', 1, 1.0))), 'k3', STRUCT('s11', 11, ARRAY(STRUCT('s11', 11, 11.0))))), + | MAP(STRUCT('k1', 1, 1.0), STRUCT('s1', 1, 1.0), STRUCT('k2', 1, 1.0), STRUCT('s11', 11, 11.0))) + |""".stripMargin) + + sql( + s""" + |INSERT INTO students VALUES ( + | 'Bob', + | STRUCT('Biology', 92.0), + | STRUCT('Jane', STRUCT('Street 2', 'City 2')), + | MAP('k2', STRUCT('s2', 2, 2.0)), + | ARRAY(STRUCT('s2', 2, 2.0), STRUCT('s22', 22, 22.0)), + | STRUCT('b', MAP('k2', STRUCT('s22', 22, ARRAY(STRUCT('s22', 22, 22.0))))), + | MAP(STRUCT('k2', 2, 2.0), STRUCT('s22', 22, 22.0))) + |""".stripMargin) + + sql(s""" + |INSERT INTO students VALUES ( + | 'Cathy', + | STRUCT('History', 95.0), + | STRUCT('Jane', STRUCT('Street 3', 'City 3')), + | MAP('k1', STRUCT('s3', 3, 3.0), 'k2', STRUCT('s33', 33, 33.0)), + | ARRAY(STRUCT('s3', 3, 3.0)), + | STRUCT('c', MAP('k1', STRUCT('s3', 3, ARRAY(STRUCT('s3', 3, 3.0))), 'k2', STRUCT('s33', 33, ARRAY(STRUCT('s33', 33, 33.0))))), + | MAP(STRUCT('k1', 3, 3.0), STRUCT('s3', 3, 3.0), STRUCT('k2', 3, 3.0), STRUCT('s33', 33, 33.0))) + |""".stripMargin) + + // Since Spark 4.0, when `spark.sql.ansi.enabled` is `true` and `array[i]` does not exist, an exception + // will be thrown instead of returning null. Here, just disabled it and return null for test. + withSQLConf("spark.sql.ansi.enabled" -> "false") { + checkAnswer( + sql(s""" + |SELECT + | course.grade, name, teacher.address, course.course_name, + | m['k1'].d, m['k1'].s, + | l[1].d, l[1].s, + | s.s2['k2'].a[0].i, + | map_keys(m2).i + |FROM students ORDER BY name + |""".stripMargin), + Seq( + Row( + 85.0, + "Alice", + Row("Street 1", "City 1"), + "Math", + 1.0, + "s1", + 11.0, + "s11", + null, + Seq(1, 1)), + Row( + 92.0, + "Bob", + Row("Street 2", "City 2"), + "Biology", + null, + null, + 22.0, + "s22", + 22, + Seq(2)), + Row( + 95.0, + "Cathy", + Row("Street 3", "City 3"), + "History", + 3.0, + "s3", + null, + null, + 33, + Seq(3, 3)) + ) + ) + } + } + } + } + } + } + + test("Paimon Query: query nested array cols") { + withTable("t") { + sql(""" + |CREATE TABLE t ( + | id INT, + | array_array ARRAY>, + | array_map ARRAY>, + | array_struct ARRAY> + |) + |""".stripMargin) + + sql(""" + |INSERT INTO t VALUES ( + | 1, + | array(array(1, 3)), + | array(map('k1', 'v1'), map('k2', 'v2')), + | array(struct(1, 's1'), struct(2, 's2')) + |) + |""".stripMargin) + + checkAnswer( + sql(s""" + |SELECT + | array_array[0][1], + | array_map[0]['k1'], + | array_struct[1].s2 + |FROM t + |""".stripMargin), + Row(3, "v1", "s2") + ) + } + } + private def getAllFiles( tableName: String, partitions: Seq[String], diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala new file mode 100644 index 000000000000..64baf6232fd8 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala @@ -0,0 +1,84 @@ +/* + * 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 + +import org.apache.paimon.spark.PaimonSparkTestBase + +import org.apache.spark.sql.Row + +class PaimonSystemTableTest extends PaimonSparkTestBase { + + test("system table: sort tags table") { + spark.sql(s""" + |CREATE TABLE T (id STRING, name STRING) + |USING PAIMON + |""".stripMargin) + + spark.sql(s"INSERT INTO T VALUES(1, 'a')") + + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => '2024-10-02')") + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => '2024-10-01')") + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => '2024-10-04')") + spark.sql("CALL paimon.sys.create_tag(table => 'test.T', tag => '2024-10-03')") + + checkAnswer( + spark.sql("select tag_name from `T$tags`"), + Row("2024-10-01") :: Row("2024-10-02") :: Row("2024-10-03") :: Row("2024-10-04") :: Nil) + } + + test("system table: sort partitions table") { + spark.sql(s""" + |CREATE TABLE T (a INT, b STRING,dt STRING,hh STRING) + |PARTITIONED BY (dt, hh) + |TBLPROPERTIES ('primary-key'='a,dt,hh', 'bucket' = '3') + |""".stripMargin) + + spark.sql("INSERT INTO T VALUES(1, 'a', '2024-10-10', '01')") + spark.sql("INSERT INTO T VALUES(3, 'c', '2024-10-10', '23')") + spark.sql("INSERT INTO T VALUES(2, 'b', '2024-10-10', '12')") + spark.sql("INSERT INTO T VALUES(5, 'f', '2024-10-09', '02')") + spark.sql("INSERT INTO T VALUES(4, 'd', '2024-10-09', '01')") + + checkAnswer(spark.sql("select count(*) from `T$partitions`"), Row(5) :: Nil) + checkAnswer( + spark.sql("select partition from `T$partitions`"), + Row("[2024-10-09, 01]") :: Row("[2024-10-09, 02]") :: Row("[2024-10-10, 01]") :: Row( + "[2024-10-10, 12]") :: Row("[2024-10-10, 23]") :: Nil + ) + } + + test("system table: sort buckets table") { + spark.sql(s""" + |CREATE TABLE T (a INT, b STRING,dt STRING,hh STRING) + |PARTITIONED BY (dt, hh) + |TBLPROPERTIES ('primary-key'='a,dt,hh', 'bucket' = '3') + |""".stripMargin) + + spark.sql("INSERT INTO T VALUES(1, 'a', '2024-10-10', '01')") + spark.sql("INSERT INTO T VALUES(2, 'b', '2024-10-10', '01')") + spark.sql("INSERT INTO T VALUES(3, 'c', '2024-10-10', '01')") + spark.sql("INSERT INTO T VALUES(4, 'd', '2024-10-10', '01')") + spark.sql("INSERT INTO T VALUES(5, 'f', '2024-10-10', '01')") + + checkAnswer(spark.sql("select count(*) from `T$partitions`"), Row(1) :: Nil) + checkAnswer( + spark.sql("select partition,bucket from `T$buckets`"), + Row("[2024-10-10, 01]", 0) :: Row("[2024-10-10, 01]", 1) :: Row("[2024-10-10, 01]", 2) :: Nil) + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala new file mode 100644 index 000000000000..5ad687b4da0f --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala @@ -0,0 +1,145 @@ +/* + * 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 + +import org.apache.paimon.spark.PaimonSparkTestBase + +import org.apache.spark.sql.Row + +abstract class PaimonTagDdlTestBase extends PaimonSparkTestBase { + test("Tag ddl: show tags syntax") { + spark.sql("""CREATE TABLE T (id INT, name STRING) + |USING PAIMON + |TBLPROPERTIES ('primary-key'='id')""".stripMargin) + + spark.sql("insert into T values(1, 'a')") + + spark.sql("alter table T create tag `2024-10-12`") + spark.sql("alter table T create tag `2024-10-11`") + spark.sql("alter table T create tag `2024-10-13`") + + checkAnswer( + spark.sql("show tags T"), + Row("2024-10-11") :: Row("2024-10-12") :: Row("2024-10-13") :: Nil) + } + + test("Tag ddl: alter table t crete tag syntax") { + spark.sql("""CREATE TABLE T (id INT, name STRING) + |USING PAIMON + |TBLPROPERTIES ('primary-key'='id')""".stripMargin) + + spark.sql("insert into T values(1, 'a')") + spark.sql("insert into T values(2, 'b')") + spark.sql("insert into T values(3, 'c')") + val table = loadTable("T") + assertResult(3)(table.snapshotManager().snapshotCount()) + + spark.sql("alter table T create tag `tag-1`") + spark.sql("alter table T create tag `tag-2` RETAIN 2 DAYS") + spark.sql("alter table T create tag `tag-3` as of version 1") + spark.sql("alter table T create tag `tag-4` as of version 2 RETAIN 3 HOURS") + assertResult(4)(spark.sql("show tags T").count()) + + checkAnswer( + spark.sql("select tag_name,snapshot_id,time_retained from `T$tags`"), + Row("tag-1", 3, null) :: Row("tag-2", 3, "PT48H") :: Row("tag-3", 1, null) :: Row( + "tag-4", + 2, + "PT3H") :: Nil + ) + + // not update tag with 'if not exists' syntax + spark.sql("alter table T create tag if not exists `tag-1` RETAIN 10 HOURS") + checkAnswer( + spark.sql("select tag_name,snapshot_id,time_retained from `T$tags` where tag_name='tag-1'"), + Row("tag-1", 3, null)) + } + + test("Tag ddl: alter table t create or replace tag syntax") { + spark.sql("""CREATE TABLE T (id INT, name STRING) + |USING PAIMON + |TBLPROPERTIES ('primary-key'='id')""".stripMargin) + + spark.sql("insert into T values(1, 'a')") + spark.sql("insert into T values(2, 'b')") + assertResult(2)(loadTable("T").snapshotManager().snapshotCount()) + + // test 'replace' syntax + spark.sql("alter table T create tag `tag-1` as of version 1") + spark.sql("alter table T replace tag `tag-1` as of version 2 RETAIN 1 HOURS") + checkAnswer( + spark.sql("select tag_name,snapshot_id,time_retained from `T$tags`"), + Row("tag-1", 2, "PT1H") :: Nil) + + // test 'create or replace' syntax + // tag-2 not exist, create it + spark.sql("alter table T create or replace tag `tag-2` as of version 1") + checkAnswer( + spark.sql("select tag_name,snapshot_id,time_retained from `T$tags` where tag_name = 'tag-2'"), + Row("tag-2", 1, null) :: Nil) + // tag-2 exists, replace it + spark.sql("alter table T create or replace tag `tag-2` as of version 2 RETAIN 1 HOURS") + checkAnswer( + spark.sql("select tag_name,snapshot_id,time_retained from `T$tags` where tag_name = 'tag-2'"), + Row("tag-2", 2, "PT1H") :: Nil) + } + + test("Tag ddl: alter table t delete tag syntax") { + spark.sql("""CREATE TABLE T (id INT, name STRING) + |USING PAIMON + |TBLPROPERTIES ('primary-key'='id')""".stripMargin) + + spark.sql("insert into T values(1, 'a')") + assertResult(1)(loadTable("T").snapshotManager().snapshotCount()) + + spark.sql("alter table T create tag `2024-10-12`") + spark.sql("alter table T create tag `2024-10-15`") + spark.sql("alter table T create tag `2024-10-13`") + spark.sql("alter table T create tag `2024-10-14`") + checkAnswer( + spark.sql("show tags T"), + Row("2024-10-12") :: Row("2024-10-13") :: Row("2024-10-14") :: Row("2024-10-15") :: Nil) + + spark.sql("alter table T delete tag `2024-10-12`") + checkAnswer( + spark.sql("show tags T"), + Row("2024-10-13") :: Row("2024-10-14") :: Row("2024-10-15") :: Nil) + + spark.sql("alter table T delete tag `2024-10-13, 2024-10-14`") + checkAnswer(spark.sql("show tags T"), Row("2024-10-15") :: Nil) + + spark.sql("alter table T delete tag if EXISTS `2024-10-18`") + checkAnswer(spark.sql("show tags T"), Row("2024-10-15") :: Nil) + } + + test("Tag ddl: alter table t rename tag syntax") { + spark.sql("""CREATE TABLE T (id INT, name STRING) + |USING PAIMON + |TBLPROPERTIES ('primary-key'='id')""".stripMargin) + + spark.sql("insert into T values(1, 'a')") + assertResult(1)(loadTable("T").snapshotManager().snapshotCount()) + + spark.sql("alter table T create tag `tag-1`") + checkAnswer(spark.sql("show tags T"), Row("tag-1")) + + spark.sql("alter table T rename tag `tag-1` to `tag-2`") + checkAnswer(spark.sql("show tags T"), Row("tag-2")) + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala new file mode 100644 index 000000000000..501e7bfb4a51 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala @@ -0,0 +1,138 @@ +/* + * 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 + +import org.apache.paimon.spark.PaimonSparkTestBase + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.plans.logical.Aggregate +import org.apache.spark.sql.execution.LocalTableScanExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.aggregate.BaseAggregateExec + +class PushDownAggregatesTest extends PaimonSparkTestBase with AdaptiveSparkPlanHelper { + + private def runAndCheckAggregate( + query: String, + expectedRows: Seq[Row], + expectedNumAggregates: Int): Unit = { + val df = spark.sql(query) + checkAnswer(df, expectedRows) + assert(df.schema.names.toSeq == df.queryExecution.executedPlan.output.map(_.name)) + assert(df.queryExecution.analyzed.find(_.isInstanceOf[Aggregate]).isDefined) + val numAggregates = collect(df.queryExecution.executedPlan) { + case agg: BaseAggregateExec => agg + }.size + assert(numAggregates == expectedNumAggregates, query) + if (numAggregates == 0) { + assert(collect(df.queryExecution.executedPlan) { + case scan: LocalTableScanExec => scan + // For compatibility with Spark3.x + case e if e.getClass.getName == "org.apache.spark.sql.execution.EmptyRelationExec" => e + }.size == 1) + } + } + + test("Push down aggregate - append table") { + withTable("T") { + spark.sql("CREATE TABLE T (c1 INT, c2 STRING) PARTITIONED BY(day STRING)") + + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(0) :: Nil, 0) + // This query does not contain aggregate due to AQE optimize it to empty relation. + runAndCheckAggregate("SELECT COUNT(*) FROM T GROUP BY c1", Nil, 0) + runAndCheckAggregate("SELECT COUNT(c1) FROM T", Row(0) :: Nil, 2) + runAndCheckAggregate("SELECT COUNT(*), COUNT(c1) FROM T", Row(0, 0) :: Nil, 2) + runAndCheckAggregate("SELECT COUNT(*), COUNT(*) + 1 FROM T", Row(0, 1) :: Nil, 0) + runAndCheckAggregate("SELECT COUNT(*) as c FROM T WHERE day='a'", Row(0) :: Nil, 0) + runAndCheckAggregate("SELECT COUNT(*) FROM T WHERE c1=1", Row(0) :: Nil, 2) + runAndCheckAggregate("SELECT COUNT(*) FROM T WHERE day='a' and c1=1", Row(0) :: Nil, 2) + + spark.sql( + "INSERT INTO T VALUES(1, 'x', 'a'), (2, 'x', 'a'), (3, 'x', 'b'), (3, 'x', 'c'), (null, 'x', 'a')") + + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(5) :: Nil, 0) + runAndCheckAggregate( + "SELECT COUNT(*) FROM T GROUP BY c1", + Row(1) :: Row(1) :: Row(1) :: Row(2) :: Nil, + 2) + runAndCheckAggregate("SELECT COUNT(c1) FROM T", Row(4) :: Nil, 2) + runAndCheckAggregate("SELECT COUNT(*), COUNT(c1) FROM T", Row(5, 4) :: Nil, 2) + runAndCheckAggregate("SELECT COUNT(*), COUNT(*) + 1 FROM T", Row(5, 6) :: Nil, 0) + runAndCheckAggregate("SELECT COUNT(*) as c FROM T WHERE day='a'", Row(3) :: Nil, 0) + runAndCheckAggregate("SELECT COUNT(*) FROM T WHERE c1=1", Row(1) :: Nil, 2) + runAndCheckAggregate("SELECT COUNT(*) FROM T WHERE day='a' and c1=1", Row(1) :: Nil, 2) + } + } + + test("Push down aggregate - group by partition column") { + withTable("T") { + spark.sql("CREATE TABLE T (c1 INT) PARTITIONED BY(day STRING, hour INT)") + + runAndCheckAggregate("SELECT COUNT(*) FROM T GROUP BY day", Nil, 0) + runAndCheckAggregate("SELECT day, COUNT(*) as c FROM T GROUP BY day, hour", Nil, 0) + runAndCheckAggregate("SELECT day, COUNT(*), hour FROM T GROUP BY day, hour", Nil, 0) + runAndCheckAggregate( + "SELECT day, COUNT(*), hour FROM T WHERE day='x' GROUP BY day, hour", + Nil, + 0) + // This query does not contain aggregate due to AQE optimize it to empty relation. + runAndCheckAggregate("SELECT day, COUNT(*) FROM T GROUP BY c1, day", Nil, 0) + + spark.sql( + "INSERT INTO T VALUES(1, 'x', 1), (2, 'x', 1), (3, 'x', 2), (3, 'x', 3), (null, 'y', null)") + + runAndCheckAggregate("SELECT COUNT(*) FROM T GROUP BY day", Row(1) :: Row(4) :: Nil, 0) + runAndCheckAggregate( + "SELECT day, COUNT(*) as c FROM T GROUP BY day, hour", + Row("x", 1) :: Row("x", 1) :: Row("x", 2) :: Row("y", 1) :: Nil, + 0) + runAndCheckAggregate( + "SELECT day, COUNT(*), hour FROM T GROUP BY day, hour", + Row("x", 1, 2) :: Row("y", 1, null) :: Row("x", 2, 1) :: Row("x", 1, 3) :: Nil, + 0) + runAndCheckAggregate( + "SELECT day, COUNT(*), hour FROM T WHERE day='x' GROUP BY day, hour", + Row("x", 1, 2) :: Row("x", 1, 3) :: Row("x", 2, 1) :: Nil, + 0) + runAndCheckAggregate( + "SELECT day, COUNT(*) FROM T GROUP BY c1, day", + Row("x", 1) :: Row("x", 1) :: Row("x", 2) :: Row("y", 1) :: Nil, + 2) + } + } + + test("Push down aggregate - primary table") { + withTable("T") { + spark.sql("CREATE TABLE T (c1 INT, c2 STRING) TBLPROPERTIES ('primary-key' = 'c1')") + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(0) :: Nil, 2) + spark.sql("INSERT INTO T VALUES(1, 'x'), (2, 'x'), (3, 'x'), (3, 'x')") + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 2) + } + } + + test("Push down aggregate - enable deletion vector") { + withTable("T") { + spark.sql( + "CREATE TABLE T (c1 INT, c2 STRING) TBLPROPERTIES('deletion-vectors.enabled' = 'true')") + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(0) :: Nil, 2) + spark.sql("INSERT INTO T VALUES(1, 'x'), (2, 'x'), (3, 'x'), (3, 'x')") + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(4) :: Nil, 2) + } + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala index fed73ba0f9e2..647b4cfdcab7 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala @@ -28,4 +28,6 @@ trait SparkVersionSupport { lazy val gteqSpark3_4: Boolean = sparkVersion >= "3.4" lazy val gteqSpark3_5: Boolean = sparkVersion >= "3.5" + + lazy val gteqSpark4_0: Boolean = sparkVersion >= "4.0" } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala index d7222a1970a2..5beaea59548f 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala @@ -328,7 +328,6 @@ abstract class UpdateTableTestBase extends PaimonSparkTestBase { "INSERT INTO T VALUES (1, map(1, 'a'), '11'), (2, map(2, 'b'), '22'), (3, map(3, 'c'), '33')") assertThatThrownBy(() => spark.sql("UPDATE T SET m.key = 11 WHERE id = 1")) - .hasMessageContaining("Unsupported update expression") spark.sql("UPDATE T SET m = map(11, 'a_new') WHERE id = 1") val rows = spark.sql("SELECT * FROM T ORDER BY id").collectAsList() diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/spark/paimon/Utils.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/spark/paimon/Utils.scala index 1a899f500153..5ea2dd861e19 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/spark/paimon/Utils.scala +++ b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/spark/paimon/Utils.scala @@ -18,6 +18,7 @@ package org.apache.spark.paimon +import org.apache.spark.sql.SparkSession import org.apache.spark.util.{Utils => SparkUtils} import java.io.File @@ -29,4 +30,7 @@ object Utils { def createTempDir: File = SparkUtils.createTempDir() + def waitUntilEventEmpty(spark: SparkSession): Unit = { + spark.sparkContext.listenerBus.waitUntilEmpty() + } } diff --git a/paimon-spark/paimon-spark3-common/pom.xml b/paimon-spark/paimon-spark3-common/pom.xml new file mode 100644 index 000000000000..03d29ea05b3a --- /dev/null +++ b/paimon-spark/paimon-spark3-common/pom.xml @@ -0,0 +1,47 @@ + + + + 4.0.0 + + + org.apache.paimon + paimon-spark + 1.0-SNAPSHOT + + + jar + + paimon-spark3-common + Paimon : Spark3 : Common + + + ${paimon-spark-common.spark.version} + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + + + \ No newline at end of file diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala new file mode 100644 index 000000000000..13ade3f3c5ac --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala @@ -0,0 +1,72 @@ +/* + * 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.spark.sql.paimon + +import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.catalyst.{InternalRow => SparkInternalRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.parser.{ParserInterface => SparkParserInterface} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate => SparkAggregate} +import org.apache.spark.sql.catalyst.util.{ArrayData => SparkArrayData} +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog => SparkTableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.StructType + +import java.util.{Map => JMap} + +/** Shims for Spark 3.x in [[org.apache.spark.sql]]. */ +object shims { + + /** In [[org.apache.spark.sql.catalyst]]. */ + + abstract class ParserInterface extends SparkParserInterface { + val delegate: SparkParserInterface + } + + abstract class ArrayData extends SparkArrayData {} + + abstract class InternalRow extends SparkInternalRow {} + + object Aggregate { + def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean = { + SparkAggregate.supportsHashAggregate(aggregateBufferAttributes) + } + } + + /** In [[org.apache.spark.sql.connector]]. */ + + def createTable( + tableCatalog: SparkTableCatalog, + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: JMap[String, String]): Table = { + tableCatalog.createTable(ident, schema, partitions, properties) + } + + /** In [[org.apache.spark.sql.internal]]. */ + + object ExpressionUtils { + def column(expr: Expression): Column = new Column(expr) + + def convertToExpression(spark: SparkSession, column: Column): Expression = column.expr + } +} diff --git a/paimon-spark/paimon-spark4-common/pom.xml b/paimon-spark/paimon-spark4-common/pom.xml new file mode 100644 index 000000000000..dcc5b370d59a --- /dev/null +++ b/paimon-spark/paimon-spark4-common/pom.xml @@ -0,0 +1,47 @@ + + + + 4.0.0 + + + org.apache.paimon + paimon-spark + 1.0-SNAPSHOT + + + jar + + paimon-spark4-common + Paimon : Spark4 : Common + + + ${paimon-spark-common.spark.version} + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + + + \ No newline at end of file diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala new file mode 100644 index 000000000000..ee6c9ad35857 --- /dev/null +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala @@ -0,0 +1,86 @@ +/* + * 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.spark.sql.paimon + +import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.catalyst.{InternalRow => SparkInternalRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.parser.{CompoundBody, ParserInterface => SparkParserInterface} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate => SparkAggregate} +import org.apache.spark.sql.catalyst.util.{ArrayData => SparkArrayData} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, Table, TableCatalog => SparkTableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.internal.{ExpressionUtils => SparkExpressionUtils} +import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.types.VariantVal + +import java.util.{Map => JMap} + +/** Shims for Spark 4.x in [[org.apache.spark.sql]]. */ +object shims { + + /** In [[org.apache.spark.sql.catalyst]]. */ + + abstract class ParserInterface extends SparkParserInterface { + val delegate: SparkParserInterface + + def parseScript(sqlScriptText: String): CompoundBody = delegate.parseScript(sqlScriptText) + } + + abstract class ArrayData extends SparkArrayData { + def getVariant(ordinal: Int): VariantVal = throw new UnsupportedOperationException + } + + abstract class InternalRow extends SparkInternalRow { + override def getVariant(i: Int): VariantVal = throw new UnsupportedOperationException + } + + object Aggregate { + def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean = { + SparkAggregate.supportsHashAggregate(aggregateBufferAttributes, groupingExpression) + } + } + + /** In [[org.apache.spark.sql.connector]]. */ + + def createTable( + tableCatalog: SparkTableCatalog, + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: JMap[String, String]): Table = { + tableCatalog.createTable( + ident, + CatalogV2Util.structTypeToV2Columns(schema), + partitions, + properties) + } + + /** In [[org.apache.spark.sql.internal]]. */ + + object ExpressionUtils { + def column(expr: Expression): Column = SparkExpressionUtils.column(expr) + + def convertToExpression(spark: SparkSession, column: Column): Expression = { + spark.expression(column) + } + } +} diff --git a/paimon-spark/pom.xml b/paimon-spark/pom.xml index efc34d67f11e..aac73baa5fec 100644 --- a/paimon-spark/pom.xml +++ b/paimon-spark/pom.xml @@ -23,8 +23,8 @@ under the License. 4.0.0 - paimon-parent org.apache.paimon + paimon-parent 1.0-SNAPSHOT @@ -39,21 +39,217 @@ under the License. paimon-spark-common - paimon-spark-3.5 - paimon-spark-3.4 - paimon-spark-3.3 - paimon-spark-3.1 - paimon-spark-3.2 + + + + org.apache.spark + spark-sql_${scala.binary.version} + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.apache.logging.log4j + log4j-slf4j2-impl + + + org.apache.orc + orc-core + + + org.apache.orc + orc-mapreduce + + + org.apache.parquet + parquet-column + + + org.apache.parquet + parquet-hadoop + + + + + + org.apache.spark + spark-core_${scala.binary.version} + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + org.apache.logging.log4j + log4j-slf4j2-impl + + + org.apache.orc + orc-core + + + org.apache.orc + orc-mapreduce + + + org.apache.parquet + parquet-column + + + com.google.protobuf + protobuf-java + + + + + + org.apache.paimon + paimon-bundle + ${project.version} + + + + + + org.apache.spark + spark-sql_${scala.binary.version} + tests + test + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.apache.logging.log4j + log4j-slf4j2-impl + + + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + tests + test + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + + + org.apache.spark + spark-core_${scala.binary.version} + tests + test + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.apache.logging.log4j + log4j-slf4j2-impl + + + + + + org.apache.spark + spark-hive_${scala.binary.version} + test + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + + + + + org.scala-lang + scala-library + ${scala.version} + + + + org.scala-lang + scala-reflect + ${scala.version} + + + + org.scala-lang + scala-compiler + ${scala.version} + + + + + + org.scalatest + scalatest_${scala.binary.version} + 3.1.0 + test + + org.apache.paimon - paimon-bundle + paimon-common ${project.version} + test-jar + test - + + org.apache.paimon + paimon-hive-common + ${project.version} + test + + + + org.apache.paimon + paimon-hive-common + ${project.version} + tests + test-jar + test + org.apache.paimon @@ -91,4 +287,90 @@ under the License. test + + + + + + net.alchim31.maven + scala-maven-plugin + ${scala-maven-plugin.version} + + + + scala-compile-first + process-resources + + add-source + compile + + + + + + scala-test-compile + process-test-resources + + testCompile + + + + + ${scala.version} + false + + -nobootcp + -target:jvm-${target.java.version} + + + + + + + org.scalatest + scalatest-maven-plugin + ${scalatest-maven-plugin.version} + + ${project.build.directory}/surefire-reports + . + -ea -Xmx4g -Xss4m -XX:MaxMetaspaceSize=2g -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraJavaTestArgs} -Dio.netty.tryReflectionSetAccessible=true + PaimonTestSuite.txt + + once + true + + + + test + + test + + + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + net.alchim31.maven + scala-maven-plugin + + + + org.scalatest + scalatest-maven-plugin + + + diff --git a/pom.xml b/pom.xml index 1f0d47860124..24f292331950 100644 --- a/pom.xml +++ b/pom.xml @@ -80,8 +80,12 @@ under the License. 4.1.100.Final 4.9.3 2.8.5 - 2.12.15 2.12 + 2.12 + 2.12.15 + 2.13.14 + ${scala212.version} + ${scala212.version} 1.1.8.4 0.27 1.8.0 @@ -97,6 +101,10 @@ under the License. 1C true 1.19.1 + 1.6.1 + 1.13.1 + 1.9.2 + 3.19.6 @@ -130,11 +138,13 @@ under the License. --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED + --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false + -Dio.netty.tryReflectionSetAccessible=true @@ -338,6 +348,55 @@ under the License. + + + spark3 + + paimon-spark/paimon-spark3-common + paimon-spark/paimon-spark-3.5 + paimon-spark/paimon-spark-3.4 + paimon-spark/paimon-spark-3.3 + paimon-spark/paimon-spark-3.2 + + + 2.12 + ${scala212.version} + 3.5.3 + paimon-spark3-common + + 3.3 + 3.3.0 + + + true + + spark3 + + + + + + spark4 + + paimon-spark/paimon-spark4-common + paimon-spark/paimon-spark-4.0 + + + 17 + 4.13.1 + 2.13 + ${scala213.version} + 4.0.0-preview2 + paimon-spark4-common + 4.0 + 4.0.0-preview2 + + + + spark4 + + + @@ -493,7 +552,7 @@ under the License. true - -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:+UseG1GC + -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:+UseG1GC ${extraJavaTestArgs} @@ -533,7 +592,6 @@ under the License. - org.apache.maven.plugins maven-enforcer-plugin @@ -615,10 +673,10 @@ under the License. - org.apache.flink:flink-table-planner_${scala.binary.version} + org.apache.flink:flink-table-planner_${flink.scala.binary.version} - org.apache.flink:flink-table-planner_${scala.binary.version}:*:*:test + org.apache.flink:flink-table-planner_${flink.scala.binary.version}:*:*:test Direct dependencies on flink-table-planner are not allowed.