diff --git a/.github/workflows/publish_snapshot.yml b/.github/workflows/publish_snapshot.yml index 7d370814da947..c7f97cab991f9 100644 --- a/.github/workflows/publish_snapshot.yml +++ b/.github/workflows/publish_snapshot.yml @@ -64,6 +64,6 @@ jobs: echo "$ASF_PASSWORD" >> $tmp_settings echo "" >> $tmp_settings - mvn --settings $tmp_settings clean deploy -Dgpg.skip -Drat.skip -DskipTests -Papache-release + mvn --settings $tmp_settings clean deploy -Dgpg.skip -Drat.skip -DskipTests -Papache-release,spark3 rm $tmp_settings diff --git a/.github/workflows/utitcase-jdk11.yml b/.github/workflows/utitcase-jdk11.yml index f1d7c25cbe44a..878ce5f968981 100644 --- a/.github/workflows/utitcase-jdk11.yml +++ b/.github/workflows/utitcase-jdk11.yml @@ -53,7 +53,7 @@ jobs: jvm_timezone=$(random_timezone) echo "JVM timezone is set to $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 + for suffix in 3.5 3.4 3.3 3.2 ut; do test_modules+="!org.apache.paimon:paimon-spark-${suffix}," done test_modules="${test_modules%,}" diff --git a/.github/workflows/utitcase-spark-3.x.yml b/.github/workflows/utitcase-spark-3.x.yml index 5edcfe49007ac..2d3df5f4d0053 100644 --- a/.github/workflows/utitcase-spark-3.x.yml +++ b/.github/workflows/utitcase-spark-3.x.yml @@ -54,7 +54,7 @@ jobs: 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 + for suffix in ut 3.5 3.4 3.3 3.2; do test_modules+="org.apache.paimon:paimon-spark-${suffix}," done test_modules="${test_modules%,}" diff --git a/.github/workflows/utitcase-spark-4.x.yml b/.github/workflows/utitcase-spark-4.x.yml index 7fbac23dda4fc..c58fd7c03be27 100644 --- a/.github/workflows/utitcase-spark-4.x.yml +++ b/.github/workflows/utitcase-spark-4.x.yml @@ -54,7 +54,7 @@ jobs: jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" test_modules="" - for suffix in common_2.13 4.0; do + for suffix in ut 4.0; do test_modules+="org.apache.paimon:paimon-spark-${suffix}," done test_modules="${test_modules%,}" diff --git a/.github/workflows/utitcase.yml b/.github/workflows/utitcase.yml index bde67cb4c2039..8aa33f5b8218e 100644 --- a/.github/workflows/utitcase.yml +++ b/.github/workflows/utitcase.yml @@ -54,7 +54,7 @@ jobs: jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" test_modules="!paimon-e2e-tests," - for suffix in 3.5 3.4 3.3 3.2 common_2.12; do + for suffix in 3.5 3.4 3.3 3.2 ut; do test_modules+="!org.apache.paimon:paimon-spark-${suffix}," done test_modules="${test_modules%,}" diff --git a/docs/content/cdc-ingestion/kafka-cdc.md b/docs/content/cdc-ingestion/kafka-cdc.md index f57260275ea83..26a5be3409424 100644 --- a/docs/content/cdc-ingestion/kafka-cdc.md +++ b/docs/content/cdc-ingestion/kafka-cdc.md @@ -198,7 +198,10 @@ To use this feature through `flink run`, run the following shell command. kafka_sync_database --warehouse \ --database \ + [--table_mapping =] \ + [--table_prefix_db ] \ [--table_prefix ] \ + [--table_suffix_db ] \ [--table_suffix ] \ [--including_tables ] \ [--excluding_tables ] \ diff --git a/docs/content/concepts/data-types.md b/docs/content/concepts/data-types.md new file mode 100644 index 0000000000000..b33dcd428399b --- /dev/null +++ b/docs/content/concepts/data-types.md @@ -0,0 +1,179 @@ +--- +title: "Data Types" +weight: 7 +type: docs +aliases: +- /concepts/data-types.html +--- + + +# Data Types + +A data type describes the logical type of a value in the table ecosystem. It can be used to declare input and/or output types of operations. + +All data types supported by Paimon are as follows: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
DataTypeDescription
BOOLEANData type of a boolean with a (possibly) three-valued logic of TRUE, FALSE, and UNKNOWN.
CHAR
+ CHAR(n) +
Data type of a fixed-length character string.

+ The type can be declared using CHAR(n) where n is the number of code points. n must have a value between 1 and 2,147,483,647 (both inclusive). If no length is specified, n is equal to 1. +
VARCHAR
+ VARCHAR(n)

+ STRING +
Data type of a variable-length character string.

+ The type can be declared using VARCHAR(n) where n is the maximum number of code points. n must have a value between 1 and 2,147,483,647 (both inclusive). If no length is specified, n is equal to 1.

+ STRING is a synonym for VARCHAR(2147483647). +
BINARY
+ BINARY(n)

+
Data type of a fixed-length binary string (=a sequence of bytes).

+ The type can be declared using BINARY(n) where n is the number of bytes. n must have a value between 1 and 2,147,483,647 (both inclusive). If no length is specified, n is equal to 1. +
VARBINARY
+ VARBINARY(n)

+ BYTES +
Data type of a variable-length binary string (=a sequence of bytes).

+ The type can be declared using VARBINARY(n) where n is the maximum number of bytes. n must have a value between 1 and 2,147,483,647 (both inclusive). If no length is specified, n is equal to 1.

+ BYTES is a synonym for VARBINARY(2147483647). +
DECIMAL
+ DECIMAL(p)
+ DECIMAL(p, s) +
Data type of a decimal number with fixed precision and scale.

+ The type can be declared using DECIMAL(p, s) where p is the number of digits in a number (precision) and s is the number of digits to the right of the decimal point in a number (scale). p must have a value between 1 and 38 (both inclusive). s must have a value between 0 and p (both inclusive). The default value for p is 10. The default value for s is 0. +
TINYINTData type of a 1-byte signed integer with values from -128 to 127.
SMALLINTData type of a 2-byte signed integer with values from -32,768 to 32,767.
INTData type of a 4-byte signed integer with values from -2,147,483,648 to 2,147,483,647.
BIGINTData type of an 8-byte signed integer with values from -9,223,372,036,854,775,808 to 9,223,372,036,854,775,807.
FLOATData type of a 4-byte single precision floating point number.

+ Compared to the SQL standard, the type does not take parameters. +
DOUBLEData type of an 8-byte double precision floating point number.
DATEData type of a date consisting of year-month-day with values ranging from 0000-01-01 to 9999-12-31.

+ Compared to the SQL standard, the range starts at year 0000. +
TIME
+ TIME(p) +
Data type of a time without time zone consisting of hour:minute:second[.fractional] with up to nanosecond precision and values ranging from 00:00:00.000000000 to 23:59:59.999999999.

+ The type can be declared using TIME(p) where p is the number of digits of fractional seconds (precision). p must have a value between 0 and 9 (both inclusive). If no precision is specified, p is equal to 0. +
TIMESTAMP
+ TIMESTAMP(p) +
Data type of a timestamp without time zone consisting of year-month-day hour:minute:second[.fractional] with up to nanosecond precision and values ranging from 0000-01-01 00:00:00.000000000 to 9999-12-31 23:59:59.999999999.

+ The type can be declared using TIMESTAMP(p) where p is the number of digits of fractional seconds (precision). p must have a value between 0 and 9 (both inclusive). If no precision is specified, p is equal to 6. +
TIMESTAMP WITH TIME ZONE
+ TIMESTAMP(p) WITH TIME ZONE +
Data type of a timestamp with time zone consisting of year-month-day hour:minute:second[.fractional] zone with up to nanosecond precision and values ranging from 0000-01-01 00:00:00.000000000 +14:59 to 9999-12-31 23:59:59.999999999 -14:59.

+ This type fills the gap between time zone free and time zone mandatory timestamp types by allowing the interpretation of UTC timestamps according to the configured session time zone. A conversion from and to int describes the number of seconds since epoch. A conversion from and to long describes the number of milliseconds since epoch. +
ARRAY<t>Data type of an array of elements with same subtype.

+ Compared to the SQL standard, the maximum cardinality of an array cannot be specified but is fixed at 2,147,483,647. Also, any valid type is supported as a subtype.

+ The type can be declared using ARRAY<t> where t is the data type of the contained elements. +
MAP<kt, vt>Data type of an associative array that maps keys (including NULL) to values (including NULL). A map cannot contain duplicate keys; each key can map to at most one value.

+ There is no restriction of element types; it is the responsibility of the user to ensure uniqueness.

+ The type can be declared using MAP<kt, vt> where kt is the data type of the key elements and vt is the data type of the value elements. +
MULTISET<t>Data type of a multiset (=bag). Unlike a set, it allows for multiple instances for each of its elements with a common subtype. Each unique value (including NULL) is mapped to some multiplicity.

+ There is no restriction of element types; it is the responsibility of the user to ensure uniqueness.

+ The type can be declared using MULTISET<t> where t is the data type of the contained elements. +
ROW<n0 t0, n1 t1, ...>
+ ROW<n0 t0 'd0', n1 t1 'd1', ...> +
Data type of a sequence of fields.

+ A field consists of a field name, field type, and an optional description. The most specific type of a row of a table is a row type. In this case, each column of the row corresponds to the field of the row type that has the same ordinal position as the column.

+ Compared to the SQL standard, an optional field description simplifies the handling with complex structures.

+ A row type is similar to the STRUCT type known from other non-standard-compliant frameworks.

+ The type can be declared using ROW<n0 t0 'd0', n1 t1 'd1', ...> where n is the unique name of a field, t is the logical type of a field, d is the description of a field. +
diff --git a/docs/content/concepts/spec/_index.md b/docs/content/concepts/spec/_index.md index 166ce4eeaa549..cc148d6a8b533 100644 --- a/docs/content/concepts/spec/_index.md +++ b/docs/content/concepts/spec/_index.md @@ -1,7 +1,7 @@ --- title: Specification bookCollapseSection: true -weight: 6 +weight: 8 --- - -# Presto - -This documentation is a guide for using Paimon in Presto. - -## Version - -Paimon currently supports Presto 0.236 and above. - -## Preparing Paimon Jar File - -{{< stable >}} - -Download from master: -https://paimon.apache.org/docs/master/project/download/ - -{{< /stable >}} - -{{< unstable >}} - -| Version | Jar | -|-----------------|-----------------------------------------------------------------------------------------------------------------------------------------------------| -| [0.236, 0.268) | [paimon-presto-0.236-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.236/{{< version >}}/) | -| [0.268, 0.273) | [paimon-presto-0.268-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.268/{{< version >}}/) | -| [0.273, latest] | [paimon-presto-0.273-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-presto-0.273/{{< version >}}/) | - -{{< /unstable >}} - -You can also manually build a bundled jar from the source code. - -To build from the source code, [clone the git repository]({{< presto_github_repo >}}). - -Build presto connector plugin with the following command. - -``` -mvn clean install -DskipTests -``` - -After the packaging is complete, you can choose the corresponding connector based on your own Presto version: - -| Version | Package | -|-----------------|----------------------------------------------------------------------------------| -| [0.236, 0.268) | `./paimon-presto-0.236/target/paimon-presto-0.236-{{< version >}}-plugin.tar.gz` | -| [0.268, 0.273) | `./paimon-presto-0.268/target/paimon-presto-0.268-{{< version >}}-plugin.tar.gz` | -| [0.273, latest] | `./paimon-presto-0.273/target/paimon-presto-0.273-{{< version >}}-plugin.tar.gz` | - -Of course, we also support different versions of Hive and Hadoop. But note that we utilize -Presto-shaded versions of Hive and Hadoop packages to address dependency conflicts. -You can check the following two links to select the appropriate versions of Hive and Hadoop: - -[hadoop-apache2](https://mvnrepository.com/artifact/com.facebook.presto.hadoop/hadoop-apache2) - -[hive-apache](https://mvnrepository.com/artifact/com.facebook.presto.hive/hive-apache) - -Both Hive 2 and 3, as well as Hadoop 2 and 3, are supported. - -For example, if your presto version is 0.274, hive and hadoop version is 2.x, you could run: - -```bash -mvn clean install -DskipTests -am -pl paimon-presto-0.273 -Dpresto.version=0.274 -Dhadoop.apache2.version=2.7.4-9 -Dhive.apache.version=1.2.2-2 -``` - -## Tmp Dir - -Paimon will unzip some jars to the tmp directory for codegen. By default, Presto will use `'/tmp'` as the temporary -directory, but `'/tmp'` may be periodically deleted. - -You can configure this environment variable when Presto starts: -```shell --Djava.io.tmpdir=/path/to/other/tmpdir -``` - -Let Paimon use a secure temporary directory. - -## Configure Paimon Catalog - -### Install Paimon Connector - -```bash -tar -zxf paimon-presto-${PRESTO_VERSION}/target/paimon-presto-${PRESTO_VERSION}-${PAIMON_VERSION}-plugin.tar.gz -C ${PRESTO_HOME}/plugin -``` - -Note that, the variable `PRESTO_VERSION` is module name, must be one of 0.236, 0.268, 0.273. - -### Configuration - -```bash -cd ${PRESTO_HOME} -mkdir -p etc/catalog -``` - -```properties -connector.name=paimon -# set your filesystem path, such as hdfs://namenode01:8020/path and s3://${YOUR_S3_BUCKET}/path -warehouse=${YOUR_FS_PATH} -``` - -If you are using HDFS FileSystem, you will also need to do one more thing: choose one of the following ways to configure your HDFS: - -- set environment variable HADOOP_HOME. -- set environment variable HADOOP_CONF_DIR. -- configure `hadoop-conf-dir` in the properties. - -If you are using S3 FileSystem, you need to add `paimon-s3-${PAIMON_VERSION}.jar` in `${PRESTO_HOME}/plugin/paimon` and additionally configure the following properties in `paimon.properties`: - -```properties -s3.endpoint=${YOUR_ENDPOINTS} -s3.access-key=${YOUR_AK} -s3.secret-key=${YOUR_SK} -``` - -**Query HiveCatalog table:** - -```bash -vim etc/catalog/paimon.properties -``` - -and set the following config: - -```properties -connector.name=paimon -# set your filesystem path, such as hdfs://namenode01:8020/path and s3://${YOUR_S3_BUCKET}/path -warehouse=${YOUR_FS_PATH} -metastore=hive -uri=thrift://${YOUR_HIVE_METASTORE}:9083 -``` - -## Kerberos - -You can configure kerberos keytab file when using KERBEROS authentication in the properties. - -``` -security.kerberos.login.principal=hadoop-user -security.kerberos.login.keytab=/etc/presto/hdfs.keytab -``` - -Keytab files must be distributed to every node in the cluster that runs Presto. - -## Create Schema - -``` -CREATE SCHEMA paimon.test_db; -``` - -## Create Table - -``` -CREATE TABLE paimon.test_db.orders ( - order_key bigint, - order_status varchar, - total_price decimal(18,4), - order_date date -) -WITH ( - file_format = 'ORC', - primary_key = ARRAY['order_key','order_date'], - partitioned_by = ARRAY['order_date'], - bucket = '2', - bucket_key = 'order_key', - changelog_producer = 'input' -) -``` - -## Add Column - -``` -CREATE TABLE paimon.test_db.orders ( - order_key bigint, - orders_tatus varchar, - total_price decimal(18,4), - order_date date -) -WITH ( - file_format = 'ORC', - primary_key = ARRAY['order_key','order_date'], - partitioned_by = ARRAY['order_date'], - bucket = '2', - bucket_key = 'order_key', - changelog_producer = 'input' -) - -ALTER TABLE paimon.test_db.orders ADD COLUMN "shipping_address varchar; -``` - -## Query - -``` -SELECT * FROM paimon.default.MyTable -``` - -## Presto to Paimon type mapping - -This section lists all supported type conversion between Presto and Paimon. -All Presto's data types are available in package ` com.facebook.presto.common.type`. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
Presto Data TypePaimon Data TypeAtomic Type
RowTypeRowTypefalse
MapTypeMapTypefalse
ArrayTypeArrayTypefalse
BooleanTypeBooleanTypetrue
TinyintTypeTinyIntTypetrue
SmallintTypeSmallIntTypetrue
IntegerTypeIntTypetrue
BigintTypeBigIntTypetrue
RealTypeFloatTypetrue
DoubleTypeDoubleTypetrue
CharType(length)CharType(length)true
VarCharType(VarCharType.MAX_LENGTH)VarCharType(VarCharType.MAX_LENGTH)true
VarCharType(length)VarCharType(length), length is less than VarCharType.MAX_LENGTHtrue
DateTypeDateTypetrue
TimestampTypeTimestampTypetrue
DecimalType(precision, scale)DecimalType(precision, scale)true
VarBinaryType(length)VarBinaryType(length)true
TimestampWithTimeZoneTypeLocalZonedTimestampTypetrue
diff --git a/docs/content/engines/starrocks.md b/docs/content/engines/starrocks.md index 1ab821a9a1033..dda22d35f76a3 100644 --- a/docs/content/engines/starrocks.md +++ b/docs/content/engines/starrocks.md @@ -81,7 +81,7 @@ SELECT * FROM paimon_catalog.test_db.partition_tbl$partitions; ## StarRocks to Paimon type mapping This section lists all supported type conversion between StarRocks and Paimon. -All StarRocks’s data types can be found in this doc [StarRocks Data type overview](https://docs.starrocks.io/docs/sql-reference/data-types/data-type-list/). +All StarRocks’s data types can be found in this doc [StarRocks Data type overview](https://docs.starrocks.io/docs/sql-reference/data-types/). diff --git a/docs/content/engines/trino.md b/docs/content/engines/trino.md index 05fc47729d151..bef10f9d2870d 100644 --- a/docs/content/engines/trino.md +++ b/docs/content/engines/trino.md @@ -30,7 +30,7 @@ This documentation is a guide for using Paimon in Trino. ## Version -Paimon currently supports Trino 420 and above. +Paimon currently supports Trino 440. ## Filesystem @@ -40,26 +40,12 @@ filesystems for Trino on Trino official website. ## Preparing Paimon Jar File -{{< stable >}} - -Download from master: -https://paimon.apache.org/docs/master/project/download/ - -{{< /stable >}} - -{{< unstable >}} - -| Version | Package | -|---------------|-----------------------------------------------------------------------------------------------------------------------------------------------| -| [420, 426] | [paimon-trino-420-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-420/{{< version >}}/) | -| [427, latest] | [paimon-trino-427-{{< version >}}-plugin.tar.gz](https://repository.apache.org/snapshots/org/apache/paimon/paimon-trino-427/{{< version >}}/) | - -{{< /unstable >}} +[Download]({{< ref "project/download" >}}) You can also manually build a bundled jar from the source code. However, there are a few preliminary steps that need to be taken before compiling: - To build from the source code, [clone the git repository]({{< trino_github_repo >}}). -- Install JDK17 locally, and configure JDK17 as a global environment variable; +- Install JDK21 locally, and configure JDK21 as a global environment variable; Then,you can build bundled jar with the following command: @@ -78,28 +64,17 @@ For example, if you want to use Hadoop 3.3.5-1, you can use the following comman mvn clean install -DskipTests -Dhadoop.apache.version=3.3.5-1 ``` -## Tmp Dir - -Paimon will unzip some jars to the tmp directory for codegen. By default, Trino will use `'/tmp'` as the temporary -directory, but `'/tmp'` may be periodically deleted. - -You can configure this environment variable when Trino starts: -```shell --Djava.io.tmpdir=/path/to/other/tmpdir -``` - -Let Paimon use a secure temporary directory. - ## Configure Paimon Catalog ### Install Paimon Connector ```bash tar -zxf paimon-trino--{{< version >}}-plugin.tar.gz -C ${TRINO_HOME}/plugin ``` -the variable `trino-version` is module name, must be one of 420, 427. -> NOTE: For JDK 17, when Deploying Trino, should add jvm options: `--add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED` + +> NOTE: For JDK 21, when Deploying Trino, should add jvm options: `--add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED` ### Configure + Catalogs are registered by creating a catalog properties file in the etc/catalog directory. For example, create etc/catalog/paimon.properties with the following contents to mount the paimon connector as the paimon catalog: ``` @@ -186,9 +161,6 @@ SELECT * FROM paimon.test_db.orders ``` ## Query with Time Traveling -{{< tabs "time-travel-example" >}} - -{{< tab "version >=420" >}} ```sql -- read the snapshot from specified timestamp @@ -208,10 +180,15 @@ you have a tag named '1' based on snapshot 2, the statement `SELECT * FROM paimo instead of snapshot 1. {{< /hint >}} -{{< /tab >}} +## Insert +``` +INSERT INTO paimon.test_db.orders VALUES (.....); +``` -{{< /tabs >}} +Supports: +- primary key table with fixed bucket. +- non-primary-key table with bucket -1. ## Trino to Paimon type mapping @@ -319,3 +296,15 @@ All Trino's data types are available in package `io.trino.spi.type`.
+ +## Tmp Dir + +Paimon will unzip some jars to the tmp directory for codegen. By default, Trino will use `'/tmp'` as the temporary +directory, but `'/tmp'` may be periodically deleted. + +You can configure this environment variable when Trino starts: +```shell +-Djava.io.tmpdir=/path/to/other/tmpdir +``` + +Let Paimon use a secure temporary directory. diff --git a/docs/content/flink/action-jars.md b/docs/content/flink/action-jars.md index 34e911ff6feba..69f9dda1fdcbe 100644 --- a/docs/content/flink/action-jars.md +++ b/docs/content/flink/action-jars.md @@ -50,7 +50,7 @@ Paimon supports "MERGE INTO" via submitting the 'merge_into' job through `flink {{< hint info >}} Important table properties setting: -1. Only [primary key table]({{< ref "primary-key-table" >}}) supports this feature. +1. Only [primary key table]({{< ref "primary-key-table/overview" >}}) supports this feature. 2. The action won't produce UPDATE_BEFORE, so it's not recommended to set 'changelog-producer' = 'input'. {{< /hint >}} diff --git a/docs/content/flink/procedures.md b/docs/content/flink/procedures.md index d55a8daddef66..7a9b238073921 100644 --- a/docs/content/flink/procedures.md +++ b/docs/content/flink/procedures.md @@ -67,7 +67,8 @@ All available procedures are listed below. order_by => 'order_by', options => 'options', `where` => 'where', - partition_idle_time => 'partition_idle_time')

+ partition_idle_time => 'partition_idle_time', + compact_strategy => 'compact_strategy')

-- Use indexed argument
CALL [catalog.]sys.compact('table')

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

@@ -76,6 +77,7 @@ All available procedures are listed below. CALL [catalog.]sys.compact('table', 'partitions', 'order_strategy', 'order_by', 'options')

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

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

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

To compact a table. Arguments: @@ -86,6 +88,7 @@ All available procedures are listed below.
  • options(optional): additional dynamic options of the table.
  • where(optional): partition predicate(Can't be used together with "partitions"). Note: as where is a keyword,a pair of backticks need to add around like `where`.
  • partition_idle_time(optional): this is used to do a full compaction for partition which had not received any new data for 'partition_idle_time'. And only these partitions will be compacted. This argument can not be used with order compact.
  • +
  • compact_strategy(optional): this determines how to pick files to be merged, the default is determined by the runtime execution mode. 'full' strategy only supports batch mode. All files will be selected for merging. 'minor' strategy: Pick the set of files that need to be merged based on specified conditions.
  • -- use partition filter
    @@ -104,7 +107,8 @@ All available procedures are listed below. including_tables => 'includingTables', excluding_tables => 'excludingTables', table_options => 'tableOptions', - partition_idle_time => 'partitionIdleTime')

    + partition_idle_time => 'partitionIdleTime', + compact_strategy => 'compact_strategy')

    -- Use indexed argument
    CALL [catalog.]sys.compact_database()

    CALL [catalog.]sys.compact_database('includingDatabases')

    @@ -112,7 +116,8 @@ All available procedures are listed below. CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables')

    CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables')

    CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables', 'tableOptions')

    - CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables', 'tableOptions', 'partitionIdleTime') + CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables', 'tableOptions', 'partitionIdleTime')

    + CALL [catalog.]sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables', 'tableOptions', 'partitionIdleTime', 'compact_strategy')

    To compact databases. Arguments: @@ -124,6 +129,7 @@ All available procedures are listed below.
  • excludingTables: to specify tables that are not compacted. You can use regular expression.
  • tableOptions: additional dynamic options of the table.
  • partition_idle_time: this is used to do a full compaction for partition which had not received any new data for 'partition_idle_time'. And only these partitions will be compacted.
  • +
  • compact_strategy(optional): this determines how to pick files to be merged, the default is determined by the runtime execution mode. 'full' strategy only supports batch mode. All files will be selected for merging. 'minor' strategy: Pick the set of files that need to be merged based on specified conditions.
  • CALL sys.compact_database( @@ -131,7 +137,8 @@ All available procedures are listed below. mode => 'combined', including_tables => 'table_.*', excluding_tables => 'ignore', - table_options => 'sink.parallelism=4') + table_options => 'sink.parallelism=4', + compat_strategy => 'full') @@ -405,11 +412,33 @@ All available procedures are listed below. CALL sys.rollback_to_timestamp(`table` => 'default.T', timestamp => 1730292023000) + + rollback_to_watermark + + -- for Flink 1.18
    + -- rollback to the snapshot which earlier or equal than watermark.
    + CALL sys.rollback_to_watermark('identifier', watermark)

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

    + + + To rollback to the snapshot which earlier or equal than watermark. Argument: +
  • identifier: the target table identifier. Cannot be empty.
  • +
  • watermark (Long): Roll back to the snapshot which earlier or equal than watermark.
  • + + + -- for Flink 1.18
    + CALL sys.rollback_to_watermark('default.T', 1730292023000) + -- for Flink 1.19 and later
    + CALL sys.rollback_to_watermark(`table` => 'default.T', watermark => 1730292023000) + + expire_snapshots -- Use named argument
    - CALL [catalog.]sys.reset_consumer(
    + CALL [catalog.]sys.expire_snapshots(
    `table` => 'identifier',
    retain_max => 'retain_max',
    retain_min => 'retain_min',
    diff --git a/docs/content/flink/savepoint.md b/docs/content/flink/savepoint.md index a0934df134535..b9d353c1de33d 100644 --- a/docs/content/flink/savepoint.md +++ b/docs/content/flink/savepoint.md @@ -70,7 +70,7 @@ to learn how to configure and trigger savepoint. **Step 3: Choose the tag corresponding to the savepoint.** The tag corresponding to the savepoint will be named in the form of `savepoint-${savepointID}`. You can refer to -[Tags Table]({{< ref "maintenance/system-tables#tags-table" >}}) to query. +[Tags Table]({{< ref "concepts/system-tables#tags-table" >}}) to query. **Step 4: Rollback the paimon table.** diff --git a/docs/content/flink/sql-alter.md b/docs/content/flink/sql-alter.md index bee616f047d42..877995cc631bb 100644 --- a/docs/content/flink/sql-alter.md +++ b/docs/content/flink/sql-alter.md @@ -78,6 +78,10 @@ If you use object storage, such as S3 or OSS, please use this syntax carefully, The following SQL adds two columns `c1` and `c2` to table `my_table`. +{{< hint info >}} +To add a column in a row type, see [Changing Column Type](#changing-column-type). +{{< /hint >}} + ```sql ALTER TABLE my_table ADD (c1 INT, c2 STRING); ``` @@ -99,6 +103,10 @@ otherwise this operation may fail, throws an exception like `The following colum ALTER TABLE my_table DROP (c1, c2); ``` +{{< hint info >}} +To drop a column in a row type, see [Changing Column Type](#changing-column-type). +{{< /hint >}} + ## Dropping Partitions The following SQL drops the partitions of the paimon table. @@ -185,6 +193,14 @@ The following SQL changes type of column `col_a` to `DOUBLE`. ALTER TABLE my_table MODIFY col_a DOUBLE; ``` +Paimon also supports changing columns of row type, array type, and map type. + +```sql +-- col_a previously has type ARRAY> +-- the following SQL changes f1 to BIGINT, drops f2, and adds f3 +ALTER TABLE my_table MODIFY col_a ARRAY>; +``` + ## Adding watermark The following SQL adds a computed column `ts` from existing column `log_ts`, and a watermark with strategy `ts - INTERVAL '1' HOUR` on column `ts` which is marked as event time attribute of table `my_table`. diff --git a/docs/content/flink/sql-ddl.md b/docs/content/flink/sql-ddl.md index 0324e6655689a..a373348861bd0 100644 --- a/docs/content/flink/sql-ddl.md +++ b/docs/content/flink/sql-ddl.md @@ -101,7 +101,7 @@ Also, you can create [FlinkGenericCatalog]({{< ref "flink/quick-start" >}}). 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 property `metastore.partitioned-table` to true. Also see [CoreOptions]({{< ref "maintenance/configurations#CoreOptions" >}}). +If you want to see a partitioned table in Hive and also synchronize newly created partitions into Hive metastore, please set the table property `metastore.partitioned-table` to true. Also see [CoreOptions]({{< ref "maintenance/configurations#coreoptions" >}}). #### Adding Parameters to a Hive Table @@ -114,7 +114,7 @@ For instance, using the option `hive.table.owner=Jon` will automatically add the If you are using an object storage , and you don't want that the location of paimon table/database is accessed by the filesystem of hive, which may lead to the error such as "No FileSystem for scheme: s3a". You can set location in the properties of table/database by the config of `location-in-properties`. See -[setting the location of table/database in properties ]({{< ref "maintenance/configurations#HiveCatalogOptions" >}}) +[setting the location of table/database in properties ]({{< ref "maintenance/configurations#hivecatalogoptions" >}}) ### Creating JDBC Catalog @@ -203,8 +203,8 @@ 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. +For the stats mode of `none`, by default `metadata.stats-dense-store` is `true`, which will significantly reduce the +storage size of the manifest. But the Paimon sdk in reading engine requires at least version 0.9.1 or 1.0.0 or higher. ### Field Default Value diff --git a/docs/content/flink/sql-write.md b/docs/content/flink/sql-write.md index 008fe498363cd..6abbfa01756ce 100644 --- a/docs/content/flink/sql-write.md +++ b/docs/content/flink/sql-write.md @@ -79,7 +79,7 @@ The data is clustered using an automatically chosen strategy (such as ORDER, ZOR by setting the `sink.clustering.strategy`. Clustering relies on sampling and sorting. If the clustering process takes too much time, you can decrease the total sample number by setting the `sink.clustering.sample-factor` or disable the sorting step by setting the `sink.clustering.sort-in-cluster` to false. -You can refer to [FlinkConnectorOptions]({{< ref "maintenance/configurations#FlinkConnectorOptions" >}}) for more info about the configurations above. +You can refer to [FlinkConnectorOptions]({{< ref "maintenance/configurations#flinkconnectoroptions" >}}) for more info about the configurations above. ## Overwriting the Whole Table @@ -175,9 +175,9 @@ PARTITION (k0 = 0, k1 = 0) SELECT v FROM my_table WHERE false; {{< hint info >}} Important table properties setting: -1. Only [primary key table]({{< ref "primary-key-table" >}}) supports this feature. -2. [MergeEngine]({{< ref "primary-key-table/merge-engine" >}}) needs to be [deduplicate]({{< ref "primary-key-table/merge-engine#deduplicate" >}}) - or [partial-update]({{< ref "primary-key-table/merge-engine#partial-update" >}}) to support this feature. +1. Only [primary key table]({{< ref "primary-key-table/overview" >}}) supports this feature. +2. [MergeEngine]({{< ref "primary-key-table/merge-engine" >}}) needs to be [deduplicate]({{< ref "primary-key-table/merge-engine/overview#deduplicate" >}}) + or [partial-update]({{< ref "primary-key-table/merge-engine/partial-update" >}}) to support this feature. 3. Do not support updating primary keys. {{< /hint >}} @@ -211,7 +211,9 @@ UPDATE my_table SET b = 1, c = 2 WHERE a = 'myTable'; {{< hint info >}} Important table properties setting: 1. Only primary key tables support this feature. -2. If the table has primary keys, [MergeEngine]({{< ref "primary-key-table/merge-engine" >}}) needs to be [deduplicate]({{< ref "primary-key-table/merge-engine#deduplicate" >}}) to support this feature. +2. If the table has primary keys, the following [MergeEngine]({{< ref "primary-key-table/merge-engine/overview" >}}) support this feature: + * [deduplicate]({{< ref "primary-key-table/merge-engine/overview#deduplicate" >}}). + * [partial-update]({{< ref "primary-key-table/merge-engine/partial-update" >}}) with option 'partial-update.remove-record-on-delete' enabled. 3. Do not support deleting from table in streaming mode. {{< /hint >}} @@ -257,7 +259,8 @@ CREATE TABLE my_partitioned_table ( 'partition.timestamp-formatter'='yyyyMMdd', 'partition.timestamp-pattern'='$dt', 'partition.time-interval'='1 d', - 'partition.idle-time-to-done'='15 m' + 'partition.idle-time-to-done'='15 m', + 'partition.mark-done-action'='done-partition' ); ``` @@ -267,4 +270,5 @@ CREATE TABLE my_partitioned_table ( and then it will be marked as done. 3. Thirdly, by default, partition mark done will create _SUCCESS file, the content of _SUCCESS file is a json, contains `creationTime` and `modificationTime`, they can help you understand if there is any delayed data. You can also - configure other actions. + configure other actions, like `'done-partition'`, for example, partition `'dt=20240501'` with produce + `'dt=20240501.done'` done partition. diff --git a/docs/content/learn-paimon/understand-files.md b/docs/content/learn-paimon/understand-files.md index 8f67f85eee2fb..fea6d30a04715 100644 --- a/docs/content/learn-paimon/understand-files.md +++ b/docs/content/learn-paimon/understand-files.md @@ -316,8 +316,8 @@ made and contains the following information: "commitKind" : "COMPACT", "timeMillis" : 1684163217960, "logOffsets" : { }, - "totalRecordCount" : 38, - "deltaRecordCount" : 20, + "totalRecordCount" : 2, + "deltaRecordCount" : -16, "changelogRecordCount" : 0, "watermark" : -9223372036854775808 } diff --git a/docs/content/maintenance/dedicated-compaction.md b/docs/content/maintenance/dedicated-compaction.md index c0010bf9cc854..63e0aa5e66e4b 100644 --- a/docs/content/maintenance/dedicated-compaction.md +++ b/docs/content/maintenance/dedicated-compaction.md @@ -107,6 +107,7 @@ Run the following command to submit a compaction job for the table. --database \ --table \ [--partition ] \ + [--compact_strategy ] \ [--table_conf ] \ [--catalog_conf [--catalog_conf ...]] ``` @@ -123,10 +124,14 @@ Example: compact table --partition dt=20221126,hh=08 \ --partition dt=20221127,hh=09 \ --table_conf sink.parallelism=10 \ + --compact_strategy minor \ --catalog_conf s3.endpoint=https://****.com \ --catalog_conf s3.access-key=***** \ --catalog_conf s3.secret-key=***** ``` +* `--compact_strategy` Determines how to pick files to be merged, the default is determined by the runtime execution mode, streaming-mode use `minor` strategy and batch-mode use `full` strategy. + * `full` : Only supports batch mode. All files will be selected for merging. + * `minor` : Pick the set of files that need to be merged based on specified conditions. You can use `-D execution.runtime-mode=batch` or `-yD execution.runtime-mode=batch` (for the ON-YARN scenario) to control batch or streaming mode. If you submit a batch job, all current table files will be compacted. If you submit a streaming job, the job will continuously monitor new changes @@ -190,6 +195,7 @@ CALL sys.compact_database( [--including_tables ] \ [--excluding_tables ] \ [--mode ] \ + [--compact_strategy ] \ [--catalog_conf [--catalog_conf ...]] \ [--table_conf [--table_conf ...]] ``` @@ -346,6 +352,7 @@ CALL sys.compact(`table` => 'default.T', 'partition_idle_time' => '1 d') --table \ --partition_idle_time \ [--partition ] \ + [--compact_strategy ] \ [--catalog_conf [--catalog_conf ...]] \ [--table_conf [--table_conf ] ...] ``` @@ -406,6 +413,7 @@ CALL sys.compact_database( [--including_tables ] \ [--excluding_tables ] \ [--mode ] \ + [--compact_strategy ] \ [--catalog_conf [--catalog_conf ...]] \ [--table_conf [--table_conf ...]] ``` diff --git a/docs/content/maintenance/manage-snapshots.md b/docs/content/maintenance/manage-snapshots.md index eed0f72c08c2c..721d5d0bafb28 100644 --- a/docs/content/maintenance/manage-snapshots.md +++ b/docs/content/maintenance/manage-snapshots.md @@ -308,9 +308,9 @@ submit a `remove_orphan_files` job to clean them: {{< tab "Spark SQL/Flink SQL" >}} ```sql -CALL sys.remove_orphan_files(`table` => "my_db.my_table", [older_than => "2023-10-31 12:00:00"]) +CALL sys.remove_orphan_files(`table` => 'my_db.my_table', [older_than => '2023-10-31 12:00:00']) -CALL sys.remove_orphan_files(`table` => "my_db.*", [older_than => "2023-10-31 12:00:00"]) +CALL sys.remove_orphan_files(`table` => 'my_db.*', [older_than => '2023-10-31 12:00:00']) ``` {{< /tab >}} diff --git a/docs/content/maintenance/metrics.md b/docs/content/maintenance/metrics.md index 139bdfff6bfe0..2c3067267fd75 100644 --- a/docs/content/maintenance/metrics.md +++ b/docs/content/maintenance/metrics.md @@ -67,16 +67,6 @@ Below is lists of Paimon built-in metrics. They are summarized into types of sca Gauge Number of scanned manifest files in the last scan. - - lastSkippedByPartitionAndStats - Gauge - Skipped table files by partition filter and value / key stats information in the last scan. - - - lastSkippedByWholeBucketFilesFilter - Gauge - Skipped table files by bucket level value filter (only primary key table) in the last scan. - lastScanSkippedTableFiles Gauge diff --git a/docs/content/maintenance/write-performance.md b/docs/content/maintenance/write-performance.md index 02362b9096e7e..ade2c3353e3cc 100644 --- a/docs/content/maintenance/write-performance.md +++ b/docs/content/maintenance/write-performance.md @@ -160,12 +160,3 @@ 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 index 17e1c65233988..b6fcaa2826155 100644 --- a/docs/content/migration/iceberg-compatibility.md +++ b/docs/content/migration/iceberg-compatibility.md @@ -29,8 +29,6 @@ under the License. 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. @@ -66,7 +64,7 @@ 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 +## Append Tables 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. @@ -200,7 +198,7 @@ germany hamburg {{< /tabs >}} -## Example: Query Paimon Primary Key Tables on Flink/Spark with Iceberg Connector +## Primary Key Tables {{< tabs "paimon-primary-key-table" >}} @@ -300,8 +298,6 @@ SELECT * FROM iceberg_catalog.`default`.orders WHERE status = 'COMPLETED'; {{< /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. @@ -338,7 +334,7 @@ You can configure the following table option, so that Paimon is forced to perfor 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 +## 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. @@ -375,10 +371,45 @@ you also need to set some (or all) of the following table options when creating + + + + + + + + + + + + + + + + + +
    String hadoop-conf-dir for Iceberg Hive catalog.
    metadata.iceberg.manifest-compression
    snappyStringCompression for Iceberg manifest files.
    metadata.iceberg.manifest-legacy-version
    falseBooleanShould use the legacy manifest version to generate Iceberg's 1.4 manifest files.
    metadata.iceberg.hive-client-class
    org.apache.hadoop.hive.metastore.HiveMetaStoreClientStringHive client class name for Iceberg Hive Catalog.
    -## Example: Query Paimon Append Only Tables on Trino with Iceberg Connector +## AWS Glue Catalog + +You can use Hive Catalog to connect AWS Glue metastore, you can use set `'metadata.iceberg.hive-client-class'` to +`'com.amazonaws.glue.catalog.metastore.AWSCatalogMetastoreClient'`. + +> **Note:** You can use this [repo](https://github.com/promotedai/aws-glue-data-catalog-client-for-apache-hive-metastore) to build the required jar, include it in your path and configure the AWSCatalogMetastoreClient. +## AWS Athena + +AWS Athena may use old manifest reader to read Iceberg manifest by names, we should let Paimon producing legacy Iceberg +manifest list file, you can enable: `'metadata.iceberg.manifest-legacy-version'`. + +## DuckDB + +Duckdb may rely on files placed in the `root/data` directory, while Paimon is usually placed directly in the `root` +directory, so you can configure this parameter for the table to achieve compatibility: +`'data-file.path-directory' = 'data'`. + +## Trino Iceberg 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. @@ -455,25 +486,28 @@ SELECT * FROM animals WHERE class = 'mammal'; 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` | +| Paimon Data Type | Iceberg Data Type | +|------------------|-------------------| +| `BOOLEAN` | `boolean` | +| `INT` | `int` | +| `BIGINT` | `long` | +| `FLOAT` | `float` | +| `DOUBLE` | `double` | +| `DECIMAL` | `decimal` | +| `CHAR` | `string` | +| `VARCHAR` | `string` | +| `BINARY` | `binary` | +| `VARBINARY` | `binary` | +| `DATE` | `date` | +| `TIMESTAMP`* | `timestamp` | +| `TIMESTAMP_LTZ`* | `timestamptz` | +| `ARRAY` | `list` | +| `MAP` | `map` | +| `ROW` | `struct` | *: `TIMESTAMP` and `TIMESTAMP_LTZ` type only support precision from 4 to 6 -## Other Related Table Options +## Table Options diff --git a/docs/content/primary-key-table/changelog-producer.md b/docs/content/primary-key-table/changelog-producer.md index bf7a23fae2a54..a9364ee9f07c5 100644 --- a/docs/content/primary-key-table/changelog-producer.md +++ b/docs/content/primary-key-table/changelog-producer.md @@ -58,9 +58,11 @@ By specifying `'changelog-producer' = 'input'`, Paimon writers rely on their inp ## Lookup -If your input can’t produce a complete changelog but you still want to get rid of the costly normalized operator, you may consider using the `'lookup'` changelog producer. +If your input can’t produce a complete changelog but you still want to get rid of the costly normalized operator, you +may consider using the `'lookup'` changelog producer. -By specifying `'changelog-producer' = 'lookup'`, Paimon will generate changelog through `'lookup'` before committing the data writing. +By specifying `'changelog-producer' = 'lookup'`, Paimon will generate changelog through `'lookup'` before committing +the data writing (You can also enable [Async Compaction]({{< ref "primary-key-table/compaction#asynchronous-compaction" >}})). {{< img src="/img/changelog-producer-lookup.png">}} @@ -105,23 +107,37 @@ important for performance). ## Full Compaction -If you think the resource consumption of 'lookup' is too large, you can consider using 'full-compaction' changelog producer, -which can decouple data writing and changelog generation, and is more suitable for scenarios with high latency (For example, 10 minutes). +You can also consider using 'full-compaction' changelog producer to generate changelog, and is more suitable for scenarios +with large latency (For example, 30 minutes). -By specifying `'changelog-producer' = 'full-compaction'`, Paimon will compare the results between full compactions and produce the differences as changelog. The latency of changelog is affected by the frequency of full compactions. +1. By specifying `'changelog-producer' = 'full-compaction'`, Paimon will compare the results between full compactions and +produce the differences as changelog. The latency of changelog is affected by the frequency of full compactions. +2. By specifying `full-compaction.delta-commits` table property, full compaction will be constantly triggered after delta +commits (checkpoints). This is set to 1 by default, so each checkpoint will have a full compression and generate a +changelog. -By specifying `full-compaction.delta-commits` table property, full compaction will be constantly triggered after delta commits (checkpoints). This is set to 1 by default, so each checkpoint will have a full compression and generate a change log. +Generally speaking, the cost and consumption of full compaction are high, so we recommend using `'lookup'` changelog +producer. {{< img src="/img/changelog-producer-full-compaction.png">}} {{< hint info >}} -Full compaction changelog producer can produce complete changelog for any type of source. However it is not as efficient as the input changelog producer and the latency to produce changelog might be high. +Full compaction changelog producer can produce complete changelog for any type of source. However it is not as +efficient as the input changelog producer and the latency to produce changelog might be high. {{< /hint >}} Full-compaction changelog-producer supports `changelog-producer.row-deduplicate` to avoid generating -U, +U changelog for the same record. -(Note: Please increase `'execution.checkpointing.max-concurrent-checkpoints'` Flink configuration, this is very -important for performance). +## Changelog Merging + +For `input`, `lookup`, `full-compaction` 'changelog-producer'. + +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/primary-key-table/compaction.md b/docs/content/primary-key-table/compaction.md index ada7e0289b359..bee8c16e46e9d 100644 --- a/docs/content/primary-key-table/compaction.md +++ b/docs/content/primary-key-table/compaction.md @@ -76,7 +76,6 @@ In compaction, you can configure record-Level expire time to expire records, you 1. `'record-level.expire-time'`: time retain for records. 2. `'record-level.time-field'`: time field for record level expire. -3. `'record-level.time-field-type'`: time field type for record level expire, it can be seconds-int,seconds-long or millis-long. Expiration happens in compaction, and there is no strong guarantee to expire records in time. diff --git a/docs/content/primary-key-table/merge-engine/aggregation.md b/docs/content/primary-key-table/merge-engine/aggregation.md index fa667ed356871..0cc6507f2b4ce 100644 --- a/docs/content/primary-key-table/merge-engine/aggregation.md +++ b/docs/content/primary-key-table/merge-engine/aggregation.md @@ -247,16 +247,17 @@ An example: uv VARBINARY ) WITH ( 'merge-engine' = 'aggregation', - 'fields.f0.aggregate-function' = 'hll_sketch' + 'fields.uv.aggregate-function' = 'hll_sketch' ); -- Register the following class as a Flink function with the name "HLL_SKETCH" + -- for example: create TEMPORARY function HLL_SKETCH as 'HllSketchFunction'; -- which is used to transform input to sketch bytes array: -- -- public static class HllSketchFunction extends ScalarFunction { -- public byte[] eval(String user_id) { -- HllSketch hllSketch = new HllSketch(); - -- hllSketch.update(id); + -- hllSketch.update(user_id); -- return hllSketch.toCompactByteArray(); -- } -- } @@ -264,6 +265,7 @@ An example: INSERT INTO UV_AGG SELECT id, HLL_SKETCH(user_id) FROM VISITS; -- Register the following class as a Flink function with the name "HLL_SKETCH_COUNT" + -- for example: create TEMPORARY function HLL_SKETCH_COUNT as 'HllSketchCountFunction'; -- which is used to get cardinality from sketch bytes array: -- -- public static class HllSketchCountFunction extends ScalarFunction { @@ -307,10 +309,11 @@ An example: uv VARBINARY ) WITH ( 'merge-engine' = 'aggregation', - 'fields.f0.aggregate-function' = 'theta_sketch' + 'fields.uv.aggregate-function' = 'theta_sketch' ); -- Register the following class as a Flink function with the name "THETA_SKETCH" + -- for example: create TEMPORARY function THETA_SKETCH as 'ThetaSketchFunction'; -- which is used to transform input to sketch bytes array: -- -- public static class ThetaSketchFunction extends ScalarFunction { @@ -324,6 +327,7 @@ An example: INSERT INTO UV_AGG SELECT id, THETA_SKETCH(user_id) FROM VISITS; -- Register the following class as a Flink function with the name "THETA_SKETCH_COUNT" + -- for example: create TEMPORARY function THETA_SKETCH_COUNT as 'ThetaSketchCountFunction'; -- which is used to get cardinality from sketch bytes array: -- -- public static class ThetaSketchCountFunction extends ScalarFunction { diff --git a/docs/content/primary-key-table/merge-engine/overview.md b/docs/content/primary-key-table/merge-engine/overview.md index 9f3b50c398bdc..4d192283aa688 100644 --- a/docs/content/primary-key-table/merge-engine/overview.md +++ b/docs/content/primary-key-table/merge-engine/overview.md @@ -3,7 +3,7 @@ title: "Overview" weight: 1 type: docs aliases: -- /primary-key-table/merge-engin/overview.html +- /primary-key-table/merge-engine/overview.html --- + + + com.squareup.okhttp3 + okhttp + ${okhttp.version} + + @@ -204,6 +213,20 @@ under the License. test + + com.squareup.okhttp3 + mockwebserver + ${okhttp.version} + test + + + org.mockito + mockito-core + ${mockito.version} + jar + test + + @@ -219,6 +242,40 @@ under the License. + + org.apache.maven.plugins + maven-shade-plugin + + + shade-paimon + package + + shade + + + + + * + + okhttp3/internal/publicsuffix/NOTICE + + + + + + com.squareup.okhttp3:okhttp + + + + + okhttp3 + org.apache.paimon.shade.okhttp3 + + + + + + diff --git a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java index 14665961a8a7e..1caff252a6547 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java @@ -54,6 +54,8 @@ import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.TagManager; +import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache; + import javax.annotation.Nullable; import java.time.Duration; @@ -79,6 +81,7 @@ abstract class AbstractFileStore implements FileStore { @Nullable private final SegmentsCache writeManifestCache; @Nullable private SegmentsCache readManifestCache; + @Nullable private Cache snapshotCache; protected AbstractFileStore( FileIO fileIO, @@ -102,21 +105,26 @@ protected AbstractFileStore( @Override public FileStorePathFactory pathFactory() { + return pathFactory(options.fileFormat().getFormatIdentifier()); + } + + protected FileStorePathFactory pathFactory(String format) { return new FileStorePathFactory( options.path(), partitionType, options.partitionDefaultName(), - options.fileFormat().getFormatIdentifier(), + format, options.dataFilePrefix(), options.changelogFilePrefix(), options.legacyPartitionName(), options.fileSuffixIncludeCompression(), - options.fileCompression()); + options.fileCompression(), + options.dataFilePathDirectory()); } @Override public SnapshotManager snapshotManager() { - return new SnapshotManager(fileIO, options.path(), options.branch()); + return new SnapshotManager(fileIO, options.path(), options.branch(), snapshotCache); } @Override @@ -215,6 +223,7 @@ public FileStoreCommitImpl newCommit(String commitUser, List cal tableName, commitUser, partitionType, + options, options.partitionDefaultName(), pathFactory(), snapshotManager(), @@ -233,7 +242,8 @@ public FileStoreCommitImpl newCommit(String commitUser, List cal bucketMode(), options.scanManifestParallelism(), callbacks, - options.commitMaxRetries()); + options.commitMaxRetries(), + options.commitTimeout()); } @Override @@ -305,7 +315,8 @@ public PartitionExpire newPartitionExpire(String commitUser) { newScan(), newCommit(commitUser), metastoreClient, - options.endInputCheckPartitionExpire()); + options.endInputCheckPartitionExpire(), + options.partitionExpireMaxNum()); } @Override @@ -340,4 +351,9 @@ public ServiceManager newServiceManager() { public void setManifestCache(SegmentsCache manifestCache) { this.readManifestCache = manifestCache; } + + @Override + public void setSnapshotCache(Cache cache) { + this.snapshotCache = cache; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/FileStore.java b/paimon-core/src/main/java/org/apache/paimon/FileStore.java index f9bf4c8440bd7..e50d4ada1397b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/FileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/FileStore.java @@ -44,6 +44,8 @@ import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.TagManager; +import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache; + import javax.annotation.Nullable; import java.util.List; @@ -107,4 +109,6 @@ public interface FileStore { List createTagCallbacks(); void setManifestCache(SegmentsCache manifestCache); + + void setSnapshotCache(Cache cache); } diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java index 1c87a087e44eb..8cf45105c01b8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java @@ -194,20 +194,7 @@ private Map format2PathFactory() { Map pathFactoryMap = new HashMap<>(); Set formats = new HashSet<>(options.fileFormatPerLevel().values()); formats.add(options.fileFormat().getFormatIdentifier()); - formats.forEach( - format -> - pathFactoryMap.put( - format, - new FileStorePathFactory( - options.path(), - partitionType, - options.partitionDefaultName(), - format, - options.dataFilePrefix(), - options.changelogFilePrefix(), - options.legacyPartitionName(), - options.fileSuffixIncludeCompression(), - options.fileCompression()))); + formats.forEach(format -> pathFactoryMap.put(format, pathFactory(format))); return pathFactoryMap; } diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueThinSerializer.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueThinSerializer.java new file mode 100644 index 0000000000000..6dd41a42506ae --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueThinSerializer.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon; + +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.JoinedRow; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.ObjectSerializer; + +/** Serialize KeyValue to InternalRow with ignorance of key. Only used to write KeyValue to disk. */ +public class KeyValueThinSerializer extends ObjectSerializer { + + private static final long serialVersionUID = 1L; + + private final GenericRow reusedMeta; + private final JoinedRow reusedKeyWithMeta; + + public KeyValueThinSerializer(RowType keyType, RowType valueType) { + super(KeyValue.schema(keyType, valueType)); + + this.reusedMeta = new GenericRow(2); + this.reusedKeyWithMeta = new JoinedRow(); + } + + public InternalRow toRow(KeyValue record) { + return toRow(record.sequenceNumber(), record.valueKind(), record.value()); + } + + public InternalRow toRow(long sequenceNumber, RowKind valueKind, InternalRow value) { + reusedMeta.setField(0, sequenceNumber); + reusedMeta.setField(1, valueKind.toByteValue()); + return reusedKeyWithMeta.replace(reusedMeta, value); + } + + @Override + public KeyValue fromRow(InternalRow row) { + throw new UnsupportedOperationException( + "KeyValue cannot be deserialized from InternalRow by this serializer."); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/Snapshot.java b/paimon-core/src/main/java/org/apache/paimon/Snapshot.java index 3b8d2fa15b4b1..baee7bad950e1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/Snapshot.java +++ b/paimon-core/src/main/java/org/apache/paimon/Snapshot.java @@ -29,9 +29,6 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonInclude; import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import javax.annotation.Nullable; import java.io.FileNotFoundException; @@ -65,7 +62,6 @@ @Public @JsonIgnoreProperties(ignoreUnknown = true) public class Snapshot { - private static final Logger LOG = LoggerFactory.getLogger(Snapshot.class); public static final long FIRST_SNAPSHOT_ID = 1; @@ -355,28 +351,6 @@ public String toJson() { return JsonSerdeUtil.toJson(this); } - public static Snapshot fromJson(String json) { - return JsonSerdeUtil.fromJson(json, Snapshot.class); - } - - public static Snapshot fromPath(FileIO fileIO, Path path) { - try { - return Snapshot.fromJson(fileIO.readFileUtf8(path)); - } catch (FileNotFoundException e) { - String errorMessage = - String.format( - "Snapshot file %s does not exist. " - + "It might have been expired by other jobs operating on this table. " - + "In this case, you can avoid concurrent modification issues by configuring " - + "write-only = true and use a dedicated compaction job, or configuring " - + "different expiration thresholds for different jobs.", - path); - throw new RuntimeException(errorMessage, e); - } catch (IOException e) { - throw new RuntimeException("Fails to read snapshot from path " + path, e); - } - } - @Override public int hashCode() { return Objects.hash( @@ -437,4 +411,36 @@ public enum CommitKind { /** Collect statistics. */ ANALYZE } + + // =================== Utils for reading ========================= + + public static Snapshot fromJson(String json) { + return JsonSerdeUtil.fromJson(json, Snapshot.class); + } + + public static Snapshot fromPath(FileIO fileIO, Path path) { + try { + return tryFromPath(fileIO, path); + } catch (FileNotFoundException e) { + String errorMessage = + String.format( + "Snapshot file %s does not exist. " + + "It might have been expired by other jobs operating on this table. " + + "In this case, you can avoid concurrent modification issues by configuring " + + "write-only = true and use a dedicated compaction job, or configuring " + + "different expiration thresholds for different jobs.", + path); + throw new RuntimeException(errorMessage, e); + } + } + + public static Snapshot tryFromPath(FileIO fileIO, Path path) throws FileNotFoundException { + try { + return Snapshot.fromJson(fileIO.readFileUtf8(path)); + } catch (FileNotFoundException e) { + throw e; + } catch (IOException e) { + throw new RuntimeException("Fails to read snapshot from path " + path, e); + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java b/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java index 47502aa707d1c..a3087e3628646 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java @@ -211,6 +211,7 @@ public CommitIncrement prepareCommit(boolean waitCompaction) throws Exception { @Override public boolean isCompacting() { + compactManager.triggerCompaction(false); return compactManager.isCompacting(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinator.java b/paimon-core/src/main/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinator.java index 9a54ea72e7cc9..490bda9d4cf1d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinator.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinator.java @@ -27,6 +27,7 @@ import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.table.FileStoreTable; @@ -379,6 +380,10 @@ public FilesIterator( if (filter != null) { snapshotReader.withFilter(filter); } + // drop stats to reduce memory + if (table.coreOptions().manifestDeleteFileDropStats()) { + snapshotReader.dropStats(); + } this.streamingMode = isStreaming; } @@ -387,6 +392,9 @@ private void assignNewIterator() { if (nextSnapshot == null) { nextSnapshot = snapshotManager.latestSnapshotId(); if (nextSnapshot == null) { + if (!streamingMode) { + throw new EndOfScanException(); + } return; } snapshotReader.withMode(ScanMode.ALL); @@ -438,7 +446,12 @@ public ManifestEntry next() { } if (currentIterator.hasNext()) { - return currentIterator.next(); + ManifestEntry entry = currentIterator.next(); + if (entry.kind() == FileKind.DELETE) { + continue; + } else { + return entry; + } } currentIterator = null; } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index c2e4afe5d533e..b56fec279ab10 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -19,11 +19,11 @@ package org.apache.paimon.catalog; import org.apache.paimon.CoreOptions; +import org.apache.paimon.TableType; import org.apache.paimon.factories.FactoryUtil; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; -import org.apache.paimon.lineage.LineageMetaFactory; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.metastore.MetastoreClient; import org.apache.paimon.operation.FileStoreCommit; @@ -48,6 +48,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -60,11 +61,11 @@ import static org.apache.paimon.CoreOptions.TYPE; import static org.apache.paimon.CoreOptions.createCommitUser; import static org.apache.paimon.options.CatalogOptions.ALLOW_UPPER_CASE; -import static org.apache.paimon.options.CatalogOptions.LINEAGE_META; import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED; import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE; import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.Preconditions.checkNotNull; /** Common implementation of {@link Catalog}. */ public abstract class AbstractCatalog implements Catalog { @@ -73,19 +74,14 @@ public abstract class AbstractCatalog implements Catalog { protected final Map tableDefaultOptions; protected final Options catalogOptions; - @Nullable protected final LineageMetaFactory lineageMetaFactory; - protected AbstractCatalog(FileIO fileIO) { this.fileIO = fileIO; - this.lineageMetaFactory = null; this.tableDefaultOptions = new HashMap<>(); this.catalogOptions = new Options(); } protected AbstractCatalog(FileIO fileIO, Options options) { this.fileIO = fileIO; - this.lineageMetaFactory = - findAndCreateLineageMeta(options, AbstractCatalog.class.getClassLoader()); this.tableDefaultOptions = Catalog.tableDefaultOptions(options.toMap()); this.catalogOptions = options; } @@ -132,6 +128,10 @@ public boolean allowUpperCase() { return catalogOptions.getOptional(ALLOW_UPPER_CASE).orElse(true); } + protected boolean allowCustomTablePath() { + return false; + } + @Override public void createDatabase(String name, boolean ignoreIfExists, Map properties) throws DatabaseAlreadyExistException { @@ -271,6 +271,7 @@ public void createTable(Identifier identifier, Schema schema, boolean ignoreIfEx validateIdentifierNameCaseInsensitive(identifier); validateFieldNameCaseInsensitive(schema.rowType().getFieldNames()); validateAutoCreateClose(schema.options()); + validateCustomTablePath(schema.options()); // check db exists getDatabase(identifier.getDatabaseName()); @@ -369,32 +370,18 @@ public void alterTable( protected abstract void alterTableImpl(Identifier identifier, List changes) throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException; - @Nullable - private LineageMetaFactory findAndCreateLineageMeta(Options options, ClassLoader classLoader) { - return options.getOptional(LINEAGE_META) - .map( - meta -> - FactoryUtil.discoverFactory( - classLoader, LineageMetaFactory.class, meta)) - .orElse(null); - } - @Override public Table getTable(Identifier identifier) throws TableNotExistException { if (isSystemDatabase(identifier.getDatabaseName())) { String tableName = identifier.getTableName(); Table table = SystemTableLoader.loadGlobal( - tableName, - fileIO, - this::allTablePaths, - catalogOptions, - lineageMetaFactory); + tableName, fileIO, this::allTablePaths, catalogOptions); if (table == null) { throw new TableNotExistException(identifier); } return table; - } else if (isSpecifiedSystemTable(identifier)) { + } else if (identifier.isSystemTable()) { Table originTable = getDataOrFormatTable( new Identifier( @@ -423,17 +410,39 @@ public Table getTable(Identifier identifier) throws TableNotExistException { protected Table getDataOrFormatTable(Identifier identifier) throws TableNotExistException { Preconditions.checkArgument(identifier.getSystemTableName() == null); - TableSchema tableSchema = getDataTableSchema(identifier); - return FileStoreTableFactory.create( - fileIO, - getTableLocation(identifier), - tableSchema, - new CatalogEnvironment( - identifier, - Lock.factory( - lockFactory().orElse(null), lockContext().orElse(null), identifier), - metastoreClientFactory(identifier, tableSchema).orElse(null), - lineageMetaFactory)); + TableMeta tableMeta = getDataTableMeta(identifier); + FileStoreTable table = + FileStoreTableFactory.create( + fileIO, + getTableLocation(identifier), + tableMeta.schema, + new CatalogEnvironment( + identifier, + tableMeta.uuid, + Lock.factory( + lockFactory().orElse(null), + lockContext().orElse(null), + identifier), + metastoreClientFactory(identifier, tableMeta.schema).orElse(null))); + CoreOptions options = table.coreOptions(); + if (options.type() == TableType.OBJECT_TABLE) { + String objectLocation = options.objectLocation(); + checkNotNull(objectLocation, "Object location should not be null for object table."); + table = + ObjectTable.builder() + .underlyingTable(table) + .objectLocation(objectLocation) + .objectFileIO(objectFileIO(objectLocation)) + .build(); + } + return table; + } + + /** + * Catalog implementation may override this method to provide {@link FileIO} to object table. + */ + protected FileIO objectFileIO(String objectLocation) { + return fileIO; } /** @@ -474,6 +483,10 @@ public Map> allTablePaths() { } } + protected TableMeta getDataTableMeta(Identifier identifier) throws TableNotExistException { + return new TableMeta(getDataTableSchema(identifier), null); + } + protected abstract TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException; @@ -506,12 +519,8 @@ protected void assertMainBranch(Identifier identifier) { } } - public static boolean isSpecifiedSystemTable(Identifier identifier) { - return identifier.getSystemTableName() != null; - } - protected static boolean isTableInSystemDatabase(Identifier identifier) { - return isSystemDatabase(identifier.getDatabaseName()) || isSpecifiedSystemTable(identifier); + return isSystemDatabase(identifier.getDatabaseName()) || identifier.isSystemTable(); } protected static void checkNotSystemTable(Identifier identifier, String method) { @@ -560,7 +569,7 @@ private void validateFieldNameCaseInsensitiveInSchemaChange(List c for (SchemaChange change : changes) { if (change instanceof SchemaChange.AddColumn) { SchemaChange.AddColumn addColumn = (SchemaChange.AddColumn) change; - fieldNames.addAll(addColumn.fieldNames()); + fieldNames.addAll(Arrays.asList(addColumn.fieldNames())); } else if (change instanceof SchemaChange.RenameColumn) { SchemaChange.RenameColumn rename = (SchemaChange.RenameColumn) change; fieldNames.add(rename.newName()); @@ -584,6 +593,15 @@ private void validateAutoCreateClose(Map options) { CoreOptions.AUTO_CREATE.key(), Boolean.FALSE)); } + private void validateCustomTablePath(Map options) { + if (!allowCustomTablePath() && options.containsKey(CoreOptions.PATH.key())) { + throw new UnsupportedOperationException( + String.format( + "The current catalog %s does not support specifying the table path when creating a table.", + this.getClass().getSimpleName())); + } + } + // =============================== Meta in File System ===================================== protected List listDatabasesInFileSystem(Path warehouse) throws IOException { @@ -609,7 +627,16 @@ protected List listTablesInFileSystem(Path databasePath) throws IOExcept } protected boolean tableExistsInFileSystem(Path tablePath, String branchName) { - return !new SchemaManager(fileIO, tablePath, branchName).listAllIds().isEmpty(); + SchemaManager schemaManager = new SchemaManager(fileIO, tablePath, branchName); + + // in order to improve the performance, check the schema-0 firstly. + boolean schemaZeroExists = schemaManager.schemaExists(0); + if (schemaZeroExists) { + return true; + } else { + // if schema-0 not exists, fallback to check other schemas + return !schemaManager.listAllIds().isEmpty(); + } } public Optional tableSchemaInFileSystem(Path tablePath, String branchName) { @@ -626,4 +653,25 @@ public Optional tableSchemaInFileSystem(Path tablePath, String bran } }); } + + /** Table metadata. */ + protected static class TableMeta { + + private final TableSchema schema; + @Nullable private final String uuid; + + public TableMeta(TableSchema schema, @Nullable String uuid) { + this.schema = schema; + this.uuid = uuid; + } + + public TableSchema schema() { + return schema; + } + + @Nullable + public String uuid() { + return uuid; + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java index 003f0edb4fa55..82d503b7a2724 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/CachingCatalog.java @@ -26,20 +26,13 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.system.SystemTableLoader; -import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.SegmentsCache; import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache; import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Caffeine; -import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.RemovalCause; -import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.RemovalListener; import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Ticker; import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Weigher; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import javax.annotation.Nullable; import java.time.Duration; @@ -48,19 +41,20 @@ import java.util.Map; import java.util.Optional; -import static org.apache.paimon.catalog.AbstractCatalog.isSpecifiedSystemTable; import static org.apache.paimon.options.CatalogOptions.CACHE_ENABLED; import static org.apache.paimon.options.CatalogOptions.CACHE_EXPIRATION_INTERVAL_MS; import static org.apache.paimon.options.CatalogOptions.CACHE_MANIFEST_MAX_MEMORY; import static org.apache.paimon.options.CatalogOptions.CACHE_MANIFEST_SMALL_FILE_MEMORY; import static org.apache.paimon.options.CatalogOptions.CACHE_MANIFEST_SMALL_FILE_THRESHOLD; import static org.apache.paimon.options.CatalogOptions.CACHE_PARTITION_MAX_NUM; -import static org.apache.paimon.table.system.SystemTableLoader.SYSTEM_TABLES; +import static org.apache.paimon.options.CatalogOptions.CACHE_SNAPSHOT_MAX_NUM_PER_TABLE; +import static org.apache.paimon.utils.Preconditions.checkNotNull; /** A {@link Catalog} to cache databases and tables and manifests. */ public class CachingCatalog extends DelegateCatalog { - private static final Logger LOG = LoggerFactory.getLogger(CachingCatalog.class); + private final Duration expirationInterval; + private final int snapshotMaxNumPerTable; protected final Cache databaseCache; protected final Cache tableCache; @@ -75,7 +69,8 @@ public CachingCatalog(Catalog wrapped) { CACHE_EXPIRATION_INTERVAL_MS.defaultValue(), CACHE_MANIFEST_SMALL_FILE_MEMORY.defaultValue(), CACHE_MANIFEST_SMALL_FILE_THRESHOLD.defaultValue().getBytes(), - CACHE_PARTITION_MAX_NUM.defaultValue()); + CACHE_PARTITION_MAX_NUM.defaultValue(), + CACHE_SNAPSHOT_MAX_NUM_PER_TABLE.defaultValue()); } public CachingCatalog( @@ -83,13 +78,15 @@ public CachingCatalog( Duration expirationInterval, MemorySize manifestMaxMemory, long manifestCacheThreshold, - long cachedPartitionMaxNum) { + long cachedPartitionMaxNum, + int snapshotMaxNumPerTable) { this( wrapped, expirationInterval, manifestMaxMemory, manifestCacheThreshold, cachedPartitionMaxNum, + snapshotMaxNumPerTable, Ticker.systemTicker()); } @@ -99,6 +96,7 @@ public CachingCatalog( MemorySize manifestMaxMemory, long manifestCacheThreshold, long cachedPartitionMaxNum, + int snapshotMaxNumPerTable, Ticker ticker) { super(wrapped); if (expirationInterval.isZero() || expirationInterval.isNegative()) { @@ -106,6 +104,9 @@ public CachingCatalog( "When cache.expiration-interval is set to negative or 0, the catalog cache should be disabled."); } + this.expirationInterval = expirationInterval; + this.snapshotMaxNumPerTable = snapshotMaxNumPerTable; + this.databaseCache = Caffeine.newBuilder() .softValues() @@ -116,11 +117,11 @@ public CachingCatalog( this.tableCache = Caffeine.newBuilder() .softValues() - .removalListener(new TableInvalidatingRemovalListener()) .executor(Runnable::run) .expireAfterAccess(expirationInterval) .ticker(ticker) .build(); + this.manifestCache = SegmentsCache.create(manifestMaxMemory, manifestCacheThreshold); this.partitionCache = cachedPartitionMaxNum == 0 ? null @@ -134,7 +135,6 @@ public CachingCatalog( .maximumWeight(cachedPartitionMaxNum) .ticker(ticker) .build(); - this.manifestCache = SegmentsCache.create(manifestMaxMemory, manifestCacheThreshold); } public static Catalog tryToCreate(Catalog catalog, Options options) { @@ -155,7 +155,8 @@ public static Catalog tryToCreate(Catalog catalog, Options options) { options.get(CACHE_EXPIRATION_INTERVAL_MS), manifestMaxMemory, manifestThreshold, - options.get(CACHE_PARTITION_MAX_NUM)); + options.get(CACHE_PARTITION_MAX_NUM), + options.get(CACHE_SNAPSHOT_MAX_NUM_PER_TABLE)); } @Override @@ -191,6 +192,13 @@ public void dropTable(Identifier identifier, boolean ignoreIfNotExists) throws TableNotExistException { super.dropTable(identifier, ignoreIfNotExists); invalidateTable(identifier); + + // clear all branch tables of this table + for (Identifier i : tableCache.asMap().keySet()) { + if (identifier.getTableName().equals(i.getTableName())) { + tableCache.invalidate(i); + } + } } @Override @@ -215,26 +223,23 @@ public Table getTable(Identifier identifier) throws TableNotExistException { return table; } - if (isSpecifiedSystemTable(identifier)) { + // For system table, do not cache it directly. Instead, cache the origin table and then wrap + // it to generate the system table. + if (identifier.isSystemTable()) { Identifier originIdentifier = new Identifier( identifier.getDatabaseName(), identifier.getTableName(), identifier.getBranchName(), null); - Table originTable = tableCache.getIfPresent(originIdentifier); - if (originTable == null) { - originTable = wrapped.getTable(originIdentifier); - putTableCache(originIdentifier, originTable); - } + Table originTable = getTable(originIdentifier); table = SystemTableLoader.load( - Preconditions.checkNotNull(identifier.getSystemTableName()), + checkNotNull(identifier.getSystemTableName()), (FileStoreTable) originTable); if (table == null) { throw new TableNotExistException(identifier); } - putTableCache(identifier, table); return table; } @@ -244,9 +249,27 @@ public Table getTable(Identifier identifier) throws TableNotExistException { } private void putTableCache(Identifier identifier, Table table) { - if (manifestCache != null && table instanceof FileStoreTable) { - ((FileStoreTable) table).setManifestCache(manifestCache); + if (table instanceof FileStoreTable) { + FileStoreTable storeTable = (FileStoreTable) table; + storeTable.setSnapshotCache( + Caffeine.newBuilder() + .softValues() + .expireAfterAccess(expirationInterval) + .maximumSize(snapshotMaxNumPerTable) + .executor(Runnable::run) + .build()); + storeTable.setStatsCache( + Caffeine.newBuilder() + .softValues() + .expireAfterAccess(expirationInterval) + .maximumSize(5) + .executor(Runnable::run) + .build()); + if (manifestCache != null) { + storeTable.setManifestCache(manifestCache); + } } + tableCache.put(identifier, table); } @@ -274,39 +297,14 @@ public void dropPartition(Identifier identifier, Map partitions) } } - private class TableInvalidatingRemovalListener implements RemovalListener { - @Override - public void onRemoval(Identifier identifier, Table table, @NonNull RemovalCause cause) { - LOG.debug("Evicted {} from the table cache ({})", identifier, cause); - if (RemovalCause.EXPIRED.equals(cause)) { - tryInvalidateSysTables(identifier); - } - } - } - @Override public void invalidateTable(Identifier identifier) { tableCache.invalidate(identifier); - tryInvalidateSysTables(identifier); if (partitionCache != null) { partitionCache.invalidate(identifier); } } - private void tryInvalidateSysTables(Identifier identifier) { - if (!isSpecifiedSystemTable(identifier)) { - tableCache.invalidateAll(allSystemTables(identifier)); - } - } - - private static Iterable allSystemTables(Identifier ident) { - List tables = new ArrayList<>(); - for (String type : SYSTEM_TABLES) { - tables.add(Identifier.fromString(ident.getFullName() + SYSTEM_TABLE_SPLITTER + type)); - } - return tables; - } - // ================================== refresh ================================================ // following caches will affect the latency of table, so refresh method is provided for engine diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java index 6a6a047bd38cd..d919c59782975 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Catalog.java @@ -52,10 +52,12 @@ public interface Catalog extends AutoCloseable { String SYSTEM_TABLE_SPLITTER = "$"; String SYSTEM_DATABASE_NAME = "sys"; String SYSTEM_BRANCH_PREFIX = "branch_"; - String COMMENT_PROP = "comment"; String TABLE_DEFAULT_OPTION_PREFIX = "table-default."; - String DB_LOCATION_PROP = "location"; String DB_SUFFIX = ".db"; + + String COMMENT_PROP = "comment"; + String OWNER_PROP = "owner"; + String DB_LOCATION_PROP = "location"; String NUM_ROWS_PROP = "numRows"; String NUM_FILES_PROP = "numFiles"; String TOTAL_SIZE_PROP = "totalSize"; diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalogOptions.java index 962b249bac677..e656742b42e92 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalogOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalogOptions.java @@ -28,6 +28,7 @@ public final class FileSystemCatalogOptions { ConfigOptions.key("case-sensitive") .booleanType() .defaultValue(true) + .withFallbackKeys("allow-upper-case") .withDescription( "Is case sensitive. If case insensitive, you need to set this option to false, and the table name and fields be converted to lowercase."); diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java b/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java index 72da69b67b839..01456f0b3ae19 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/Identifier.java @@ -65,6 +65,10 @@ public Identifier(String database, String object) { this.object = object; } + public Identifier(String database, String table, @Nullable String branch) { + this(database, table, branch, null); + } + public Identifier( String database, String table, @Nullable String branch, @Nullable String systemTable) { this.database = database; @@ -119,6 +123,10 @@ public String getBranchNameOrDefault() { return systemTable; } + public boolean isSystemTable() { + return getSystemTableName() != null; + } + private void splitObjectName() { if (table != null) { return; diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java index 18ab033fb2762..2fc292e54d340 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/ApplyDeletionVectorReader.java @@ -20,23 +20,22 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.reader.FileRecordIterator; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader; import javax.annotation.Nullable; import java.io.IOException; -import static org.apache.paimon.utils.Preconditions.checkArgument; - /** A {@link RecordReader} which apply {@link DeletionVector} to filter record. */ -public class ApplyDeletionVectorReader implements RecordReader { +public class ApplyDeletionVectorReader implements FileRecordReader { - private final RecordReader reader; + private final FileRecordReader reader; private final DeletionVector deletionVector; public ApplyDeletionVectorReader( - RecordReader reader, DeletionVector deletionVector) { + FileRecordReader reader, DeletionVector deletionVector) { this.reader = reader; this.deletionVector = deletionVector; } @@ -51,19 +50,14 @@ public DeletionVector deletionVector() { @Nullable @Override - public RecordIterator readBatch() throws IOException { - RecordIterator batch = reader.readBatch(); + public FileRecordIterator readBatch() throws IOException { + FileRecordIterator batch = reader.readBatch(); if (batch == null) { return null; } - checkArgument( - batch instanceof FileRecordIterator, - "There is a bug, RecordIterator in ApplyDeletionVectorReader must be RecordWithPositionIterator"); - - return new ApplyDeletionFileRecordIterator( - (FileRecordIterator) batch, deletionVector); + return new ApplyDeletionFileRecordIterator(batch, deletionVector); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java index a2c5925966464..51ae729c2193b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/BitmapDeletionVector.java @@ -117,4 +117,9 @@ public boolean equals(Object o) { BitmapDeletionVector that = (BitmapDeletionVector) o; return Objects.equals(this.roaringBitmap, that.roaringBitmap); } + + @Override + public int hashCode() { + return Objects.hashCode(roaringBitmap); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java index f8c8330f190ca..5246d35d4b31a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorIndexFileWriter.java @@ -20,9 +20,9 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.options.MemorySize; -import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.PathFactory; import org.apache.paimon.utils.Preconditions; @@ -104,13 +104,13 @@ private class SingleIndexFileWriter implements Closeable { private final Path path; private final DataOutputStream dataOutputStream; - private final LinkedHashMap> dvRanges; + private final LinkedHashMap dvMetas; private SingleIndexFileWriter() throws IOException { this.path = indexPathFactory.newPath(); this.dataOutputStream = new DataOutputStream(fileIO.newOutputStream(path, true)); dataOutputStream.writeByte(VERSION_ID_V1); - this.dvRanges = new LinkedHashMap<>(); + this.dvMetas = new LinkedHashMap<>(); } private long writtenSizeInBytes() { @@ -121,7 +121,10 @@ private void write(String key, DeletionVector deletionVector) throws IOException Preconditions.checkNotNull(dataOutputStream); byte[] data = deletionVector.serializeToBytes(); int size = data.length; - dvRanges.put(key, Pair.of(dataOutputStream.size(), size)); + dvMetas.put( + key, + new DeletionVectorMeta( + key, dataOutputStream.size(), size, deletionVector.getCardinality())); dataOutputStream.writeInt(size); dataOutputStream.write(data); dataOutputStream.writeInt(calculateChecksum(data)); @@ -132,8 +135,8 @@ public IndexFileMeta writtenIndexFile() { DELETION_VECTORS_INDEX, path.getName(), writtenSizeInBytes(), - dvRanges.size(), - dvRanges); + dvMetas.size(), + dvMetas); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsIndexFile.java b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsIndexFile.java index 798404e001e52..77abb2d729859 100644 --- a/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsIndexFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/deletionvectors/DeletionVectorsIndexFile.java @@ -21,11 +21,11 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.SeekableInputStream; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFile; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.options.MemorySize; import org.apache.paimon.table.source.DeletionFile; -import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.PathFactory; import java.io.DataInputStream; @@ -63,9 +63,9 @@ public DeletionVectorsIndexFile( * @throws UncheckedIOException If an I/O error occurs while reading from the file. */ public Map readAllDeletionVectors(IndexFileMeta fileMeta) { - LinkedHashMap> deletionVectorRanges = - fileMeta.deletionVectorsRanges(); - checkNotNull(deletionVectorRanges); + LinkedHashMap deletionVectorMetas = + fileMeta.deletionVectorMetas(); + checkNotNull(deletionVectorMetas); String indexFileName = fileMeta.fileName(); Map deletionVectors = new HashMap<>(); @@ -73,18 +73,17 @@ public Map readAllDeletionVectors(IndexFileMeta fileMeta try (SeekableInputStream inputStream = fileIO.newInputStream(filePath)) { checkVersion(inputStream); DataInputStream dataInputStream = new DataInputStream(inputStream); - for (Map.Entry> entry : - deletionVectorRanges.entrySet()) { + for (DeletionVectorMeta deletionVectorMeta : deletionVectorMetas.values()) { deletionVectors.put( - entry.getKey(), - readDeletionVector(dataInputStream, entry.getValue().getRight())); + deletionVectorMeta.dataFileName(), + readDeletionVector(dataInputStream, deletionVectorMeta.length())); } } catch (Exception e) { throw new RuntimeException( "Unable to read deletion vectors from file: " + filePath - + ", deletionVectorRanges: " - + deletionVectorRanges, + + ", deletionVectorMetas: " + + deletionVectorMetas, e); } return deletionVectors; diff --git a/paimon-core/src/main/java/org/apache/paimon/disk/FileChannelManagerImpl.java b/paimon-core/src/main/java/org/apache/paimon/disk/FileChannelManagerImpl.java index ce175e90bbd1e..99690d426fb52 100644 --- a/paimon-core/src/main/java/org/apache/paimon/disk/FileChannelManagerImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/disk/FileChannelManagerImpl.java @@ -29,7 +29,9 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.Random; import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; @@ -63,24 +65,32 @@ public FileChannelManagerImpl(String[] tempDirs, String prefix) { } private static File[] createFiles(String[] tempDirs, String prefix) { - File[] files = new File[tempDirs.length]; + List filesList = new ArrayList<>(); for (int i = 0; i < tempDirs.length; i++) { File baseDir = new File(tempDirs[i]); String subfolder = String.format("paimon-%s-%s", prefix, UUID.randomUUID()); File storageDir = new File(baseDir, subfolder); if (!storageDir.exists() && !storageDir.mkdirs()) { - throw new RuntimeException( - "Could not create storage directory for FileChannelManager: " - + storageDir.getAbsolutePath()); + LOG.warn( + "Failed to create directory {}, temp directory {} will not be used", + storageDir.getAbsolutePath(), + tempDirs[i]); + continue; } - files[i] = storageDir; + + filesList.add(storageDir); LOG.debug( "FileChannelManager uses directory {} for spill files.", storageDir.getAbsolutePath()); } - return files; + + if (filesList.isEmpty()) { + throw new RuntimeException("No available temporary directories"); + } + + return filesList.toArray(new File[0]); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java index 1b952c1716cf9..f561546e8bb31 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java @@ -112,22 +112,7 @@ public AbstractIcebergCommitCallback(FileStoreTable table, String commitUser) { break; case HADOOP_CATALOG: case HIVE_CATALOG: - Path dbPath = table.location().getParent(); - final String dbSuffix = ".db"; - if (dbPath.getName().endsWith(dbSuffix)) { - String dbName = - dbPath.getName() - .substring(0, dbPath.getName().length() - dbSuffix.length()); - String tableName = table.location().getName(); - Path separatePath = - new Path( - dbPath.getParent(), - String.format("iceberg/%s/%s/metadata", dbName, tableName)); - this.pathFactory = new IcebergPathFactory(separatePath); - } else { - throw new UnsupportedOperationException( - "Storage type ICEBERG_WAREHOUSE can only be used on Paimon tables in a Paimon warehouse."); - } + this.pathFactory = new IcebergPathFactory(catalogTableMetadataPath(table)); break; default: throw new UnsupportedOperationException( @@ -152,6 +137,24 @@ public AbstractIcebergCommitCallback(FileStoreTable table, String commitUser) { this.manifestList = IcebergManifestList.create(table, pathFactory); } + public static Path catalogTableMetadataPath(FileStoreTable table) { + Path icebergDBPath = catalogDatabasePath(table); + return new Path(icebergDBPath, String.format("%s/metadata", table.location().getName())); + } + + public static Path catalogDatabasePath(FileStoreTable table) { + Path dbPath = table.location().getParent(); + final String dbSuffix = ".db"; + if (dbPath.getName().endsWith(dbSuffix)) { + String dbName = + dbPath.getName().substring(0, dbPath.getName().length() - dbSuffix.length()); + return new Path(dbPath.getParent(), String.format("iceberg/%s/", dbName)); + } else { + throw new UnsupportedOperationException( + "Storage type ICEBERG_WAREHOUSE can only be used on Paimon tables in a Paimon warehouse."); + } + } + @Override public void close() throws Exception {} @@ -292,7 +295,8 @@ private List dataSplitToManifestEntries( rawFile.rowCount(), rawFile.fileSize(), schemaCache.get(paimonFileMeta.schemaId()), - paimonFileMeta.valueStats()); + paimonFileMeta.valueStats(), + paimonFileMeta.valueStatsCols()); result.add( new IcebergManifestEntry( IcebergManifestEntry.Status.ADDED, @@ -506,7 +510,8 @@ private List createNewlyAddedManifestFileMetas( paimonFileMeta.rowCount(), paimonFileMeta.fileSize(), schemaCache.get(paimonFileMeta.schemaId()), - paimonFileMeta.valueStats()); + paimonFileMeta.valueStats(), + paimonFileMeta.valueStatsCols()); return new IcebergManifestEntry( IcebergManifestEntry.Status.ADDED, currentSnapshotId, diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java index 769ce6b1614af..55fbab5158fa0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/IcebergOptions.java @@ -70,6 +70,26 @@ public class IcebergOptions { .noDefaultValue() .withDescription("hadoop-conf-dir for Iceberg Hive catalog."); + public static final ConfigOption MANIFEST_COMPRESSION = + key("metadata.iceberg.manifest-compression") + .stringType() + .defaultValue( + "snappy") // some Iceberg reader cannot support zstd, for example DuckDB + .withDescription("Compression for Iceberg manifest files."); + + public static final ConfigOption MANIFEST_LEGACY_VERSION = + key("metadata.iceberg.manifest-legacy-version") + .booleanType() + .defaultValue(false) + .withDescription( + "Should use the legacy manifest version to generate Iceberg's 1.4 manifest files."); + + public static final ConfigOption HIVE_CLIENT_CLASS = + key("metadata.iceberg.hive-client-class") + .stringType() + .defaultValue("org.apache.hadoop.hive.metastore.HiveMetaStoreClient") + .withDescription("Hive client class name for Iceberg Hive Catalog."); + /** Where to store Iceberg metadata. */ public enum StorageType implements DescribedEnum { DISABLED("disabled", "Disable Iceberg compatibility support."), diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergDataFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergDataFileMeta.java index 10dbf3d237de3..d171962becad0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergDataFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergDataFileMeta.java @@ -22,13 +22,15 @@ import org.apache.paimon.data.GenericMap; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.iceberg.metadata.IcebergDataField; import org.apache.paimon.iceberg.metadata.IcebergSchema; import org.apache.paimon.stats.SimpleStats; import org.apache.paimon.types.DataField; -import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import javax.annotation.Nullable; + import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -110,27 +112,44 @@ public static IcebergDataFileMeta create( long recordCount, long fileSizeInBytes, IcebergSchema icebergSchema, - SimpleStats stats) { + SimpleStats stats, + @Nullable List statsColumns) { + int numFields = icebergSchema.fields().size(); + Map indexMap = new HashMap<>(); + if (statsColumns == null) { + for (int i = 0; i < numFields; i++) { + indexMap.put(icebergSchema.fields().get(i).name(), i); + } + } else { + for (int i = 0; i < statsColumns.size(); i++) { + indexMap.put(statsColumns.get(i), i); + } + } + Map nullValueCounts = new HashMap<>(); Map lowerBounds = new HashMap<>(); Map upperBounds = new HashMap<>(); - List fieldGetters = new ArrayList<>(); - int numFields = icebergSchema.fields().size(); for (int i = 0; i < numFields; i++) { - fieldGetters.add( - InternalRow.createFieldGetter(icebergSchema.fields().get(i).dataType(), i)); - } + IcebergDataField field = icebergSchema.fields().get(i); + if (!indexMap.containsKey(field.name())) { + continue; + } - for (int i = 0; i < numFields; i++) { - int fieldId = icebergSchema.fields().get(i).id(); - DataType type = icebergSchema.fields().get(i).dataType(); - nullValueCounts.put(fieldId, stats.nullCounts().getLong(i)); - Object minValue = fieldGetters.get(i).getFieldOrNull(stats.minValues()); - Object maxValue = fieldGetters.get(i).getFieldOrNull(stats.maxValues()); + int idx = indexMap.get(field.name()); + nullValueCounts.put(field.id(), stats.nullCounts().getLong(idx)); + + InternalRow.FieldGetter fieldGetter = + InternalRow.createFieldGetter(field.dataType(), idx); + Object minValue = fieldGetter.getFieldOrNull(stats.minValues()); + Object maxValue = fieldGetter.getFieldOrNull(stats.maxValues()); if (minValue != null && maxValue != null) { - lowerBounds.put(fieldId, IcebergConversions.toByteBuffer(type, minValue).array()); - upperBounds.put(fieldId, IcebergConversions.toByteBuffer(type, maxValue).array()); + lowerBounds.put( + field.id(), + IcebergConversions.toByteBuffer(field.dataType(), minValue).array()); + upperBounds.put( + field.id(), + IcebergConversions.toByteBuffer(field.dataType(), maxValue).array()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java index d04cf3576a111..5955da6220f87 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java @@ -18,7 +18,7 @@ package org.apache.paimon.iceberg.manifest; -import org.apache.paimon.CoreOptions; +import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.FormatWriterFactory; @@ -26,6 +26,7 @@ import org.apache.paimon.format.SimpleStatsCollector; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.iceberg.IcebergOptions; import org.apache.paimon.iceberg.IcebergPathFactory; import org.apache.paimon.iceberg.manifest.IcebergManifestFileMeta.Content; import org.apache.paimon.iceberg.metadata.IcebergPartitionSpec; @@ -82,29 +83,34 @@ public IcebergManifestFile( this.targetFileSize = targetFileSize; } + @VisibleForTesting + public String compression() { + return compression; + } + public static IcebergManifestFile create(FileStoreTable table, IcebergPathFactory pathFactory) { RowType partitionType = table.schema().logicalPartitionType(); RowType entryType = IcebergManifestEntry.schema(partitionType); - Options manifestFileAvroOptions = Options.fromMap(table.options()); + Options avroOptions = Options.fromMap(table.options()); // https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/ManifestReader.java - manifestFileAvroOptions.set( + avroOptions.set( "avro.row-name-mapping", "org.apache.paimon.avro.generated.record:manifest_entry," + "manifest_entry_data_file:r2," + "r2_partition:r102"); - FileFormat manifestFileAvro = FileFormat.fromIdentifier("avro", manifestFileAvroOptions); + FileFormat manifestFileAvro = FileFormat.fromIdentifier("avro", avroOptions); return new IcebergManifestFile( table.fileIO(), partitionType, manifestFileAvro.createReaderFactory(entryType), manifestFileAvro.createWriterFactory(entryType), - table.coreOptions().manifestCompression(), + avroOptions.get(IcebergOptions.MANIFEST_COMPRESSION), pathFactory.manifestFileFactory(), table.coreOptions().manifestTargetSize()); } public List rollingWrite( - Iterator entries, long sequenceNumber) throws IOException { + Iterator entries, long sequenceNumber) { RollingFileWriter writer = new RollingFileWriter<>( () -> createWriter(sequenceNumber), targetFileSize.getBytes()); @@ -120,10 +126,7 @@ public List rollingWrite( public SingleFileWriter createWriter( long sequenceNumber) { return new IcebergManifestEntryWriter( - writerFactory, - pathFactory.newPath(), - CoreOptions.FILE_COMPRESSION.defaultValue(), - sequenceNumber); + writerFactory, pathFactory.newPath(), compression, sequenceNumber); } private class IcebergManifestEntryWriter diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMeta.java index fd9c2daf7e208..c5fcb6005fcb8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMeta.java @@ -165,7 +165,11 @@ public List partitions() { return partitions; } - public static RowType schema() { + public static RowType schema(boolean legacyVersion) { + return legacyVersion ? schemaForIceberg1_4() : schemaForIcebergNew(); + } + + private static RowType schemaForIcebergNew() { List fields = new ArrayList<>(); fields.add(new DataField(500, "manifest_path", DataTypes.STRING().notNull())); fields.add(new DataField(501, "manifest_length", DataTypes.BIGINT().notNull())); @@ -186,6 +190,29 @@ public static RowType schema() { return new RowType(false, fields); } + private static RowType schemaForIceberg1_4() { + // see https://github.com/apache/iceberg/pull/5338 + // some reader still want old schema, for example, AWS athena + List fields = new ArrayList<>(); + fields.add(new DataField(500, "manifest_path", DataTypes.STRING().notNull())); + fields.add(new DataField(501, "manifest_length", DataTypes.BIGINT().notNull())); + fields.add(new DataField(502, "partition_spec_id", DataTypes.INT().notNull())); + fields.add(new DataField(517, "content", DataTypes.INT().notNull())); + fields.add(new DataField(515, "sequence_number", DataTypes.BIGINT().notNull())); + fields.add(new DataField(516, "min_sequence_number", DataTypes.BIGINT().notNull())); + fields.add(new DataField(503, "added_snapshot_id", DataTypes.BIGINT())); + fields.add(new DataField(504, "added_data_files_count", DataTypes.INT().notNull())); + fields.add(new DataField(505, "existing_data_files_count", DataTypes.INT().notNull())); + fields.add(new DataField(506, "deleted_data_files_count", DataTypes.INT().notNull())); + fields.add(new DataField(512, "added_rows_count", DataTypes.BIGINT().notNull())); + fields.add(new DataField(513, "existing_rows_count", DataTypes.BIGINT().notNull())); + fields.add(new DataField(514, "deleted_rows_count", DataTypes.BIGINT().notNull())); + fields.add( + new DataField( + 508, "partitions", DataTypes.ARRAY(IcebergPartitionSummary.schema()))); + return new RowType(false, fields); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMetaSerializer.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMetaSerializer.java index c40a26e8fdf8f..2b4c9b771c599 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMetaSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFileMetaSerializer.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalRow; import org.apache.paimon.iceberg.manifest.IcebergManifestFileMeta.Content; +import org.apache.paimon.types.RowType; import org.apache.paimon.utils.ObjectSerializer; import java.util.ArrayList; @@ -36,8 +37,8 @@ public class IcebergManifestFileMetaSerializer extends ObjectSerializer { public IcebergManifestList( FileIO fileIO, - FormatReaderFactory readerFactory, - FormatWriterFactory writerFactory, + FileFormat fileFormat, + RowType manifestType, String compression, PathFactory pathFactory) { super( fileIO, - new IcebergManifestFileMetaSerializer(), - IcebergManifestFileMeta.schema(), - readerFactory, - writerFactory, + new IcebergManifestFileMetaSerializer(manifestType), + manifestType, + fileFormat.createReaderFactory(manifestType), + fileFormat.createWriterFactory(manifestType), compression, pathFactory, null); } + @VisibleForTesting + public String compression() { + return compression; + } + public static IcebergManifestList create(FileStoreTable table, IcebergPathFactory pathFactory) { - Options manifestListAvroOptions = Options.fromMap(table.options()); + Options avroOptions = Options.fromMap(table.options()); // https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/ManifestLists.java - manifestListAvroOptions.set( + avroOptions.set( "avro.row-name-mapping", "org.apache.paimon.avro.generated.record:manifest_file," + "manifest_file_partitions:r508"); - FileFormat manifestListAvro = FileFormat.fromIdentifier("avro", manifestListAvroOptions); + FileFormat fileFormat = FileFormat.fromIdentifier("avro", avroOptions); + RowType manifestType = + IcebergManifestFileMeta.schema( + avroOptions.get(IcebergOptions.MANIFEST_LEGACY_VERSION)); return new IcebergManifestList( table.fileIO(), - manifestListAvro.createReaderFactory(IcebergManifestFileMeta.schema()), - manifestListAvro.createWriterFactory(IcebergManifestFileMeta.schema()), - table.coreOptions().manifestCompression(), + fileFormat, + manifestType, + avroOptions.get(IcebergOptions.MANIFEST_COMPRESSION), pathFactory.manifestListFactory()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java index 93cb2ab6de239..4ecc77a13581e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java @@ -18,10 +18,14 @@ package org.apache.paimon.iceberg.metadata; +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.DecimalType; import org.apache.paimon.types.LocalZonedTimestampType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.RowType; import org.apache.paimon.types.TimestampType; import org.apache.paimon.utils.Preconditions; @@ -32,6 +36,7 @@ import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.util.Objects; +import java.util.stream.Collectors; /** * {@link DataField} in Iceberg. @@ -57,7 +62,7 @@ public class IcebergDataField { private final boolean required; @JsonProperty(FIELD_TYPE) - private final String type; + private final Object type; @JsonIgnore private final DataType dataType; @@ -69,7 +74,7 @@ public IcebergDataField(DataField dataField) { dataField.id(), dataField.name(), !dataField.type().isNullable(), - toTypeString(dataField.type()), + toTypeObject(dataField.type(), dataField.id(), 0), dataField.type(), dataField.description()); } @@ -79,13 +84,13 @@ public IcebergDataField( @JsonProperty(FIELD_ID) int id, @JsonProperty(FIELD_NAME) String name, @JsonProperty(FIELD_REQUIRED) boolean required, - @JsonProperty(FIELD_TYPE) String type, + @JsonProperty(FIELD_TYPE) Object type, @JsonProperty(FIELD_DOC) String doc) { this(id, name, required, type, null, doc); } public IcebergDataField( - int id, String name, boolean required, String type, DataType dataType, String doc) { + int id, String name, boolean required, Object type, DataType dataType, String doc) { this.id = id; this.name = name; this.required = required; @@ -110,7 +115,7 @@ public boolean required() { } @JsonGetter(FIELD_TYPE) - public String type() { + public Object type() { return type; } @@ -124,7 +129,7 @@ public DataType dataType() { return Preconditions.checkNotNull(dataType); } - private static String toTypeString(DataType dataType) { + private static Object toTypeObject(DataType dataType, int fieldId, int depth) { switch (dataType.getTypeRoot()) { case BOOLEAN: return "boolean"; @@ -160,6 +165,26 @@ private static String toTypeString(DataType dataType) { timestampLtzPrecision > 3 && timestampLtzPrecision <= 6, "Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6."); return "timestamptz"; + case ARRAY: + ArrayType arrayType = (ArrayType) dataType; + return new IcebergListType( + SpecialFields.getArrayElementFieldId(fieldId, depth + 1), + !dataType.isNullable(), + toTypeObject(arrayType.getElementType(), fieldId, depth + 1)); + case MAP: + MapType mapType = (MapType) dataType; + return new IcebergMapType( + SpecialFields.getMapKeyFieldId(fieldId, depth + 1), + toTypeObject(mapType.getKeyType(), fieldId, depth + 1), + SpecialFields.getMapValueFieldId(fieldId, depth + 1), + !mapType.getValueType().isNullable(), + toTypeObject(mapType.getValueType(), fieldId, depth + 1)); + case ROW: + RowType rowType = (RowType) dataType; + return new IcebergStructType( + rowType.getFields().stream() + .map(IcebergDataField::new) + .collect(Collectors.toList())); default: throw new UnsupportedOperationException("Unsupported data type: " + dataType); } diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergListType.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergListType.java new file mode 100644 index 0000000000000..d25ead64fcb5a --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergListType.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.iceberg.metadata; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +/** + * {@link org.apache.paimon.types.ArrayType} in Iceberg. + * + *

    See Iceberg spec. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class IcebergListType { + + private static final String FIELD_TYPE = "type"; + private static final String FIELD_ELEMENT_ID = "element-id"; + private static final String FIELD_ELEMENT_REQUIRED = "element-required"; + private static final String FIELD_ELEMENT = "element"; + + @JsonProperty(FIELD_TYPE) + private final String type; + + @JsonProperty(FIELD_ELEMENT_ID) + private final int elementId; + + @JsonProperty(FIELD_ELEMENT_REQUIRED) + private final boolean elementRequired; + + @JsonProperty(FIELD_ELEMENT) + private final Object element; + + public IcebergListType(int elementId, boolean elementRequired, Object element) { + this("list", elementId, elementRequired, element); + } + + @JsonCreator + public IcebergListType( + @JsonProperty(FIELD_TYPE) String type, + @JsonProperty(FIELD_ELEMENT_ID) int elementId, + @JsonProperty(FIELD_ELEMENT_REQUIRED) boolean elementRequired, + @JsonProperty(FIELD_ELEMENT) Object element) { + this.type = type; + this.elementId = elementId; + this.elementRequired = elementRequired; + this.element = element; + } + + @JsonGetter(FIELD_TYPE) + public String type() { + return type; + } + + @JsonGetter(FIELD_ELEMENT_ID) + public int elementId() { + return elementId; + } + + @JsonGetter(FIELD_ELEMENT_REQUIRED) + public boolean elementRequired() { + return elementRequired; + } + + @JsonGetter(FIELD_ELEMENT) + public Object element() { + return element; + } + + @Override + public int hashCode() { + return Objects.hash(type, elementId, elementRequired, element); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IcebergListType)) { + return false; + } + + IcebergListType that = (IcebergListType) o; + return Objects.equals(type, that.type) + && elementId == that.elementId + && elementRequired == that.elementRequired + && Objects.equals(element, that.element); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergMapType.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergMapType.java new file mode 100644 index 0000000000000..81a3a04b1f413 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergMapType.java @@ -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.iceberg.metadata; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +/** + * {@link org.apache.paimon.types.MapType} in Iceberg. + * + *

    See Iceberg spec. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class IcebergMapType { + + private static final String FIELD_TYPE = "type"; + private static final String FIELD_KEY_ID = "key-id"; + private static final String FIELD_KEY = "key"; + private static final String FIELD_VALUE_ID = "value-id"; + private static final String FIELD_VALUE_REQUIRED = "value-required"; + private static final String FIELD_VALUE = "value"; + + @JsonProperty(FIELD_TYPE) + private final String type; + + @JsonProperty(FIELD_KEY_ID) + private final int keyId; + + @JsonProperty(FIELD_KEY) + private final Object key; + + @JsonProperty(FIELD_VALUE_ID) + private final int valueId; + + @JsonProperty(FIELD_VALUE_REQUIRED) + private final boolean valueRequired; + + @JsonProperty(FIELD_VALUE) + private final Object value; + + public IcebergMapType(int keyId, Object key, int valueId, boolean valueRequired, Object value) { + this("map", keyId, key, valueId, valueRequired, value); + } + + @JsonCreator + public IcebergMapType( + @JsonProperty(FIELD_TYPE) String type, + @JsonProperty(FIELD_KEY_ID) int keyId, + @JsonProperty(FIELD_KEY) Object key, + @JsonProperty(FIELD_VALUE_ID) int valueId, + @JsonProperty(FIELD_VALUE_REQUIRED) boolean valueRequired, + @JsonProperty(FIELD_VALUE) Object value) { + this.type = type; + this.keyId = keyId; + this.key = key; + this.valueId = valueId; + this.valueRequired = valueRequired; + this.value = value; + } + + @JsonGetter(FIELD_TYPE) + public String type() { + return type; + } + + @JsonGetter(FIELD_KEY_ID) + public int keyId() { + return keyId; + } + + @JsonGetter(FIELD_KEY) + public Object key() { + return key; + } + + @JsonGetter(FIELD_VALUE_ID) + public int valueId() { + return valueId; + } + + @JsonGetter(FIELD_VALUE_REQUIRED) + public boolean valueRequired() { + return valueRequired; + } + + @JsonGetter(FIELD_VALUE) + public Object value() { + return value; + } + + @Override + public int hashCode() { + return Objects.hash(type, keyId, key, valueId, valueRequired, value); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IcebergMapType)) { + return false; + } + IcebergMapType that = (IcebergMapType) o; + return Objects.equals(type, that.type) + && keyId == that.keyId + && Objects.equals(key, that.key) + && valueId == that.valueId + && valueRequired == that.valueRequired + && Objects.equals(value, that.value); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergStructType.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergStructType.java new file mode 100644 index 0000000000000..84b0d430e4385 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergStructType.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.iceberg.metadata; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Objects; + +/** + * {@link org.apache.paimon.types.RowType} in Iceberg. + * + *

    See Iceberg spec. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class IcebergStructType { + + private static final String FIELD_TYPE = "type"; + private static final String FIELD_FIELDS = "fields"; + + @JsonProperty(FIELD_TYPE) + private final String type; + + @JsonProperty(FIELD_FIELDS) + private final List fields; + + public IcebergStructType(List fields) { + this("struct", fields); + } + + @JsonCreator + public IcebergStructType( + @JsonProperty(FIELD_TYPE) String type, + @JsonProperty(FIELD_FIELDS) List fields) { + this.type = type; + this.fields = fields; + } + + @JsonGetter(FIELD_TYPE) + public String type() { + return type; + } + + @JsonGetter(FIELD_FIELDS) + public List fields() { + return fields; + } + + @Override + public int hashCode() { + return Objects.hash(type, fields); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof IcebergStructType)) { + return false; + } + + IcebergStructType that = (IcebergStructType) o; + return Objects.equals(type, that.type) && Objects.equals(fields, that.fields); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/index/DeletionVectorMeta.java b/paimon-core/src/main/java/org/apache/paimon/index/DeletionVectorMeta.java new file mode 100644 index 0000000000000..9eb38818f694f --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/index/DeletionVectorMeta.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.index; + +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.RowType; + +import javax.annotation.Nullable; + +import java.util.Objects; + +import static org.apache.paimon.utils.SerializationUtils.newStringType; + +/** Metadata of deletion vector. */ +public class DeletionVectorMeta { + + public static final RowType SCHEMA = + RowType.of( + new DataField(0, "f0", newStringType(false)), + new DataField(1, "f1", new IntType(false)), + new DataField(2, "f2", new IntType(false)), + new DataField(3, "_CARDINALITY", new BigIntType(true))); + + private final String dataFileName; + private final int offset; + private final int length; + @Nullable private final Long cardinality; + + public DeletionVectorMeta( + String dataFileName, int start, int size, @Nullable Long cardinality) { + this.dataFileName = dataFileName; + this.offset = start; + this.length = size; + this.cardinality = cardinality; + } + + public String dataFileName() { + return dataFileName; + } + + public int offset() { + return offset; + } + + public int length() { + return length; + } + + @Nullable + public Long cardinality() { + return cardinality; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + DeletionVectorMeta that = (DeletionVectorMeta) o; + return offset == that.offset + && length == that.length + && Objects.equals(dataFileName, that.dataFileName) + && Objects.equals(cardinality, that.cardinality); + } + + @Override + public int hashCode() { + return Objects.hash(dataFileName, offset, length, cardinality); + } + + @Override + public String toString() { + return "DeletionVectorMeta{" + + "dataFileName='" + + dataFileName + + '\'' + + ", offset=" + + offset + + ", length=" + + length + + ", cardinality=" + + cardinality + + '}'; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileHandler.java b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileHandler.java index 7e5efccdd813a..8b0e5c5021f68 100644 --- a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileHandler.java +++ b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileHandler.java @@ -100,15 +100,16 @@ public Map scanDVIndex( if (meta.indexType().equals(DELETION_VECTORS_INDEX) && file.partition().equals(partition) && file.bucket() == bucket) { - LinkedHashMap> dvRanges = - meta.deletionVectorsRanges(); - checkNotNull(dvRanges); - for (String dataFile : dvRanges.keySet()) { - Pair pair = dvRanges.get(dataFile); - DeletionFile deletionFile = + LinkedHashMap dvMetas = meta.deletionVectorMetas(); + checkNotNull(dvMetas); + for (DeletionVectorMeta dvMeta : dvMetas.values()) { + result.put( + dvMeta.dataFileName(), new DeletionFile( - filePath(meta).toString(), pair.getLeft(), pair.getRight()); - result.put(dataFile, deletionFile); + filePath(meta).toString(), + dvMeta.offset(), + dvMeta.length(), + dvMeta.cardinality())); } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta.java index 24ba6992a5d9c..aae4f8c4731bd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta.java @@ -23,9 +23,7 @@ import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.DataField; -import org.apache.paimon.types.IntType; import org.apache.paimon.types.RowType; -import org.apache.paimon.utils.Pair; import javax.annotation.Nullable; @@ -54,12 +52,7 @@ public class IndexFileMeta { new DataField( 4, "_DELETIONS_VECTORS_RANGES", - new ArrayType( - true, - RowType.of( - newStringType(false), - new IntType(false), - new IntType(false)))))); + new ArrayType(true, DeletionVectorMeta.SCHEMA)))); private final String indexType; private final String fileName; @@ -68,9 +61,9 @@ public class IndexFileMeta { /** * Metadata only used by {@link DeletionVectorsIndexFile}, use LinkedHashMap to ensure that the - * order of DeletionVectorRanges and the written DeletionVectors is consistent. + * order of DeletionVectorMetas and the written DeletionVectors is consistent. */ - private final @Nullable LinkedHashMap> deletionVectorsRanges; + private final @Nullable LinkedHashMap deletionVectorMetas; public IndexFileMeta(String indexType, String fileName, long fileSize, long rowCount) { this(indexType, fileName, fileSize, rowCount, null); @@ -81,12 +74,12 @@ public IndexFileMeta( String fileName, long fileSize, long rowCount, - @Nullable LinkedHashMap> deletionVectorsRanges) { + @Nullable LinkedHashMap deletionVectorMetas) { this.indexType = indexType; this.fileName = fileName; this.fileSize = fileSize; this.rowCount = rowCount; - this.deletionVectorsRanges = deletionVectorsRanges; + this.deletionVectorMetas = deletionVectorMetas; } public String indexType() { @@ -105,8 +98,8 @@ public long rowCount() { return rowCount; } - public @Nullable LinkedHashMap> deletionVectorsRanges() { - return deletionVectorsRanges; + public @Nullable LinkedHashMap deletionVectorMetas() { + return deletionVectorMetas; } @Override @@ -122,12 +115,12 @@ public boolean equals(Object o) { && Objects.equals(fileName, that.fileName) && fileSize == that.fileSize && rowCount == that.rowCount - && Objects.equals(deletionVectorsRanges, that.deletionVectorsRanges); + && Objects.equals(deletionVectorMetas, that.deletionVectorMetas); } @Override public int hashCode() { - return Objects.hash(indexType, fileName, fileSize, rowCount, deletionVectorsRanges); + return Objects.hash(indexType, fileName, fileSize, rowCount, deletionVectorMetas); } @Override @@ -142,8 +135,8 @@ public String toString() { + fileSize + ", rowCount=" + rowCount - + ", deletionVectorsRanges=" - + deletionVectorsRanges + + ", deletionVectorMetas=" + + deletionVectorMetas + '}'; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta09Serializer.java b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta09Serializer.java new file mode 100644 index 0000000000000..915d904569d70 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMeta09Serializer.java @@ -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.index; + +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.serializer.InternalRowSerializer; +import org.apache.paimon.data.serializer.InternalSerializers; +import org.apache.paimon.io.DataInputView; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.RowType; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; + +import static org.apache.paimon.utils.SerializationUtils.newStringType; + +/** Serializer for {@link IndexFileMeta} with 0.9 version. */ +public class IndexFileMeta09Serializer implements Serializable { + + private static final long serialVersionUID = 1L; + + public static final RowType SCHEMA = + new RowType( + false, + Arrays.asList( + new DataField(0, "_INDEX_TYPE", newStringType(false)), + new DataField(1, "_FILE_NAME", newStringType(false)), + new DataField(2, "_FILE_SIZE", new BigIntType(false)), + new DataField(3, "_ROW_COUNT", new BigIntType(false)), + new DataField( + 4, + "_DELETIONS_VECTORS_RANGES", + new ArrayType( + true, + RowType.of( + newStringType(false), + new IntType(false), + new IntType(false)))))); + + protected final InternalRowSerializer rowSerializer; + + public IndexFileMeta09Serializer() { + this.rowSerializer = InternalSerializers.create(SCHEMA); + } + + public IndexFileMeta fromRow(InternalRow row) { + return new IndexFileMeta( + row.getString(0).toString(), + row.getString(1).toString(), + row.getLong(2), + row.getLong(3), + row.isNullAt(4) ? null : rowArrayDataToDvMetas(row.getArray(4))); + } + + public final List deserializeList(DataInputView source) throws IOException { + int size = source.readInt(); + List records = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + records.add(deserialize(source)); + } + return records; + } + + public IndexFileMeta deserialize(DataInputView in) throws IOException { + return fromRow(rowSerializer.deserialize(in)); + } + + public static LinkedHashMap rowArrayDataToDvMetas( + InternalArray arrayData) { + LinkedHashMap dvMetas = new LinkedHashMap<>(arrayData.size()); + for (int i = 0; i < arrayData.size(); i++) { + InternalRow row = arrayData.getRow(i, 3); + dvMetas.put( + row.getString(0).toString(), + new DeletionVectorMeta( + row.getString(0).toString(), row.getInt(1), row.getInt(2), null)); + } + return dvMetas; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java index 4b52932623f2a..db4a44838fbf2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/index/IndexFileMetaSerializer.java @@ -24,9 +24,9 @@ import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalRow; import org.apache.paimon.utils.ObjectSerializer; -import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.VersionedObjectSerializer; +import java.util.Collection; import java.util.LinkedHashMap; /** A {@link VersionedObjectSerializer} for {@link IndexFileMeta}. */ @@ -43,9 +43,9 @@ public InternalRow toRow(IndexFileMeta record) { BinaryString.fromString(record.fileName()), record.fileSize(), record.rowCount(), - record.deletionVectorsRanges() == null + record.deletionVectorMetas() == null ? null - : dvRangesToRowArrayData(record.deletionVectorsRanges())); + : dvMetasToRowArrayData(record.deletionVectorMetas().values())); } @Override @@ -55,30 +55,35 @@ public IndexFileMeta fromRow(InternalRow row) { row.getString(1).toString(), row.getLong(2), row.getLong(3), - row.isNullAt(4) ? null : rowArrayDataToDvRanges(row.getArray(4))); + row.isNullAt(4) ? null : rowArrayDataToDvMetas(row.getArray(4))); } - public static InternalArray dvRangesToRowArrayData( - LinkedHashMap> dvRanges) { + public static InternalArray dvMetasToRowArrayData(Collection dvMetas) { return new GenericArray( - dvRanges.entrySet().stream() + dvMetas.stream() .map( - entry -> + dvMeta -> GenericRow.of( - BinaryString.fromString(entry.getKey()), - entry.getValue().getLeft(), - entry.getValue().getRight())) + BinaryString.fromString(dvMeta.dataFileName()), + dvMeta.offset(), + dvMeta.length(), + dvMeta.cardinality())) .toArray(GenericRow[]::new)); } - public static LinkedHashMap> rowArrayDataToDvRanges( + public static LinkedHashMap rowArrayDataToDvMetas( InternalArray arrayData) { - LinkedHashMap> dvRanges = - new LinkedHashMap<>(arrayData.size()); + LinkedHashMap dvMetas = new LinkedHashMap<>(arrayData.size()); for (int i = 0; i < arrayData.size(); i++) { - InternalRow row = arrayData.getRow(i, 3); - dvRanges.put(row.getString(0).toString(), Pair.of(row.getInt(1), row.getInt(2))); + InternalRow row = arrayData.getRow(i, DeletionVectorMeta.SCHEMA.getFieldCount()); + dvMetas.put( + row.getString(0).toString(), + new DeletionVectorMeta( + row.getString(0).toString(), + row.getInt(1), + row.getInt(2), + row.isNullAt(3) ? null : row.getLong(3))); } - return dvRanges; + return dvMetas; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java index b6cac5ae51304..bb9e45ff002da 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java @@ -409,6 +409,27 @@ public DataFileMeta rename(String newFileName) { valueStatsCols); } + public DataFileMeta copyWithoutStats() { + return new DataFileMeta( + fileName, + fileSize, + rowCount, + minKey, + maxKey, + keyStats, + EMPTY_STATS, + minSequenceNumber, + maxSequenceNumber, + schemaId, + level, + extraFiles, + creationTime, + deleteRowCount, + embeddedIndex, + fileSource, + Collections.emptyList()); + } + public List collectFiles(DataFilePathFactory pathFactory) { List paths = new ArrayList<>(); paths.add(pathFactory.toPath(fileName)); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/FileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java similarity index 88% rename from paimon-core/src/main/java/org/apache/paimon/io/FileRecordReader.java rename to paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java index 1e12025ba5330..16fad55a49a27 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/FileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFileRecordReader.java @@ -25,7 +25,8 @@ import org.apache.paimon.data.PartitionInfo; import org.apache.paimon.data.columnar.ColumnarRowIterator; import org.apache.paimon.format.FormatReaderFactory; -import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.reader.FileRecordIterator; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.utils.FileUtils; import org.apache.paimon.utils.ProjectedRow; @@ -34,17 +35,35 @@ import java.io.IOException; /** Reads {@link InternalRow} from data files. */ -public class FileRecordReader implements RecordReader { +public class DataFileRecordReader implements FileRecordReader { - private final RecordReader reader; + private final FileRecordReader reader; @Nullable private final int[] indexMapping; @Nullable private final PartitionInfo partitionInfo; @Nullable private final CastFieldGetter[] castMapping; + public DataFileRecordReader( + FormatReaderFactory readerFactory, + FormatReaderFactory.Context context, + @Nullable int[] indexMapping, + @Nullable CastFieldGetter[] castMapping, + @Nullable PartitionInfo partitionInfo) + throws IOException { + try { + this.reader = readerFactory.createReader(context); + } catch (Exception e) { + FileUtils.checkExists(context.fileIO(), context.filePath()); + throw e; + } + this.indexMapping = indexMapping; + this.partitionInfo = partitionInfo; + this.castMapping = castMapping; + } + @Nullable @Override - public RecordReader.RecordIterator readBatch() throws IOException { - RecordIterator iterator = reader.readBatch(); + public FileRecordIterator readBatch() throws IOException { + FileRecordIterator iterator = reader.readBatch(); if (iterator == null) { return null; } @@ -57,6 +76,7 @@ public RecordReader.RecordIterator readBatch() throws IOException { PartitionSettedRow.from(partitionInfo); iterator = iterator.transform(partitionSettedRow::replaceRow); } + if (indexMapping != null) { final ProjectedRow projectedRow = ProjectedRow.from(indexMapping); iterator = iterator.transform(projectedRow::replaceRow); @@ -71,24 +91,6 @@ public RecordReader.RecordIterator readBatch() throws IOException { return iterator; } - public FileRecordReader( - FormatReaderFactory readerFactory, - FormatReaderFactory.Context context, - @Nullable int[] indexMapping, - @Nullable CastFieldGetter[] castMapping, - @Nullable PartitionInfo partitionInfo) - throws IOException { - try { - this.reader = readerFactory.createReader(context); - } catch (Exception e) { - FileUtils.checkExists(context.fileIO(), context.filePath()); - throw e; - } - this.indexMapping = indexMapping; - this.partitionInfo = partitionInfo; - this.castMapping = castMapping; - } - @Override public void close() throws IOException { reader.close(); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java b/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java index c34d1b0d3ba75..530b87165322f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java @@ -40,6 +40,15 @@ public static FileIndexResult evaluate( DataFileMeta file) throws IOException { if (dataFilter != null && !dataFilter.isEmpty()) { + byte[] embeddedIndex = file.embeddedIndex(); + if (embeddedIndex != null) { + try (FileIndexPredicate predicate = + new FileIndexPredicate(embeddedIndex, dataSchema.logicalRowType())) { + return predicate.evaluate( + PredicateBuilder.and(dataFilter.toArray(new Predicate[0]))); + } + } + List indexFiles = file.extraFiles().stream() .filter(name -> name.endsWith(DataFilePathFactory.INDEX_PATH_SUFFIX)) diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java index e44ad79ff53ee..6cf08769703fc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileRecordReader.java @@ -21,6 +21,8 @@ import org.apache.paimon.KeyValue; import org.apache.paimon.KeyValueSerializer; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.reader.FileRecordIterator; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.types.RowType; @@ -29,14 +31,14 @@ import java.io.IOException; /** {@link RecordReader} for reading {@link KeyValue} data files. */ -public class KeyValueDataFileRecordReader implements RecordReader { +public class KeyValueDataFileRecordReader implements FileRecordReader { - private final RecordReader reader; + private final FileRecordReader reader; private final KeyValueSerializer serializer; private final int level; public KeyValueDataFileRecordReader( - RecordReader reader, RowType keyType, RowType valueType, int level) { + FileRecordReader reader, RowType keyType, RowType valueType, int level) { this.reader = reader; this.serializer = new KeyValueSerializer(keyType, valueType); this.level = level; @@ -44,8 +46,8 @@ public KeyValueDataFileRecordReader( @Nullable @Override - public RecordIterator readBatch() throws IOException { - RecordReader.RecordIterator iterator = reader.readBatch(); + public FileRecordIterator readBatch() throws IOException { + FileRecordIterator iterator = reader.readBatch(); if (iterator == null) { return null; } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java index ce0b3b02840ba..651c6a6f7b564 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java @@ -42,7 +42,6 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.function.Function; @@ -56,13 +55,13 @@ *

    NOTE: records given to the writer must be sorted because it does not compare the min max keys * to produce {@link DataFileMeta}. */ -public class KeyValueDataFileWriter +public abstract class KeyValueDataFileWriter extends StatsCollectingSingleFileWriter { private static final Logger LOG = LoggerFactory.getLogger(KeyValueDataFileWriter.class); - private final RowType keyType; - private final RowType valueType; + protected final RowType keyType; + protected final RowType valueType; private final long schemaId; private final int level; @@ -85,6 +84,7 @@ public KeyValueDataFileWriter( Function converter, RowType keyType, RowType valueType, + RowType writeRowType, @Nullable SimpleStatsExtractor simpleStatsExtractor, long schemaId, int level, @@ -97,11 +97,11 @@ public KeyValueDataFileWriter( factory, path, converter, - KeyValue.schema(keyType, valueType), + writeRowType, simpleStatsExtractor, compression, StatsCollectorFactories.createStatsFactories( - options, KeyValue.schema(keyType, valueType).getFieldNames()), + options, writeRowType.getFieldNames(), keyType.getFieldNames()), options.asyncFileWrite()); this.keyType = keyType; @@ -166,17 +166,11 @@ public DataFileMeta result() throws IOException { return null; } - SimpleColStats[] rowStats = fieldStats(); - int numKeyFields = keyType.getFieldCount(); - - SimpleColStats[] keyFieldStats = Arrays.copyOfRange(rowStats, 0, numKeyFields); - SimpleStats keyStats = keyStatsConverter.toBinaryAllMode(keyFieldStats); - - SimpleColStats[] valFieldStats = - Arrays.copyOfRange(rowStats, numKeyFields + 2, rowStats.length); + Pair keyValueStats = fetchKeyValueStats(fieldStats()); + SimpleStats keyStats = keyStatsConverter.toBinaryAllMode(keyValueStats.getKey()); Pair, SimpleStats> valueStatsPair = - valueStatsConverter.toBinary(valFieldStats); + valueStatsConverter.toBinary(keyValueStats.getValue()); DataFileIndexWriter.FileIndexResult indexResult = dataFileIndexWriter == null @@ -204,6 +198,8 @@ public DataFileMeta result() throws IOException { valueStatsPair.getKey()); } + abstract Pair fetchKeyValueStats(SimpleColStats[] rowStats); + @Override public void close() throws IOException { if (dataFileIndexWriter != null) { diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriterImpl.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriterImpl.java new file mode 100644 index 0000000000000..27a1aef64e36f --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriterImpl.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.io; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.KeyValue; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fileindex.FileIndexOptions; +import org.apache.paimon.format.FormatWriterFactory; +import org.apache.paimon.format.SimpleColStats; +import org.apache.paimon.format.SimpleStatsExtractor; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.manifest.FileSource; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.Pair; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.function.Function; + +/** Write data files containing {@link KeyValue}s. */ +public class KeyValueDataFileWriterImpl extends KeyValueDataFileWriter { + + public KeyValueDataFileWriterImpl( + FileIO fileIO, + FormatWriterFactory factory, + Path path, + Function converter, + RowType keyType, + RowType valueType, + @Nullable SimpleStatsExtractor simpleStatsExtractor, + long schemaId, + int level, + String compression, + CoreOptions options, + FileSource fileSource, + FileIndexOptions fileIndexOptions) { + super( + fileIO, + factory, + path, + converter, + keyType, + valueType, + KeyValue.schema(keyType, valueType), + simpleStatsExtractor, + schemaId, + level, + compression, + options, + fileSource, + fileIndexOptions); + } + + @Override + Pair fetchKeyValueStats(SimpleColStats[] rowStats) { + int numKeyFields = keyType.getFieldCount(); + return Pair.of( + Arrays.copyOfRange(rowStats, 0, numKeyFields), + Arrays.copyOfRange(rowStats, numKeyFields + 2, rowStats.length)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java index fdbb727e56747..7e272fc97c654 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java @@ -32,6 +32,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.partition.PartitionUtils; import org.apache.paimon.predicate.Predicate; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.KeyValueFieldsExtractor; import org.apache.paimon.schema.SchemaManager; @@ -39,9 +40,8 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.AsyncRecordReader; -import org.apache.paimon.utils.BulkFormatMapping; -import org.apache.paimon.utils.BulkFormatMapping.BulkFormatMappingBuilder; import org.apache.paimon.utils.FileStorePathFactory; +import org.apache.paimon.utils.FormatReaderMapping; import javax.annotation.Nullable; @@ -63,11 +63,11 @@ public class KeyValueFileReaderFactory implements FileReaderFactory { private final RowType keyType; private final RowType valueType; - private final BulkFormatMappingBuilder bulkFormatMappingBuilder; + private final FormatReaderMapping.Builder formatReaderMappingBuilder; private final DataFilePathFactory pathFactory; private final long asyncThreshold; - private final Map bulkFormatMappings; + private final Map formatReaderMappings; private final BinaryRow partition; private final DeletionVector.Factory dvFactory; @@ -77,7 +77,7 @@ private KeyValueFileReaderFactory( TableSchema schema, RowType keyType, RowType valueType, - BulkFormatMappingBuilder bulkFormatMappingBuilder, + FormatReaderMapping.Builder formatReaderMappingBuilder, DataFilePathFactory pathFactory, long asyncThreshold, BinaryRow partition, @@ -87,11 +87,11 @@ private KeyValueFileReaderFactory( this.schema = schema; this.keyType = keyType; this.valueType = valueType; - this.bulkFormatMappingBuilder = bulkFormatMappingBuilder; + this.formatReaderMappingBuilder = formatReaderMappingBuilder; this.pathFactory = pathFactory; this.asyncThreshold = asyncThreshold; this.partition = partition; - this.bulkFormatMappings = new HashMap<>(); + this.formatReaderMappings = new HashMap<>(); this.dvFactory = dvFactory; } @@ -109,7 +109,7 @@ public RecordReader createRecordReader( return createRecordReader(schemaId, fileName, level, true, null, fileSize); } - private RecordReader createRecordReader( + private FileRecordReader createRecordReader( long schemaId, String fileName, int level, @@ -119,31 +119,31 @@ private RecordReader createRecordReader( throws IOException { String formatIdentifier = DataFilePathFactory.formatIdentifier(fileName); - Supplier formatSupplier = + Supplier formatSupplier = () -> - bulkFormatMappingBuilder.build( + formatReaderMappingBuilder.build( formatIdentifier, schema, schemaId == schema.id() ? schema : schemaManager.schema(schemaId)); - BulkFormatMapping bulkFormatMapping = + FormatReaderMapping formatReaderMapping = reuseFormat - ? bulkFormatMappings.computeIfAbsent( + ? formatReaderMappings.computeIfAbsent( new FormatKey(schemaId, formatIdentifier), key -> formatSupplier.get()) : formatSupplier.get(); Path filePath = pathFactory.toPath(fileName); - RecordReader fileRecordReader = - new FileRecordReader( - bulkFormatMapping.getReaderFactory(), + FileRecordReader fileRecordReader = + new DataFileRecordReader( + formatReaderMapping.getReaderFactory(), orcPoolSize == null ? new FormatReaderContext(fileIO, filePath, fileSize) : new OrcFormatReaderContext( fileIO, filePath, fileSize, orcPoolSize), - bulkFormatMapping.getIndexMapping(), - bulkFormatMapping.getCastMapping(), - PartitionUtils.create(bulkFormatMapping.getPartitionPair(), partition)); + formatReaderMapping.getIndexMapping(), + formatReaderMapping.getCastMapping(), + PartitionUtils.create(formatReaderMapping.getPartitionPair(), partition)); Optional deletionVector = dvFactory.create(fileName); if (deletionVector.isPresent() && !deletionVector.get().isEmpty()) { @@ -275,7 +275,7 @@ public KeyValueFileReaderFactory build( schema, finalReadKeyType, readValueType, - new BulkFormatMappingBuilder( + new FormatReaderMapping.Builder( formatDiscover, readTableFields, fieldsExtractor, filters), pathFactory.createDataFilePathFactory(partition, bucket), options.fileReaderAsyncThreshold().getBytes(), diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java index a6fddb43283a1..a6aae3985bd4c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.KeyValue; import org.apache.paimon.KeyValueSerializer; +import org.apache.paimon.KeyValueThinSerializer; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.fileindex.FileIndexOptions; @@ -31,6 +32,8 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.FileSource; import org.apache.paimon.statistics.SimpleColStatsCollector; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.types.DataField; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.StatsCollectorFactories; @@ -38,10 +41,13 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.function.Function; +import java.util.stream.Collectors; /** A factory to create {@link FileWriter}s for writing {@link KeyValue} files. */ public class KeyValueFileWriterFactory { @@ -58,15 +64,13 @@ public class KeyValueFileWriterFactory { private KeyValueFileWriterFactory( FileIO fileIO, long schemaId, - RowType keyType, - RowType valueType, WriteFormatContext formatContext, long suggestedFileSize, CoreOptions options) { this.fileIO = fileIO; this.schemaId = schemaId; - this.keyType = keyType; - this.valueType = valueType; + this.keyType = formatContext.keyType; + this.valueType = formatContext.valueType; this.formatContext = formatContext; this.suggestedFileSize = suggestedFileSize; this.options = options; @@ -107,21 +111,35 @@ public RollingFileWriter createRollingChangelogFileWrite private KeyValueDataFileWriter createDataFileWriter( Path path, int level, FileSource fileSource) { - KeyValueSerializer kvSerializer = new KeyValueSerializer(keyType, valueType); - return new KeyValueDataFileWriter( - fileIO, - formatContext.writerFactory(level), - path, - kvSerializer::toRow, - keyType, - valueType, - formatContext.extractor(level), - schemaId, - level, - formatContext.compression(level), - options, - fileSource, - fileIndexOptions); + return formatContext.thinModeEnabled() + ? new KeyValueThinDataFileWriterImpl( + fileIO, + formatContext.writerFactory(level), + path, + new KeyValueThinSerializer(keyType, valueType)::toRow, + keyType, + valueType, + formatContext.extractor(level), + schemaId, + level, + formatContext.compression(level), + options, + fileSource, + fileIndexOptions) + : new KeyValueDataFileWriterImpl( + fileIO, + formatContext.writerFactory(level), + path, + new KeyValueSerializer(keyType, valueType)::toRow, + keyType, + valueType, + formatContext.extractor(level), + schemaId, + level, + formatContext.compression(level), + options, + fileSource, + fileIndexOptions); } public void deleteFile(String filename, int level) { @@ -191,17 +209,17 @@ private Builder( public KeyValueFileWriterFactory build( BinaryRow partition, int bucket, CoreOptions options) { - RowType fileRowType = KeyValue.schema(keyType, valueType); WriteFormatContext context = new WriteFormatContext( partition, bucket, - fileRowType, + keyType, + valueType, fileFormat, format2PathFactory, options); return new KeyValueFileWriterFactory( - fileIO, schemaId, keyType, valueType, context, suggestedFileSize, options); + fileIO, schemaId, context, suggestedFileSize, options); } } @@ -214,13 +232,24 @@ private static class WriteFormatContext { private final Map format2PathFactory; private final Map format2WriterFactory; + private final RowType keyType; + private final RowType valueType; + private final boolean thinModeEnabled; + private WriteFormatContext( BinaryRow partition, int bucket, - RowType rowType, + RowType keyType, + RowType valueType, FileFormat defaultFormat, Map parentFactories, CoreOptions options) { + this.keyType = keyType; + this.valueType = valueType; + this.thinModeEnabled = + options.dataFileThinMode() && supportsThinMode(keyType, valueType); + RowType writeRowType = + KeyValue.schema(thinModeEnabled ? RowType.of() : keyType, valueType); Map fileFormatPerLevel = options.fileFormatPerLevel(); this.level2Format = level -> @@ -236,7 +265,10 @@ private WriteFormatContext( this.format2PathFactory = new HashMap<>(); this.format2WriterFactory = new HashMap<>(); SimpleColStatsCollector.Factory[] statsCollectorFactories = - StatsCollectorFactories.createStatsFactories(options, rowType.getFieldNames()); + StatsCollectorFactories.createStatsFactories( + options, + writeRowType.getFieldNames(), + thinModeEnabled ? keyType.getFieldNames() : Collections.emptyList()); for (String format : parentFactories.keySet()) { format2PathFactory.put( format, @@ -252,11 +284,30 @@ private WriteFormatContext( format.equals("avro") ? Optional.empty() : fileFormat.createStatsExtractor( - rowType, statsCollectorFactories)); - format2WriterFactory.put(format, fileFormat.createWriterFactory(rowType)); + writeRowType, statsCollectorFactories)); + format2WriterFactory.put(format, fileFormat.createWriterFactory(writeRowType)); } } + private boolean supportsThinMode(RowType keyType, RowType valueType) { + Set keyFieldIds = + valueType.getFields().stream().map(DataField::id).collect(Collectors.toSet()); + + for (DataField field : keyType.getFields()) { + if (!SpecialFields.isKeyField(field.name())) { + return false; + } + if (!keyFieldIds.contains(field.id() - SpecialFields.KEY_FIELD_ID_START)) { + return false; + } + } + return true; + } + + private boolean thinModeEnabled() { + return thinModeEnabled; + } + @Nullable private SimpleStatsExtractor extractor(int level) { return format2Extractor.get(level2Format.apply(level)).orElse(null); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueThinDataFileWriterImpl.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueThinDataFileWriterImpl.java new file mode 100644 index 0000000000000..dd7ebb006764e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueThinDataFileWriterImpl.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.io; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.KeyValue; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fileindex.FileIndexOptions; +import org.apache.paimon.format.FormatWriterFactory; +import org.apache.paimon.format.SimpleColStats; +import org.apache.paimon.format.SimpleStatsExtractor; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.manifest.FileSource; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.Pair; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.function.Function; + +/** + * Implementation of KeyValueDataFileWriter for thin data files. Thin data files only contain + * _SEQUENCE_NUMBER_, _ROW_KIND_ and value fields. + */ +public class KeyValueThinDataFileWriterImpl extends KeyValueDataFileWriter { + + private final int[] keyStatMapping; + + /** + * Constructs a KeyValueThinDataFileWriterImpl. + * + * @param fileIO The file IO interface. + * @param factory The format writer factory. + * @param path The path to the file. + * @param converter The function to convert KeyValue to InternalRow. + * @param keyType The row type of the key. + * @param valueType The row type of the value. + * @param simpleStatsExtractor The simple stats extractor, can be null. + * @param schemaId The schema ID. + * @param level The level. + * @param compression The compression type. + * @param options The core options. + * @param fileSource The file source. + * @param fileIndexOptions The file index options. + */ + public KeyValueThinDataFileWriterImpl( + FileIO fileIO, + FormatWriterFactory factory, + Path path, + Function converter, + RowType keyType, + RowType valueType, + @Nullable SimpleStatsExtractor simpleStatsExtractor, + long schemaId, + int level, + String compression, + CoreOptions options, + FileSource fileSource, + FileIndexOptions fileIndexOptions) { + super( + fileIO, + factory, + path, + converter, + keyType, + valueType, + KeyValue.schema(RowType.of(), valueType), + simpleStatsExtractor, + schemaId, + level, + compression, + options, + fileSource, + fileIndexOptions); + Map idToIndex = new HashMap<>(valueType.getFieldCount()); + for (int i = 0; i < valueType.getFieldCount(); i++) { + idToIndex.put(valueType.getFields().get(i).id(), i); + } + this.keyStatMapping = new int[keyType.getFieldCount()]; + for (int i = 0; i < keyType.getFieldCount(); i++) { + keyStatMapping[i] = + idToIndex.get( + keyType.getFields().get(i).id() - SpecialFields.KEY_FIELD_ID_START); + } + } + + /** + * Fetches the key and value statistics. + * + * @param rowStats The row statistics. + * @return A pair of key statistics and value statistics. + */ + @Override + Pair fetchKeyValueStats(SimpleColStats[] rowStats) { + int numKeyFields = keyType.getFieldCount(); + // In thin mode, there is no key stats in rowStats, so we only jump + // _SEQUNCE_NUMBER_ and _ROW_KIND_ stats. Therefore, the 'from' value is 2. + SimpleColStats[] valFieldStats = Arrays.copyOfRange(rowStats, 2, rowStats.length); + // Thin mode on, so need to map value stats to key stats. + SimpleColStats[] keyStats = new SimpleColStats[numKeyFields]; + for (int i = 0; i < keyStatMapping.length; i++) { + keyStats[i] = valFieldStats[keyStatMapping[i]]; + } + + return Pair.of(keyStats, valFieldStats); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/io/RollingFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/RollingFileWriter.java index 109b7574304ed..29b9223b9a372 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/RollingFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/RollingFileWriter.java @@ -64,10 +64,9 @@ public long targetFileSize() { return targetFileSize; } - @VisibleForTesting - boolean rollingFile() throws IOException { + private boolean rollingFile(boolean forceCheck) throws IOException { return currentWriter.reachTargetSize( - recordCount % CHECK_ROLLING_RECORD_CNT == 0, targetFileSize); + forceCheck || recordCount % CHECK_ROLLING_RECORD_CNT == 0, targetFileSize); } @Override @@ -81,7 +80,7 @@ public void write(T row) throws IOException { currentWriter.write(row); recordCount += 1; - if (rollingFile()) { + if (rollingFile(false)) { closeCurrentWriter(); } } catch (Throwable e) { @@ -105,7 +104,7 @@ public void writeBundle(BundleRecords bundle) throws IOException { currentWriter.writeBundle(bundle); recordCount += bundle.rowCount(); - if (rollingFile()) { + if (rollingFile(true)) { closeCurrentWriter(); } } catch (Throwable e) { diff --git a/paimon-core/src/main/java/org/apache/paimon/io/SingleFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/SingleFileWriter.java index d41040e05bb78..f303e8597870c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/SingleFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/SingleFileWriter.java @@ -49,7 +49,7 @@ public abstract class SingleFileWriter implements FileWriter { protected final Path path; private final Function converter; - private final FormatWriter writer; + private FormatWriter writer; private PositionOutputStream out; private long recordCount; @@ -144,7 +144,14 @@ public boolean reachTargetSize(boolean suggestedCheck, long targetSize) throws I @Override public void abort() { - IOUtils.closeQuietly(out); + if (writer != null) { + IOUtils.closeQuietly(writer); + writer = null; + } + if (out != null) { + IOUtils.closeQuietly(out); + out = null; + } fileIO.deleteQuietly(path); } @@ -167,9 +174,15 @@ public void close() throws IOException { } try { - writer.close(); - out.flush(); - out.close(); + if (writer != null) { + writer.close(); + writer = null; + } + if (out != null) { + out.flush(); + out.close(); + out = null; + } } catch (IOException e) { LOG.warn("Exception occurs when closing file {}. Cleaning up.", path, e); abort(); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java index 2f4190a049dc4..67a3fa6d1acea 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java @@ -25,6 +25,7 @@ import org.apache.paimon.format.SimpleStatsExtractor; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.statistics.NoneSimpleColStatsCollector; import org.apache.paimon.statistics.SimpleColStatsCollector; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Preconditions; @@ -32,7 +33,9 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Arrays; import java.util.function.Function; +import java.util.stream.IntStream; /** * A {@link SingleFileWriter} which also produces statistics for each written field. @@ -44,6 +47,8 @@ public abstract class StatsCollectingSingleFileWriter extends SingleFileWr @Nullable private final SimpleStatsExtractor simpleStatsExtractor; @Nullable private SimpleStatsCollector simpleStatsCollector = null; + @Nullable private SimpleColStats[] noneStats = null; + private final boolean isStatsDisabled; public StatsCollectingSingleFileWriter( FileIO fileIO, @@ -63,6 +68,15 @@ public StatsCollectingSingleFileWriter( Preconditions.checkArgument( statsCollectors.length == writeSchema.getFieldCount(), "The stats collector is not aligned to write schema."); + this.isStatsDisabled = + Arrays.stream(SimpleColStatsCollector.create(statsCollectors)) + .allMatch(p -> p instanceof NoneSimpleColStatsCollector); + if (isStatsDisabled) { + this.noneStats = + IntStream.range(0, statsCollectors.length) + .mapToObj(i -> SimpleColStats.NONE) + .toArray(SimpleColStats[]::new); + } } @Override @@ -85,7 +99,11 @@ public void writeBundle(BundleRecords bundle) throws IOException { public SimpleColStats[] fieldStats() throws IOException { Preconditions.checkState(closed, "Cannot access metric unless the writer is closed."); if (simpleStatsExtractor != null) { - return simpleStatsExtractor.extract(fileIO, path); + if (isStatsDisabled) { + return noneStats; + } else { + return simpleStatsExtractor.extract(fileIO, path); + } } else { return simpleStatsCollector.extract(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ExpireFileEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ExpireFileEntry.java new file mode 100644 index 0000000000000..060360623cd09 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ExpireFileEntry.java @@ -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.paimon.manifest; + +import org.apache.paimon.data.BinaryRow; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** A {@link SimpleFileEntry} with {@link #fileSource}. */ +public class ExpireFileEntry extends SimpleFileEntry { + + @Nullable private final FileSource fileSource; + + public ExpireFileEntry( + FileKind kind, + BinaryRow partition, + int bucket, + int level, + String fileName, + List extraFiles, + @Nullable byte[] embeddedIndex, + BinaryRow minKey, + BinaryRow maxKey, + @Nullable FileSource fileSource) { + super(kind, partition, bucket, level, fileName, extraFiles, embeddedIndex, minKey, maxKey); + this.fileSource = fileSource; + } + + public Optional fileSource() { + return Optional.ofNullable(fileSource); + } + + public static ExpireFileEntry from(ManifestEntry entry) { + return new ExpireFileEntry( + entry.kind(), + entry.partition(), + entry.bucket(), + entry.level(), + entry.fileName(), + entry.file().extraFiles(), + entry.file().embeddedIndex(), + entry.minKey(), + entry.maxKey(), + entry.file().fileSource().orElse(null)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + ExpireFileEntry that = (ExpireFileEntry) o; + return fileSource == that.fileSource; + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), fileSource); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java index 3b3e514e0a1e2..a2569beac61cb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java @@ -28,15 +28,17 @@ import java.util.Arrays; import java.util.Collection; -import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.paimon.utils.ManifestReadThreadPool.randomlyExecuteSequentialReturn; import static org.apache.paimon.utils.ManifestReadThreadPool.sequentialBatchedExecute; /** Entry representing a file. */ @@ -58,6 +60,8 @@ public interface FileEntry { BinaryRow maxKey(); + List extraFiles(); + /** * The same {@link Identifier} indicates that the {@link ManifestEntry} refers to the same data * file. @@ -214,7 +218,11 @@ static Set readDeletedEntries( return readDeletedEntries( m -> manifestFile.read( - m.fileName(), m.fileSize(), Filter.alwaysTrue(), deletedFilter()), + m.fileName(), + m.fileSize(), + Filter.alwaysTrue(), + deletedFilter(), + Filter.alwaysTrue()), manifestFiles, manifestReadParallelism); } @@ -234,11 +242,11 @@ static Set readDeletedEntries( .filter(e -> e.kind() == FileKind.DELETE) .map(FileEntry::identifier) .collect(Collectors.toList()); - Iterable identifiers = - sequentialBatchedExecute(processor, manifestFiles, manifestReadParallelism); - Set result = new HashSet<>(); - for (Identifier identifier : identifiers) { - result.add(identifier); + Iterator identifiers = + randomlyExecuteSequentialReturn(processor, manifestFiles, manifestReadParallelism); + Set result = ConcurrentHashMap.newKeySet(); + while (identifiers.hasNext()) { + result.add(identifiers.next()); } return result; } @@ -247,4 +255,9 @@ static Filter deletedFilter() { Function getter = ManifestEntrySerializer.kindGetter(); return row -> getter.apply(row) == FileKind.DELETE; } + + static Filter addFilter() { + Function getter = ManifestEntrySerializer.kindGetter(); + return row -> getter.apply(row) == FileKind.ADD; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/FilteredManifestEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/FilteredManifestEntry.java new file mode 100644 index 0000000000000..29ae6f6389c6b --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/FilteredManifestEntry.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.manifest; + +/** Wrap a {@link ManifestEntry} to contain {@link #selected}. */ +public class FilteredManifestEntry extends ManifestEntry { + + private final boolean selected; + + public FilteredManifestEntry(ManifestEntry entry, boolean selected) { + super(entry.kind(), entry.partition(), entry.bucket(), entry.totalBuckets(), entry.file()); + this.selected = selected; + } + + public boolean selected() { + return selected; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntry.java index a52d9e8af40f2..2431a1c26412f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntry.java @@ -20,6 +20,7 @@ import org.apache.paimon.annotation.Public; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; @@ -57,12 +58,7 @@ public class IndexManifestEntry { new DataField( 7, "_DELETIONS_VECTORS_RANGES", - new ArrayType( - true, - RowType.of( - newStringType(false), - new IntType(false), - new IntType(false)))))); + new ArrayType(true, DeletionVectorMeta.SCHEMA)))); private final FileKind kind; private final BinaryRow partition; diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntrySerializer.java b/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntrySerializer.java index 574e935550eb6..6f2ec17dda8c8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntrySerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/IndexManifestEntrySerializer.java @@ -22,10 +22,9 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.index.IndexFileMetaSerializer; import org.apache.paimon.utils.VersionedObjectSerializer; -import static org.apache.paimon.index.IndexFileMetaSerializer.dvRangesToRowArrayData; -import static org.apache.paimon.index.IndexFileMetaSerializer.rowArrayDataToDvRanges; import static org.apache.paimon.utils.SerializationUtils.deserializeBinaryRow; import static org.apache.paimon.utils.SerializationUtils.serializeBinaryRow; @@ -52,9 +51,10 @@ public InternalRow convertTo(IndexManifestEntry record) { BinaryString.fromString(indexFile.fileName()), indexFile.fileSize(), indexFile.rowCount(), - record.indexFile().deletionVectorsRanges() == null + record.indexFile().deletionVectorMetas() == null ? null - : dvRangesToRowArrayData(record.indexFile().deletionVectorsRanges())); + : IndexFileMetaSerializer.dvMetasToRowArrayData( + record.indexFile().deletionVectorMetas().values())); } @Override @@ -72,6 +72,8 @@ public IndexManifestEntry convertFrom(int version, InternalRow row) { row.getString(4).toString(), row.getLong(5), row.getLong(6), - row.isNullAt(7) ? null : rowArrayDataToDvRanges(row.getArray(7)))); + row.isNullAt(7) + ? null + : IndexFileMetaSerializer.rowArrayDataToDvMetas(row.getArray(7)))); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java index 61c4619bd6d6a..b4abd0e9ec0ed 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestCommittable.java @@ -62,13 +62,14 @@ public void addFileCommittable(CommitMessage commitMessage) { commitMessages.add(commitMessage); } - public void addLogOffset(int bucket, long offset) { - if (logOffsets.containsKey(bucket)) { + public void addLogOffset(int bucket, long offset, boolean allowDuplicate) { + if (!allowDuplicate && logOffsets.containsKey(bucket)) { throw new RuntimeException( String.format( "bucket-%d appears multiple times, which is not possible.", bucket)); } - logOffsets.put(bucket, offset); + long newOffset = Math.max(logOffsets.getOrDefault(bucket, offset), offset); + logOffsets.put(bucket, newOffset); } public long identifier() { diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java index f7c5c4639a6f2..626e0a5d468ff 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java @@ -102,6 +102,11 @@ public BinaryRow maxKey() { return file.maxKey(); } + @Override + public List extraFiles() { + return file.extraFiles(); + } + public int totalBuckets() { return totalBuckets; } @@ -121,6 +126,10 @@ public Identifier identifier() { file.embeddedIndex()); } + public ManifestEntry copyWithoutStats() { + return new ManifestEntry(kind, partition, bucket, totalBuckets, file.copyWithoutStats()); + } + @Override public boolean equals(Object o) { if (!(o instanceof ManifestEntry)) { diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java index 41b480e8a427a..f70cfdd91f19c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java @@ -40,6 +40,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; import java.util.List; /** @@ -85,6 +86,15 @@ public long suggestedFileSize() { return suggestedFileSize; } + public List readExpireFileEntries(String fileName, @Nullable Long fileSize) { + List entries = read(fileName, fileSize); + List result = new ArrayList<>(entries.size()); + for (ManifestEntry entry : entries) { + result.add(ExpireFileEntry.from(entry)); + } + return result; + } + /** * Write several {@link ManifestEntry}s into manifest files. * @@ -216,18 +226,5 @@ public ManifestFile create(List filters) { suggestedFileSize, cache); } - - public ObjectsFile createSimpleFileEntryReader() { - RowType entryType = VersionedObjectSerializer.versionType(ManifestEntry.SCHEMA); - return new ObjectsFile<>( - fileIO, - new SimpleFileEntrySerializer(), - entryType, - fileFormat.createReaderFactory(entryType), - fileFormat.createWriterFactory(entryType), - compression, - pathFactory.manifestFileFactory(), - cache); - } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntry.java index 8d33ede0c4a19..fdaed2b85aafd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntry.java @@ -117,6 +117,11 @@ public BinaryRow maxKey() { return maxKey; } + @Override + public List extraFiles() { + return extraFiles; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java index b73be6941e545..f2a964bae16a6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java @@ -279,6 +279,7 @@ public CommitIncrement prepareCommit(boolean waitCompaction) throws Exception { @Override public boolean isCompacting() { + compactManager.triggerCompaction(false); return compactManager.isCompacting(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunction.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunction.java index 4d720cb3f0750..ab25794129baf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunction.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunction.java @@ -52,6 +52,7 @@ import static org.apache.paimon.CoreOptions.FIELDS_PREFIX; import static org.apache.paimon.CoreOptions.FIELDS_SEPARATOR; import static org.apache.paimon.CoreOptions.PARTIAL_UPDATE_REMOVE_RECORD_ON_DELETE; +import static org.apache.paimon.CoreOptions.PARTIAL_UPDATE_REMOVE_RECORD_ON_SEQUENCE_GROUP; import static org.apache.paimon.utils.InternalRowUtils.createFieldGetters; /** @@ -68,6 +69,7 @@ public class PartialUpdateMergeFunction implements MergeFunction { private final boolean fieldSequenceEnabled; private final Map fieldAggregators; private final boolean removeRecordOnDelete; + private final Set sequenceGroupPartialDelete; private InternalRow currentKey; private long latestSequenceNumber; @@ -81,13 +83,15 @@ protected PartialUpdateMergeFunction( Map fieldSeqComparators, Map fieldAggregators, boolean fieldSequenceEnabled, - boolean removeRecordOnDelete) { + boolean removeRecordOnDelete, + Set sequenceGroupPartialDelete) { this.getters = getters; this.ignoreDelete = ignoreDelete; this.fieldSeqComparators = fieldSeqComparators; this.fieldAggregators = fieldAggregators; this.fieldSequenceEnabled = fieldSequenceEnabled; this.removeRecordOnDelete = removeRecordOnDelete; + this.sequenceGroupPartialDelete = sequenceGroupPartialDelete; } @Override @@ -220,8 +224,15 @@ private void retractWithSequenceGroup(KeyValue kv) { .anyMatch(field -> field == index)) { for (int field : seqComparator.compareFields()) { if (!updatedSequenceFields.contains(field)) { - row.setField(field, getters[field].getFieldOrNull(kv.value())); - updatedSequenceFields.add(field); + if (kv.valueKind() == RowKind.DELETE + && sequenceGroupPartialDelete.contains(field)) { + currentDeleteRow = true; + row = new GenericRow(getters.length); + return; + } else { + row.setField(field, getters[field].getFieldOrNull(kv.value())); + updatedSequenceFields.add(field); + } } } } else { @@ -278,13 +289,21 @@ private static class Factory implements MergeFunctionFactory { private final boolean removeRecordOnDelete; + private final String removeRecordOnSequenceGroup; + + private Set sequenceGroupPartialDelete; + private Factory(Options options, RowType rowType, List primaryKeys) { this.ignoreDelete = options.get(CoreOptions.IGNORE_DELETE); this.rowType = rowType; this.tableTypes = rowType.getFieldTypes(); + this.removeRecordOnSequenceGroup = + options.get(PARTIAL_UPDATE_REMOVE_RECORD_ON_SEQUENCE_GROUP); + this.sequenceGroupPartialDelete = new HashSet<>(); List fieldNames = rowType.getFieldNames(); this.fieldSeqComparators = new HashMap<>(); + Map sequenceGroupMap = new HashMap<>(); for (Map.Entry entry : options.toMap().entrySet()) { String k = entry.getKey(); String v = entry.getValue(); @@ -323,6 +342,7 @@ private Factory(Options options, RowType rowType, List primaryKeys) { fieldName -> { int index = fieldNames.indexOf(fieldName); fieldSeqComparators.put(index, userDefinedSeqComparator); + sequenceGroupMap.put(fieldName, index); }); } } @@ -345,6 +365,21 @@ private Factory(Options options, RowType rowType, List primaryKeys) { String.format( "sequence group and %s have conflicting behavior so should not be enabled at the same time.", PARTIAL_UPDATE_REMOVE_RECORD_ON_DELETE)); + + if (removeRecordOnSequenceGroup != null) { + String[] sequenceGroupArr = removeRecordOnSequenceGroup.split(FIELDS_SEPARATOR); + Preconditions.checkState( + sequenceGroupMap.keySet().containsAll(Arrays.asList(sequenceGroupArr)), + String.format( + "field '%s' defined in '%s' option must be part of sequence groups", + removeRecordOnSequenceGroup, + PARTIAL_UPDATE_REMOVE_RECORD_ON_SEQUENCE_GROUP.key())); + sequenceGroupPartialDelete = + Arrays.stream(sequenceGroupArr) + .filter(sequenceGroupMap::containsKey) + .map(sequenceGroupMap::get) + .collect(Collectors.toSet()); + } } @Override @@ -405,7 +440,8 @@ public MergeFunction create(@Nullable int[][] projection) { projectedSeqComparators, projectedAggregators, !fieldSeqComparators.isEmpty(), - removeRecordOnDelete); + removeRecordOnDelete, + sequenceGroupPartialDelete); } else { Map fieldSeqComparators = new HashMap<>(); this.fieldSeqComparators.forEach( @@ -419,7 +455,8 @@ public MergeFunction create(@Nullable int[][] projection) { fieldSeqComparators, fieldAggregators, !fieldSeqComparators.isEmpty(), - removeRecordOnDelete); + removeRecordOnDelete, + sequenceGroupPartialDelete); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionCommitCallback.java b/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionCommitCallback.java index 06002161a68e4..599f88e512c03 100644 --- a/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionCommitCallback.java +++ b/paimon-core/src/main/java/org/apache/paimon/metastore/AddPartitionCommitCallback.java @@ -72,20 +72,6 @@ public void retry(ManifestCommittable committable) { addPartitions(partitions); } - private void addPartition(BinaryRow partition) { - try { - boolean added = cache.get(partition, () -> false); - if (added) { - return; - } - - client.addPartition(partition); - cache.put(partition, true); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - private void addPartitions(Set partitions) { try { List newPartitions = new ArrayList<>(); diff --git a/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java b/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java index 60e28c59f45dc..75f7af5abbdc1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java +++ b/paimon-core/src/main/java/org/apache/paimon/metastore/MetastoreClient.java @@ -55,7 +55,8 @@ default void addPartitionsSpec(List> partitionSpec default void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { throw new UnsupportedOperationException(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java index 306fd6f5a7c84..135d2fbe21b4a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java @@ -23,7 +23,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.manifest.BucketEntry; import org.apache.paimon.manifest.FileEntry; -import org.apache.paimon.manifest.FileKind; +import org.apache.paimon.manifest.FileEntry.Identifier; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestEntrySerializer; @@ -47,8 +47,6 @@ import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.SnapshotManager; -import org.apache.paimon.shade.guava30.com.google.common.collect.Iterators; - import javax.annotation.Nullable; import java.util.ArrayList; @@ -66,9 +64,9 @@ import java.util.stream.Collectors; import static org.apache.paimon.utils.ManifestReadThreadPool.getExecutorService; +import static org.apache.paimon.utils.ManifestReadThreadPool.randomlyExecuteSequentialReturn; import static org.apache.paimon.utils.ManifestReadThreadPool.sequentialBatchedExecute; import static org.apache.paimon.utils.Preconditions.checkArgument; -import static org.apache.paimon.utils.Preconditions.checkState; import static org.apache.paimon.utils.ThreadPoolUtils.randomlyOnlyExecute; /** Default implementation of {@link FileStoreScan}. */ @@ -87,7 +85,6 @@ public abstract class AbstractFileStoreScan implements FileStoreScan { private Filter bucketFilter = null; private Collection buckets; private BiFilter totalAwareBucketFilter = null; - private List specifiedManifests = null; protected ScanMode scanMode = ScanMode.ALL; private Filter levelFilter = null; private Filter manifestEntryFilter = null; @@ -95,6 +92,7 @@ public abstract class AbstractFileStoreScan implements FileStoreScan { private ManifestCacheFilter manifestCacheFilter = null; private ScanMetrics scanMetrics = null; + private boolean dropStats; public AbstractFileStoreScan( ManifestsReader manifestsReader, @@ -110,6 +108,7 @@ public AbstractFileStoreScan( this.manifestFileFactory = manifestFileFactory; this.tableSchemas = new ConcurrentHashMap<>(); this.parallelism = parallelism; + this.dropStats = false; } @Override @@ -173,25 +172,16 @@ public FileStoreScan withPartitionBucket(BinaryRow partition, int bucket) { @Override public FileStoreScan withSnapshot(long snapshotId) { - checkState(specifiedManifests == null, "Cannot set both snapshot and manifests."); this.specifiedSnapshot = snapshotManager.snapshot(snapshotId); return this; } @Override public FileStoreScan withSnapshot(Snapshot snapshot) { - checkState(specifiedManifests == null, "Cannot set both snapshot and manifests."); this.specifiedSnapshot = snapshot; return this; } - @Override - public FileStoreScan withManifestList(List manifests) { - checkState(specifiedSnapshot == null, "Cannot set both snapshot and manifests."); - this.specifiedManifests = manifests; - return this; - } - @Override public FileStoreScan withKind(ScanMode scanMode) { this.scanMode = scanMode; @@ -204,6 +194,11 @@ public FileStoreScan withLevelFilter(Filter levelFilter) { return this; } + @Override + public FileStoreScan enableValueFilter() { + return this; + } + @Override public FileStoreScan withManifestEntryFilter(Filter filter) { this.manifestEntryFilter = filter; @@ -228,6 +223,12 @@ public FileStoreScan withMetrics(ScanMetrics metrics) { return this; } + @Override + public FileStoreScan dropStats() { + this.dropStats = true; + return this; + } + @Nullable @Override public Integer parallelism() { @@ -246,47 +247,46 @@ public Plan plan() { Snapshot snapshot = manifestsResult.snapshot; List manifests = manifestsResult.filteredManifests; - long startDataFiles = - manifestsResult.allManifests.stream() - .mapToLong(f -> f.numAddedFiles() - f.numDeletedFiles()) - .sum(); - - Collection mergedEntries = - readAndMergeFileEntries(manifests, this::readManifest); - - long skippedByPartitionAndStats = startDataFiles - mergedEntries.size(); - - // We group files by bucket here, and filter them by the whole bucket filter. - // Why do this: because in primary key table, we can't just filter the value - // by the stat in files (see `PrimaryKeyFileStoreTable.nonPartitionFilterConsumer`), - // but we can do this by filter the whole bucket files - List files = - mergedEntries.stream() - .collect( - Collectors.groupingBy( - // we use LinkedHashMap to avoid disorder - file -> Pair.of(file.partition(), file.bucket()), - LinkedHashMap::new, - Collectors.toList())) - .values() - .stream() - .map(this::filterWholeBucketByStats) - .flatMap(Collection::stream) - .collect(Collectors.toList()); + Iterator iterator = readManifestEntries(manifests, false); + List files = new ArrayList<>(); + while (iterator.hasNext()) { + files.add(iterator.next()); + } + + if (wholeBucketFilterEnabled()) { + // We group files by bucket here, and filter them by the whole bucket filter. + // Why do this: because in primary key table, we can't just filter the value + // by the stat in files (see `PrimaryKeyFileStoreTable.nonPartitionFilterConsumer`), + // but we can do this by filter the whole bucket files + files = + files.stream() + .collect( + Collectors.groupingBy( + // we use LinkedHashMap to avoid disorder + file -> Pair.of(file.partition(), file.bucket()), + LinkedHashMap::new, + Collectors.toList())) + .values() + .stream() + .map(this::filterWholeBucketByStats) + .flatMap(Collection::stream) + .collect(Collectors.toList()); + } + + List result = files; - long skippedByWholeBucketFiles = mergedEntries.size() - files.size(); long scanDuration = (System.nanoTime() - started) / 1_000_000; - checkState( - startDataFiles - skippedByPartitionAndStats - skippedByWholeBucketFiles - == files.size()); if (scanMetrics != null) { + long allDataFiles = + manifestsResult.allManifests.stream() + .mapToLong(f -> f.numAddedFiles() - f.numDeletedFiles()) + .sum(); scanMetrics.reportScan( new ScanStats( scanDuration, manifests.size(), - skippedByPartitionAndStats, - skippedByWholeBucketFiles, - files.size())); + allDataFiles - result.size(), + result.size())); } return new Plan() { @@ -304,7 +304,7 @@ public Snapshot snapshot() { @Override public List files() { - return files; + return result; } }; } @@ -312,9 +312,15 @@ public List files() { @Override public List readSimpleEntries() { List manifests = readManifests().filteredManifests; - Collection mergedEntries = - readAndMergeFileEntries(manifests, this::readSimpleEntries); - return new ArrayList<>(mergedEntries); + Iterator iterator = + scanMode == ScanMode.ALL + ? readAndMergeFileEntries(manifests, SimpleFileEntry::from, false) + : readAndNoMergeFileEntries(manifests, SimpleFileEntry::from, false); + List result = new ArrayList<>(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + return result; } @Override @@ -343,30 +349,60 @@ public List readBucketEntries() { @Override public Iterator readFileIterator() { - List manifests = readManifests().filteredManifests; - Set deleteEntries = - FileEntry.readDeletedEntries(this::readSimpleEntries, manifests, parallelism); - Iterator iterator = - sequentialBatchedExecute(this::readManifest, manifests, parallelism).iterator(); - return Iterators.filter( - iterator, - entry -> - entry != null - && entry.kind() == FileKind.ADD - && !deleteEntries.contains(entry.identifier())); - } + // useSequential: reduce memory and iterator can be stopping + return readManifestEntries(readManifests().filteredManifests, true); + } + + private Iterator readManifestEntries( + List manifests, boolean useSequential) { + return scanMode == ScanMode.ALL + ? readAndMergeFileEntries(manifests, Function.identity(), useSequential) + : readAndNoMergeFileEntries(manifests, Function.identity(), useSequential); + } + + private Iterator readAndMergeFileEntries( + List manifests, + Function, List> converter, + boolean useSequential) { + Set deletedEntries = + FileEntry.readDeletedEntries( + manifest -> readManifest(manifest, FileEntry.deletedFilter(), null), + manifests, + parallelism); + + manifests = + manifests.stream() + .filter(file -> file.numAddedFiles() > 0) + .collect(Collectors.toList()); - public Collection readAndMergeFileEntries( - List manifests, Function> manifestReader) { - return FileEntry.mergeEntries( - sequentialBatchedExecute(manifestReader, manifests, parallelism)); + Function> processor = + manifest -> + converter.apply( + readManifest( + manifest, + FileEntry.addFilter(), + entry -> !deletedEntries.contains(entry.identifier()))); + if (useSequential) { + return sequentialBatchedExecute(processor, manifests, parallelism).iterator(); + } else { + return randomlyExecuteSequentialReturn(processor, manifests, parallelism); + } } - private ManifestsReader.Result readManifests() { - if (specifiedManifests != null) { - return new ManifestsReader.Result(null, specifiedManifests, specifiedManifests); + private Iterator readAndNoMergeFileEntries( + List manifests, + Function, List> converter, + boolean useSequential) { + Function> reader = + manifest -> converter.apply(readManifest(manifest)); + if (useSequential) { + return sequentialBatchedExecute(reader, manifests, parallelism).iterator(); + } else { + return randomlyExecuteSequentialReturn(reader, manifests, parallelism); } + } + private ManifestsReader.Result readManifests() { return manifestsReader.read(specifiedSnapshot, scanMode); } @@ -384,12 +420,24 @@ protected TableSchema scanTableSchema(long id) { /** Note: Keep this thread-safe. */ protected abstract boolean filterByStats(ManifestEntry entry); - /** Note: Keep this thread-safe. */ - protected abstract List filterWholeBucketByStats(List entries); + protected boolean wholeBucketFilterEnabled() { + return false; + } + + protected List filterWholeBucketByStats(List entries) { + return entries; + } /** Note: Keep this thread-safe. */ @Override public List readManifest(ManifestFileMeta manifest) { + return readManifest(manifest, null, null); + } + + private List readManifest( + ManifestFileMeta manifest, + @Nullable Filter additionalFilter, + @Nullable Filter additionalTFilter) { List entries = manifestFileFactory .create(createPushDownFilter(buckets)) @@ -397,29 +445,24 @@ public List readManifest(ManifestFileMeta manifest) { manifest.fileName(), manifest.fileSize(), createCacheRowFilter(), - createEntryRowFilter()); - List filteredEntries = new ArrayList<>(entries.size()); - for (ManifestEntry entry : entries) { - if ((manifestEntryFilter == null || manifestEntryFilter.test(entry)) - && filterByStats(entry)) { - filteredEntries.add(entry); + createEntryRowFilter().and(additionalFilter), + entry -> + (additionalTFilter == null || additionalTFilter.test(entry)) + && (manifestEntryFilter == null + || manifestEntryFilter.test(entry)) + && filterByStats(entry)); + if (dropStats) { + List copied = new ArrayList<>(entries.size()); + for (ManifestEntry entry : entries) { + copied.add(dropStats(entry)); } + entries = copied; } - return filteredEntries; + return entries; } - /** Note: Keep this thread-safe. */ - private List readSimpleEntries(ManifestFileMeta manifest) { - return manifestFileFactory - .createSimpleFileEntryReader() - .read( - manifest.fileName(), - manifest.fileSize(), - // use filter for ManifestEntry - // currently, projection is not pushed down to file format - // see SimpleFileEntrySerializer - createCacheRowFilter(), - createEntryRowFilter()); + protected ManifestEntry dropStats(ManifestEntry entry) { + return entry.copyWithoutStats(); } /** diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java index d63887030090e..14dfe75a6e357 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java @@ -18,6 +18,7 @@ package org.apache.paimon.operation; +import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.compact.CompactDeletionFile; @@ -96,12 +97,19 @@ protected AbstractFileStoreWrite( @Nullable IndexMaintainer.Factory indexFactory, @Nullable DeletionVectorsMaintainer.Factory dvMaintainerFactory, String tableName, + CoreOptions options, int totalBuckets, RowType partitionType, int writerNumberMax, boolean legacyPartitionName) { this.snapshotManager = snapshotManager; this.scan = scan; + // Statistic is useless in writer + if (options.manifestDeleteFileDropStats()) { + if (this.scan != null) { + this.scan.dropStats(); + } + } this.indexFactory = indexFactory; this.dvMaintainerFactory = dvMaintainerFactory; this.totalBuckets = totalBuckets; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java index 60b4e7933cb19..d2ca5da42249a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java @@ -34,7 +34,6 @@ import java.io.IOException; import java.util.HashMap; -import java.util.List; import java.util.Map; /** {@link FileStoreScan} for {@link AppendOnlyFileStore}. */ @@ -100,12 +99,6 @@ protected boolean filterByStats(ManifestEntry entry) { && (!fileIndexReadEnabled || testFileIndex(entry.file().embeddedIndex(), entry)); } - @Override - protected List filterWholeBucketByStats(List entries) { - // We don't need to filter per-bucket entries here - return entries; - } - private boolean testFileIndex(@Nullable byte[] embeddedIndexBytes, ManifestEntry entry) { if (embeddedIndexBytes == null) { return true; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java index 3ce019c916389..4a6196453df62 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java @@ -212,6 +212,9 @@ protected void forceBufferSpill() throws Exception { if (ioManager == null) { return; } + if (forceBufferSpill) { + return; + } forceBufferSpill = true; LOG.info( "Force buffer spill for append-only file store write, writer number is: {}", diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/ChangelogDeletion.java b/paimon-core/src/main/java/org/apache/paimon/operation/ChangelogDeletion.java index c20405ff26c96..069e57bb3daf6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/ChangelogDeletion.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/ChangelogDeletion.java @@ -23,8 +23,8 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.manifest.ExpireFileEntry; import org.apache.paimon.manifest.IndexManifestEntry; -import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.manifest.ManifestFileMeta; import org.apache.paimon.manifest.ManifestList; @@ -60,7 +60,7 @@ public ChangelogDeletion( } @Override - public void cleanUnusedDataFiles(Changelog changelog, Predicate skipper) { + public void cleanUnusedDataFiles(Changelog changelog, Predicate skipper) { if (changelog.changelogManifestList() != null) { deleteAddedDataFiles(changelog.changelogManifestList()); } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.java b/paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.java new file mode 100644 index 0000000000000..d29eede720ac9 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/operation/CleanOrphanFilesResult.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.operation; + +import org.apache.paimon.fs.Path; + +import javax.annotation.Nullable; + +import java.util.List; + +/** The result of OrphanFilesClean. */ +public class CleanOrphanFilesResult { + + private final long deletedFileCount; + private final long deletedFileTotalLenInBytes; + + @Nullable private final List deletedFilesPath; + + public CleanOrphanFilesResult(long deletedFileCount, long deletedFileTotalLenInBytes) { + this(deletedFileCount, deletedFileTotalLenInBytes, null); + } + + public CleanOrphanFilesResult( + long deletedFileCount, + long deletedFileTotalLenInBytes, + @Nullable List deletedFilesPath) { + this.deletedFilesPath = deletedFilesPath; + this.deletedFileCount = deletedFileCount; + this.deletedFileTotalLenInBytes = deletedFileTotalLenInBytes; + } + + public long getDeletedFileCount() { + return deletedFileCount; + } + + public long getDeletedFileTotalLenInBytes() { + return deletedFileTotalLenInBytes; + } + + @Nullable + public List getDeletedFilesPath() { + return deletedFilesPath; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java index 303a074b0cb89..cfecd767b6fb0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java @@ -24,10 +24,11 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.manifest.ExpireFileEntry; import org.apache.paimon.manifest.FileEntry; +import org.apache.paimon.manifest.FileEntry.Identifier; import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.IndexManifestEntry; -import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.manifest.ManifestFileMeta; import org.apache.paimon.manifest.ManifestList; @@ -46,7 +47,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; @@ -54,7 +54,6 @@ import java.util.concurrent.Executor; import java.util.function.Consumer; import java.util.function.Predicate; -import java.util.stream.Collectors; /** * Base class for file deletion including methods for clean data files, manifest files and empty @@ -110,7 +109,7 @@ public FileDeletionBase( * @param skipper if the test result of a data file is true, it will be skipped when deleting; * else it will be deleted */ - public abstract void cleanUnusedDataFiles(T snapshot, Predicate skipper); + public abstract void cleanUnusedDataFiles(T snapshot, Predicate skipper); /** * Clean metadata files that will not be used anymore of a snapshot, including data manifests, @@ -164,21 +163,23 @@ public void cleanEmptyDirectories() { deletionBuckets.clear(); } - protected void recordDeletionBuckets(ManifestEntry entry) { + protected void recordDeletionBuckets(ExpireFileEntry entry) { deletionBuckets .computeIfAbsent(entry.partition(), p -> new HashSet<>()) .add(entry.bucket()); } - public void cleanUnusedDataFiles(String manifestList, Predicate skipper) { + public void cleanUnusedDataFiles(String manifestList, Predicate skipper) { // try read manifests - List manifestFileNames = readManifestFileNames(tryReadManifestList(manifestList)); - List manifestEntries; + List manifests = tryReadManifestList(manifestList); + List manifestEntries; // data file path -> (original manifest entry, extra file paths) - Map>> dataFileToDelete = new HashMap<>(); - for (String manifest : manifestFileNames) { + Map>> dataFileToDelete = new HashMap<>(); + for (ManifestFileMeta manifest : manifests) { try { - manifestEntries = manifestFile.read(manifest); + manifestEntries = + manifestFile.readExpireFileEntries( + manifest.fileName(), manifest.fileSize()); } catch (Exception e) { // cancel deletion if any exception occurs LOG.warn("Failed to read some manifest files. Cancel deletion.", e); @@ -192,12 +193,12 @@ public void cleanUnusedDataFiles(String manifestList, Predicate s } protected void doCleanUnusedDataFile( - Map>> dataFileToDelete, - Predicate skipper) { + Map>> dataFileToDelete, + Predicate skipper) { List actualDataFileToDelete = new ArrayList<>(); dataFileToDelete.forEach( (path, pair) -> { - ManifestEntry entry = pair.getLeft(); + ExpireFileEntry entry = pair.getLeft(); // check whether we should skip the data file if (!skipper.test(entry)) { // delete data files @@ -211,20 +212,20 @@ protected void doCleanUnusedDataFile( } protected void getDataFileToDelete( - Map>> dataFileToDelete, - List dataFileEntries) { + Map>> dataFileToDelete, + List dataFileEntries) { // we cannot delete a data file directly when we meet a DELETE entry, because that // file might be upgraded - for (ManifestEntry entry : dataFileEntries) { + for (ExpireFileEntry entry : dataFileEntries) { Path bucketPath = pathFactory.bucketPath(entry.partition(), entry.bucket()); - Path dataFilePath = new Path(bucketPath, entry.file().fileName()); + Path dataFilePath = new Path(bucketPath, entry.fileName()); switch (entry.kind()) { case ADD: dataFileToDelete.remove(dataFilePath); break; case DELETE: - List extraFiles = new ArrayList<>(entry.file().extraFiles().size()); - for (String file : entry.file().extraFiles()) { + List extraFiles = new ArrayList<>(entry.extraFiles().size()); + for (String file : entry.extraFiles()) { extraFiles.add(new Path(bucketPath, file)); } dataFileToDelete.put(dataFilePath, Pair.of(entry, extraFiles)); @@ -242,27 +243,28 @@ protected void getDataFileToDelete( * @param manifestListName name of manifest list */ public void deleteAddedDataFiles(String manifestListName) { - List manifestFileNames = - readManifestFileNames(tryReadManifestList(manifestListName)); - for (String file : manifestFileNames) { + List manifests = tryReadManifestList(manifestListName); + for (ManifestFileMeta manifest : manifests) { try { - List manifestEntries = manifestFile.read(file); + List manifestEntries = + manifestFile.readExpireFileEntries( + manifest.fileName(), manifest.fileSize()); deleteAddedDataFiles(manifestEntries); } catch (Exception e) { // We want to delete the data file, so just ignore the unavailable files - LOG.info("Failed to read manifest " + file + ". Ignore it.", e); + LOG.info("Failed to read manifest " + manifest.fileName() + ". Ignore it.", e); } } } - private void deleteAddedDataFiles(List manifestEntries) { + private void deleteAddedDataFiles(List manifestEntries) { List dataFileToDelete = new ArrayList<>(); - for (ManifestEntry entry : manifestEntries) { + for (ExpireFileEntry entry : manifestEntries) { if (entry.kind() == FileKind.ADD) { dataFileToDelete.add( new Path( pathFactory.bucketPath(entry.partition(), entry.bucket()), - entry.file().fileName())); + entry.fileName())); recordDeletionBuckets(entry); } } @@ -327,7 +329,7 @@ protected void cleanUnusedManifests( cleanUnusedStatisticsManifests(snapshot, skippingSet); } - public Predicate createDataFileSkipperForTags( + public Predicate createDataFileSkipperForTags( List taggedSnapshots, long expiringSnapshotId) throws Exception { int index = SnapshotManager.findPreviousSnapshot(taggedSnapshots, expiringSnapshotId); // refresh tag data files @@ -358,18 +360,6 @@ protected List tryReadManifestList(String manifestListName) { } } - protected List tryReadDataManifests(Snapshot snapshot) { - List manifestFileMetas = tryReadManifestList(snapshot.baseManifestList()); - manifestFileMetas.addAll(tryReadManifestList(snapshot.deltaManifestList())); - return readManifestFileNames(manifestFileMetas); - } - - protected List readManifestFileNames(List manifestFileMetas) { - return manifestFileMetas.stream() - .map(ManifestFileMeta::fileName) - .collect(Collectors.toCollection(LinkedList::new)); - } - /** * NOTE: This method is used for building data file skipping set. If failed to read some * manifests, it will throw exception which callers must handle. @@ -377,23 +367,26 @@ protected List readManifestFileNames(List manifestFile protected void addMergedDataFiles( Map>> dataFiles, Snapshot snapshot) throws IOException { - for (ManifestEntry entry : readMergedDataFiles(snapshot)) { + for (ExpireFileEntry entry : readMergedDataFiles(snapshot)) { dataFiles .computeIfAbsent(entry.partition(), p -> new HashMap<>()) .computeIfAbsent(entry.bucket(), b -> new HashSet<>()) - .add(entry.file().fileName()); + .add(entry.fileName()); } } - protected Collection readMergedDataFiles(Snapshot snapshot) throws IOException { + protected Collection readMergedDataFiles(Snapshot snapshot) + throws IOException { // read data manifests - List files = tryReadDataManifests(snapshot); + + List manifests = tryReadManifestList(snapshot.baseManifestList()); + manifests.addAll(tryReadManifestList(snapshot.deltaManifestList())); // read and merge manifest entries - Map map = new HashMap<>(); - for (String manifest : files) { - List entries; - entries = manifestFile.readWithIOException(manifest); + Map map = new HashMap<>(); + for (ManifestFileMeta manifest : manifests) { + List entries = + manifestFile.readExpireFileEntries(manifest.fileName(), manifest.fileSize()); FileEntry.mergeEntries(entries, map); } @@ -401,12 +394,12 @@ protected Collection readMergedDataFiles(Snapshot snapshot) throw } protected boolean containsDataFile( - Map>> dataFiles, ManifestEntry testee) { - Map> buckets = dataFiles.get(testee.partition()); + Map>> dataFiles, ExpireFileEntry entry) { + Map> buckets = dataFiles.get(entry.partition()); if (buckets != null) { - Set fileNames = buckets.get(testee.bucket()); + Set fileNames = buckets.get(entry.bucket()); if (fileNames != null) { - return fileNames.contains(testee.file().fileName()); + return fileNames.contains(entry.fileName()); } } return false; diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java index 69efce9189514..001132e1671c1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java @@ -18,6 +18,7 @@ package org.apache.paimon.operation; +import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.data.BinaryRow; @@ -82,6 +83,7 @@ import static org.apache.paimon.manifest.ManifestEntry.recordCount; import static org.apache.paimon.manifest.ManifestEntry.recordCountAdd; import static org.apache.paimon.manifest.ManifestEntry.recordCountDelete; +import static org.apache.paimon.partition.PartitionPredicate.createBinaryPartitions; import static org.apache.paimon.partition.PartitionPredicate.createPartitionPredicate; import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH; import static org.apache.paimon.utils.InternalRowPartitionComputer.partToSimpleString; @@ -133,6 +135,7 @@ public class FileStoreCommitImpl implements FileStoreCommit { private final List commitCallbacks; private final StatsFileHandler statsFileHandler; private final BucketMode bucketMode; + private long commitTimeout; private final int commitMaxRetries; @Nullable private Lock lock; @@ -146,6 +149,7 @@ public FileStoreCommitImpl( String tableName, String commitUser, RowType partitionType, + CoreOptions options, String partitionDefaultName, FileStorePathFactory pathFactory, SnapshotManager snapshotManager, @@ -164,7 +168,8 @@ public FileStoreCommitImpl( BucketMode bucketMode, @Nullable Integer manifestReadParallelism, List commitCallbacks, - int commitMaxRetries) { + int commitMaxRetries, + long commitTimeout) { this.fileIO = fileIO; this.schemaManager = schemaManager; this.tableName = tableName; @@ -177,6 +182,10 @@ public FileStoreCommitImpl( this.manifestList = manifestListFactory.create(); this.indexManifestFile = indexManifestFileFactory.create(); this.scan = scan; + // Stats in DELETE Manifest Entries is useless + if (options.manifestDeleteFileDropStats()) { + this.scan.dropStats(); + } this.numBucket = numBucket; this.manifestTargetSize = manifestTargetSize; this.manifestFullCompactionSize = manifestFullCompactionSize; @@ -187,6 +196,7 @@ public FileStoreCommitImpl( this.manifestReadParallelism = manifestReadParallelism; this.commitCallbacks = commitCallbacks; this.commitMaxRetries = commitMaxRetries; + this.commitTimeout = commitTimeout; this.lock = null; this.ignoreEmptyCommit = true; @@ -524,17 +534,26 @@ public void dropPartitions(List> partitions, long commitIden partitions.stream().map(Objects::toString).collect(Collectors.joining(","))); } - // partitions may be partial partition fields, so here must to use predicate way. - Predicate predicate = - partitions.stream() - .map( - partition -> - createPartitionPredicate( - partition, partitionType, partitionDefaultName)) - .reduce(PredicateBuilder::or) - .orElseThrow(() -> new RuntimeException("Failed to get partition filter.")); - PartitionPredicate partitionFilter = - PartitionPredicate.fromPredicate(partitionType, predicate); + boolean fullMode = + partitions.stream().allMatch(part -> part.size() == partitionType.getFieldCount()); + PartitionPredicate partitionFilter; + if (fullMode) { + List binaryPartitions = + createBinaryPartitions(partitions, partitionType, partitionDefaultName); + partitionFilter = PartitionPredicate.fromMultiple(partitionType, binaryPartitions); + } else { + // partitions may be partial partition fields, so here must to use predicate way. + Predicate predicate = + partitions.stream() + .map( + partition -> + createPartitionPredicate( + partition, partitionType, partitionDefaultName)) + .reduce(PredicateBuilder::or) + .orElseThrow( + () -> new RuntimeException("Failed to get partition filter.")); + partitionFilter = PartitionPredicate.fromPredicate(partitionType, predicate); + } tryOverwrite( partitionFilter, @@ -715,21 +734,11 @@ private int tryCommit( ConflictCheck conflictCheck, String branchName, @Nullable String statsFileName) { - int cnt = 0; + int retryCount = 0; RetryResult retryResult = null; + long startMillis = System.currentTimeMillis(); while (true) { Snapshot latestSnapshot = snapshotManager.latestSnapshot(); - cnt++; - if (cnt >= commitMaxRetries) { - if (retryResult != null) { - retryResult.cleanAll(); - } - throw new RuntimeException( - String.format( - "Commit failed after %s attempts, there maybe exist commit conflicts between multiple jobs.", - commitMaxRetries)); - } - CommitResult result = tryCommitOnce( retryResult, @@ -750,8 +759,19 @@ private int tryCommit( } retryResult = (RetryResult) result; + + if (System.currentTimeMillis() - startMillis > commitTimeout + || retryCount >= commitMaxRetries) { + retryResult.cleanAll(); + throw new RuntimeException( + String.format( + "Commit failed after %s millis with %s retries, there maybe exist commit conflicts between multiple jobs.", + commitTimeout, retryCount)); + } + + retryCount++; } - return cnt; + return retryCount + 1; } private int tryOverwrite( @@ -761,74 +781,52 @@ private int tryOverwrite( long identifier, @Nullable Long watermark, Map logOffsets) { - int cnt = 0; - while (true) { - Snapshot latestSnapshot = snapshotManager.latestSnapshot(); - - cnt++; - if (cnt >= commitMaxRetries) { - throw new RuntimeException( - String.format( - "Commit failed after %s attempts, there maybe exist commit conflicts between multiple jobs.", - commitMaxRetries)); + // collect all files with overwrite + Snapshot latestSnapshot = snapshotManager.latestSnapshot(); + List changesWithOverwrite = new ArrayList<>(); + List indexChangesWithOverwrite = new ArrayList<>(); + if (latestSnapshot != null) { + List currentEntries = + scan.withSnapshot(latestSnapshot) + .withPartitionFilter(partitionFilter) + .withKind(ScanMode.ALL) + .plan() + .files(); + for (ManifestEntry entry : currentEntries) { + changesWithOverwrite.add( + new ManifestEntry( + FileKind.DELETE, + entry.partition(), + entry.bucket(), + entry.totalBuckets(), + entry.file())); } - List changesWithOverwrite = new ArrayList<>(); - List indexChangesWithOverwrite = new ArrayList<>(); - if (latestSnapshot != null) { - List currentEntries = - scan.withSnapshot(latestSnapshot) - .withPartitionFilter(partitionFilter) - .withKind(ScanMode.ALL) - .plan() - .files(); - for (ManifestEntry entry : currentEntries) { - changesWithOverwrite.add( - new ManifestEntry( - FileKind.DELETE, - entry.partition(), - entry.bucket(), - entry.totalBuckets(), - entry.file())); - } - // collect index files - if (latestSnapshot.indexManifest() != null) { - List entries = - indexManifestFile.read(latestSnapshot.indexManifest()); - for (IndexManifestEntry entry : entries) { - if (partitionFilter == null || partitionFilter.test(entry.partition())) { - indexChangesWithOverwrite.add(entry.toDeleteEntry()); - } + // collect index files + if (latestSnapshot.indexManifest() != null) { + List entries = + indexManifestFile.read(latestSnapshot.indexManifest()); + for (IndexManifestEntry entry : entries) { + if (partitionFilter == null || partitionFilter.test(entry.partition())) { + indexChangesWithOverwrite.add(entry.toDeleteEntry()); } } } - changesWithOverwrite.addAll(changes); - indexChangesWithOverwrite.addAll(indexFiles); - - CommitResult result = - tryCommitOnce( - null, - changesWithOverwrite, - Collections.emptyList(), - indexChangesWithOverwrite, - identifier, - watermark, - logOffsets, - Snapshot.CommitKind.OVERWRITE, - latestSnapshot, - mustConflictCheck(), - branchName, - null); - - if (result.isSuccess()) { - break; - } - - // TODO optimize OVERWRITE too - RetryResult retryResult = (RetryResult) result; - retryResult.cleanAll(); } - return cnt; + changesWithOverwrite.addAll(changes); + indexChangesWithOverwrite.addAll(indexFiles); + + return tryCommit( + changesWithOverwrite, + Collections.emptyList(), + indexChangesWithOverwrite, + identifier, + watermark, + logOffsets, + Snapshot.CommitKind.OVERWRITE, + mustConflictCheck(), + branchName, + null); } @VisibleForTesting @@ -1068,22 +1066,25 @@ CommitResult tryCommitOnce( } public void compactManifest() { - int cnt = 0; + int retryCount = 0; ManifestCompactResult retryResult = null; + long startMillis = System.currentTimeMillis(); while (true) { - cnt++; retryResult = compactManifest(retryResult); if (retryResult.isSuccess()) { break; } - if (cnt >= commitMaxRetries) { + if (System.currentTimeMillis() - startMillis > commitTimeout + || retryCount >= commitMaxRetries) { retryResult.cleanAll(); throw new RuntimeException( String.format( - "Commit compact manifest failed after %s attempts, there maybe exist commit conflicts between multiple jobs.", - commitMaxRetries)); + "Commit failed after %s millis with %s retries, there maybe exist commit conflicts between multiple jobs.", + commitTimeout, retryCount)); } + + retryCount++; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java index 89e7e7aace908..8e9dc31757fe9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreScan.java @@ -70,12 +70,12 @@ public interface FileStoreScan { FileStoreScan withSnapshot(Snapshot snapshot); - FileStoreScan withManifestList(List manifests); - FileStoreScan withKind(ScanMode scanMode); FileStoreScan withLevelFilter(Filter levelFilter); + FileStoreScan enableValueFilter(); + FileStoreScan withManifestEntryFilter(Filter filter); FileStoreScan withManifestCacheFilter(ManifestCacheFilter manifestFilter); @@ -84,6 +84,8 @@ public interface FileStoreScan { FileStoreScan withMetrics(ScanMetrics metrics); + FileStoreScan dropStats(); + @Nullable Integer parallelism(); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java index c368d9e510b02..8d8c51996cfe9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java @@ -23,6 +23,7 @@ import org.apache.paimon.KeyValueFileStore; import org.apache.paimon.fileindex.FileIndexPredicate; import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.manifest.FilteredManifestEntry; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.predicate.Predicate; @@ -45,7 +46,6 @@ import java.util.Map; import static org.apache.paimon.CoreOptions.MergeEngine.AGGREGATE; -import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; import static org.apache.paimon.CoreOptions.MergeEngine.PARTIAL_UPDATE; /** {@link FileStoreScan} for {@link KeyValueFileStore}. */ @@ -64,6 +64,8 @@ public class KeyValueFileStoreScan extends AbstractFileStoreScan { private final boolean fileIndexReadEnabled; private final Map schemaId2DataFilter = new HashMap<>(); + private boolean valueFilterForceEnabled = false; + public KeyValueFileStoreScan( ManifestsReader manifestsReader, BucketSelectConverter bucketSelectConverter, @@ -110,11 +112,17 @@ public KeyValueFileStoreScan withValueFilter(Predicate predicate) { return this; } + @Override + public FileStoreScan enableValueFilter() { + this.valueFilterForceEnabled = true; + return this; + } + /** Note: Keep this thread-safe. */ @Override protected boolean filterByStats(ManifestEntry entry) { DataFileMeta file = entry.file(); - if (isValueFilterEnabled(entry) && !filterByValueFilter(entry)) { + if (isValueFilterEnabled() && !filterByValueFilter(entry)) { return false; } @@ -130,6 +138,14 @@ protected boolean filterByStats(ManifestEntry entry) { return true; } + @Override + protected ManifestEntry dropStats(ManifestEntry entry) { + if (!isValueFilterEnabled() && wholeBucketFilterEnabled()) { + return new FilteredManifestEntry(entry.copyWithoutStats(), filterByValueFilter(entry)); + } + return entry.copyWithoutStats(); + } + private boolean filterByFileIndex(@Nullable byte[] embeddedIndexBytes, ManifestEntry entry) { if (embeddedIndexBytes == null) { return true; @@ -150,14 +166,14 @@ private boolean filterByFileIndex(@Nullable byte[] embeddedIndexBytes, ManifestE } } - private boolean isValueFilterEnabled(ManifestEntry entry) { + private boolean isValueFilterEnabled() { if (valueFilter == null) { return false; } switch (scanMode) { case ALL: - return (deletionVectorsEnabled || mergeEngine == FIRST_ROW) && entry.level() > 0; + return valueFilterForceEnabled; case DELTA: return false; case CHANGELOG: @@ -168,13 +184,13 @@ private boolean isValueFilterEnabled(ManifestEntry entry) { } } - /** Note: Keep this thread-safe. */ @Override - protected List filterWholeBucketByStats(List entries) { - if (valueFilter == null || scanMode != ScanMode.ALL) { - return entries; - } + protected boolean wholeBucketFilterEnabled() { + return valueFilter != null && scanMode == ScanMode.ALL; + } + @Override + protected List filterWholeBucketByStats(List entries) { return noOverlapping(entries) ? filterWholeBucketPerFile(entries) : filterWholeBucketAllFiles(entries); @@ -207,6 +223,10 @@ private List filterWholeBucketAllFiles(List entrie } private boolean filterByValueFilter(ManifestEntry entry) { + if (entry instanceof FilteredManifestEntry) { + return ((FilteredManifestEntry) entry).selected(); + } + DataFileMeta file = entry.file(); SimpleStatsEvolution.Result result = fieldValueStatsConverters diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java b/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java index 3ee108c10359a..6a4276662468b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/LocalOrphanFilesClean.java @@ -21,12 +21,12 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; +import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.SerializableConsumer; import javax.annotation.Nullable; @@ -47,13 +47,14 @@ import java.util.concurrent.Future; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.paimon.utils.ThreadPoolUtils.createCachedThreadPool; -import static org.apache.paimon.utils.ThreadPoolUtils.randomlyExecute; +import static org.apache.paimon.utils.ThreadPoolUtils.randomlyExecuteSequentialReturn; import static org.apache.paimon.utils.ThreadPoolUtils.randomlyOnlyExecute; /** @@ -68,6 +69,8 @@ public class LocalOrphanFilesClean extends OrphanFilesClean { private final List deleteFiles; + private final AtomicLong deletedFilesLenInBytes = new AtomicLong(0); + private Set candidateDeletes; public LocalOrphanFilesClean(FileStoreTable table) { @@ -87,16 +90,18 @@ public LocalOrphanFilesClean( table.coreOptions().deleteFileThreadNum(), "ORPHAN_FILES_CLEAN"); } - public List clean() throws IOException, ExecutionException, InterruptedException { + public CleanOrphanFilesResult clean() + throws IOException, ExecutionException, InterruptedException { List branches = validBranches(); // specially handle to clear snapshot dir - cleanSnapshotDir(branches, deleteFiles::add); + cleanSnapshotDir(branches, deleteFiles::add, deletedFilesLenInBytes::addAndGet); // delete candidate files - Map candidates = getCandidateDeletingFiles(); + Map> candidates = getCandidateDeletingFiles(); if (candidates.isEmpty()) { - return deleteFiles; + return new CleanOrphanFilesResult( + deleteFiles.size(), deletedFilesLenInBytes.get(), deleteFiles); } candidateDeletes = new HashSet<>(candidates.keySet()); @@ -108,12 +113,22 @@ public List clean() throws IOException, ExecutionException, InterruptedExc // delete unused files candidateDeletes.removeAll(usedFiles); - candidateDeletes.stream().map(candidates::get).forEach(fileCleaner); + candidateDeletes.stream() + .map(candidates::get) + .forEach( + deleteFileInfo -> { + deletedFilesLenInBytes.addAndGet(deleteFileInfo.getRight()); + fileCleaner.accept(deleteFileInfo.getLeft()); + }); deleteFiles.addAll( - candidateDeletes.stream().map(candidates::get).collect(Collectors.toList())); + candidateDeletes.stream() + .map(candidates::get) + .map(Pair::getLeft) + .collect(Collectors.toList())); candidateDeletes.clear(); - return deleteFiles; + return new CleanOrphanFilesResult( + deleteFiles.size(), deletedFilesLenInBytes.get(), deleteFiles); } private void collectWithoutDataFile( @@ -172,19 +187,20 @@ private Set getUsedFiles(String branch) { * Get all the candidate deleting files in the specified directories and filter them by * olderThanMillis. */ - private Map getCandidateDeletingFiles() { + private Map> getCandidateDeletingFiles() { List fileDirs = listPaimonFileDirs(); - Function> processor = + Function>> processor = path -> tryBestListingDirs(path).stream() .filter(this::oldEnough) - .map(FileStatus::getPath) + .map(status -> Pair.of(status.getPath(), status.getLen())) .collect(Collectors.toList()); - Iterator allPaths = randomlyExecute(executor, processor, fileDirs); - Map result = new HashMap<>(); - while (allPaths.hasNext()) { - Path next = allPaths.next(); - result.put(next.getName(), next); + Iterator> allFilesInfo = + randomlyExecuteSequentialReturn(executor, processor, fileDirs); + Map> result = new HashMap<>(); + while (allFilesInfo.hasNext()) { + Pair fileInfo = allFilesInfo.next(); + result.put(fileInfo.getLeft().getName(), fileInfo); } return result; } @@ -197,7 +213,6 @@ public static List createOrphanFilesCleans( SerializableConsumer fileCleaner, @Nullable Integer parallelism) throws Catalog.DatabaseNotExistException, Catalog.TableNotExistException { - List orphanFilesCleans = new ArrayList<>(); List tableNames = Collections.singletonList(tableName); if (tableName == null || "*".equals(tableName)) { tableNames = catalog.listTables(databaseName); @@ -214,6 +229,7 @@ public static List createOrphanFilesCleans( } }; + List orphanFilesCleans = new ArrayList<>(tableNames.size()); for (String t : tableNames) { Identifier identifier = new Identifier(databaseName, t); Table table = catalog.getTable(identifier).copy(dynamicOptions); @@ -230,7 +246,7 @@ public static List createOrphanFilesCleans( return orphanFilesCleans; } - public static long executeDatabaseOrphanFiles( + public static CleanOrphanFilesResult executeDatabaseOrphanFiles( Catalog catalog, String databaseName, @Nullable String tableName, @@ -249,15 +265,17 @@ public static long executeDatabaseOrphanFiles( ExecutorService executorService = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); - List>> tasks = new ArrayList<>(); + List> tasks = new ArrayList<>(tableCleans.size()); for (LocalOrphanFilesClean clean : tableCleans) { tasks.add(executorService.submit(clean::clean)); } - List cleanOrphanFiles = new ArrayList<>(); - for (Future> task : tasks) { + long deletedFileCount = 0; + long deletedFileTotalLenInBytes = 0; + for (Future task : tasks) { try { - cleanOrphanFiles.addAll(task.get()); + deletedFileCount += task.get().getDeletedFileCount(); + deletedFileTotalLenInBytes += task.get().getDeletedFileTotalLenInBytes(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new RuntimeException(e); @@ -267,6 +285,6 @@ public static long executeDatabaseOrphanFiles( } executorService.shutdownNow(); - return cleanOrphanFiles.size(); + return new CleanOrphanFilesResult(deletedFileCount, deletedFileTotalLenInBytes); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/MemoryFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/MemoryFileStoreWrite.java index ff99f06510c93..a2733121eece1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/MemoryFileStoreWrite.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/MemoryFileStoreWrite.java @@ -73,6 +73,7 @@ public MemoryFileStoreWrite( indexFactory, dvMaintainerFactory, tableName, + options, options.bucket(), partitionType, options.writeMaxWritersToSpill(), diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java index 5698908cb9b0c..274cdd52fe140 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java @@ -105,7 +105,8 @@ protected List validBranches() { List abnormalBranches = new ArrayList<>(); for (String branch : branches) { - if (!new SchemaManager(table.fileIO(), table.location(), branch).latest().isPresent()) { + SchemaManager schemaManager = table.schemaManager().copyWithBranch(branch); + if (!schemaManager.latest().isPresent()) { abnormalBranches.add(branch); } } @@ -119,23 +120,47 @@ protected List validBranches() { return branches; } - protected void cleanSnapshotDir(List branches, Consumer deletedFileConsumer) { + protected void cleanSnapshotDir( + List branches, + Consumer deletedFilesConsumer, + Consumer deletedFilesLenInBytesConsumer) { for (String branch : branches) { FileStoreTable branchTable = table.switchToBranch(branch); SnapshotManager snapshotManager = branchTable.snapshotManager(); // specially handle the snapshot directory - List nonSnapshotFiles = snapshotManager.tryGetNonSnapshotFiles(this::oldEnough); - nonSnapshotFiles.forEach(fileCleaner); - nonSnapshotFiles.forEach(deletedFileConsumer); + List> nonSnapshotFiles = + snapshotManager.tryGetNonSnapshotFiles(this::oldEnough); + nonSnapshotFiles.forEach( + nonSnapshotFile -> + cleanFile( + nonSnapshotFile, + deletedFilesConsumer, + deletedFilesLenInBytesConsumer)); // specially handle the changelog directory - List nonChangelogFiles = snapshotManager.tryGetNonChangelogFiles(this::oldEnough); - nonChangelogFiles.forEach(fileCleaner); - nonChangelogFiles.forEach(deletedFileConsumer); + List> nonChangelogFiles = + snapshotManager.tryGetNonChangelogFiles(this::oldEnough); + nonChangelogFiles.forEach( + nonChangelogFile -> + cleanFile( + nonChangelogFile, + deletedFilesConsumer, + deletedFilesLenInBytesConsumer)); } } + private void cleanFile( + Pair deleteFileInfo, + Consumer deletedFilesConsumer, + Consumer deletedFilesLenInBytesConsumer) { + Path filePath = deleteFileInfo.getLeft(); + Long fileSize = deleteFileInfo.getRight(); + deletedFilesConsumer.accept(filePath); + deletedFilesLenInBytesConsumer.accept(fileSize); + fileCleaner.accept(filePath); + } + protected Set safelyGetAllSnapshots(String branch) throws IOException { FileStoreTable branchTable = table.switchToBranch(branch); SnapshotManager snapshotManager = branchTable.snapshotManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java b/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java index 62a9b796476ab..d432a37dfd9cc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/PartitionExpire.java @@ -54,7 +54,7 @@ public class PartitionExpire { private LocalDateTime lastCheck; private final PartitionExpireStrategy strategy; private final boolean endInputCheckPartitionExpire; - private int maxExpires; + private int maxExpireNum; public PartitionExpire( Duration expirationTime, @@ -63,7 +63,8 @@ public PartitionExpire( FileStoreScan scan, FileStoreCommit commit, @Nullable MetastoreClient metastoreClient, - boolean endInputCheckPartitionExpire) { + boolean endInputCheckPartitionExpire, + int maxExpireNum) { this.expirationTime = expirationTime; this.checkInterval = checkInterval; this.strategy = strategy; @@ -72,7 +73,7 @@ public PartitionExpire( this.metastoreClient = metastoreClient; this.lastCheck = LocalDateTime.now(); this.endInputCheckPartitionExpire = endInputCheckPartitionExpire; - this.maxExpires = Integer.MAX_VALUE; + this.maxExpireNum = maxExpireNum; } public PartitionExpire( @@ -81,8 +82,17 @@ public PartitionExpire( PartitionExpireStrategy strategy, FileStoreScan scan, FileStoreCommit commit, - @Nullable MetastoreClient metastoreClient) { - this(expirationTime, checkInterval, strategy, scan, commit, metastoreClient, false); + @Nullable MetastoreClient metastoreClient, + int maxExpireNum) { + this( + expirationTime, + checkInterval, + strategy, + scan, + commit, + metastoreClient, + false, + maxExpireNum); } public PartitionExpire withLock(Lock lock) { @@ -90,8 +100,8 @@ public PartitionExpire withLock(Lock lock) { return this; } - public PartitionExpire withMaxExpires(int maxExpires) { - this.maxExpires = maxExpires; + public PartitionExpire withMaxExpireNum(int maxExpireNum) { + this.maxExpireNum = maxExpireNum; return this; } @@ -145,6 +155,7 @@ private List> doExpire( List> expired = new ArrayList<>(); if (!expiredPartValues.isEmpty()) { + // convert partition value to partition string, and limit the partition num expired = convertToPartitionString(expiredPartValues); LOG.info("Expire Partitions: {}", expired); if (metastoreClient != null) { @@ -175,7 +186,7 @@ private List> convertToPartitionString( .sorted() .map(s -> s.split(DELIMITER)) .map(strategy::toPartitionString) - .limit(Math.min(expiredPartValues.size(), maxExpires)) + .limit(Math.min(expiredPartValues.size(), maxExpireNum)) .collect(Collectors.toList()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java index 9c612a9f8cf06..4fda82f4e88f2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java @@ -24,18 +24,21 @@ import org.apache.paimon.deletionvectors.DeletionVector; import org.apache.paimon.disk.IOManager; import org.apache.paimon.fileindex.FileIndexResult; +import org.apache.paimon.fileindex.bitmap.ApplyBitmapIndexRecordReader; +import org.apache.paimon.fileindex.bitmap.BitmapIndexResult; import org.apache.paimon.format.FileFormatDiscover; import org.apache.paimon.format.FormatKey; import org.apache.paimon.format.FormatReaderContext; import org.apache.paimon.fs.FileIO; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFilePathFactory; +import org.apache.paimon.io.DataFileRecordReader; import org.apache.paimon.io.FileIndexEvaluator; -import org.apache.paimon.io.FileRecordReader; import org.apache.paimon.mergetree.compact.ConcatRecordReader; import org.apache.paimon.partition.PartitionUtils; import org.apache.paimon.predicate.Predicate; -import org.apache.paimon.reader.EmptyRecordReader; +import org.apache.paimon.reader.EmptyFileRecordReader; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.ReaderSupplier; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.SchemaManager; @@ -43,9 +46,9 @@ import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.types.DataField; import org.apache.paimon.types.RowType; -import org.apache.paimon.utils.BulkFormatMapping; -import org.apache.paimon.utils.BulkFormatMapping.BulkFormatMappingBuilder; import org.apache.paimon.utils.FileStorePathFactory; +import org.apache.paimon.utils.FormatReaderMapping; +import org.apache.paimon.utils.FormatReaderMapping.Builder; import org.apache.paimon.utils.IOExceptionSupplier; import org.slf4j.Logger; @@ -72,7 +75,7 @@ public class RawFileSplitRead implements SplitRead { private final TableSchema schema; private final FileFormatDiscover formatDiscover; private final FileStorePathFactory pathFactory; - private final Map bulkFormatMappings; + private final Map formatReaderMappings; private final boolean fileIndexReadEnabled; private RowType readRowType; @@ -91,7 +94,7 @@ public RawFileSplitRead( this.schema = schema; this.formatDiscover = formatDiscover; this.pathFactory = pathFactory; - this.bulkFormatMappings = new HashMap<>(); + this.formatReaderMappings = new HashMap<>(); this.fileIndexReadEnabled = fileIndexReadEnabled; this.readRowType = rowType; } @@ -147,26 +150,25 @@ public RecordReader createReader( List> suppliers = new ArrayList<>(); List readTableFields = readRowType.getFields(); - BulkFormatMappingBuilder bulkFormatMappingBuilder = - new BulkFormatMappingBuilder( - formatDiscover, readTableFields, TableSchema::fields, filters); + Builder formatReaderMappingBuilder = + new Builder(formatDiscover, readTableFields, TableSchema::fields, filters); for (int i = 0; i < files.size(); i++) { DataFileMeta file = files.get(i); String formatIdentifier = DataFilePathFactory.formatIdentifier(file.fileName()); long schemaId = file.schemaId(); - Supplier formatSupplier = + Supplier formatSupplier = () -> - bulkFormatMappingBuilder.build( + formatReaderMappingBuilder.build( formatIdentifier, schema, schemaId == schema.id() ? schema : schemaManager.schema(schemaId)); - BulkFormatMapping bulkFormatMapping = - bulkFormatMappings.computeIfAbsent( + FormatReaderMapping formatReaderMapping = + formatReaderMappings.computeIfAbsent( new FormatKey(file.schemaId(), formatIdentifier), key -> formatSupplier.get()); @@ -178,18 +180,18 @@ public RecordReader createReader( partition, file, dataFilePathFactory, - bulkFormatMapping, + formatReaderMapping, dvFactory)); } return ConcatRecordReader.create(suppliers); } - private RecordReader createFileReader( + private FileRecordReader createFileReader( BinaryRow partition, DataFileMeta file, DataFilePathFactory dataFilePathFactory, - BulkFormatMapping bulkFormatMapping, + FormatReaderMapping formatReaderMapping, IOExceptionSupplier dvFactory) throws IOException { FileIndexResult fileIndexResult = null; @@ -197,12 +199,12 @@ private RecordReader createFileReader( fileIndexResult = FileIndexEvaluator.evaluate( fileIO, - bulkFormatMapping.getDataSchema(), - bulkFormatMapping.getDataFilters(), + formatReaderMapping.getDataSchema(), + formatReaderMapping.getDataFilters(), dataFilePathFactory, file); if (!fileIndexResult.remain()) { - return new EmptyRecordReader<>(); + return new EmptyFileRecordReader<>(); } } @@ -212,13 +214,19 @@ private RecordReader createFileReader( dataFilePathFactory.toPath(file.fileName()), file.fileSize(), fileIndexResult); - FileRecordReader fileRecordReader = - new FileRecordReader( - bulkFormatMapping.getReaderFactory(), + FileRecordReader fileRecordReader = + new DataFileRecordReader( + formatReaderMapping.getReaderFactory(), formatReaderContext, - bulkFormatMapping.getIndexMapping(), - bulkFormatMapping.getCastMapping(), - PartitionUtils.create(bulkFormatMapping.getPartitionPair(), partition)); + formatReaderMapping.getIndexMapping(), + formatReaderMapping.getCastMapping(), + PartitionUtils.create(formatReaderMapping.getPartitionPair(), partition)); + + if (fileIndexResult instanceof BitmapIndexResult) { + fileRecordReader = + new ApplyBitmapIndexRecordReader( + fileRecordReader, (BitmapIndexResult) fileIndexResult); + } DeletionVector deletionVector = dvFactory == null ? null : dvFactory.get(); if (deletionVector != null && !deletionVector.isEmpty()) { diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/SnapshotDeletion.java b/paimon-core/src/main/java/org/apache/paimon/operation/SnapshotDeletion.java index d86907ecea546..7d55b64c8eac5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/SnapshotDeletion.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/SnapshotDeletion.java @@ -23,8 +23,8 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.index.IndexFileHandler; +import org.apache.paimon.manifest.ExpireFileEntry; import org.apache.paimon.manifest.FileSource; -import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.manifest.ManifestList; import org.apache.paimon.stats.StatsFileHandler; @@ -65,15 +65,15 @@ public SnapshotDeletion( } @Override - public void cleanUnusedDataFiles(Snapshot snapshot, Predicate skipper) { + public void cleanUnusedDataFiles(Snapshot snapshot, Predicate skipper) { if (changelogDecoupled && !produceChangelog) { // Skip clean the 'APPEND' data files.If we do not have the file source information // eg: the old version table file, we just skip clean this here, let it done by // ExpireChangelogImpl - Predicate enriched = + Predicate enriched = manifestEntry -> skipper.test(manifestEntry) - || (manifestEntry.file().fileSource().orElse(FileSource.APPEND) + || (manifestEntry.fileSource().orElse(FileSource.APPEND) == FileSource.APPEND); cleanUnusedDataFiles(snapshot.deltaManifestList(), enriched); } else { @@ -92,8 +92,8 @@ public void cleanUnusedManifests(Snapshot snapshot, Set skippingSet) { } @VisibleForTesting - void cleanUnusedDataFile(List dataFileLog) { - Map>> dataFileToDelete = new HashMap<>(); + void cleanUnusedDataFile(List dataFileLog) { + Map>> dataFileToDelete = new HashMap<>(); getDataFileToDelete(dataFileToDelete, dataFileLog); doCleanUnusedDataFile(dataFileToDelete, f -> false); } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/TagDeletion.java b/paimon-core/src/main/java/org/apache/paimon/operation/TagDeletion.java index a6cd338d5859c..2722ed0c7ec81 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/TagDeletion.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/TagDeletion.java @@ -23,7 +23,7 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.index.IndexFileHandler; -import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.manifest.ExpireFileEntry; import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.manifest.ManifestList; import org.apache.paimon.stats.StatsFileHandler; @@ -68,8 +68,8 @@ public TagDeletion( } @Override - public void cleanUnusedDataFiles(Snapshot taggedSnapshot, Predicate skipper) { - Collection manifestEntries; + public void cleanUnusedDataFiles(Snapshot taggedSnapshot, Predicate skipper) { + Collection manifestEntries; try { manifestEntries = readMergedDataFiles(taggedSnapshot); } catch (IOException e) { @@ -78,11 +78,11 @@ public void cleanUnusedDataFiles(Snapshot taggedSnapshot, Predicate dataFileToDelete = new HashSet<>(); - for (ManifestEntry entry : manifestEntries) { + for (ExpireFileEntry entry : manifestEntries) { if (!skipper.test(entry)) { Path bucketPath = pathFactory.bucketPath(entry.partition(), entry.bucket()); - dataFileToDelete.add(new Path(bucketPath, entry.file().fileName())); - for (String file : entry.file().extraFiles()) { + dataFileToDelete.add(new Path(bucketPath, entry.fileName())); + for (String file : entry.extraFiles()) { dataFileToDelete.add(new Path(bucketPath, file)); } @@ -98,11 +98,12 @@ public void cleanUnusedManifests(Snapshot taggedSnapshot, Set skippingSe cleanUnusedManifests(taggedSnapshot, skippingSet, true, false); } - public Predicate dataFileSkipper(Snapshot fromSnapshot) throws Exception { + public Predicate dataFileSkipper(Snapshot fromSnapshot) throws Exception { return dataFileSkipper(Collections.singletonList(fromSnapshot)); } - public Predicate dataFileSkipper(List fromSnapshots) throws Exception { + public Predicate dataFileSkipper(List fromSnapshots) + throws Exception { Map>> skipped = new HashMap<>(); for (Snapshot snapshot : fromSnapshots) { addMergedDataFiles(skipped, snapshot); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java b/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java index 9fcbb8960fc50..96f0aec1c0b21 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanMetrics.java @@ -49,12 +49,6 @@ public MetricGroup getMetricGroup() { public static final String SCAN_DURATION = "scanDuration"; public static final String LAST_SCANNED_MANIFESTS = "lastScannedManifests"; - public static final String LAST_SKIPPED_BY_PARTITION_AND_STATS = - "lastSkippedByPartitionAndStats"; - - public static final String LAST_SKIPPED_BY_WHOLE_BUCKET_FILES_FILTER = - "lastSkippedByWholeBucketFilesFilter"; - public static final String LAST_SCAN_SKIPPED_TABLE_FILES = "lastScanSkippedTableFiles"; public static final String LAST_SCAN_RESULTED_TABLE_FILES = "lastScanResultedTableFiles"; @@ -66,12 +60,6 @@ private void registerGenericScanMetrics() { metricGroup.gauge( LAST_SCANNED_MANIFESTS, () -> latestScan == null ? 0L : latestScan.getScannedManifests()); - metricGroup.gauge( - LAST_SKIPPED_BY_PARTITION_AND_STATS, - () -> latestScan == null ? 0L : latestScan.getSkippedByPartitionAndStats()); - metricGroup.gauge( - LAST_SKIPPED_BY_WHOLE_BUCKET_FILES_FILTER, - () -> latestScan == null ? 0L : latestScan.getSkippedByWholeBucketFiles()); metricGroup.gauge( LAST_SCAN_SKIPPED_TABLE_FILES, () -> latestScan == null ? 0L : latestScan.getSkippedTableFiles()); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanStats.java b/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanStats.java index e760282e687a5..700619c3680f0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanStats.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/metrics/ScanStats.java @@ -25,23 +25,15 @@ public class ScanStats { // the unit is milliseconds private final long duration; private final long scannedManifests; - private final long skippedByPartitionAndStats; - private final long skippedByWholeBucketFiles; private final long skippedTableFiles; private final long resultedTableFiles; public ScanStats( - long duration, - long scannedManifests, - long skippedByPartitionAndStats, - long skippedByWholeBucketFiles, - long resultedTableFiles) { + long duration, long scannedManifests, long skippedTableFiles, long resultedTableFiles) { this.duration = duration; this.scannedManifests = scannedManifests; - this.skippedByPartitionAndStats = skippedByPartitionAndStats; - this.skippedByWholeBucketFiles = skippedByWholeBucketFiles; - this.skippedTableFiles = skippedByPartitionAndStats + skippedByWholeBucketFiles; + this.skippedTableFiles = skippedTableFiles; this.resultedTableFiles = resultedTableFiles; } @@ -60,16 +52,6 @@ protected long getResultedTableFiles() { return resultedTableFiles; } - @VisibleForTesting - protected long getSkippedByPartitionAndStats() { - return skippedByPartitionAndStats; - } - - @VisibleForTesting - protected long getSkippedByWholeBucketFiles() { - return skippedByWholeBucketFiles; - } - @VisibleForTesting protected long getDuration() { return duration; diff --git a/paimon-core/src/main/java/org/apache/paimon/partition/PartitionPredicate.java b/paimon-core/src/main/java/org/apache/paimon/partition/PartitionPredicate.java index 12ea884be15fb..1f6c2cfe454e4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/partition/PartitionPredicate.java +++ b/paimon-core/src/main/java/org/apache/paimon/partition/PartitionPredicate.java @@ -19,8 +19,10 @@ package org.apache.paimon.partition; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.data.serializer.InternalSerializers; import org.apache.paimon.data.serializer.Serializer; import org.apache.paimon.format.SimpleColStats; @@ -33,6 +35,7 @@ import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -40,6 +43,7 @@ import java.util.Set; import static org.apache.paimon.utils.InternalRowPartitionComputer.convertSpecToInternal; +import static org.apache.paimon.utils.InternalRowPartitionComputer.convertSpecToInternalRow; import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.paimon.utils.Preconditions.checkNotNull; @@ -231,4 +235,15 @@ static Predicate createPartitionPredicate( .map(p -> createPartitionPredicate(p, rowType, defaultPartValue)) .toArray(Predicate[]::new)); } + + static List createBinaryPartitions( + List> partitions, RowType partitionType, String defaultPartValue) { + InternalRowSerializer serializer = new InternalRowSerializer(partitionType); + List result = new ArrayList<>(); + for (Map spec : partitions) { + GenericRow row = convertSpecToInternalRow(spec, partitionType, defaultPartValue); + result.add(serializer.toBinaryRow(row).copy()); + } + return result; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/partition/PartitionValuesTimeExpireStrategy.java b/paimon-core/src/main/java/org/apache/paimon/partition/PartitionValuesTimeExpireStrategy.java index 80ae633fd2970..51c53282c476a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/partition/PartitionValuesTimeExpireStrategy.java +++ b/paimon-core/src/main/java/org/apache/paimon/partition/PartitionValuesTimeExpireStrategy.java @@ -81,23 +81,34 @@ public boolean test(BinaryRow partition) { LocalDateTime partTime = timeExtractor.extract(partitionKeys, Arrays.asList(array)); return expireDateTime.isAfter(partTime); } catch (DateTimeParseException e) { - String partitionInfo = - IntStream.range(0, partitionKeys.size()) - .mapToObj(i -> partitionKeys.get(i) + ":" + array[i]) - .collect(Collectors.joining(",")); LOG.warn( "Can't extract datetime from partition {}. If you want to configure partition expiration, please:\n" + " 1. Check the expiration configuration.\n" + " 2. Manually delete the partition using the drop-partition command if the partition" + " value is non-date formatted.\n" + " 3. Use '{}' expiration strategy by set '{}', which supports non-date formatted partition.", - partitionInfo, + formatPartitionInfo(array), + CoreOptions.PartitionExpireStrategy.UPDATE_TIME, + CoreOptions.PARTITION_EXPIRATION_STRATEGY.key()); + return false; + } catch (NullPointerException e) { + // there might exist NULL partition value + LOG.warn( + "This partition {} cannot be expired because it contains null value. " + + "You can try to drop it manually or use '{}' expiration strategy by set '{}'.", + formatPartitionInfo(array), CoreOptions.PartitionExpireStrategy.UPDATE_TIME, CoreOptions.PARTITION_EXPIRATION_STRATEGY.key()); return false; } } + private String formatPartitionInfo(Object[] array) { + return IntStream.range(0, partitionKeys.size()) + .mapToObj(i -> partitionKeys.get(i) + ":" + array[i]) + .collect(Collectors.joining(",")); + } + @Override public boolean test( long rowCount, diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java index c9b9c21937be8..2e88213a24b95 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedCatalog.java @@ -127,7 +127,7 @@ public void alterTable( public Table getTable(Identifier identifier) throws TableNotExistException { Table table = wrapped.getTable(identifier); if (table instanceof FileStoreTable) { - return new PrivilegedFileStoreTable( + return PrivilegedFileStoreTable.wrap( (FileStoreTable) table, privilegeManager.getPrivilegeChecker(), identifier); } else { return table; diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStore.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStore.java index 59243a53569ed..3ee0d5fa9b013 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStore.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.FileStore; +import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.fs.Path; import org.apache.paimon.index.IndexFileHandler; @@ -47,6 +48,8 @@ import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.TagManager; +import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache; + import javax.annotation.Nullable; import java.util.List; @@ -210,4 +213,9 @@ public List createTagCallbacks() { public void setManifestCache(SegmentsCache manifestCache) { wrapped.setManifestCache(manifestCache); } + + @Override + public void setSnapshotCache(Cache cache) { + wrapped.setSnapshotCache(cache); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java index 37990ed5a1f3b..52c806c7c53b2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java @@ -27,6 +27,7 @@ import org.apache.paimon.table.DelegatedFileStoreTable; import org.apache.paimon.table.ExpireSnapshots; import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.object.ObjectTable; import org.apache.paimon.table.query.LocalTableQuery; import org.apache.paimon.table.sink.TableCommitImpl; import org.apache.paimon.table.sink.TableWriteImpl; @@ -48,10 +49,10 @@ /** {@link FileStoreTable} with privilege checks. */ public class PrivilegedFileStoreTable extends DelegatedFileStoreTable { - private final PrivilegeChecker privilegeChecker; - private final Identifier identifier; + protected final PrivilegeChecker privilegeChecker; + protected final Identifier identifier; - public PrivilegedFileStoreTable( + protected PrivilegedFileStoreTable( FileStoreTable wrapped, PrivilegeChecker privilegeChecker, Identifier identifier) { super(wrapped); this.privilegeChecker = privilegeChecker; @@ -106,18 +107,6 @@ public Optional statistics() { return wrapped.statistics(); } - @Override - public FileStoreTable copy(Map dynamicOptions) { - return new PrivilegedFileStoreTable( - wrapped.copy(dynamicOptions), privilegeChecker, identifier); - } - - @Override - public FileStoreTable copy(TableSchema newTableSchema) { - return new PrivilegedFileStoreTable( - wrapped.copy(newTableSchema), privilegeChecker, identifier); - } - @Override public void rollbackTo(long snapshotId) { privilegeChecker.assertCanInsert(identifier); @@ -202,18 +191,6 @@ public ExpireSnapshots newExpireChangelog() { return wrapped.newExpireChangelog(); } - @Override - public FileStoreTable copyWithoutTimeTravel(Map dynamicOptions) { - return new PrivilegedFileStoreTable( - wrapped.copyWithoutTimeTravel(dynamicOptions), privilegeChecker, identifier); - } - - @Override - public FileStoreTable copyWithLatestSchema() { - return new PrivilegedFileStoreTable( - wrapped.copyWithLatestSchema(), privilegeChecker, identifier); - } - @Override public DataTableScan newScan() { privilegeChecker.assertCanSelect(identifier); @@ -262,11 +239,7 @@ public LocalTableQuery newLocalTableQuery() { return wrapped.newLocalTableQuery(); } - @Override - public FileStoreTable switchToBranch(String branchName) { - return new PrivilegedFileStoreTable( - wrapped.switchToBranch(branchName), privilegeChecker, identifier); - } + // ======================= equals ============================ @Override public boolean equals(Object o) { @@ -281,4 +254,45 @@ public boolean equals(Object o) { && Objects.equals(privilegeChecker, that.privilegeChecker) && Objects.equals(identifier, that.identifier); } + + // ======================= copy ============================ + + @Override + public PrivilegedFileStoreTable copy(Map dynamicOptions) { + return new PrivilegedFileStoreTable( + wrapped.copy(dynamicOptions), privilegeChecker, identifier); + } + + @Override + public PrivilegedFileStoreTable copy(TableSchema newTableSchema) { + return new PrivilegedFileStoreTable( + wrapped.copy(newTableSchema), privilegeChecker, identifier); + } + + @Override + public PrivilegedFileStoreTable copyWithoutTimeTravel(Map dynamicOptions) { + return new PrivilegedFileStoreTable( + wrapped.copyWithoutTimeTravel(dynamicOptions), privilegeChecker, identifier); + } + + @Override + public PrivilegedFileStoreTable copyWithLatestSchema() { + return new PrivilegedFileStoreTable( + wrapped.copyWithLatestSchema(), privilegeChecker, identifier); + } + + @Override + public PrivilegedFileStoreTable switchToBranch(String branchName) { + return new PrivilegedFileStoreTable( + wrapped.switchToBranch(branchName), privilegeChecker, identifier); + } + + public static PrivilegedFileStoreTable wrap( + FileStoreTable table, PrivilegeChecker privilegeChecker, Identifier identifier) { + if (table instanceof ObjectTable) { + return new PrivilegedObjectTable((ObjectTable) table, privilegeChecker, identifier); + } else { + return new PrivilegedFileStoreTable(table, privilegeChecker, identifier); + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedObjectTable.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedObjectTable.java new file mode 100644 index 0000000000000..c5a319c1fedd4 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedObjectTable.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.privilege; + +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.object.ObjectTable; + +import java.util.Map; + +/** A {@link PrivilegedFileStoreTable} for {@link ObjectTable}. */ +public class PrivilegedObjectTable extends PrivilegedFileStoreTable implements ObjectTable { + + private final ObjectTable objectTable; + + protected PrivilegedObjectTable( + ObjectTable wrapped, PrivilegeChecker privilegeChecker, Identifier identifier) { + super(wrapped, privilegeChecker, identifier); + this.objectTable = wrapped; + } + + @Override + public String objectLocation() { + return objectTable.objectLocation(); + } + + @Override + public FileStoreTable underlyingTable() { + return objectTable.underlyingTable(); + } + + @Override + public FileIO objectFileIO() { + return objectTable.objectFileIO(); + } + + @Override + public long refresh() { + privilegeChecker.assertCanInsert(identifier); + return objectTable.refresh(); + } + + // ======================= copy ============================ + + @Override + public PrivilegedObjectTable copy(Map dynamicOptions) { + return new PrivilegedObjectTable( + objectTable.copy(dynamicOptions), privilegeChecker, identifier); + } + + @Override + public PrivilegedObjectTable copy(TableSchema newTableSchema) { + return new PrivilegedObjectTable( + objectTable.copy(newTableSchema), privilegeChecker, identifier); + } + + @Override + public PrivilegedObjectTable copyWithoutTimeTravel(Map dynamicOptions) { + return new PrivilegedObjectTable( + objectTable.copyWithoutTimeTravel(dynamicOptions), privilegeChecker, identifier); + } + + @Override + public PrivilegedObjectTable copyWithLatestSchema() { + return new PrivilegedObjectTable( + objectTable.copyWithLatestSchema(), privilegeChecker, identifier); + } + + @Override + public PrivilegedObjectTable switchToBranch(String branchName) { + return new PrivilegedObjectTable( + objectTable.switchToBranch(branchName), privilegeChecker, identifier); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java b/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java new file mode 100644 index 0000000000000..ce2cbb56ae248 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/DefaultErrorHandler.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +import org.apache.paimon.rest.exceptions.AlreadyExistsException; +import org.apache.paimon.rest.exceptions.BadRequestException; +import org.apache.paimon.rest.exceptions.ForbiddenException; +import org.apache.paimon.rest.exceptions.NoSuchResourceException; +import org.apache.paimon.rest.exceptions.NotAuthorizedException; +import org.apache.paimon.rest.exceptions.RESTException; +import org.apache.paimon.rest.exceptions.ServiceFailureException; +import org.apache.paimon.rest.exceptions.ServiceUnavailableException; +import org.apache.paimon.rest.responses.ErrorResponse; + +/** Default error handler. */ +public class DefaultErrorHandler extends ErrorHandler { + + private static final ErrorHandler INSTANCE = new DefaultErrorHandler(); + + public static ErrorHandler getInstance() { + return INSTANCE; + } + + @Override + public void accept(ErrorResponse error) { + int code = error.getCode(); + String message = error.getMessage(); + switch (code) { + case 400: + throw new BadRequestException(String.format("Malformed request: %s", message)); + case 401: + throw new NotAuthorizedException("Not authorized: %s", message); + case 403: + throw new ForbiddenException("Forbidden: %s", message); + case 404: + throw new NoSuchResourceException("%s", message); + case 405: + case 406: + break; + case 409: + throw new AlreadyExistsException("%s", message); + case 500: + throw new ServiceFailureException("Server error: %s", message); + case 501: + throw new UnsupportedOperationException(message); + case 503: + throw new ServiceUnavailableException("Service unavailable: %s", message); + default: + break; + } + + throw new RESTException("Unable to process: %s", message); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java b/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java new file mode 100644 index 0000000000000..cdfa4bcdfaac6 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ErrorHandler.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest; + +import org.apache.paimon.rest.responses.ErrorResponse; + +import java.util.function.Consumer; + +/** Error handler for REST client. */ +public abstract class ErrorHandler implements Consumer {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java new file mode 100644 index 0000000000000..87f3fad9b2fd0 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClient.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest; + +import org.apache.paimon.rest.exceptions.RESTException; +import org.apache.paimon.rest.responses.ErrorResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import okhttp3.Dispatcher; +import okhttp3.Headers; +import okhttp3.MediaType; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.RequestBody; +import okhttp3.Response; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; + +import static okhttp3.ConnectionSpec.CLEARTEXT; +import static okhttp3.ConnectionSpec.COMPATIBLE_TLS; +import static okhttp3.ConnectionSpec.MODERN_TLS; +import static org.apache.paimon.utils.ThreadPoolUtils.createCachedThreadPool; + +/** HTTP client for REST catalog. */ +public class HttpClient implements RESTClient { + + private final OkHttpClient okHttpClient; + private final String uri; + private final ObjectMapper mapper; + private final ErrorHandler errorHandler; + + private static final String THREAD_NAME = "REST-CATALOG-HTTP-CLIENT-THREAD-POOL"; + private static final MediaType MEDIA_TYPE = MediaType.parse("application/json"); + + public HttpClient(HttpClientOptions httpClientOptions) { + this.uri = httpClientOptions.uri(); + this.mapper = httpClientOptions.mapper(); + this.okHttpClient = createHttpClient(httpClientOptions); + this.errorHandler = httpClientOptions.errorHandler(); + } + + @Override + public T get( + String path, Class responseType, Map headers) { + Request request = + new Request.Builder().url(uri + path).get().headers(Headers.of(headers)).build(); + return exec(request, responseType); + } + + @Override + public T post( + String path, RESTRequest body, Class responseType, Map headers) { + try { + RequestBody requestBody = buildRequestBody(body); + Request request = + new Request.Builder() + .url(uri + path) + .post(requestBody) + .headers(Headers.of(headers)) + .build(); + return exec(request, responseType); + } catch (JsonProcessingException e) { + throw new RESTException(e, "build request failed."); + } + } + + @Override + public T delete(String path, Map headers) { + Request request = + new Request.Builder().url(uri + path).delete().headers(Headers.of(headers)).build(); + return exec(request, null); + } + + @Override + public void close() throws IOException { + okHttpClient.dispatcher().cancelAll(); + okHttpClient.connectionPool().evictAll(); + } + + private T exec(Request request, Class responseType) { + try (Response response = okHttpClient.newCall(request).execute()) { + String responseBodyStr = response.body() != null ? response.body().string() : null; + if (!response.isSuccessful()) { + ErrorResponse error = + new ErrorResponse( + responseBodyStr != null ? responseBodyStr : "response body is null", + response.code()); + errorHandler.accept(error); + } + if (responseType != null && responseBodyStr != null) { + return mapper.readValue(responseBodyStr, responseType); + } else if (responseType == null) { + return null; + } else { + throw new RESTException("response body is null."); + } + } catch (RESTException e) { + throw e; + } catch (Exception e) { + throw new RESTException(e, "rest exception"); + } + } + + private RequestBody buildRequestBody(RESTRequest body) throws JsonProcessingException { + return RequestBody.create(mapper.writeValueAsBytes(body), MEDIA_TYPE); + } + + private static OkHttpClient createHttpClient(HttpClientOptions httpClientOptions) { + BlockingQueue workQueue = new SynchronousQueue<>(); + ExecutorService executorService = + createCachedThreadPool(httpClientOptions.threadPoolSize(), THREAD_NAME, workQueue); + + OkHttpClient.Builder builder = + new OkHttpClient.Builder() + .dispatcher(new Dispatcher(executorService)) + .retryOnConnectionFailure(true) + .connectionSpecs(Arrays.asList(MODERN_TLS, COMPATIBLE_TLS, CLEARTEXT)); + httpClientOptions.connectTimeout().ifPresent(builder::connectTimeout); + httpClientOptions.readTimeout().ifPresent(builder::readTimeout); + + return builder.build(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.java new file mode 100644 index 0000000000000..694779cfdb86f --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/HttpClientOptions.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.rest; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import java.time.Duration; +import java.util.Optional; + +/** Options for Http Client. */ +public class HttpClientOptions { + + private final String uri; + private final Optional connectTimeout; + private final Optional readTimeout; + private final ObjectMapper mapper; + private final int threadPoolSize; + private final ErrorHandler errorHandler; + + public HttpClientOptions( + String uri, + Optional connectTimeout, + Optional readTimeout, + ObjectMapper mapper, + int threadPoolSize, + ErrorHandler errorHandler) { + this.uri = uri; + this.connectTimeout = connectTimeout; + this.readTimeout = readTimeout; + this.mapper = mapper; + this.threadPoolSize = threadPoolSize; + this.errorHandler = errorHandler; + } + + public String uri() { + return uri; + } + + public Optional connectTimeout() { + return connectTimeout; + } + + public Optional readTimeout() { + return readTimeout; + } + + public ObjectMapper mapper() { + return mapper; + } + + public int threadPoolSize() { + return threadPoolSize; + } + + public ErrorHandler errorHandler() { + return errorHandler; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java new file mode 100644 index 0000000000000..03b257efbf86e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalog.java @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Database; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.manifest.PartitionEntry; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; +import org.apache.paimon.rest.auth.AuthSession; +import org.apache.paimon.rest.auth.CredentialsProvider; +import org.apache.paimon.rest.auth.CredentialsProviderFactory; +import org.apache.paimon.rest.exceptions.AlreadyExistsException; +import org.apache.paimon.rest.exceptions.NoSuchResourceException; +import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.rest.responses.CreateDatabaseResponse; +import org.apache.paimon.rest.responses.DatabaseName; +import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.ListDatabasesResponse; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.table.Table; + +import org.apache.paimon.shade.guava30.com.google.common.annotations.VisibleForTesting; +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ScheduledExecutorService; +import java.util.stream.Collectors; + +import static org.apache.paimon.utils.ThreadPoolUtils.createScheduledThreadPool; + +/** A catalog implementation for REST. */ +public class RESTCatalog implements Catalog { + + private static final ObjectMapper OBJECT_MAPPER = RESTObjectMapper.create(); + + private final RESTClient client; + private final ResourcePaths resourcePaths; + private final Map options; + private final Map baseHeader; + private final AuthSession catalogAuth; + + private volatile ScheduledExecutorService refreshExecutor = null; + + public RESTCatalog(Options options) { + if (options.getOptional(CatalogOptions.WAREHOUSE).isPresent()) { + throw new IllegalArgumentException("Can not config warehouse in RESTCatalog."); + } + String uri = options.get(RESTCatalogOptions.URI); + Optional connectTimeout = + options.getOptional(RESTCatalogOptions.CONNECTION_TIMEOUT); + Optional readTimeout = options.getOptional(RESTCatalogOptions.READ_TIMEOUT); + Integer threadPoolSize = options.get(RESTCatalogOptions.THREAD_POOL_SIZE); + HttpClientOptions httpClientOptions = + new HttpClientOptions( + uri, + connectTimeout, + readTimeout, + OBJECT_MAPPER, + threadPoolSize, + DefaultErrorHandler.getInstance()); + this.client = new HttpClient(httpClientOptions); + this.baseHeader = configHeaders(options.toMap()); + CredentialsProvider credentialsProvider = + CredentialsProviderFactory.createCredentialsProvider( + options, RESTCatalog.class.getClassLoader()); + if (credentialsProvider.keepRefreshed()) { + this.catalogAuth = + AuthSession.fromRefreshCredentialsProvider( + tokenRefreshExecutor(), this.baseHeader, credentialsProvider); + + } else { + this.catalogAuth = new AuthSession(this.baseHeader, credentialsProvider); + } + Map initHeaders = + RESTUtil.merge(configHeaders(options.toMap()), this.catalogAuth.getHeaders()); + this.options = fetchOptionsFromServer(initHeaders, options.toMap()); + this.resourcePaths = + ResourcePaths.forCatalogProperties( + this.options.get(RESTCatalogInternalOptions.PREFIX)); + } + + @Override + public String warehouse() { + throw new UnsupportedOperationException(); + } + + @Override + public Map options() { + return this.options; + } + + @Override + public FileIO fileIO() { + throw new UnsupportedOperationException(); + } + + @Override + public List listDatabases() { + ListDatabasesResponse response = + client.get(resourcePaths.databases(), ListDatabasesResponse.class, headers()); + if (response.getDatabases() != null) { + return response.getDatabases().stream() + .map(DatabaseName::getName) + .collect(Collectors.toList()); + } + return ImmutableList.of(); + } + + @Override + public void createDatabase(String name, boolean ignoreIfExists, Map properties) + throws DatabaseAlreadyExistException { + CreateDatabaseRequest request = new CreateDatabaseRequest(name, ignoreIfExists, properties); + try { + client.post( + resourcePaths.databases(), request, CreateDatabaseResponse.class, headers()); + } catch (AlreadyExistsException e) { + throw new DatabaseAlreadyExistException(name); + } + } + + @Override + public Database getDatabase(String name) throws DatabaseNotExistException { + try { + GetDatabaseResponse response = + client.get(resourcePaths.database(name), GetDatabaseResponse.class, headers()); + return new Database.DatabaseImpl( + name, response.options(), response.comment().orElseGet(() -> null)); + } catch (NoSuchResourceException e) { + throw new DatabaseNotExistException(name); + } + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException { + try { + if (!cascade && !this.listTables(name).isEmpty()) { + throw new DatabaseNotEmptyException(name); + } + client.delete(resourcePaths.database(name), headers()); + } catch (NoSuchResourceException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(name); + } + } + } + + @Override + public Table getTable(Identifier identifier) throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public Path getTableLocation(Identifier identifier) { + throw new UnsupportedOperationException(); + } + + @Override + public List listTables(String databaseName) throws DatabaseNotExistException { + return new ArrayList(); + } + + @Override + public void dropTable(Identifier identifier, boolean ignoreIfNotExists) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void createTable(Identifier identifier, Schema schema, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void renameTable(Identifier fromTable, Identifier toTable, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTable( + Identifier identifier, List changes, boolean ignoreIfNotExists) + throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void createPartition(Identifier identifier, Map partitionSpec) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartition(Identifier identifier, Map partitions) + throws TableNotExistException, PartitionNotExistException {} + + @Override + public List listPartitions(Identifier identifier) + throws TableNotExistException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean allowUpperCase() { + return false; + } + + @Override + public void close() throws Exception { + if (refreshExecutor != null) { + refreshExecutor.shutdownNow(); + } + if (client != null) { + client.close(); + } + } + + @VisibleForTesting + Map fetchOptionsFromServer( + Map headers, Map clientProperties) { + ConfigResponse response = + client.get(ResourcePaths.V1_CONFIG, ConfigResponse.class, headers); + return response.merge(clientProperties); + } + + private static Map configHeaders(Map properties) { + return RESTUtil.extractPrefixMap(properties, "header."); + } + + private Map headers() { + return catalogAuth.getHeaders(); + } + + private ScheduledExecutorService tokenRefreshExecutor() { + if (refreshExecutor == null) { + synchronized (this) { + if (refreshExecutor == null) { + this.refreshExecutor = createScheduledThreadPool(1, "token-refresh-thread"); + } + } + } + + return refreshExecutor; + } +} diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/RowPosition.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java similarity index 61% rename from paimon-format/src/main/java/org/apache/paimon/format/parquet/position/RowPosition.java rename to paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java index fb6378349007d..a5c773cb4bd5e 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/RowPosition.java +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogFactory.java @@ -16,25 +16,23 @@ * limitations under the License. */ -package org.apache.paimon.format.parquet.position; +package org.apache.paimon.rest; -import javax.annotation.Nullable; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.CatalogFactory; -/** To represent struct's position in repeated type. */ -public class RowPosition { - @Nullable private final boolean[] isNull; - private final int positionsCount; +/** Factory to create {@link RESTCatalog}. */ +public class RESTCatalogFactory implements CatalogFactory { + public static final String IDENTIFIER = "rest"; - public RowPosition(boolean[] isNull, int positionsCount) { - this.isNull = isNull; - this.positionsCount = positionsCount; + @Override + public String identifier() { + return IDENTIFIER; } - public boolean[] getIsNull() { - return isNull; - } - - public int getPositionsCount() { - return positionsCount; + @Override + public Catalog create(CatalogContext context) { + return new RESTCatalog(context.options()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java new file mode 100644 index 0000000000000..722010923c469 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogInternalOptions.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +import org.apache.paimon.options.ConfigOption; +import org.apache.paimon.options.ConfigOptions; + +/** Internal options for REST Catalog. */ +public class RESTCatalogInternalOptions { + public static final ConfigOption PREFIX = + ConfigOptions.key("prefix") + .stringType() + .noDefaultValue() + .withDescription("REST Catalog uri's prefix."); + public static final ConfigOption CREDENTIALS_PROVIDER = + ConfigOptions.key("credentials-provider") + .stringType() + .noDefaultValue() + .withDescription("REST Catalog auth credentials provider."); + public static final ConfigOption DATABASE_COMMENT = + ConfigOptions.key("comment") + .stringType() + .defaultValue(null) + .withDescription("REST Catalog database comment."); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java new file mode 100644 index 0000000000000..1af64def4f714 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.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.rest; + +import org.apache.paimon.options.ConfigOption; +import org.apache.paimon.options.ConfigOptions; + +import java.time.Duration; + +/** Options for REST Catalog. */ +public class RESTCatalogOptions { + + public static final ConfigOption URI = + ConfigOptions.key("uri") + .stringType() + .noDefaultValue() + .withDescription("REST Catalog server's uri."); + + public static final ConfigOption CONNECTION_TIMEOUT = + ConfigOptions.key("rest.client.connection-timeout") + .durationType() + .noDefaultValue() + .withDescription("REST Catalog http client connect timeout."); + + public static final ConfigOption READ_TIMEOUT = + ConfigOptions.key("rest.client.read-timeout") + .durationType() + .noDefaultValue() + .withDescription("REST Catalog http client read timeout."); + + public static final ConfigOption THREAD_POOL_SIZE = + ConfigOptions.key("rest.client.num-threads") + .intType() + .defaultValue(1) + .withDescription("REST Catalog http client thread num."); + + public static final ConfigOption TOKEN = + ConfigOptions.key("token") + .stringType() + .noDefaultValue() + .withDescription("REST Catalog auth token."); + + public static final ConfigOption TOKEN_EXPIRATION_TIME = + ConfigOptions.key("token.expiration-time") + .durationType() + .defaultValue(Duration.ofHours(1)) + .withDescription( + "REST Catalog auth token expires time.The token generates system refresh frequency is t1," + + " the token expires time is t2, we need to guarantee that t2 > t1," + + " the token validity time is [t2 - t1, t2]," + + " and the expires time defined here needs to be less than (t2 - t1)"); + + public static final ConfigOption TOKEN_PROVIDER_PATH = + ConfigOptions.key("token.provider.path") + .stringType() + .noDefaultValue() + .withDescription("REST Catalog auth token provider path."); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.java new file mode 100644 index 0000000000000..a255d688bc52a --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTClient.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.rest; + +import java.io.Closeable; +import java.util.Map; + +/** Interface for a basic HTTP Client for interfacing with the REST catalog. */ +public interface RESTClient extends Closeable { + + T get(String path, Class responseType, Map headers); + + T post( + String path, RESTRequest body, Class responseType, Map headers); + + T delete(String path, Map headers); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java new file mode 100644 index 0000000000000..31d46df7ef0ff --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTMessage.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; + +/** Interface to mark both REST requests and responses. */ +@JsonIgnoreProperties(ignoreUnknown = true) +public interface RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.java new file mode 100644 index 0000000000000..b1c83e90224ad --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTObjectMapper.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.rest; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.DeserializationFeature; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.SerializationFeature; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; + +/** Object mapper for REST request and response. */ +public class RESTObjectMapper { + public static ObjectMapper create() { + ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + mapper.configure(SerializationFeature.FAIL_ON_EMPTY_BEANS, false); + mapper.registerModule(new JavaTimeModule()); + return mapper; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.java new file mode 100644 index 0000000000000..9c6758df14f04 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTRequest.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.rest; + +/** Interface to mark a REST request. */ +public interface RESTRequest extends RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.java new file mode 100644 index 0000000000000..a4149d3fda145 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTResponse.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.rest; + +/** Interface to mark a REST response. */ +public interface RESTResponse extends RESTMessage {} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java b/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java new file mode 100644 index 0000000000000..3d42e99fa6d59 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTUtil.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +import org.apache.paimon.utils.Preconditions; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; + +import java.util.Map; + +/** Util for REST. */ +public class RESTUtil { + public static Map extractPrefixMap( + Map properties, String prefix) { + Preconditions.checkNotNull(properties, "Invalid properties map: null"); + Map result = Maps.newHashMap(); + for (Map.Entry entry : properties.entrySet()) { + if (entry.getKey() != null && entry.getKey().startsWith(prefix)) { + result.put( + entry.getKey().substring(prefix.length()), properties.get(entry.getKey())); + } + } + return result; + } + + public static Map merge( + Map target, Map updates) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (Map.Entry entry : target.entrySet()) { + if (!updates.containsKey(entry.getKey())) { + builder.put(entry.getKey(), entry.getValue()); + } + } + updates.forEach(builder::put); + + return builder.build(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.java new file mode 100644 index 0000000000000..b58053374daac --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/ResourcePaths.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.rest; + +import java.util.StringJoiner; + +/** Resource paths for REST catalog. */ +public class ResourcePaths { + + public static final String V1_CONFIG = "/v1/config"; + private static final StringJoiner SLASH = new StringJoiner("/"); + + public static ResourcePaths forCatalogProperties(String prefix) { + return new ResourcePaths(prefix); + } + + private final String prefix; + + public ResourcePaths(String prefix) { + this.prefix = prefix; + } + + public String databases() { + return SLASH.add("v1").add(prefix).add("databases").toString(); + } + + public String database(String databaseName) { + return SLASH.add("v1").add(prefix).add("databases").add(databaseName).toString(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/auth/AuthSession.java b/paimon-core/src/main/java/org/apache/paimon/rest/auth/AuthSession.java new file mode 100644 index 0000000000000..3ca7590e5f962 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/AuthSession.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.auth; + +import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.rest.RESTUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** Auth session. */ +public class AuthSession { + + static final int TOKEN_REFRESH_NUM_RETRIES = 5; + static final long MIN_REFRESH_WAIT_MILLIS = 10; + static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes + + private static final Logger log = LoggerFactory.getLogger(AuthSession.class); + private final CredentialsProvider credentialsProvider; + private volatile Map headers; + + public AuthSession(Map headers, CredentialsProvider credentialsProvider) { + this.headers = headers; + this.credentialsProvider = credentialsProvider; + } + + public static AuthSession fromRefreshCredentialsProvider( + ScheduledExecutorService executor, + Map headers, + CredentialsProvider credentialsProvider) { + AuthSession session = new AuthSession(headers, credentialsProvider); + + long startTimeMillis = System.currentTimeMillis(); + Optional expiresAtMillisOpt = credentialsProvider.expiresAtMillis(); + + // when init session if credentials expire time is in the past, refresh it and update + // expiresAtMillis + if (expiresAtMillisOpt.isPresent() && expiresAtMillisOpt.get() <= startTimeMillis) { + boolean refreshSuccessful = session.refresh(); + if (refreshSuccessful) { + expiresAtMillisOpt = session.credentialsProvider.expiresAtMillis(); + } + } + + if (null != executor && expiresAtMillisOpt.isPresent()) { + scheduleTokenRefresh(executor, session, expiresAtMillisOpt.get()); + } + + return session; + } + + public Map getHeaders() { + if (this.credentialsProvider.keepRefreshed() && this.credentialsProvider.willSoonExpire()) { + refresh(); + } + return headers; + } + + public Boolean refresh() { + if (this.credentialsProvider.supportRefresh() + && this.credentialsProvider.keepRefreshed() + && this.credentialsProvider.expiresInMills().isPresent()) { + boolean isSuccessful = this.credentialsProvider.refresh(); + if (isSuccessful) { + Map currentHeaders = this.headers; + this.headers = + RESTUtil.merge(currentHeaders, this.credentialsProvider.authHeader()); + } + return isSuccessful; + } + + return false; + } + + @VisibleForTesting + static void scheduleTokenRefresh( + ScheduledExecutorService executor, AuthSession session, long expiresAtMillis) { + scheduleTokenRefresh(executor, session, expiresAtMillis, 0); + } + + @VisibleForTesting + static long getTimeToWaitByExpiresInMills(long expiresInMillis) { + // how much ahead of time to start the refresh to allow it to complete + long refreshWindowMillis = Math.min(expiresInMillis, MAX_REFRESH_WINDOW_MILLIS); + // how much time to wait before expiration + long waitIntervalMillis = expiresInMillis - refreshWindowMillis; + // how much time to actually wait + return Math.max(waitIntervalMillis, MIN_REFRESH_WAIT_MILLIS); + } + + private static void scheduleTokenRefresh( + ScheduledExecutorService executor, + AuthSession session, + long expiresAtMillis, + int retryTimes) { + if (retryTimes < TOKEN_REFRESH_NUM_RETRIES) { + long expiresInMillis = expiresAtMillis - System.currentTimeMillis(); + long timeToWait = getTimeToWaitByExpiresInMills(expiresInMillis); + + executor.schedule( + () -> { + long refreshStartTime = System.currentTimeMillis(); + boolean isSuccessful = session.refresh(); + if (isSuccessful) { + scheduleTokenRefresh( + executor, + session, + refreshStartTime + + session.credentialsProvider.expiresInMills().get(), + 0); + } else { + scheduleTokenRefresh( + executor, session, expiresAtMillis, retryTimes + 1); + } + }, + timeToWait, + TimeUnit.MILLISECONDS); + } else { + log.warn("Failed to refresh token after {} retries.", TOKEN_REFRESH_NUM_RETRIES); + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/auth/BaseBearTokenCredentialsProvider.java b/paimon-core/src/main/java/org/apache/paimon/rest/auth/BaseBearTokenCredentialsProvider.java new file mode 100644 index 0000000000000..d3df878261645 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/BaseBearTokenCredentialsProvider.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.auth; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; + +import java.util.Map; + +/** Base bear token credentials provider. */ +public abstract class BaseBearTokenCredentialsProvider implements CredentialsProvider { + + private static final String AUTHORIZATION_HEADER = "Authorization"; + private static final String BEARER_PREFIX = "Bearer "; + + @Override + public Map authHeader() { + return ImmutableMap.of(AUTHORIZATION_HEADER, BEARER_PREFIX + token()); + } + + abstract String token(); +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenCredentialsProvider.java b/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenCredentialsProvider.java new file mode 100644 index 0000000000000..89228fe10b287 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenCredentialsProvider.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.rest.auth; + +/** credentials provider for bear token. */ +public class BearTokenCredentialsProvider extends BaseBearTokenCredentialsProvider { + + private final String token; + + public BearTokenCredentialsProvider(String token) { + this.token = token; + } + + @Override + String token() { + return this.token; + } + + @Override + public boolean refresh() { + return true; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenCredentialsProviderFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenCredentialsProviderFactory.java new file mode 100644 index 0000000000000..e63ac5606b01c --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenCredentialsProviderFactory.java @@ -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.rest.auth; + +import org.apache.paimon.options.Options; +import org.apache.paimon.rest.RESTCatalogOptions; +import org.apache.paimon.utils.StringUtils; + +/** factory for create {@link BearTokenCredentialsProvider}. */ +public class BearTokenCredentialsProviderFactory implements CredentialsProviderFactory { + + @Override + public String identifier() { + return CredentialsProviderType.BEAR_TOKEN.name(); + } + + @Override + public CredentialsProvider create(Options options) { + if (options.getOptional(RESTCatalogOptions.TOKEN) + .map(StringUtils::isNullOrWhitespaceOnly) + .orElse(true)) { + throw new IllegalArgumentException( + RESTCatalogOptions.TOKEN.key() + " is required and not empty"); + } + return new BearTokenCredentialsProvider(options.get(RESTCatalogOptions.TOKEN)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenFileCredentialsProvider.java b/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenFileCredentialsProvider.java new file mode 100644 index 0000000000000..d479caa67fd0e --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenFileCredentialsProvider.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.rest.auth; + +import org.apache.paimon.utils.FileIOUtils; +import org.apache.paimon.utils.StringUtils; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Optional; + +/** credentials provider for get bear token from file. */ +public class BearTokenFileCredentialsProvider extends BaseBearTokenCredentialsProvider { + + public static final double EXPIRED_FACTOR = 0.4; + + private final String tokenFilePath; + private String token; + private boolean keepRefreshed = false; + private Long expiresAtMillis = null; + private Long expiresInMills = null; + + public BearTokenFileCredentialsProvider(String tokenFilePath) { + this.tokenFilePath = tokenFilePath; + this.token = getTokenFromFile(); + } + + public BearTokenFileCredentialsProvider(String tokenFilePath, Long expiresInMills) { + this(tokenFilePath); + this.keepRefreshed = true; + this.expiresAtMillis = -1L; + this.expiresInMills = expiresInMills; + } + + @Override + String token() { + return this.token; + } + + @Override + public boolean refresh() { + long start = System.currentTimeMillis(); + String newToken = getTokenFromFile(); + if (StringUtils.isNullOrWhitespaceOnly(newToken)) { + return false; + } + this.expiresAtMillis = start + this.expiresInMills; + this.token = newToken; + return true; + } + + @Override + public boolean supportRefresh() { + return true; + } + + @Override + public boolean keepRefreshed() { + return this.keepRefreshed; + } + + @Override + public boolean willSoonExpire() { + if (keepRefreshed()) { + return expiresAtMillis().get() - System.currentTimeMillis() + < expiresInMills().get() * EXPIRED_FACTOR; + } else { + return false; + } + } + + @Override + public Optional expiresAtMillis() { + return Optional.ofNullable(this.expiresAtMillis); + } + + @Override + public Optional expiresInMills() { + return Optional.ofNullable(this.expiresInMills); + } + + private String getTokenFromFile() { + try { + return FileIOUtils.readFileUtf8(new File(tokenFilePath)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenFileCredentialsProviderFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenFileCredentialsProviderFactory.java new file mode 100644 index 0000000000000..a0fa6b405d622 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/BearTokenFileCredentialsProviderFactory.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest.auth; + +import org.apache.paimon.options.Options; + +import static org.apache.paimon.rest.RESTCatalogOptions.TOKEN_EXPIRATION_TIME; +import static org.apache.paimon.rest.RESTCatalogOptions.TOKEN_PROVIDER_PATH; + +/** factory for create {@link BearTokenCredentialsProvider}. */ +public class BearTokenFileCredentialsProviderFactory implements CredentialsProviderFactory { + + @Override + public String identifier() { + return CredentialsProviderType.BEAR_TOKEN_FILE.name(); + } + + @Override + public CredentialsProvider create(Options options) { + if (!options.getOptional(TOKEN_PROVIDER_PATH).isPresent()) { + throw new IllegalArgumentException(TOKEN_PROVIDER_PATH.key() + " is required"); + } + String tokenFilePath = options.get(TOKEN_PROVIDER_PATH); + if (options.getOptional(TOKEN_EXPIRATION_TIME).isPresent()) { + long tokenExpireInMills = options.get(TOKEN_EXPIRATION_TIME).toMillis(); + return new BearTokenFileCredentialsProvider(tokenFilePath, tokenExpireInMills); + + } else { + return new BearTokenFileCredentialsProvider(tokenFilePath); + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/auth/CredentialsProvider.java b/paimon-core/src/main/java/org/apache/paimon/rest/auth/CredentialsProvider.java new file mode 100644 index 0000000000000..7fe8008e59475 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/CredentialsProvider.java @@ -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.rest.auth; + +import java.util.Map; +import java.util.Optional; + +/** Credentials provider. */ +public interface CredentialsProvider { + + Map authHeader(); + + boolean refresh(); + + default boolean supportRefresh() { + return false; + } + + default boolean keepRefreshed() { + return false; + } + + default boolean willSoonExpire() { + return false; + } + + default Optional expiresAtMillis() { + return Optional.empty(); + } + + default Optional expiresInMills() { + return Optional.empty(); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/auth/CredentialsProviderFactory.java b/paimon-core/src/main/java/org/apache/paimon/rest/auth/CredentialsProviderFactory.java new file mode 100644 index 0000000000000..50c3564ad8c68 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/CredentialsProviderFactory.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.rest.auth; + +import org.apache.paimon.factories.Factory; +import org.apache.paimon.factories.FactoryUtil; +import org.apache.paimon.options.Options; +import org.apache.paimon.rest.RESTCatalogOptions; + +import static org.apache.paimon.rest.RESTCatalogInternalOptions.CREDENTIALS_PROVIDER; + +/** Factory for creating {@link CredentialsProvider}. */ +public interface CredentialsProviderFactory extends Factory { + + default CredentialsProvider create(Options options) { + throw new UnsupportedOperationException( + "Use create(context) for " + this.getClass().getSimpleName()); + } + + static CredentialsProvider createCredentialsProvider(Options options, ClassLoader classLoader) { + String credentialsProviderIdentifier = getCredentialsProviderTypeByConf(options).name(); + CredentialsProviderFactory credentialsProviderFactory = + FactoryUtil.discoverFactory( + classLoader, + CredentialsProviderFactory.class, + credentialsProviderIdentifier); + return credentialsProviderFactory.create(options); + } + + static CredentialsProviderType getCredentialsProviderTypeByConf(Options options) { + if (options.getOptional(CREDENTIALS_PROVIDER).isPresent()) { + return CredentialsProviderType.valueOf(options.get(CREDENTIALS_PROVIDER)); + } else if (options.getOptional(RESTCatalogOptions.TOKEN_PROVIDER_PATH).isPresent()) { + return CredentialsProviderType.BEAR_TOKEN_FILE; + } + return CredentialsProviderType.BEAR_TOKEN; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/auth/CredentialsProviderType.java b/paimon-core/src/main/java/org/apache/paimon/rest/auth/CredentialsProviderType.java new file mode 100644 index 0000000000000..28c344d70eee2 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/CredentialsProviderType.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.auth; + +/** Credentials provider type. */ +public enum CredentialsProviderType { + BEAR_TOKEN, + BEAR_TOKEN_FILE +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java new file mode 100644 index 0000000000000..8e30c8375bf91 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/AlreadyExistsException.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest.exceptions; + +/** Exception thrown on HTTP 409 means a resource already exists. */ +public class AlreadyExistsException extends RESTException { + + public AlreadyExistsException(String message, Object... args) { + super(message, args); + } +} diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/spark/paimon/Utils.scala b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java similarity index 74% rename from paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/spark/paimon/Utils.scala rename to paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java index 1a899f5001536..301f3bd63f88d 100644 --- a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/spark/paimon/Utils.scala +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/BadRequestException.java @@ -16,17 +16,12 @@ * limitations under the License. */ -package org.apache.spark.paimon +package org.apache.paimon.rest.exceptions; -import org.apache.spark.util.{Utils => SparkUtils} - -import java.io.File - -/** - * A wrapper that some Objects or Classes is limited to access beyond [[org.apache.spark]] package. - */ -object Utils { - - def createTempDir: File = SparkUtils.createTempDir() +/** Exception thrown on HTTP 400 - Bad Request. */ +public class BadRequestException extends RESTException { + public BadRequestException(String message, Object... args) { + super(message, args); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java new file mode 100644 index 0000000000000..3982e5b704177 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ForbiddenException.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest.exceptions; + +/** Exception thrown on HTTP 403 Forbidden. */ +public class ForbiddenException extends RESTException { + public ForbiddenException(String message, Object... args) { + super(message, args); + } +} diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/spark/paimon/Utils.scala b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java similarity index 74% rename from paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/spark/paimon/Utils.scala rename to paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java index 1a899f5001536..cc4c7881f465a 100644 --- a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/spark/paimon/Utils.scala +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NoSuchResourceException.java @@ -16,17 +16,12 @@ * limitations under the License. */ -package org.apache.spark.paimon +package org.apache.paimon.rest.exceptions; -import org.apache.spark.util.{Utils => SparkUtils} - -import java.io.File - -/** - * A wrapper that some Objects or Classes is limited to access beyond [[org.apache.spark]] package. - */ -object Utils { - - def createTempDir: File = SparkUtils.createTempDir() +/** Exception thrown on HTTP 404 means a resource not exists. */ +public class NoSuchResourceException extends RESTException { + public NoSuchResourceException(String message, Object... args) { + super(message, args); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java new file mode 100644 index 0000000000000..43c13b1a1c979 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/NotAuthorizedException.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest.exceptions; + +/** Exception thrown on HTTP 401 Unauthorized. */ +public class NotAuthorizedException extends RESTException { + public NotAuthorizedException(String message, Object... args) { + super(String.format(message, args)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java new file mode 100644 index 0000000000000..532936f43032d --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/RESTException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.exceptions; + +/** Base class for REST client exceptions. */ +public class RESTException extends RuntimeException { + public RESTException(String message, Object... args) { + super(String.format(message, args)); + } + + public RESTException(Throwable cause, String message, Object... args) { + super(String.format(message, args), cause); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java new file mode 100644 index 0000000000000..45c48ec0de094 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceFailureException.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest.exceptions; + +/** Exception thrown on HTTP 500 - Bad Request. */ +public class ServiceFailureException extends RESTException { + public ServiceFailureException(String message, Object... args) { + super(String.format(message, args)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java new file mode 100644 index 0000000000000..fb6a05e89f9f3 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/exceptions/ServiceUnavailableException.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest.exceptions; + +/** Exception thrown on HTTP 503 - service is unavailable. */ +public class ServiceUnavailableException extends RESTException { + public ServiceUnavailableException(String message, Object... args) { + super(String.format(message, args)); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java new file mode 100644 index 0000000000000..6067bf544b875 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/requests/CreateDatabaseRequest.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.requests; + +import org.apache.paimon.rest.RESTRequest; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Map; + +/** Request for creating database. */ +public class CreateDatabaseRequest implements RESTRequest { + + private static final String FIELD_NAME = "name"; + private static final String FIELD_IGNORE_IF_EXISTS = "ignoreIfExists"; + private static final String FIELD_OPTIONS = "options"; + + @JsonProperty(FIELD_NAME) + private String name; + + @JsonProperty(FIELD_IGNORE_IF_EXISTS) + private boolean ignoreIfExists; + + @JsonProperty(FIELD_OPTIONS) + private Map options; + + @JsonCreator + public CreateDatabaseRequest( + @JsonProperty(FIELD_NAME) String name, + @JsonProperty(FIELD_IGNORE_IF_EXISTS) boolean ignoreIfExists, + @JsonProperty(FIELD_OPTIONS) Map options) { + this.name = name; + this.ignoreIfExists = ignoreIfExists; + this.options = options; + } + + @JsonGetter(FIELD_NAME) + public String getName() { + return name; + } + + @JsonGetter(FIELD_IGNORE_IF_EXISTS) + public boolean getIgnoreIfExists() { + return ignoreIfExists; + } + + @JsonGetter(FIELD_OPTIONS) + public Map getOptions() { + return options; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.java new file mode 100644 index 0000000000000..e8fff88b09c2d --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ConfigResponse.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.rest.responses; + +import org.apache.paimon.rest.RESTResponse; +import org.apache.paimon.utils.Preconditions; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Map; +import java.util.Objects; + +/** Response for getting config. */ +public class ConfigResponse implements RESTResponse { + + private static final String FIELD_DEFAULTS = "defaults"; + private static final String FIELD_OVERRIDES = "overrides"; + + @JsonProperty(FIELD_DEFAULTS) + private Map defaults; + + @JsonProperty(FIELD_OVERRIDES) + private Map overrides; + + @JsonCreator + public ConfigResponse( + @JsonProperty(FIELD_DEFAULTS) Map defaults, + @JsonProperty(FIELD_OVERRIDES) Map overrides) { + this.defaults = defaults; + this.overrides = overrides; + } + + public Map merge(Map clientProperties) { + Preconditions.checkNotNull( + clientProperties, + "Cannot merge client properties with server-provided properties. Invalid client configuration: null"); + Map merged = + defaults != null ? Maps.newHashMap(defaults) : Maps.newHashMap(); + merged.putAll(clientProperties); + + if (overrides != null) { + merged.putAll(overrides); + } + + return ImmutableMap.copyOf(Maps.filterValues(merged, Objects::nonNull)); + } + + @JsonGetter(FIELD_DEFAULTS) + public Map getDefaults() { + return defaults; + } + + @JsonGetter(FIELD_OVERRIDES) + public Map getOverrides() { + return overrides; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java new file mode 100644 index 0000000000000..43c99254f3990 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/CreateDatabaseResponse.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.responses; + +import org.apache.paimon.rest.RESTResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Map; + +/** Response for creating database. */ +public class CreateDatabaseResponse implements RESTResponse { + + private static final String FIELD_NAME = "name"; + private static final String FIELD_OPTIONS = "options"; + + @JsonProperty(FIELD_NAME) + private String name; + + @JsonProperty(FIELD_OPTIONS) + private Map options; + + @JsonCreator + public CreateDatabaseResponse( + @JsonProperty(FIELD_NAME) String name, + @JsonProperty(FIELD_OPTIONS) Map options) { + this.name = name; + this.options = options; + } + + @JsonGetter(FIELD_NAME) + public String getName() { + return name; + } + + @JsonGetter(FIELD_OPTIONS) + public Map getOptions() { + return options; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java new file mode 100644 index 0000000000000..9a93b2fd1e3da --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/DatabaseName.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest.responses; + +import org.apache.paimon.rest.RESTMessage; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +/** Class for Database entity. */ +public class DatabaseName implements RESTMessage { + + private static final String FIELD_NAME = "name"; + + @JsonProperty(FIELD_NAME) + private String name; + + @JsonCreator + public DatabaseName(@JsonProperty(FIELD_NAME) String name) { + this.name = name; + } + + @JsonGetter(FIELD_NAME) + public String getName() { + return this.name; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java new file mode 100644 index 0000000000000..d24c8f0f9936b --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ErrorResponse.java @@ -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.rest.responses; + +import org.apache.paimon.rest.RESTResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** Response for error. */ +public class ErrorResponse implements RESTResponse { + + private static final String FIELD_MESSAGE = "message"; + private static final String FIELD_CODE = "code"; + private static final String FIELD_STACK = "stack"; + + @JsonProperty(FIELD_MESSAGE) + private final String message; + + @JsonProperty(FIELD_CODE) + private final Integer code; + + @JsonProperty(FIELD_STACK) + private final List stack; + + public ErrorResponse(String message, Integer code) { + this.code = code; + this.message = message; + this.stack = new ArrayList(); + } + + @JsonCreator + public ErrorResponse( + @JsonProperty(FIELD_MESSAGE) String message, + @JsonProperty(FIELD_CODE) int code, + @JsonProperty(FIELD_STACK) List stack) { + this.message = message; + this.code = code; + this.stack = stack; + } + + public ErrorResponse(String message, int code, Throwable throwable) { + this.message = message; + this.code = code; + this.stack = getStackFromThrowable(throwable); + } + + @JsonGetter(FIELD_MESSAGE) + public String getMessage() { + return message; + } + + @JsonGetter(FIELD_CODE) + public Integer getCode() { + return code; + } + + @JsonGetter(FIELD_STACK) + public List getStack() { + return stack; + } + + private List getStackFromThrowable(Throwable throwable) { + if (throwable == null) { + return new ArrayList(); + } + StringWriter sw = new StringWriter(); + try (PrintWriter pw = new PrintWriter(sw)) { + throwable.printStackTrace(pw); + } + + return Arrays.asList(sw.toString().split("\n")); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java new file mode 100644 index 0000000000000..f8f7c8794b7b4 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/GetDatabaseResponse.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest.responses; + +import org.apache.paimon.catalog.Database; +import org.apache.paimon.rest.RESTResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Map; +import java.util.Optional; + +import static org.apache.paimon.rest.RESTCatalogInternalOptions.DATABASE_COMMENT; + +/** Response for getting database. */ +public class GetDatabaseResponse implements RESTResponse, Database { + + private static final String FIELD_NAME = "name"; + private static final String FIELD_OPTIONS = "options"; + + @JsonProperty(FIELD_NAME) + private final String name; + + @JsonProperty(FIELD_OPTIONS) + private final Map options; + + @JsonCreator + public GetDatabaseResponse( + @JsonProperty(FIELD_NAME) String name, + @JsonProperty(FIELD_OPTIONS) Map options) { + this.name = name; + this.options = options; + } + + @JsonGetter(FIELD_NAME) + public String getName() { + return name; + } + + @JsonGetter(FIELD_OPTIONS) + public Map getOptions() { + return options; + } + + @Override + public String name() { + return this.getName(); + } + + @Override + public Map options() { + return this.getOptions(); + } + + @Override + public Optional comment() { + return Optional.ofNullable( + this.options.getOrDefault(DATABASE_COMMENT.key(), DATABASE_COMMENT.defaultValue())); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java new file mode 100644 index 0000000000000..38773f354b771 --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/rest/responses/ListDatabasesResponse.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest.responses; + +import org.apache.paimon.rest.RESTResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** Response for listing databases. */ +public class ListDatabasesResponse implements RESTResponse { + private static final String FIELD_DATABASES = "databases"; + + @JsonProperty(FIELD_DATABASES) + private List databases; + + @JsonCreator + public ListDatabasesResponse(@JsonProperty(FIELD_DATABASES) List databases) { + this.databases = databases; + } + + @JsonGetter(FIELD_DATABASES) + public List getDatabases() { + return this.databases; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java index 9984e3feef0c8..33309a7cecc91 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/Schema.java @@ -341,13 +341,4 @@ public Schema build() { return new Schema(columns, partitionKeys, primaryKeys, options, comment); } } - - public static Schema fromTableSchema(TableSchema tableSchema) { - return new Schema( - tableSchema.fields(), - tableSchema.partitionKeys(), - tableSchema.primaryKeys(), - tableSchema.options(), - tableSchema.comment()); - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java index 1c1d601bced8a..cefa3c6eb9e74 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaChange.java @@ -25,8 +25,6 @@ import java.io.Serializable; import java.util.Arrays; -import java.util.Collections; -import java.util.List; import java.util.Objects; /** @@ -54,46 +52,45 @@ static SchemaChange addColumn(String fieldName, DataType dataType) { } static SchemaChange addColumn(String fieldName, DataType dataType, String comment) { - return new AddColumn(Collections.singletonList(fieldName), dataType, comment, null); + return new AddColumn(new String[] {fieldName}, dataType, comment, null); } static SchemaChange addColumn(String fieldName, DataType dataType, String comment, Move move) { - return new AddColumn(Collections.singletonList(fieldName), dataType, comment, move); + return new AddColumn(new String[] {fieldName}, dataType, comment, move); } static SchemaChange addColumn( - List fieldNames, DataType dataType, String comment, Move move) { + String[] fieldNames, DataType dataType, String comment, Move move) { return new AddColumn(fieldNames, dataType, comment, move); } static SchemaChange renameColumn(String fieldName, String newName) { - return new RenameColumn(Collections.singletonList(fieldName), newName); + return new RenameColumn(new String[] {fieldName}, newName); } - static SchemaChange renameColumn(List fieldNames, String newName) { + static SchemaChange renameColumn(String[] fieldNames, String newName) { return new RenameColumn(fieldNames, newName); } static SchemaChange dropColumn(String fieldName) { - return new DropColumn(Collections.singletonList(fieldName)); + return new DropColumn(new String[] {fieldName}); } - static SchemaChange dropColumn(List fieldNames) { + static SchemaChange dropColumn(String[] fieldNames) { return new DropColumn(fieldNames); } static SchemaChange updateColumnType(String fieldName, DataType newDataType) { - return new UpdateColumnType(Collections.singletonList(fieldName), newDataType, false); + return new UpdateColumnType(new String[] {fieldName}, newDataType, false); } static SchemaChange updateColumnType( String fieldName, DataType newDataType, boolean keepNullability) { - return new UpdateColumnType( - Collections.singletonList(fieldName), newDataType, keepNullability); + return new UpdateColumnType(new String[] {fieldName}, newDataType, keepNullability); } static SchemaChange updateColumnType( - List fieldNames, DataType newDataType, boolean keepNullability) { + String[] fieldNames, DataType newDataType, boolean keepNullability) { return new UpdateColumnType(fieldNames, newDataType, keepNullability); } @@ -228,20 +225,19 @@ final class AddColumn implements SchemaChange { private static final long serialVersionUID = 1L; - private final List fieldNames; + private final String[] fieldNames; private final DataType dataType; private final String description; private final Move move; - private AddColumn( - List fieldNames, DataType dataType, String description, Move move) { + private AddColumn(String[] fieldNames, DataType dataType, String description, Move move) { this.fieldNames = fieldNames; this.dataType = dataType; this.description = description; this.move = move; } - public List fieldNames() { + public String[] fieldNames() { return fieldNames; } @@ -268,7 +264,7 @@ public boolean equals(Object o) { return false; } AddColumn addColumn = (AddColumn) o; - return Objects.equals(fieldNames, addColumn.fieldNames) + return Arrays.equals(fieldNames, addColumn.fieldNames) && dataType.equals(addColumn.dataType) && Objects.equals(description, addColumn.description) && move.equals(addColumn.move); @@ -288,15 +284,15 @@ final class RenameColumn implements SchemaChange { private static final long serialVersionUID = 1L; - private final List fieldNames; + private final String[] fieldNames; private final String newName; - private RenameColumn(List fieldNames, String newName) { + private RenameColumn(String[] fieldNames, String newName) { this.fieldNames = fieldNames; this.newName = newName; } - public List fieldNames() { + public String[] fieldNames() { return fieldNames; } @@ -313,7 +309,7 @@ public boolean equals(Object o) { return false; } RenameColumn that = (RenameColumn) o; - return Objects.equals(fieldNames, that.fieldNames) + return Arrays.equals(fieldNames, that.fieldNames) && Objects.equals(newName, that.newName); } @@ -330,13 +326,13 @@ final class DropColumn implements SchemaChange { private static final long serialVersionUID = 1L; - private final List fieldNames; + private final String[] fieldNames; - private DropColumn(List fieldNames) { + private DropColumn(String[] fieldNames) { this.fieldNames = fieldNames; } - public List fieldNames() { + public String[] fieldNames() { return fieldNames; } @@ -349,7 +345,7 @@ public boolean equals(Object o) { return false; } DropColumn that = (DropColumn) o; - return Objects.equals(fieldNames, that.fieldNames); + return Arrays.equals(fieldNames, that.fieldNames); } @Override @@ -363,19 +359,19 @@ final class UpdateColumnType implements SchemaChange { private static final long serialVersionUID = 1L; - private final List fieldNames; + private final String[] fieldNames; private final DataType newDataType; // If true, do not change the target field nullability private final boolean keepNullability; private UpdateColumnType( - List fieldNames, DataType newDataType, boolean keepNullability) { + String[] fieldNames, DataType newDataType, boolean keepNullability) { this.fieldNames = fieldNames; this.newDataType = newDataType; this.keepNullability = keepNullability; } - public List fieldNames() { + public String[] fieldNames() { return fieldNames; } @@ -396,7 +392,7 @@ public boolean equals(Object o) { return false; } UpdateColumnType that = (UpdateColumnType) o; - return Objects.equals(fieldNames, that.fieldNames) + return Arrays.equals(fieldNames, that.fieldNames) && newDataType.equals(that.newDataType); } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java index b5d7307073595..0ae2798c29e00 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaEvolutionUtil.java @@ -19,10 +19,15 @@ package org.apache.paimon.schema; import org.apache.paimon.KeyValue; +import org.apache.paimon.casting.CastElementGetter; import org.apache.paimon.casting.CastExecutor; import org.apache.paimon.casting.CastExecutors; import org.apache.paimon.casting.CastFieldGetter; +import org.apache.paimon.casting.CastedArray; +import org.apache.paimon.casting.CastedMap; import org.apache.paimon.casting.CastedRow; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; import org.apache.paimon.predicate.LeafPredicate; import org.apache.paimon.predicate.Predicate; @@ -31,7 +36,6 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.MapType; -import org.apache.paimon.types.MultisetType; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.InternalRowUtils; import org.apache.paimon.utils.ProjectedRow; @@ -372,6 +376,7 @@ private static CastFieldGetter[] createCastFieldGetterMapping( List tableFields, List dataFields, int[] indexMapping) { CastFieldGetter[] converterMapping = new CastFieldGetter[tableFields.size()]; boolean castExist = false; + for (int i = 0; i < tableFields.size(); i++) { int dataIndex = indexMapping == null ? i : indexMapping[i]; if (dataIndex < 0) { @@ -380,53 +385,39 @@ private static CastFieldGetter[] createCastFieldGetterMapping( } else { DataField tableField = tableFields.get(i); DataField dataField = dataFields.get(dataIndex); - if (dataField.type().equalsIgnoreNullable(tableField.type())) { - // Create getter with index i and projected row data will convert to underlying - // data - converterMapping[i] = - new CastFieldGetter( - InternalRowUtils.createNullCheckingFieldGetter( - dataField.type(), i), - CastExecutors.identityCastExecutor()); - } else { - // TODO support column type evolution in nested type - checkState( - !(tableField.type() instanceof MapType - || dataField.type() instanceof ArrayType - || dataField.type() instanceof MultisetType), - "Only support column type evolution in atomic and row data type."); - - CastExecutor castExecutor; - if (tableField.type() instanceof RowType - && dataField.type() instanceof RowType) { - castExecutor = - createRowCastExecutor( - (RowType) dataField.type(), (RowType) tableField.type()); - } else { - castExecutor = CastExecutors.resolve(dataField.type(), tableField.type()); - } - checkNotNull( - castExecutor, - "Cannot cast from type " - + dataField.type() - + " to type " - + tableField.type()); - - // Create getter with index i and projected row data will convert to underlying - // data - converterMapping[i] = - new CastFieldGetter( - InternalRowUtils.createNullCheckingFieldGetter( - dataField.type(), i), - castExecutor); + if (!dataField.type().equalsIgnoreNullable(tableField.type())) { castExist = true; } + + // Create getter with index i and projected row data will convert to underlying data + converterMapping[i] = + new CastFieldGetter( + InternalRowUtils.createNullCheckingFieldGetter(dataField.type(), i), + createCastExecutor(dataField.type(), tableField.type())); } } return castExist ? converterMapping : null; } + private static CastExecutor createCastExecutor(DataType inputType, DataType targetType) { + if (targetType.equalsIgnoreNullable(inputType)) { + return CastExecutors.identityCastExecutor(); + } else if (inputType instanceof RowType && targetType instanceof RowType) { + return createRowCastExecutor((RowType) inputType, (RowType) targetType); + } else if (inputType instanceof ArrayType && targetType instanceof ArrayType) { + return createArrayCastExecutor((ArrayType) inputType, (ArrayType) targetType); + } else if (inputType instanceof MapType && targetType instanceof MapType) { + return createMapCastExecutor((MapType) inputType, (MapType) targetType); + } else { + return checkNotNull( + CastExecutors.resolve(inputType, targetType), + "Cannot cast from type %s to type %s", + inputType, + targetType); + } + } + private static CastExecutor createRowCastExecutor( RowType inputType, RowType targetType) { int[] indexMapping = createIndexMapping(targetType.getFields(), inputType.getFields()); @@ -446,4 +437,32 @@ private static CastExecutor createRowCastExecutor( return value; }; } + + private static CastExecutor createArrayCastExecutor( + ArrayType inputType, ArrayType targetType) { + CastElementGetter castElementGetter = + new CastElementGetter( + InternalArray.createElementGetter(inputType.getElementType()), + createCastExecutor( + inputType.getElementType(), targetType.getElementType())); + + CastedArray castedArray = CastedArray.from(castElementGetter); + return castedArray::replaceArray; + } + + private static CastExecutor createMapCastExecutor( + MapType inputType, MapType targetType) { + checkState( + inputType.getKeyType().equals(targetType.getKeyType()), + "Cannot cast map type %s to map type %s, because they have different key types.", + inputType.getKeyType(), + targetType.getKeyType()); + CastElementGetter castElementGetter = + new CastElementGetter( + InternalArray.createElementGetter(inputType.getValueType()), + createCastExecutor(inputType.getValueType(), targetType.getValueType())); + + CastedMap castedMap = CastedMap.from(castElementGetter); + return castedMap::replaceMap; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java index 86ed96d5b01ba..2139dca4a9907 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java @@ -37,13 +37,14 @@ import org.apache.paimon.schema.SchemaChange.UpdateColumnType; import org.apache.paimon.schema.SchemaChange.UpdateComment; import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeCasts; +import org.apache.paimon.types.MapType; import org.apache.paimon.types.ReassignFieldId; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.BranchManager; -import org.apache.paimon.utils.JsonSerdeUtil; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.StringUtils; @@ -78,6 +79,7 @@ import static org.apache.paimon.catalog.Identifier.UNKNOWN_DATABASE; import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH; import static org.apache.paimon.utils.FileUtils.listVersionedFiles; +import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.paimon.utils.Preconditions.checkState; /** Schema Manager to manage schema versions. */ @@ -123,6 +125,24 @@ public Optional latest() { } } + public long earliestCreationTime() { + try { + long earliest = 0; + if (!schemaExists(0)) { + Optional min = + listVersionedFiles(fileIO, schemaDirectory(), SCHEMA_PREFIX) + .reduce(Math::min); + checkArgument(min.isPresent()); + earliest = min.get(); + } + + Path schemaPath = toSchemaPath(earliest); + return fileIO.getFileStatus(schemaPath).getModificationTime(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + public List listAll() { return listAllIds().stream().map(this::schema).collect(Collectors.toList()); } @@ -188,24 +208,18 @@ public TableSchema createTable(Schema schema) throws Exception { return createTable(schema, false); } - public TableSchema createTable(Schema schema, boolean ignoreIfExistsSame) throws Exception { + public TableSchema createTable(Schema schema, boolean externalTable) throws Exception { while (true) { Optional latest = latest(); if (latest.isPresent()) { - TableSchema oldSchema = latest.get(); - boolean isSame = - Objects.equals(oldSchema.fields(), schema.fields()) - && Objects.equals(oldSchema.partitionKeys(), schema.partitionKeys()) - && Objects.equals(oldSchema.primaryKeys(), schema.primaryKeys()) - && Objects.equals(oldSchema.options(), schema.options()); - if (ignoreIfExistsSame && isSame) { - return oldSchema; + TableSchema latestSchema = latest.get(); + if (externalTable) { + checkSchemaForExternalTable(latestSchema, schema); + return latestSchema; + } else { + throw new IllegalStateException( + "Schema in filesystem exists, creation is not allowed."); } - - throw new IllegalStateException( - "Schema in filesystem exists, please use updating," - + " latest schema is: " - + oldSchema); } List fields = schema.fields(); @@ -234,6 +248,38 @@ public TableSchema createTable(Schema schema, boolean ignoreIfExistsSame) throws } } + private void checkSchemaForExternalTable(TableSchema existsSchema, Schema newSchema) { + // When creating an external table, if the table already exists in the location, we can + // choose not to specify the fields. + if (newSchema.fields().isEmpty() + // When the fields are explicitly specified, we need check for consistency. + || (Objects.equals(existsSchema.fields(), newSchema.fields()) + && Objects.equals(existsSchema.partitionKeys(), newSchema.partitionKeys()) + && Objects.equals(existsSchema.primaryKeys(), newSchema.primaryKeys()))) { + // check for options + Map existsOptions = existsSchema.options(); + Map newOptions = newSchema.options(); + newOptions.forEach( + (key, value) -> { + if (!key.equals(Catalog.OWNER_PROP) + && (!existsOptions.containsKey(key) + || !existsOptions.get(key).equals(value))) { + throw new RuntimeException( + "New schema's options are not equal to the exists schema's, new schema: " + + newOptions + + ", exists schema: " + + existsOptions); + } + }); + } else { + throw new RuntimeException( + "New schema is not equal to exists schema, new schema: " + + newSchema + + ", exists schema: " + + existsSchema); + } + } + /** Update {@link SchemaChange}s. */ public TableSchema commitChanges(SchemaChange... changes) throws Exception { return commitChanges(Arrays.asList(changes)); @@ -290,7 +336,7 @@ public TableSchema commitChanges(List changes) DataType dataType = ReassignFieldId.reassign(addColumn.dataType(), highestFieldId); - new NestedColumnModifier(addColumn.fieldNames().toArray(new String[0])) { + new NestedColumnModifier(addColumn.fieldNames()) { @Override protected void updateLastColumn(List newFields, String fieldName) throws Catalog.ColumnAlreadyExistException { @@ -320,7 +366,7 @@ protected void updateLastColumn(List newFields, String fieldName) } else if (change instanceof RenameColumn) { RenameColumn rename = (RenameColumn) change; assertNotUpdatingPrimaryKeys(oldTableSchema, rename.fieldNames(), "rename"); - new NestedColumnModifier(rename.fieldNames().toArray(new String[0])) { + new NestedColumnModifier(rename.fieldNames()) { @Override protected void updateLastColumn(List newFields, String fieldName) throws Catalog.ColumnNotExistException, @@ -347,7 +393,7 @@ protected void updateLastColumn(List newFields, String fieldName) } else if (change instanceof DropColumn) { DropColumn drop = (DropColumn) change; dropColumnValidation(oldTableSchema, drop); - new NestedColumnModifier(drop.fieldNames().toArray(new String[0])) { + new NestedColumnModifier(drop.fieldNames()) { @Override protected void updateLastColumn(List newFields, String fieldName) throws Catalog.ColumnNotExistException { @@ -364,7 +410,7 @@ protected void updateLastColumn(List newFields, String fieldName) assertNotUpdatingPrimaryKeys(oldTableSchema, update.fieldNames(), "update"); updateNestedColumn( newFields, - update.fieldNames().toArray(new String[0]), + update.fieldNames(), (field) -> { DataType targetType = update.newDataType(); if (update.keepNullability()) { @@ -558,8 +604,8 @@ private static List applyNotNestedColumnRename( Map columnNames = Maps.newHashMap(); for (RenameColumn renameColumn : renames) { - if (renameColumn.fieldNames().size() == 1) { - columnNames.put(renameColumn.fieldNames().get(0), renameColumn.newName()); + if (renameColumn.fieldNames().length == 1) { + columnNames.put(renameColumn.fieldNames()[0], renameColumn.newName()); } } @@ -571,10 +617,10 @@ private static List applyNotNestedColumnRename( private static void dropColumnValidation(TableSchema schema, DropColumn change) { // primary keys and partition keys can't be nested columns - if (change.fieldNames().size() > 1) { + if (change.fieldNames().length > 1) { return; } - String columnToDrop = change.fieldNames().get(0); + String columnToDrop = change.fieldNames()[0]; if (schema.partitionKeys().contains(columnToDrop) || schema.primaryKeys().contains(columnToDrop)) { throw new UnsupportedOperationException( @@ -583,12 +629,12 @@ private static void dropColumnValidation(TableSchema schema, DropColumn change) } private static void assertNotUpdatingPrimaryKeys( - TableSchema schema, List fieldNames, String operation) { + TableSchema schema, String[] fieldNames, String operation) { // partition keys can't be nested columns - if (fieldNames.size() > 1) { + if (fieldNames.length > 1) { return; } - String columnToRename = fieldNames.get(0); + String columnToRename = fieldNames[0]; if (schema.partitionKeys().contains(columnToRename)) { throw new UnsupportedOperationException( String.format( @@ -617,17 +663,18 @@ public void updateIntermediateColumn(List newFields, int depth) continue; } - List nestedFields = - new ArrayList<>( - ((org.apache.paimon.types.RowType) field.type()).getFields()); - updateIntermediateColumn(nestedFields, depth + 1); + String fullFieldName = + String.join(".", Arrays.asList(updateFieldNames).subList(0, depth + 1)); + List nestedFields = new ArrayList<>(); + int newDepth = + depth + extractRowDataFields(field.type(), fullFieldName, nestedFields); + updateIntermediateColumn(nestedFields, newDepth); newFields.set( i, new DataField( field.id(), field.name(), - new org.apache.paimon.types.RowType( - field.type().isNullable(), nestedFields), + wrapNewRowType(field.type(), nestedFields), field.description())); return; } @@ -637,6 +684,48 @@ public void updateIntermediateColumn(List newFields, int depth) String.join(".", Arrays.asList(updateFieldNames).subList(0, depth + 1))); } + private int extractRowDataFields( + DataType type, String fullFieldName, List nestedFields) { + switch (type.getTypeRoot()) { + case ROW: + nestedFields.addAll(((RowType) type).getFields()); + return 1; + case ARRAY: + return extractRowDataFields( + ((ArrayType) type).getElementType(), + fullFieldName, + nestedFields) + + 1; + case MAP: + return extractRowDataFields( + ((MapType) type).getValueType(), fullFieldName, nestedFields) + + 1; + default: + throw new IllegalArgumentException( + fullFieldName + " is not a structured type."); + } + } + + private DataType wrapNewRowType(DataType type, List nestedFields) { + switch (type.getTypeRoot()) { + case ROW: + return new RowType(type.isNullable(), nestedFields); + case ARRAY: + return new ArrayType( + type.isNullable(), + wrapNewRowType(((ArrayType) type).getElementType(), nestedFields)); + case MAP: + MapType mapType = (MapType) type; + return new MapType( + type.isNullable(), + mapType.getKeyType(), + wrapNewRowType(mapType.getValueType(), nestedFields)); + default: + throw new IllegalStateException( + "Trying to wrap a row type in " + type + ". This is unexpected."); + } + } + protected abstract void updateLastColumn(List newFields, String fieldName) throws Catalog.ColumnNotExistException, Catalog.ColumnAlreadyExistException; @@ -714,11 +803,7 @@ boolean commit(TableSchema newSchema) throws Exception { /** Read schema for schema id. */ public TableSchema schema(long id) { - try { - return JsonSerdeUtil.fromJson(fileIO.readFileUtf8(toSchemaPath(id)), TableSchema.class); - } catch (IOException e) { - throw new UncheckedIOException(e); - } + return TableSchema.fromPath(fileIO, toSchemaPath(id)); } /** Check if a schema exists. */ @@ -734,14 +819,6 @@ public boolean schemaExists(long id) { } } - public static TableSchema fromPath(FileIO fileIO, Path path) { - try { - return JsonSerdeUtil.fromJson(fileIO.readFileUtf8(path), TableSchema.class); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - private String branchPath() { return BranchManager.branchPath(tableRoot, branch); } diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java index b5bdeccf10f6f..a0a149d1ae9b6 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java @@ -29,8 +29,10 @@ import javax.annotation.Nullable; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.Serializable; +import java.io.UncheckedIOException; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; @@ -296,19 +298,6 @@ public TableSchema copy(Map newOptions) { timeMillis); } - public static TableSchema fromJson(String json) { - return JsonSerdeUtil.fromJson(json, TableSchema.class); - } - - public static TableSchema fromPath(FileIO fileIO, Path path) { - try { - String json = fileIO.readFileUtf8(path); - return TableSchema.fromJson(json); - } catch (IOException e) { - throw new RuntimeException("Fails to read schema from path " + path, e); - } - } - @Override public String toString() { return JsonSerdeUtil.toJson(this); @@ -341,4 +330,28 @@ public int hashCode() { public static List newFields(RowType rowType) { return rowType.getFields(); } + + // =================== Utils for reading ========================= + + public static TableSchema fromJson(String json) { + return JsonSerdeUtil.fromJson(json, TableSchema.class); + } + + public static TableSchema fromPath(FileIO fileIO, Path path) { + try { + return tryFromPath(fileIO, path); + } catch (FileNotFoundException e) { + throw new RuntimeException(e.getMessage(), e); + } + } + + public static TableSchema tryFromPath(FileIO fileIO, Path path) throws FileNotFoundException { + try { + return fromJson(fileIO.readFileUtf8(path)); + } catch (FileNotFoundException e) { + throw e; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolution.java b/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolution.java index d3f6d4cd62af6..079300a89dd2a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolution.java +++ b/paimon-core/src/main/java/org/apache/paimon/stats/SimpleStatsEvolution.java @@ -22,6 +22,8 @@ import org.apache.paimon.casting.CastedRow; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.GenericArray; +import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalArray; import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; @@ -33,9 +35,9 @@ import javax.annotation.Nullable; -import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** Converter for array of {@link SimpleColStats}. */ public class SimpleStatsEvolution { @@ -46,6 +48,9 @@ public class SimpleStatsEvolution { private final Map, int[]> indexMappings; + private final GenericRow emptyValues; + private final GenericArray emptyNullCounts; + public SimpleStatsEvolution( RowType rowType, @Nullable int[] indexMapping, @@ -53,7 +58,9 @@ public SimpleStatsEvolution( this.fieldNames = rowType.getFieldNames(); this.indexMapping = indexMapping; this.castFieldGetters = castFieldGetters; - this.indexMappings = new HashMap<>(); + this.indexMappings = new ConcurrentHashMap<>(); + this.emptyValues = new GenericRow(fieldNames.size()); + this.emptyNullCounts = new GenericArray(new Object[fieldNames.size()]); } public Result evolution( @@ -62,7 +69,12 @@ public Result evolution( InternalRow maxValues = stats.maxValues(); InternalArray nullCounts = stats.nullCounts(); - if (denseFields != null) { + if (denseFields != null && denseFields.isEmpty()) { + // optimize for empty dense fields + minValues = emptyValues; + maxValues = emptyValues; + nullCounts = emptyNullCounts; + } else if (denseFields != null) { int[] denseIndexMapping = indexMappings.computeIfAbsent( denseFields, diff --git a/paimon-core/src/main/java/org/apache/paimon/stats/StatsFileHandler.java b/paimon-core/src/main/java/org/apache/paimon/stats/StatsFileHandler.java index f9e057c7cbb32..5cb88f7257a7e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/stats/StatsFileHandler.java +++ b/paimon-core/src/main/java/org/apache/paimon/stats/StatsFileHandler.java @@ -71,13 +71,14 @@ public Optional readStats(long snapshotId) { } public Optional readStats(Snapshot snapshot) { - if (snapshot.statistics() == null) { - return Optional.empty(); - } else { - Statistics stats = statsFile.read(snapshot.statistics()); - stats.deserializeFieldsFromString(schemaManager.schema(stats.schemaId())); - return Optional.of(stats); - } + String file = snapshot.statistics(); + return file == null ? Optional.empty() : Optional.of(readStats(file)); + } + + public Statistics readStats(String file) { + Statistics stats = statsFile.read(file); + stats.deserializeFieldsFromString(schemaManager.schema(stats.schemaId())); + return stats; } /** Delete stats of the specified snapshot. */ diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index af0c3d71ec1ca..57966d24ce474 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -68,6 +68,8 @@ import org.apache.paimon.utils.SnapshotNotExistException; import org.apache.paimon.utils.TagManager; +import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache; + import javax.annotation.Nullable; import java.io.IOException; @@ -90,6 +92,7 @@ abstract class AbstractFileStoreTable implements FileStoreTable { private static final long serialVersionUID = 1L; + private static final String WATERMARK_PREFIX = "watermark-"; protected final FileIO fileIO; @@ -97,6 +100,10 @@ abstract class AbstractFileStoreTable implements FileStoreTable { protected final TableSchema tableSchema; protected final CatalogEnvironment catalogEnvironment; + @Nullable protected transient SegmentsCache manifestCache; + @Nullable protected transient Cache snapshotCache; + @Nullable protected transient Cache statsCache; + protected AbstractFileStoreTable( FileIO fileIO, Path path, @@ -120,9 +127,21 @@ public String currentBranch() { @Override public void setManifestCache(SegmentsCache manifestCache) { + this.manifestCache = manifestCache; store().setManifestCache(manifestCache); } + @Override + public void setSnapshotCache(Cache cache) { + this.snapshotCache = cache; + store().setSnapshotCache(cache); + } + + @Override + public void setStatsCache(Cache cache) { + this.statsCache = cache; + } + @Override public OptionalLong latestSnapshotId() { Long snapshot = store().snapshotManager().latestSnapshotId(); @@ -167,11 +186,34 @@ public Identifier identifier() { : identifier; } + @Override + public String uuid() { + if (catalogEnvironment.uuid() != null) { + return catalogEnvironment.uuid(); + } + long earliestCreationTime = schemaManager().earliestCreationTime(); + return fullName() + "." + earliestCreationTime; + } + @Override public Optional statistics() { Snapshot snapshot = TimeTravelUtil.resolveSnapshot(this); if (snapshot != null) { - return store().newStatsFileHandler().readStats(snapshot); + String file = snapshot.statistics(); + if (file == null) { + return Optional.empty(); + } + if (statsCache != null) { + Statistics stats = statsCache.getIfPresent(file); + if (stats != null) { + return Optional.of(stats); + } + } + Statistics stats = store().newStatsFileHandler().readStats(file); + if (statsCache != null) { + statsCache.put(file, stats); + } + return Optional.of(stats); } return Optional.empty(); } @@ -326,12 +368,26 @@ public FileStoreTable copyWithLatestSchema() { @Override public FileStoreTable copy(TableSchema newTableSchema) { - return newTableSchema.primaryKeys().isEmpty() - ? new AppendOnlyFileStoreTable(fileIO, path, newTableSchema, catalogEnvironment) - : new PrimaryKeyFileStoreTable(fileIO, path, newTableSchema, catalogEnvironment); + AbstractFileStoreTable copied = + newTableSchema.primaryKeys().isEmpty() + ? new AppendOnlyFileStoreTable( + fileIO, path, newTableSchema, catalogEnvironment) + : new PrimaryKeyFileStoreTable( + fileIO, path, newTableSchema, catalogEnvironment); + if (snapshotCache != null) { + copied.setSnapshotCache(snapshotCache); + } + if (manifestCache != null) { + copied.setManifestCache(manifestCache); + } + if (statsCache != null) { + copied.setStatsCache(statsCache); + } + return copied; } - protected SchemaManager schemaManager() { + @Override + public SchemaManager schemaManager() { return new SchemaManager(fileIO(), path, currentBranch()); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java b/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java index ebaff12661556..a722d9e21ada3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/CatalogEnvironment.java @@ -19,7 +19,6 @@ package org.apache.paimon.table; import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.lineage.LineageMetaFactory; import org.apache.paimon.metastore.MetastoreClient; import org.apache.paimon.operation.Lock; @@ -27,32 +26,29 @@ import java.io.Serializable; -/** - * Catalog environment in table which contains log factory, metastore client factory and lineage - * meta. - */ +/** Catalog environment in table which contains log factory, metastore client factory. */ public class CatalogEnvironment implements Serializable { private static final long serialVersionUID = 1L; @Nullable private final Identifier identifier; + @Nullable private final String uuid; private final Lock.Factory lockFactory; @Nullable private final MetastoreClient.Factory metastoreClientFactory; - @Nullable private final LineageMetaFactory lineageMetaFactory; public CatalogEnvironment( @Nullable Identifier identifier, + @Nullable String uuid, Lock.Factory lockFactory, - @Nullable MetastoreClient.Factory metastoreClientFactory, - @Nullable LineageMetaFactory lineageMetaFactory) { + @Nullable MetastoreClient.Factory metastoreClientFactory) { this.identifier = identifier; + this.uuid = uuid; this.lockFactory = lockFactory; this.metastoreClientFactory = metastoreClientFactory; - this.lineageMetaFactory = lineageMetaFactory; } public static CatalogEnvironment empty() { - return new CatalogEnvironment(null, Lock.emptyFactory(), null, null); + return new CatalogEnvironment(null, null, Lock.emptyFactory(), null); } @Nullable @@ -60,6 +56,11 @@ public Identifier identifier() { return identifier; } + @Nullable + public String uuid() { + return uuid; + } + public Lock.Factory lockFactory() { return lockFactory; } @@ -68,9 +69,4 @@ public Lock.Factory lockFactory() { public MetastoreClient.Factory metastoreClientFactory() { return metastoreClientFactory; } - - @Nullable - public LineageMetaFactory lineageMetaFactory() { - return lineageMetaFactory; - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java b/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java index e330db0e04a43..7979daccf7563 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.table.source.DataTableScan; import org.apache.paimon.table.source.snapshot.SnapshotReader; import org.apache.paimon.utils.BranchManager; @@ -39,6 +40,8 @@ public interface DataTable extends InnerTable { SnapshotManager snapshotManager(); + SchemaManager schemaManager(); + TagManager tagManager(); BranchManager branchManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java index f6f3930baade8..0a548941bedcd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java @@ -27,6 +27,7 @@ import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFileMeta; +import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.stats.Statistics; import org.apache.paimon.table.query.LocalTableQuery; @@ -44,6 +45,8 @@ import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.TagManager; +import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache; + import java.time.Duration; import java.util.Objects; import java.util.Optional; @@ -72,6 +75,11 @@ public String fullName() { return wrapped.fullName(); } + @Override + public String uuid() { + return wrapped.uuid(); + } + @Override public SnapshotReader newSnapshotReader() { return wrapped.newSnapshotReader(); @@ -87,6 +95,11 @@ public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); } + @Override + public SchemaManager schemaManager() { + return wrapped.schemaManager(); + } + @Override public TagManager tagManager() { return wrapped.tagManager(); @@ -112,6 +125,16 @@ public void setManifestCache(SegmentsCache manifestCache) { wrapped.setManifestCache(manifestCache); } + @Override + public void setSnapshotCache(Cache cache) { + wrapped.setSnapshotCache(cache); + } + + @Override + public void setStatsCache(Cache cache) { + wrapped.setStatsCache(cache); + } + @Override public TableSchema schema() { return wrapped.schema(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/ExpireChangelogImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/ExpireChangelogImpl.java index ce54975450559..1ffa7485aee5d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/ExpireChangelogImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/ExpireChangelogImpl.java @@ -21,7 +21,7 @@ import org.apache.paimon.Changelog; import org.apache.paimon.Snapshot; import org.apache.paimon.consumer.ConsumerManager; -import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.manifest.ExpireFileEntry; import org.apache.paimon.operation.ChangelogDeletion; import org.apache.paimon.options.ExpireConfig; import org.apache.paimon.utils.Preconditions; @@ -147,7 +147,7 @@ public int expireUntil(long earliestId, long endExclusiveId) { LOG.debug("Ready to delete changelog files from changelog #" + id); } Changelog changelog = snapshotManager.longLivedChangelog(id); - Predicate skipper; + Predicate skipper; try { skipper = changelogDeletion.createDataFileSkipperForTags(taggedSnapshots, id); } catch (Exception e) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/ExpireSnapshotsImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/ExpireSnapshotsImpl.java index 2c83b63c97ef5..dc1c2d6bdbc5d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/ExpireSnapshotsImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/ExpireSnapshotsImpl.java @@ -22,7 +22,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.consumer.ConsumerManager; -import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.manifest.ExpireFileEntry; import org.apache.paimon.operation.SnapshotDeletion; import org.apache.paimon.options.ExpireConfig; import org.apache.paimon.utils.Preconditions; @@ -176,7 +176,7 @@ public int expireUntil(long earliestId, long endExclusiveId) { continue; } // expire merge tree files and collect changed buckets - Predicate skipper; + Predicate skipper; try { skipper = snapshotDeletion.createDataFileSkipperForTags(taggedSnapshots, id); } catch (Exception e) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java index 663933f337b52..050b0841074db 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java @@ -28,7 +28,6 @@ import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.source.DataFilePlan; import org.apache.paimon.table.source.DataSplit; @@ -105,7 +104,7 @@ public FileStoreTable switchToBranch(String branchName) { private FileStoreTable switchWrappedToBranch(String branchName) { Optional optionalSchema = - new SchemaManager(wrapped.fileIO(), wrapped.location(), branchName).latest(); + wrapped.schemaManager().copyWithBranch(branchName).latest(); Preconditions.checkArgument( optionalSchema.isPresent(), "Branch " + branchName + " does not exist"); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java index 01227dd35407c..61aa77d5f36a5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java @@ -19,10 +19,12 @@ package org.apache.paimon.table; import org.apache.paimon.FileStore; +import org.apache.paimon.Snapshot; import org.apache.paimon.data.InternalRow; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.stats.Statistics; import org.apache.paimon.table.query.LocalTableQuery; import org.apache.paimon.table.sink.RowKeyExtractor; import org.apache.paimon.table.sink.TableCommitImpl; @@ -30,6 +32,8 @@ import org.apache.paimon.types.RowType; import org.apache.paimon.utils.SegmentsCache; +import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache; + import java.util.List; import java.util.Map; import java.util.Optional; @@ -42,6 +46,10 @@ public interface FileStoreTable extends DataTable { void setManifestCache(SegmentsCache manifestCache); + void setSnapshotCache(Cache cache); + + void setStatsCache(Cache cache); + @Override default RowType rowType() { return schema().logicalRowType(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java index 47d8777241d6c..423dc17263195 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java @@ -19,14 +19,12 @@ package org.apache.paimon.table; import org.apache.paimon.CoreOptions; -import org.apache.paimon.TableType; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.table.object.ObjectTable; import org.apache.paimon.utils.StringUtils; import java.io.IOException; @@ -35,7 +33,6 @@ import static org.apache.paimon.CoreOptions.PATH; import static org.apache.paimon.utils.Preconditions.checkArgument; -import static org.apache.paimon.utils.Preconditions.checkNotNull; /** Factory to create {@link FileStoreTable}. */ public class FileStoreTableFactory { @@ -127,17 +124,6 @@ public static FileStoreTable createWithoutFallbackBranch( fileIO, tablePath, tableSchema, catalogEnvironment) : new PrimaryKeyFileStoreTable( fileIO, tablePath, tableSchema, catalogEnvironment); - table = table.copy(dynamicOptions.toMap()); - CoreOptions options = table.coreOptions(); - if (options.type() == TableType.OBJECT_TABLE) { - String objectLocation = options.objectLocation(); - checkNotNull(objectLocation, "Object location should not be null for object table."); - table = - ObjectTable.builder() - .underlyingTable(table) - .objectLocation(objectLocation) - .build(); - } - return table; + return table.copy(dynamicOptions.toMap()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FormatTable.java b/paimon-core/src/main/java/org/apache/paimon/table/FormatTable.java index a53ba545c25e4..a4c7788c38afe 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FormatTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FormatTable.java @@ -34,6 +34,7 @@ import javax.annotation.Nullable; import java.time.Duration; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -70,6 +71,19 @@ enum Format { CSV } + /** Parses a file format string to a corresponding {@link Format} enum constant. */ + static Format parseFormat(String fileFormat) { + try { + return Format.valueOf(fileFormat.toUpperCase()); + } catch (IllegalArgumentException e) { + throw new UnsupportedOperationException( + "Format table unsupported file format: " + + fileFormat + + ". Supported formats: " + + Arrays.toString(Format.values())); + } + } + /** Create a new builder for {@link FormatTable}. */ static FormatTable.Builder builder() { return new FormatTable.Builder(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/RollbackHelper.java b/paimon-core/src/main/java/org/apache/paimon/table/RollbackHelper.java index 1eb4ccf001362..29fecec113536 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/RollbackHelper.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/RollbackHelper.java @@ -21,7 +21,7 @@ import org.apache.paimon.Changelog; import org.apache.paimon.Snapshot; import org.apache.paimon.fs.FileIO; -import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.manifest.ExpireFileEntry; import org.apache.paimon.operation.ChangelogDeletion; import org.apache.paimon.operation.SnapshotDeletion; import org.apache.paimon.operation.TagDeletion; @@ -205,7 +205,7 @@ private List cleanTagsDataFiles(Snapshot retainedSnapshot) { } // delete data files - Predicate dataFileSkipper = null; + Predicate dataFileSkipper = null; boolean success = true; try { dataFileSkipper = tagDeletion.dataFileSkipper(retainedSnapshot); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/Table.java b/paimon-core/src/main/java/org/apache/paimon/table/Table.java index db6848f5f1a84..7ed7ba48a8ebd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/Table.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/Table.java @@ -52,10 +52,19 @@ public interface Table extends Serializable { /** A name to identify this table. */ String name(); + /** Full name of the table, default is database.tableName. */ default String fullName() { return name(); } + /** + * UUID of the table, metastore can provide the true UUID of this table, default is the full + * name. + */ + default String uuid() { + return fullName(); + } + /** Returns the row type of this table. */ RowType rowType(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectRefresh.java b/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectRefresh.java index 326efbc0eac83..b1be840c51536 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectRefresh.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectRefresh.java @@ -26,7 +26,6 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; -import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.BatchTableCommit; import org.apache.paimon.table.sink.BatchTableWrite; import org.apache.paimon.table.sink.BatchWriteBuilder; @@ -41,13 +40,14 @@ public class ObjectRefresh { public static long refresh(ObjectTable table) throws Exception { String location = table.objectLocation(); - FileStoreTable underlyingTable = table.underlyingTable(); - FileIO fileIO = underlyingTable.fileIO(); + // 1. collect all files for object table List fileCollector = new ArrayList<>(); - listAllFiles(fileIO, new Path(location), fileCollector); + listAllFiles(table.objectFileIO(), new Path(location), fileCollector); - BatchWriteBuilder writeBuilder = underlyingTable.newBatchWriteBuilder().withOverwrite(); + // 2. write to underlying table + BatchWriteBuilder writeBuilder = + table.underlyingTable().newBatchWriteBuilder().withOverwrite(); try (BatchTableWrite write = writeBuilder.newWrite(); BatchTableCommit commit = writeBuilder.newCommit()) { for (FileStatus file : fileCollector) { @@ -78,6 +78,7 @@ private static void listAllFiles(FileIO fileIO, Path directory, List private static InternalRow toRow(FileStatus file) { return toRow( file.getPath().toString(), + file.getPath().getParent().toString(), file.getPath().getName(), file.getLen(), Timestamp.fromEpochMillis(file.getModificationTime()), diff --git a/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java b/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java index 65689108caae8..97acfe7299c51 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java @@ -18,6 +18,7 @@ package org.apache.paimon.table.object; +import org.apache.paimon.fs.FileIO; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.DelegatedFileStoreTable; @@ -46,6 +47,7 @@ public interface ObjectTable extends FileStoreTable { RowType SCHEMA = RowType.builder() .field("path", DataTypes.STRING().notNull()) + .field("parent_path", DataTypes.STRING().notNull()) .field("name", DataTypes.STRING().notNull()) .field("length", DataTypes.BIGINT().notNull()) .field("mtime", DataTypes.TIMESTAMP_LTZ_MILLIS()) @@ -66,11 +68,26 @@ public interface ObjectTable extends FileStoreTable { /** Underlying table to store metadata. */ FileStoreTable underlyingTable(); + /** File io for object file system. */ + FileIO objectFileIO(); + long refresh(); @Override ObjectTable copy(Map dynamicOptions); + @Override + ObjectTable copy(TableSchema newTableSchema); + + @Override + ObjectTable copyWithoutTimeTravel(Map dynamicOptions); + + @Override + ObjectTable copyWithLatestSchema(); + + @Override + ObjectTable switchToBranch(String branchName); + /** Create a new builder for {@link ObjectTable}. */ static ObjectTable.Builder builder() { return new ObjectTable.Builder(); @@ -80,6 +97,7 @@ static ObjectTable.Builder builder() { class Builder { private FileStoreTable underlyingTable; + private FileIO objectFileIO; private String objectLocation; public ObjectTable.Builder underlyingTable(FileStoreTable underlyingTable) { @@ -93,23 +111,31 @@ public ObjectTable.Builder underlyingTable(FileStoreTable underlyingTable) { return this; } + public ObjectTable.Builder objectFileIO(FileIO objectFileIO) { + this.objectFileIO = objectFileIO; + return this; + } + public ObjectTable.Builder objectLocation(String objectLocation) { this.objectLocation = objectLocation; return this; } public ObjectTable build() { - return new ObjectTableImpl(underlyingTable, objectLocation); + return new ObjectTableImpl(underlyingTable, objectFileIO, objectLocation); } } /** An implementation for {@link ObjectTable}. */ class ObjectTableImpl extends DelegatedFileStoreTable implements ObjectTable { + private final FileIO objectFileIO; private final String objectLocation; - public ObjectTableImpl(FileStoreTable underlyingTable, String objectLocation) { + public ObjectTableImpl( + FileStoreTable underlyingTable, FileIO objectFileIO, String objectLocation) { super(underlyingTable); + this.objectFileIO = objectFileIO; this.objectLocation = objectLocation; } @@ -148,6 +174,11 @@ public FileStoreTable underlyingTable() { return wrapped; } + @Override + public FileIO objectFileIO() { + return objectFileIO; + } + @Override public long refresh() { try { @@ -159,28 +190,30 @@ public long refresh() { @Override public ObjectTable copy(Map dynamicOptions) { - return new ObjectTableImpl(wrapped.copy(dynamicOptions), objectLocation); + return new ObjectTableImpl(wrapped.copy(dynamicOptions), objectFileIO, objectLocation); } @Override - public FileStoreTable copy(TableSchema newTableSchema) { - return new ObjectTableImpl(wrapped.copy(newTableSchema), objectLocation); + public ObjectTable copy(TableSchema newTableSchema) { + return new ObjectTableImpl(wrapped.copy(newTableSchema), objectFileIO, objectLocation); } @Override - public FileStoreTable copyWithoutTimeTravel(Map dynamicOptions) { + public ObjectTable copyWithoutTimeTravel(Map dynamicOptions) { return new ObjectTableImpl( - wrapped.copyWithoutTimeTravel(dynamicOptions), objectLocation); + wrapped.copyWithoutTimeTravel(dynamicOptions), objectFileIO, objectLocation); } @Override - public FileStoreTable copyWithLatestSchema() { - return new ObjectTableImpl(wrapped.copyWithLatestSchema(), objectLocation); + public ObjectTable copyWithLatestSchema() { + return new ObjectTableImpl( + wrapped.copyWithLatestSchema(), objectFileIO, objectLocation); } @Override - public FileStoreTable switchToBranch(String branchName) { - return new ObjectTableImpl(wrapped.switchToBranch(branchName), objectLocation); + public ObjectTable switchToBranch(String branchName) { + return new ObjectTableImpl( + wrapped.switchToBranch(branchName), objectFileIO, objectLocation); } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java index 7918914b2c635..9fc251c366729 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/CommitMessageSerializer.java @@ -19,6 +19,8 @@ package org.apache.paimon.table.sink; import org.apache.paimon.data.serializer.VersionedSerializer; +import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.index.IndexFileMeta09Serializer; import org.apache.paimon.index.IndexFileMetaSerializer; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; @@ -45,12 +47,14 @@ /** {@link VersionedSerializer} for {@link CommitMessage}. */ public class CommitMessageSerializer implements VersionedSerializer { - private static final int CURRENT_VERSION = 4; + private static final int CURRENT_VERSION = 5; private final DataFileMetaSerializer dataFileSerializer; private final IndexFileMetaSerializer indexEntrySerializer; + private DataFileMeta09Serializer dataFile09Serializer; private DataFileMeta08Serializer dataFile08Serializer; + private IndexFileMeta09Serializer indexEntry09Serializer; public CommitMessageSerializer() { this.dataFileSerializer = new DataFileMetaSerializer(); @@ -107,48 +111,48 @@ public List deserializeList(int version, DataInputView view) thro } private CommitMessage deserialize(int version, DataInputView view) throws IOException { - if (version >= 3) { - IOExceptionSupplier> fileDeserializer = - () -> dataFileSerializer.deserializeList(view); - if (version == 3) { - DataFileMeta09Serializer serializer = new DataFileMeta09Serializer(); - fileDeserializer = () -> serializer.deserializeList(view); - } - return new CommitMessageImpl( - deserializeBinaryRow(view), - view.readInt(), - new DataIncrement( - fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()), - new CompactIncrement( - fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()), - new IndexIncrement( - indexEntrySerializer.deserializeList(view), - indexEntrySerializer.deserializeList(view))); - } else { - return deserialize08(version, view); - } - } - - private CommitMessage deserialize08(int version, DataInputView view) throws IOException { - if (dataFile08Serializer == null) { - dataFile08Serializer = new DataFileMeta08Serializer(); - } + IOExceptionSupplier> fileDeserializer = fileDeserializer(version, view); + IOExceptionSupplier> indexEntryDeserializer = + indexEntryDeserializer(version, view); return new CommitMessageImpl( deserializeBinaryRow(view), view.readInt(), new DataIncrement( - dataFile08Serializer.deserializeList(view), - dataFile08Serializer.deserializeList(view), - dataFile08Serializer.deserializeList(view)), + fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()), new CompactIncrement( - dataFile08Serializer.deserializeList(view), - dataFile08Serializer.deserializeList(view), - dataFile08Serializer.deserializeList(view)), + fileDeserializer.get(), fileDeserializer.get(), fileDeserializer.get()), new IndexIncrement( - indexEntrySerializer.deserializeList(view), - version <= 2 - ? Collections.emptyList() - : indexEntrySerializer.deserializeList(view))); + indexEntryDeserializer.get(), + version <= 2 ? Collections.emptyList() : indexEntryDeserializer.get())); + } + + private IOExceptionSupplier> fileDeserializer( + int version, DataInputView view) { + if (version >= 4) { + return () -> dataFileSerializer.deserializeList(view); + } else if (version == 3) { + if (dataFile09Serializer == null) { + dataFile09Serializer = new DataFileMeta09Serializer(); + } + return () -> dataFile09Serializer.deserializeList(view); + } else { + if (dataFile08Serializer == null) { + dataFile08Serializer = new DataFileMeta08Serializer(); + } + return () -> dataFile08Serializer.deserializeList(view); + } + } + + private IOExceptionSupplier> indexEntryDeserializer( + int version, DataInputView view) { + if (version >= 5) { + return () -> indexEntrySerializer.deserializeList(view); + } else { + if (indexEntry09Serializer == null) { + indexEntry09Serializer = new IndexFileMeta09Serializer(); + } + return () -> indexEntry09Serializer.deserializeList(view); + } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java index b4f8fa47dbb1e..73c55942a56aa 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/TableCommitImpl.java @@ -68,7 +68,7 @@ import static org.apache.paimon.table.sink.BatchWriteBuilder.COMMIT_IDENTIFIER; import static org.apache.paimon.utils.ManifestReadThreadPool.getExecutorService; import static org.apache.paimon.utils.Preconditions.checkState; -import static org.apache.paimon.utils.ThreadPoolUtils.randomlyExecute; +import static org.apache.paimon.utils.ThreadPoolUtils.randomlyExecuteSequentialReturn; /** An abstraction layer above {@link FileStoreCommit} to provide snapshot commit and expiration. */ public class TableCommitImpl implements InnerTableCommit { @@ -292,7 +292,7 @@ private void checkFilesExistence(List committables) { List nonExistFiles = Lists.newArrayList( - randomlyExecute( + randomlyExecuteSequentialReturn( getExecutorService(null), f -> nonExists.test(f) ? singletonList(f) : emptyList(), files)); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java index 9299b86aa4205..1f478f283b68a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractDataTableScan.java @@ -109,6 +109,12 @@ public AbstractDataTableScan withMetricsRegistry(MetricRegistry metricsRegistry) return this; } + @Override + public AbstractDataTableScan dropStats() { + snapshotReader.dropStats(); + return this; + } + public CoreOptions options() { return options; } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java b/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java index 1dac6584d6988..b9460f28b4e79 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/DataSplit.java @@ -44,13 +44,14 @@ import static org.apache.paimon.io.DataFilePathFactory.INDEX_PATH_SUFFIX; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.Preconditions.checkState; /** Input splits. Needed by most batch computation engines. */ public class DataSplit implements Split { private static final long serialVersionUID = 7L; private static final long MAGIC = -2394839472490812314L; - private static final int VERSION = 3; + private static final int VERSION = 4; private long snapshotId = 0; private BinaryRow partition; @@ -126,6 +127,45 @@ public long rowCount() { return rowCount; } + /** Whether it is possible to calculate the merged row count. */ + public boolean mergedRowCountAvailable() { + return rawConvertible + && (dataDeletionFiles == null + || dataDeletionFiles.stream() + .allMatch(f -> f == null || f.cardinality() != null)); + } + + public long mergedRowCount() { + checkState(mergedRowCountAvailable()); + return partialMergedRowCount(); + } + + /** + * Obtain merged row count as much as possible. There are two scenarios where accurate row count + * can be calculated: + * + *

    1. raw file and no deletion file. + * + *

    2. raw file + deletion file with cardinality. + */ + public long partialMergedRowCount() { + long sum = 0L; + if (rawConvertible) { + List rawFiles = convertToRawFiles().orElse(null); + if (rawFiles != null) { + for (int i = 0; i < rawFiles.size(); i++) { + RawFile rawFile = rawFiles.get(i); + if (dataDeletionFiles == null || dataDeletionFiles.get(i) == null) { + sum += rawFile.rowCount(); + } else if (dataDeletionFiles.get(i).cardinality() != null) { + sum += rawFile.rowCount() - dataDeletionFiles.get(i).cardinality(); + } + } + } + } + return sum; + } + @Override public Optional> convertToRawFiles() { if (rawConvertible) { @@ -272,13 +312,16 @@ public static DataSplit deserialize(DataInputView in) throws IOException { FunctionWithIOException dataFileSer = getFileMetaSerde(version); + FunctionWithIOException deletionFileSerde = + getDeletionFileSerde(version); int beforeNumber = in.readInt(); List beforeFiles = new ArrayList<>(beforeNumber); for (int i = 0; i < beforeNumber; i++) { beforeFiles.add(dataFileSer.apply(in)); } - List beforeDeletionFiles = DeletionFile.deserializeList(in); + List beforeDeletionFiles = + DeletionFile.deserializeList(in, deletionFileSerde); int fileNumber = in.readInt(); List dataFiles = new ArrayList<>(fileNumber); @@ -286,7 +329,7 @@ public static DataSplit deserialize(DataInputView in) throws IOException { dataFiles.add(dataFileSer.apply(in)); } - List dataDeletionFiles = DeletionFile.deserializeList(in); + List dataDeletionFiles = DeletionFile.deserializeList(in, deletionFileSerde); boolean isStreaming = in.readBoolean(); boolean rawConvertible = in.readBoolean(); @@ -319,16 +362,22 @@ private static FunctionWithIOException getFileMetaS } else if (version == 2) { DataFileMeta09Serializer serializer = new DataFileMeta09Serializer(); return serializer::deserialize; - } else if (version == 3) { + } else if (version >= 3) { DataFileMetaSerializer serializer = new DataFileMetaSerializer(); return serializer::deserialize; } else { - throw new UnsupportedOperationException( - "Expecting DataSplit version to be smaller or equal than " - + VERSION - + ", but found " - + version - + "."); + throw new UnsupportedOperationException("Unsupported version: " + version); + } + } + + private static FunctionWithIOException getDeletionFileSerde( + int version) { + if (version >= 1 && version <= 3) { + return DeletionFile::deserializeV3; + } else if (version >= 4) { + return DeletionFile::deserialize; + } else { + throw new UnsupportedOperationException("Unsupported version: " + version); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java b/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java index d3e8a2adb6979..a4fe6d73bba19 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableBatchScan.java @@ -28,7 +28,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Objects; import static org.apache.paimon.CoreOptions.MergeEngine.FIRST_ROW; @@ -51,7 +50,7 @@ public DataTableBatchScan( this.hasNext = true; this.defaultValueAssigner = defaultValueAssigner; if (pkTable && (options.deletionVectorsEnabled() || options.mergeEngine() == FIRST_ROW)) { - snapshotReader.withLevelFilter(level -> level > 0); + snapshotReader.withLevelFilter(level -> level > 0).enableValueFilter(); } } @@ -103,9 +102,9 @@ private StartingScanner.Result applyPushDownLimit(StartingScanner.Result result) List limitedSplits = new ArrayList<>(); for (DataSplit dataSplit : splits) { if (dataSplit.rawConvertible()) { - long splitRowCount = getRowCountForSplit(dataSplit); + long partialMergedRowCount = dataSplit.partialMergedRowCount(); limitedSplits.add(dataSplit); - scannedRowCount += splitRowCount; + scannedRowCount += partialMergedRowCount; if (scannedRowCount >= pushDownLimit) { SnapshotReader.Plan newPlan = new PlanImpl(plan.watermark(), plan.snapshotId(), limitedSplits); @@ -117,20 +116,6 @@ private StartingScanner.Result applyPushDownLimit(StartingScanner.Result result) return result; } - /** - * 0 represents that we can't compute the row count of this split: 1. the split needs to be - * merged; 2. the table enabled deletion vector and there are some deletion files. - */ - private long getRowCountForSplit(DataSplit split) { - if (split.deletionFiles().isPresent() - && split.deletionFiles().get().stream().anyMatch(Objects::nonNull)) { - return 0L; - } - return split.convertToRawFiles() - .map(files -> files.stream().map(RawFile::rowCount).reduce(Long::sum).orElse(0L)) - .orElse(0L); - } - @Override public DataTableScan withShard(int indexOfThisSubtask, int numberOfParallelSubtasks) { snapshotReader.withShard(indexOfThisSubtask, numberOfParallelSubtasks); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableStreamScan.java b/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableStreamScan.java index a68c7b1cb46d7..e8c4ddfa1c7c4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableStreamScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/DataTableStreamScan.java @@ -194,16 +194,16 @@ private Plan nextPlan() { return SnapshotNotExistPlan.INSTANCE; } - // first check changes of overwrite - if (snapshot.commitKind() == Snapshot.CommitKind.OVERWRITE - && supportStreamingReadOverwrite) { - LOG.debug("Find overwrite snapshot id {}.", nextSnapshotId); - SnapshotReader.Plan overwritePlan = - followUpScanner.getOverwriteChangesPlan(snapshot, snapshotReader); - currentWatermark = overwritePlan.watermark(); - nextSnapshotId++; - return overwritePlan; - } else if (followUpScanner.shouldScanSnapshot(snapshot)) { + // first try to get overwrite changes + if (snapshot.commitKind() == Snapshot.CommitKind.OVERWRITE) { + SnapshotReader.Plan overwritePlan = handleOverwriteSnapshot(snapshot); + if (overwritePlan != null) { + nextSnapshotId++; + return overwritePlan; + } + } + + if (followUpScanner.shouldScanSnapshot(snapshot)) { LOG.debug("Find snapshot id {}.", nextSnapshotId); SnapshotReader.Plan plan = followUpScanner.scan(snapshot, snapshotReader); currentWatermark = plan.watermark(); @@ -228,6 +228,18 @@ private boolean shouldDelaySnapshot(long snapshotId) { return false; } + @Nullable + protected SnapshotReader.Plan handleOverwriteSnapshot(Snapshot snapshot) { + if (supportStreamingReadOverwrite) { + LOG.debug("Find overwrite snapshot id {}.", nextSnapshotId); + SnapshotReader.Plan overwritePlan = + followUpScanner.getOverwriteChangesPlan(snapshot, snapshotReader); + currentWatermark = overwritePlan.watermark(); + return overwritePlan; + } + return null; + } + protected FollowUpScanner createFollowUpScanner() { CoreOptions.StreamScanMode type = options.toConfiguration().get(CoreOptions.STREAM_SCAN_MODE); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/DeletionFile.java b/paimon-core/src/main/java/org/apache/paimon/table/source/DeletionFile.java index 94dfc615729c8..5bcf6898ed991 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/DeletionFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/DeletionFile.java @@ -22,6 +22,7 @@ import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataInputView; import org.apache.paimon.io.DataOutputView; +import org.apache.paimon.utils.FunctionWithIOException; import javax.annotation.Nullable; @@ -52,11 +53,13 @@ public class DeletionFile implements Serializable { private final String path; private final long offset; private final long length; + @Nullable private final Long cardinality; - public DeletionFile(String path, long offset, long length) { + public DeletionFile(String path, long offset, long length, @Nullable Long cardinality) { this.path = path; this.offset = offset; this.length = length; + this.cardinality = cardinality; } /** Path of the file. */ @@ -74,6 +77,12 @@ public long length() { return length; } + /** the number of deleted rows. */ + @Nullable + public Long cardinality() { + return cardinality; + } + public static void serialize(DataOutputView out, @Nullable DeletionFile file) throws IOException { if (file == null) { @@ -83,6 +92,7 @@ public static void serialize(DataOutputView out, @Nullable DeletionFile file) out.writeUTF(file.path); out.writeLong(file.offset); out.writeLong(file.length); + out.writeLong(file.cardinality == null ? -1 : file.cardinality); } } @@ -108,17 +118,32 @@ public static DeletionFile deserialize(DataInputView in) throws IOException { String path = in.readUTF(); long offset = in.readLong(); long length = in.readLong(); - return new DeletionFile(path, offset, length); + long cardinality = in.readLong(); + return new DeletionFile(path, offset, length, cardinality == -1 ? null : cardinality); } @Nullable - public static List deserializeList(DataInputView in) throws IOException { + public static DeletionFile deserializeV3(DataInputView in) throws IOException { + if (in.readByte() == 0) { + return null; + } + + String path = in.readUTF(); + long offset = in.readLong(); + long length = in.readLong(); + return new DeletionFile(path, offset, length, null); + } + + @Nullable + public static List deserializeList( + DataInputView in, FunctionWithIOException deserialize) + throws IOException { List files = null; if (in.readByte() == 1) { int size = in.readInt(); files = new ArrayList<>(size); for (int i = 0; i < size; i++) { - files.add(DeletionFile.deserialize(in)); + files.add(deserialize.apply(in)); } } return files; @@ -126,22 +151,34 @@ public static List deserializeList(DataInputView in) throws IOExce @Override public boolean equals(Object o) { - if (!(o instanceof DeletionFile)) { + if (o == null || getClass() != o.getClass()) { return false; } - - DeletionFile other = (DeletionFile) o; - return Objects.equals(path, other.path) && offset == other.offset && length == other.length; + DeletionFile that = (DeletionFile) o; + return offset == that.offset + && length == that.length + && Objects.equals(path, that.path) + && Objects.equals(cardinality, that.cardinality); } @Override public int hashCode() { - return Objects.hash(path, offset, length); + return Objects.hash(path, offset, length, cardinality); } @Override public String toString() { - return String.format("{path = %s, offset = %d, length = %d}", path, offset, length); + return "DeletionFile{" + + "path='" + + path + + '\'' + + ", offset=" + + offset + + ", length=" + + length + + ", cardinality=" + + cardinality + + '}'; } static Factory emptyFactory() { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java index 1e17e001694eb..58f8845280542 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java @@ -66,4 +66,9 @@ default InnerTableScan withMetricsRegistry(MetricRegistry metricRegistry) { // do nothing, should implement this if need return this; } + + default InnerTableScan dropStats() { + // do nothing, should implement this if need + return this; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java index 91d5f1004e914..0c1386ce441d8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java @@ -150,6 +150,9 @@ default ReadBuilder withProjection(int[][] projection) { */ ReadBuilder withShard(int indexOfThisSubtask, int numberOfParallelSubtasks); + /** Delete stats in scan plan result. */ + ReadBuilder dropStats(); + /** Create a {@link TableScan} to perform batch planning. */ TableScan newScan(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java index 577b0a20a99b3..95bfe6f24bc77 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java @@ -51,6 +51,8 @@ public class ReadBuilderImpl implements ReadBuilder { private @Nullable RowType readType; + private boolean dropStats = false; + public ReadBuilderImpl(InnerTable table) { this.table = table; } @@ -124,6 +126,12 @@ public ReadBuilder withBucketFilter(Filter bucketFilter) { return this; } + @Override + public ReadBuilder dropStats() { + this.dropStats = true; + return this; + } + @Override public TableScan newScan() { InnerTableScan tableScan = configureScan(table.newScan()); @@ -156,6 +164,9 @@ private InnerTableScan configureScan(InnerTableScan scan) { if (bucketFilter != null) { scan.withBucketFilter(bucketFilter); } + if (dropStats) { + scan.dropStats(); + } return scan; } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java index 358d86cbe948a..9bfb54f2cf608 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java @@ -31,7 +31,6 @@ import org.apache.paimon.table.source.ScanMode; import org.apache.paimon.table.source.Split; import org.apache.paimon.table.source.SplitGenerator; -import org.apache.paimon.utils.ManifestReadThreadPool; import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.SnapshotManager; @@ -50,6 +49,7 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; +import static org.apache.paimon.utils.ManifestReadThreadPool.randomlyExecuteSequentialReturn; import static org.apache.paimon.utils.Preconditions.checkArgument; /** {@link StartingScanner} for incremental changes by snapshot. */ @@ -84,7 +84,7 @@ public Result scan(SnapshotReader reader) { .collect(Collectors.toList()); Iterator manifests = - ManifestReadThreadPool.randomlyExecute( + randomlyExecuteSequentialReturn( id -> { Snapshot snapshot = snapshotManager.snapshot(id); switch (scanMode) { @@ -111,7 +111,7 @@ public Result scan(SnapshotReader reader) { reader.parallelism()); Iterator entries = - ManifestReadThreadPool.randomlyExecute( + randomlyExecuteSequentialReturn( reader::readManifest, Lists.newArrayList(manifests), reader.parallelism()); while (entries.hasNext()) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java index 026e529dc0c84..50a5b6940c3eb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReader.java @@ -78,6 +78,8 @@ public interface SnapshotReader { SnapshotReader withLevelFilter(Filter levelFilter); + SnapshotReader enableValueFilter(); + SnapshotReader withManifestEntryFilter(Filter filter); SnapshotReader withBucket(int bucket); @@ -88,6 +90,8 @@ public interface SnapshotReader { SnapshotReader withDataFileNameFilter(Filter fileNameFilter); + SnapshotReader dropStats(); + SnapshotReader withShard(int indexOfThisSubtask, int numberOfParallelSubtasks); SnapshotReader withMetricRegistry(MetricRegistry registry); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java index c3f027da962f0..f24b0760e6f39 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java @@ -24,6 +24,7 @@ import org.apache.paimon.codegen.RecordComparator; import org.apache.paimon.consumer.ConsumerManager; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.io.DataFileMeta; @@ -235,6 +236,12 @@ public SnapshotReader withLevelFilter(Filter levelFilter) { return this; } + @Override + public SnapshotReader enableValueFilter() { + scan.enableValueFilter(); + return this; + } + @Override public SnapshotReader withManifestEntryFilter(Filter filter) { scan.withManifestEntryFilter(filter); @@ -270,6 +277,12 @@ public SnapshotReader withDataFileNameFilter(Filter fileNameFilter) { return this; } + @Override + public SnapshotReader dropStats() { + scan.dropStats(); + return this; + } + @Override public SnapshotReader withShard(int indexOfThisSubtask, int numberOfParallelSubtasks) { if (splitGenerator.alwaysRawConvertible()) { @@ -492,23 +505,24 @@ private List getDeletionFiles( List deletionFiles = new ArrayList<>(dataFiles.size()); Map dataFileToIndexFileMeta = new HashMap<>(); for (IndexFileMeta indexFileMeta : indexFileMetas) { - if (indexFileMeta.deletionVectorsRanges() != null) { - for (String dataFileName : indexFileMeta.deletionVectorsRanges().keySet()) { - dataFileToIndexFileMeta.put(dataFileName, indexFileMeta); + if (indexFileMeta.deletionVectorMetas() != null) { + for (DeletionVectorMeta dvMeta : indexFileMeta.deletionVectorMetas().values()) { + dataFileToIndexFileMeta.put(dvMeta.dataFileName(), indexFileMeta); } } } for (DataFileMeta file : dataFiles) { IndexFileMeta indexFileMeta = dataFileToIndexFileMeta.get(file.fileName()); if (indexFileMeta != null) { - Map> ranges = indexFileMeta.deletionVectorsRanges(); - if (ranges != null && ranges.containsKey(file.fileName())) { - Pair range = ranges.get(file.fileName()); + LinkedHashMap dvMetas = + indexFileMeta.deletionVectorMetas(); + if (dvMetas != null && dvMetas.containsKey(file.fileName())) { deletionFiles.add( new DeletionFile( indexFileHandler.filePath(indexFileMeta).toString(), - range.getKey(), - range.getValue())); + dvMetas.get(file.fileName()).offset(), + dvMetas.get(file.fileName()).length(), + dvMetas.get(file.fileName()).cardinality())); continue; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/splitread/IncrementalChangelogReadProvider.java b/paimon-core/src/main/java/org/apache/paimon/table/source/splitread/IncrementalChangelogReadProvider.java index 308c09d142046..eb41d02669fc0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/splitread/IncrementalChangelogReadProvider.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/splitread/IncrementalChangelogReadProvider.java @@ -60,20 +60,20 @@ private SplitRead create(Supplier supplier) { ConcatRecordReader.create( () -> new ReverseReader( - read.createNoMergeReader( + read.createMergeReader( split.partition(), split.bucket(), split.beforeFiles(), split.beforeDeletionFiles() .orElse(null), - true)), + false)), () -> - read.createNoMergeReader( + read.createMergeReader( split.partition(), split.bucket(), split.dataFiles(), split.deletionFiles().orElse(null), - true)); + false)); return unwrap(reader); }; diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java index a88bde9e5d720..8c0eed4d6b8bf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java @@ -18,7 +18,6 @@ package org.apache.paimon.table.system; -import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; @@ -27,7 +26,6 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.ReadonlyTable; @@ -78,19 +76,13 @@ public class AggregationFieldsTable implements ReadonlyTable { private final FileIO fileIO; private final Path location; - private final String branch; - public AggregationFieldsTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - CoreOptions.branch(dataTable.schema().options())); - } + private final FileStoreTable dataTable; - public AggregationFieldsTable(FileIO fileIO, Path location, String branchName) { - this.fileIO = fileIO; - this.location = location; - this.branch = branchName; + public AggregationFieldsTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.dataTable = dataTable; } @Override @@ -120,7 +112,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new AggregationFieldsTable(fileIO, location, branch); + return new AggregationFieldsTable(dataTable.copy(dynamicOptions)); } private class SchemasScan extends ReadOnceTableScan { @@ -196,8 +188,7 @@ public RecordReader createReader(Split split) { if (!(split instanceof AggregationSplit)) { throw new IllegalArgumentException("Unsupported split: " + split.getClass()); } - Path location = ((AggregationSplit) split).location; - TableSchema schemas = new SchemaManager(fileIO, location, branch).latest().get(); + TableSchema schemas = dataTable.schemaManager().latest().get(); Iterator rows = createInternalRowIterator(schemas); if (readType != null) { rows = diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java index 5105f6fc8fd26..f1dc3331b4ee8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java @@ -39,6 +39,7 @@ import org.apache.paimon.predicate.PredicateBuilder; import org.apache.paimon.predicate.PredicateReplaceVisitor; import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.table.DataTable; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.ReadonlyTable; @@ -188,6 +189,11 @@ public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); } + @Override + public SchemaManager schemaManager() { + return wrapped.schemaManager(); + } + @Override public TagManager tagManager() { return wrapped.tagManager(); @@ -320,6 +326,12 @@ public SnapshotReader withLevelFilter(Filter levelFilter) { return this; } + @Override + public SnapshotReader enableValueFilter() { + wrapped.enableValueFilter(); + return this; + } + @Override public SnapshotReader withManifestEntryFilter(Filter filter) { wrapped.withManifestEntryFilter(filter); @@ -349,6 +361,12 @@ public SnapshotReader withDataFileNameFilter(Filter fileNameFilter) { return this; } + @Override + public SnapshotReader dropStats() { + wrapped.dropStats(); + return this; + } + @Override public SnapshotReader withShard(int indexOfThisSubtask, int numberOfParallelSubtasks) { wrapped.withShard(indexOfThisSubtask, numberOfParallelSubtasks); @@ -551,13 +569,13 @@ public DataTableScan withShard(int indexOfThisSubtask, int numberOfParallelSubta } } - private class AuditLogRead implements InnerTableRead { + class AuditLogRead implements InnerTableRead { - private final InnerTableRead dataRead; + protected final InnerTableRead dataRead; - private int[] readProjection; + protected int[] readProjection; - private AuditLogRead(InnerTableRead dataRead) { + protected AuditLogRead(InnerTableRead dataRead) { this.dataRead = dataRead.forceKeepDelete(); this.readProjection = defaultProjection(); } @@ -625,9 +643,9 @@ private InternalRow convertRow(InternalRow data) { } /** A {@link ProjectedRow} which returns row kind when mapping index is negative. */ - private static class AuditLogRow extends ProjectedRow { + static class AuditLogRow extends ProjectedRow { - private AuditLogRow(int[] indexMapping, InternalRow row) { + AuditLogRow(int[] indexMapping, InternalRow row) { super(indexMapping); replaceRow(row); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.java new file mode 100644 index 0000000000000..08eea468ea70c --- /dev/null +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/BinlogTable.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.table.system; + +import org.apache.paimon.data.GenericArray; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.reader.PackChangelogReader; +import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.table.Table; +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.types.ArrayType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.RowType; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER; + +/** + * A {@link Table} for reading binlog of table. The binlog format is as below. + * + *

    INSERT: [+I, [co1], [col2]] + * + *

    UPDATE: [+U, [co1_ub, col1_ua], [col2_ub, col2_ua]] + * + *

    DELETE: [-D, [co1], [col2]] + */ +public class BinlogTable extends AuditLogTable { + + public static final String BINLOG = "binlog"; + + private final FileStoreTable wrapped; + + public BinlogTable(FileStoreTable wrapped) { + super(wrapped); + this.wrapped = wrapped; + } + + @Override + public String name() { + return wrapped.name() + SYSTEM_TABLE_SPLITTER + BINLOG; + } + + @Override + public RowType rowType() { + List fields = new ArrayList<>(); + fields.add(SpecialFields.ROW_KIND); + for (DataField field : wrapped.rowType().getFields()) { + // convert to nullable + fields.add(field.newType(new ArrayType(field.type().nullable()))); + } + return new RowType(fields); + } + + @Override + public InnerTableRead newRead() { + return new BinlogRead(wrapped.newRead()); + } + + @Override + public Table copy(Map dynamicOptions) { + return new BinlogTable(wrapped.copy(dynamicOptions)); + } + + private class BinlogRead extends AuditLogRead { + + private BinlogRead(InnerTableRead dataRead) { + super(dataRead); + } + + @Override + public InnerTableRead withReadType(RowType readType) { + List fields = new ArrayList<>(); + for (DataField field : readType.getFields()) { + if (field.name().equals(SpecialFields.ROW_KIND.name())) { + fields.add(field); + } else { + fields.add(field.newType(((ArrayType) field.type()).getElementType())); + } + } + return super.withReadType(readType.copy(fields)); + } + + @Override + public RecordReader createReader(Split split) throws IOException { + DataSplit dataSplit = (DataSplit) split; + if (dataSplit.isStreaming()) { + return new PackChangelogReader( + dataRead.createReader(split), + (row1, row2) -> + new AuditLogRow( + readProjection, + convertToArray( + row1, row2, wrapped.rowType().fieldGetters())), + wrapped.rowType()); + } else { + return dataRead.createReader(split) + .transform( + (row) -> + new AuditLogRow( + readProjection, + convertToArray( + row, + null, + wrapped.rowType().fieldGetters()))); + } + } + + private InternalRow convertToArray( + InternalRow row1, + @Nullable InternalRow row2, + InternalRow.FieldGetter[] fieldGetters) { + GenericRow row = new GenericRow(row1.getFieldCount()); + for (int i = 0; i < row1.getFieldCount(); i++) { + Object o1 = fieldGetters[i].getFieldOrNull(row1); + Object o2; + if (row2 != null) { + o2 = fieldGetters[i].getFieldOrNull(row2); + row.setField(i, new GenericArray(new Object[] {o1, o2})); + } else { + row.setField(i, new GenericArray(new Object[] {o1})); + } + } + // If no row2 provided, then follow the row1 kind. + if (row2 == null) { + row.setRowKind(row1.getRowKind()); + } else { + row.setRowKind(row2.getRowKind()); + } + return row; + } + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java index f523f20e9d200..384a2eee92c82 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java @@ -81,13 +81,12 @@ public class BranchesTable implements ReadonlyTable { private final FileIO fileIO; private final Path location; - public BranchesTable(FileStoreTable dataTable) { - this(dataTable.fileIO(), dataTable.location()); - } + private final FileStoreTable dataTable; - public BranchesTable(FileIO fileIO, Path location) { - this.fileIO = fileIO; - this.location = location; + public BranchesTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.dataTable = dataTable; } @Override @@ -117,7 +116,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new BranchesTable(fileIO, location); + return new BranchesTable(dataTable.copy(dynamicOptions)); } private class BranchesScan extends ReadOnceTableScan { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java index ff40c9502eb78..31cecbfb15c21 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java @@ -33,6 +33,7 @@ import org.apache.paimon.manifest.ManifestFileMeta; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.table.DataTable; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.ReadonlyTable; @@ -145,6 +146,11 @@ public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); } + @Override + public SchemaManager schemaManager() { + return wrapped.schemaManager(); + } + @Override public TagManager tagManager() { return wrapped.tagManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java index 9f7d12961e2f1..7e4816b13510b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java @@ -74,17 +74,13 @@ public class ConsumersTable implements ReadonlyTable { private final Path location; private final String branch; - public ConsumersTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - CoreOptions.branch(dataTable.schema().options())); - } + private final FileStoreTable dataTable; - public ConsumersTable(FileIO fileIO, Path location, String branchName) { - this.fileIO = fileIO; - this.location = location; - this.branch = branchName; + public ConsumersTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.branch = CoreOptions.branch(dataTable.schema().options()); + this.dataTable = dataTable; } @Override @@ -114,7 +110,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new ConsumersTable(fileIO, location, branch); + return new ConsumersTable(dataTable.copy(dynamicOptions)); } private class ConsumersScan extends ReadOnceTableScan { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java index fc1bb2a5b1670..522335aaa6c99 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java @@ -34,6 +34,7 @@ import org.apache.paimon.manifest.ManifestFileMeta; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.table.DataTable; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.ReadonlyTable; @@ -131,6 +132,11 @@ public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); } + @Override + public SchemaManager schemaManager() { + return wrapped.schemaManager(); + } + @Override public TagManager tagManager() { return wrapped.tagManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java index 53d2078126737..6dcbb322d6d0a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/FilesTable.java @@ -143,8 +143,7 @@ public InnerTableScan newScan() { @Override public InnerTableRead newRead() { - return new FilesRead( - new SchemaManager(storeTable.fileIO(), storeTable.location()), storeTable); + return new FilesRead(storeTable.schemaManager(), storeTable); } @Override @@ -385,7 +384,9 @@ private LazyGenericRow toRow( partitionConverter.convert( dataSplit.partition()))), dataSplit::bucket, - () -> BinaryString.fromString(dataFileMeta.fileName()), + () -> + BinaryString.fromString( + dataSplit.bucketPath() + "/" + dataFileMeta.fileName()), () -> BinaryString.fromString( DataFilePathFactory.formatIdentifier( diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java index b4a3b82a2f5f7..ed20896646b2d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java @@ -18,7 +18,6 @@ package org.apache.paimon.table.system; -import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; @@ -27,7 +26,6 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.predicate.Predicate; import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.ReadonlyTable; import org.apache.paimon.table.Table; @@ -44,7 +42,6 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.Iterators; -import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.Iterator; @@ -70,19 +67,13 @@ public class OptionsTable implements ReadonlyTable { private final FileIO fileIO; private final Path location; - private final String branch; - public OptionsTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - CoreOptions.branch(dataTable.schema().options())); - } + private final FileStoreTable dataTable; - public OptionsTable(FileIO fileIO, Path location, String branchName) { - this.fileIO = fileIO; - this.location = location; - this.branch = branchName; + public OptionsTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.dataTable = dataTable; } @Override @@ -112,7 +103,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new OptionsTable(fileIO, location, branch); + return new OptionsTable(dataTable.copy(dynamicOptions)); } private class OptionsScan extends ReadOnceTableScan { @@ -182,14 +173,20 @@ public TableRead withIOManager(IOManager ioManager) { } @Override - public RecordReader createReader(Split split) throws IOException { + public RecordReader createReader(Split split) { if (!(split instanceof OptionsSplit)) { throw new IllegalArgumentException("Unsupported split: " + split.getClass()); } - Path location = ((OptionsSplit) split).location; Iterator rows = Iterators.transform( - options(fileIO, location, branch).entrySet().iterator(), this::toRow); + dataTable + .schemaManager() + .latest() + .orElseThrow(() -> new RuntimeException("Table not exists.")) + .options() + .entrySet() + .iterator(), + this::toRow); if (readType != null) { rows = Iterators.transform( @@ -207,11 +204,4 @@ private InternalRow toRow(Map.Entry option) { BinaryString.fromString(option.getValue())); } } - - private static Map options(FileIO fileIO, Path location, String branchName) { - return new SchemaManager(fileIO, location, branchName) - .latest() - .orElseThrow(() -> new RuntimeException("Table not exists.")) - .options(); - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java index 736347b9aff3e..7e0b1f1d75682 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/PartitionsTable.java @@ -178,7 +178,7 @@ public RecordReader createReader(Split split) throws IOException { throw new IllegalArgumentException("Unsupported split: " + split.getClass()); } - List partitions = fileStoreTable.newSnapshotReader().partitionEntries(); + List partitions = fileStoreTable.newScan().listPartitionEntries(); RowDataToObjectArrayConverter converter = new RowDataToObjectArrayConverter( diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java index e28ae3760534f..5308005053c8d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java @@ -26,6 +26,7 @@ import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFileMeta; import org.apache.paimon.operation.DefaultValueAssigner; +import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.table.DataTable; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.ReadonlyTable; @@ -120,7 +121,8 @@ public List primaryKeys() { public SnapshotReader newSnapshotReader() { if (wrapped.schema().primaryKeys().size() > 0) { return wrapped.newSnapshotReader() - .withLevelFilter(level -> level == coreOptions().numLevels() - 1); + .withLevelFilter(level -> level == coreOptions().numLevels() - 1) + .enableValueFilter(); } else { return wrapped.newSnapshotReader(); } @@ -164,6 +166,11 @@ public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); } + @Override + public SchemaManager schemaManager() { + return wrapped.schemaManager(); + } + @Override public TagManager tagManager() { return wrapped.tagManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java index 86e2598c609cf..3cb0ff4783e91 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java @@ -18,13 +18,11 @@ package org.apache.paimon.table.system; -import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.Timestamp; import org.apache.paimon.disk.IOManager; -import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.predicate.And; import org.apache.paimon.predicate.CompoundPredicate; @@ -61,8 +59,6 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.Iterators; -import javax.annotation.Nullable; - import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; @@ -73,7 +69,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import static org.apache.paimon.catalog.Catalog.SYSTEM_TABLE_SPLITTER; @@ -98,21 +93,13 @@ public class SchemasTable implements ReadonlyTable { new DataField(5, "comment", SerializationUtils.newStringType(true)), new DataField(6, "update_time", new TimestampType(false, 3)))); - private final FileIO fileIO; private final Path location; - private final String branch; - public SchemasTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - CoreOptions.branch(dataTable.schema().options())); - } + private final FileStoreTable dataTable; - public SchemasTable(FileIO fileIO, Path location, String branchName) { - this.fileIO = fileIO; - this.location = location; - this.branch = branchName; + public SchemasTable(FileStoreTable dataTable) { + this.location = dataTable.location(); + this.dataTable = dataTable; } @Override @@ -137,32 +124,24 @@ public InnerTableScan newScan() { @Override public InnerTableRead newRead() { - return new SchemasRead(fileIO); + return new SchemasRead(); } @Override public Table copy(Map dynamicOptions) { - return new SchemasTable(fileIO, location, branch); + return new SchemasTable(dataTable.copy(dynamicOptions)); } - private class SchemasScan extends ReadOnceTableScan { - private @Nullable LeafPredicate schemaId; + private static class SchemasScan extends ReadOnceTableScan { @Override - public InnerTableScan withFilter(Predicate predicate) { - if (predicate == null) { - return this; - } - - Map leafPredicates = - predicate.visit(LeafPredicateExtractor.INSTANCE); - schemaId = leafPredicates.get("schema_id"); - return this; + public Plan innerPlan() { + return () -> Collections.singletonList(new SchemasSplit()); } @Override - public Plan innerPlan() { - return () -> Collections.singletonList(new SchemasSplit(location, schemaId)); + public InnerTableScan withFilter(Predicate predicate) { + return this; } } @@ -171,47 +150,29 @@ private static class SchemasSplit extends SingletonSplit { private static final long serialVersionUID = 1L; - private final Path location; - - private final @Nullable LeafPredicate schemaId; - - private SchemasSplit(Path location, @Nullable LeafPredicate schemaId) { - this.location = location; - this.schemaId = schemaId; - } - + @Override public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { - return false; - } - SchemasSplit that = (SchemasSplit) o; - return Objects.equals(location, that.location) - && Objects.equals(schemaId, that.schemaId); + return o != null && getClass() == o.getClass(); } @Override public int hashCode() { - return Objects.hash(location, schemaId); + return 0; } } /** {@link TableRead} implementation for {@link SchemasTable}. */ private class SchemasRead implements InnerTableRead { - private final FileIO fileIO; private RowType readType; private Optional optionalFilterSchemaIdMax = Optional.empty(); private Optional optionalFilterSchemaIdMin = Optional.empty(); private final List schemaIds = new ArrayList<>(); - public SchemasRead(FileIO fileIO) { - this.fileIO = fileIO; - } - @Override public InnerTableRead withFilter(Predicate predicate) { if (predicate == null) { @@ -291,9 +252,7 @@ public RecordReader createReader(Split split) { if (!(split instanceof SchemasSplit)) { throw new IllegalArgumentException("Unsupported split: " + split.getClass()); } - SchemasSplit schemasSplit = (SchemasSplit) split; - Path location = schemasSplit.location; - SchemaManager manager = new SchemaManager(fileIO, location, branch); + SchemaManager manager = dataTable.schemaManager(); Collection tableSchemas; if (!schemaIds.isEmpty()) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SinkTableLineageTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SinkTableLineageTable.java deleted file mode 100644 index 71efce0704713..0000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SinkTableLineageTable.java +++ /dev/null @@ -1,64 +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.table.system; - -import org.apache.paimon.lineage.LineageMeta; -import org.apache.paimon.lineage.LineageMetaFactory; -import org.apache.paimon.options.Options; -import org.apache.paimon.table.Table; -import org.apache.paimon.table.source.InnerTableRead; - -import java.util.Map; - -/** - * This is a system table to display all the sink table lineages. - * - *

    - *  For example:
    - *     If we select * from sys.sink_table_lineage, we will get
    - *     database_name       table_name       job_name      create_time
    - *        default            test0            job1    2023-10-22 20:35:12
    - *       database1           test1            job1    2023-10-28 21:35:52
    - *          ...               ...             ...             ...
    - *     We can write sql to fetch the information we need.
    - * 
    - */ -public class SinkTableLineageTable extends TableLineageTable { - - public static final String SINK_TABLE_LINEAGE = "sink_table_lineage"; - - public SinkTableLineageTable(LineageMetaFactory lineageMetaFactory, Options options) { - super(lineageMetaFactory, options); - } - - @Override - public InnerTableRead newRead() { - return new TableLineageRead(lineageMetaFactory, options, LineageMeta::sinkTableLineages); - } - - @Override - public String name() { - return SINK_TABLE_LINEAGE; - } - - @Override - public Table copy(Map dynamicOptions) { - return new SinkTableLineageTable(lineageMetaFactory, options); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java index a958432194400..2af13ee937bd5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java @@ -18,7 +18,6 @@ package org.apache.paimon.table.system; -import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.GenericRow; @@ -111,24 +110,13 @@ public class SnapshotsTable implements ReadonlyTable { private final FileIO fileIO; private final Path location; - private final String branch; private final FileStoreTable dataTable; public SnapshotsTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - dataTable, - CoreOptions.branch(dataTable.schema().options())); - } - - public SnapshotsTable( - FileIO fileIO, Path location, FileStoreTable dataTable, String branchName) { - this.fileIO = fileIO; - this.location = location; + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); this.dataTable = dataTable; - this.branch = branchName; } @Override @@ -158,7 +146,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new SnapshotsTable(fileIO, location, dataTable.copy(dynamicOptions), branch); + return new SnapshotsTable(dataTable.copy(dynamicOptions)); } private class SnapshotsScan extends ReadOnceTableScan { @@ -298,9 +286,8 @@ public RecordReader createReader(Split split) throws IOException { if (!(split instanceof SnapshotsSplit)) { throw new IllegalArgumentException("Unsupported split: " + split.getClass()); } - SnapshotManager snapshotManager = - new SnapshotManager(fileIO, ((SnapshotsSplit) split).location, branch); + SnapshotManager snapshotManager = dataTable.snapshotManager(); Iterator snapshots; if (!snapshotIds.isEmpty()) { snapshots = snapshotManager.snapshotsWithId(snapshotIds); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SourceTableLineageTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SourceTableLineageTable.java deleted file mode 100644 index 5d9904fa66754..0000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SourceTableLineageTable.java +++ /dev/null @@ -1,64 +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.table.system; - -import org.apache.paimon.lineage.LineageMeta; -import org.apache.paimon.lineage.LineageMetaFactory; -import org.apache.paimon.options.Options; -import org.apache.paimon.table.Table; -import org.apache.paimon.table.source.InnerTableRead; - -import java.util.Map; - -/** - * This is a system table to display all the source table lineages. - * - *
    - *  For example:
    - *     If we select * from sys.source_table_lineage, we will get
    - *     database_name       table_name       job_name      create_time
    - *        default            test0            job1    2023-10-22 20:35:12
    - *       database1           test1            job1    2023-10-28 21:35:52
    - *          ...               ...             ...             ...
    - *     We can write sql to fetch the information we need.
    - * 
    - */ -public class SourceTableLineageTable extends TableLineageTable { - - public static final String SOURCE_TABLE_LINEAGE = "source_table_lineage"; - - public SourceTableLineageTable(LineageMetaFactory lineageMetaFactory, Options options) { - super(lineageMetaFactory, options); - } - - @Override - public InnerTableRead newRead() { - return new TableLineageRead(lineageMetaFactory, options, LineageMeta::sourceTableLineages); - } - - @Override - public String name() { - return SOURCE_TABLE_LINEAGE; - } - - @Override - public Table copy(Map dynamicOptions) { - return new SourceTableLineageTable(lineageMetaFactory, options); - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java index a84f41ec1a514..763e4d1216731 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SystemTableLoader.java @@ -20,7 +20,6 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; -import org.apache.paimon.lineage.LineageMetaFactory; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; @@ -37,10 +36,10 @@ import java.util.function.Function; import java.util.function.Supplier; -import static org.apache.paimon.options.CatalogOptions.LINEAGE_META; import static org.apache.paimon.table.system.AggregationFieldsTable.AGGREGATION_FIELDS; import static org.apache.paimon.table.system.AllTableOptionsTable.ALL_TABLE_OPTIONS; import static org.apache.paimon.table.system.AuditLogTable.AUDIT_LOG; +import static org.apache.paimon.table.system.BinlogTable.BINLOG; import static org.apache.paimon.table.system.BranchesTable.BRANCHES; import static org.apache.paimon.table.system.BucketsTable.BUCKETS; import static org.apache.paimon.table.system.CatalogOptionsTable.CATALOG_OPTIONS; @@ -51,12 +50,9 @@ import static org.apache.paimon.table.system.PartitionsTable.PARTITIONS; import static org.apache.paimon.table.system.ReadOptimizedTable.READ_OPTIMIZED; import static org.apache.paimon.table.system.SchemasTable.SCHEMAS; -import static org.apache.paimon.table.system.SinkTableLineageTable.SINK_TABLE_LINEAGE; import static org.apache.paimon.table.system.SnapshotsTable.SNAPSHOTS; -import static org.apache.paimon.table.system.SourceTableLineageTable.SOURCE_TABLE_LINEAGE; import static org.apache.paimon.table.system.StatisticTable.STATISTICS; import static org.apache.paimon.table.system.TagsTable.TAGS; -import static org.apache.paimon.utils.Preconditions.checkNotNull; /** Loader to load system {@link Table}s. */ public class SystemTableLoader { @@ -77,6 +73,7 @@ public class SystemTableLoader { .put(READ_OPTIMIZED, ReadOptimizedTable::new) .put(AGGREGATION_FIELDS, AggregationFieldsTable::new) .put(STATISTICS, StatisticTable::new) + .put(BINLOG, BinlogTable::new) .build(); public static final List SYSTEM_TABLES = new ArrayList<>(SYSTEM_TABLE_LOADERS.keySet()); @@ -93,38 +90,18 @@ public static Table loadGlobal( String tableName, FileIO fileIO, Supplier>> allTablePaths, - Options catalogOptions, - @Nullable LineageMetaFactory lineageMetaFactory) { + Options catalogOptions) { switch (tableName.toLowerCase()) { case ALL_TABLE_OPTIONS: return new AllTableOptionsTable(fileIO, allTablePaths.get()); case CATALOG_OPTIONS: return new CatalogOptionsTable(catalogOptions); - case SOURCE_TABLE_LINEAGE: - { - checkNotNull( - lineageMetaFactory, - String.format( - "Lineage meta should be configured for catalog with %s", - LINEAGE_META.key())); - return new SourceTableLineageTable(lineageMetaFactory, catalogOptions); - } - case SINK_TABLE_LINEAGE: - { - checkNotNull( - lineageMetaFactory, - String.format( - "Lineage meta should be configured for catalog with %s", - LINEAGE_META.key())); - return new SinkTableLineageTable(lineageMetaFactory, catalogOptions); - } default: return null; } } public static List loadGlobalTableNames() { - return Arrays.asList( - ALL_TABLE_OPTIONS, CATALOG_OPTIONS, SOURCE_TABLE_LINEAGE, SINK_TABLE_LINEAGE); + return Arrays.asList(ALL_TABLE_OPTIONS, CATALOG_OPTIONS); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/TableLineageTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/TableLineageTable.java deleted file mode 100644 index aeaf3ca3b1330..0000000000000 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/TableLineageTable.java +++ /dev/null @@ -1,168 +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.table.system; - -import org.apache.paimon.data.BinaryString; -import org.apache.paimon.data.GenericRow; -import org.apache.paimon.data.InternalRow; -import org.apache.paimon.disk.IOManager; -import org.apache.paimon.lineage.LineageMeta; -import org.apache.paimon.lineage.LineageMetaFactory; -import org.apache.paimon.lineage.TableLineageEntity; -import org.apache.paimon.options.Options; -import org.apache.paimon.predicate.Predicate; -import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.table.ReadonlyTable; -import org.apache.paimon.table.source.InnerTableRead; -import org.apache.paimon.table.source.InnerTableScan; -import org.apache.paimon.table.source.ReadOnceTableScan; -import org.apache.paimon.table.source.Split; -import org.apache.paimon.table.source.TableRead; -import org.apache.paimon.types.DataField; -import org.apache.paimon.types.RowType; -import org.apache.paimon.types.TimestampType; -import org.apache.paimon.types.VarCharType; -import org.apache.paimon.utils.IteratorRecordReader; -import org.apache.paimon.utils.ProjectedRow; - -import org.apache.paimon.shade.guava30.com.google.common.collect.Iterators; - -import javax.annotation.Nullable; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.function.BiFunction; - -import static org.apache.paimon.utils.Preconditions.checkNotNull; - -/** Base lineage table for source and sink table lineage. */ -public abstract class TableLineageTable implements ReadonlyTable { - protected final LineageMetaFactory lineageMetaFactory; - protected final Options options; - - public static final RowType TABLE_TYPE = - new RowType( - Arrays.asList( - new DataField( - 0, "database_name", new VarCharType(VarCharType.MAX_LENGTH)), - new DataField(1, "table_name", new VarCharType(VarCharType.MAX_LENGTH)), - new DataField(2, "job_name", new VarCharType(VarCharType.MAX_LENGTH)), - new DataField(3, "create_time", new TimestampType()))); - - protected TableLineageTable(LineageMetaFactory lineageMetaFactory, Options options) { - this.lineageMetaFactory = lineageMetaFactory; - this.options = options; - } - - @Override - public InnerTableScan newScan() { - return new ReadOnceTableScan() { - @Override - public InnerTableScan withFilter(Predicate predicate) { - return this; - } - - @Override - protected Plan innerPlan() { - /// TODO get the real row count for plan. - return () -> Collections.singletonList((Split) () -> 1L); - } - }; - } - - @Override - public RowType rowType() { - return TABLE_TYPE; - } - - @Override - public List primaryKeys() { - return Arrays.asList("database_name", "table_name", "job_name"); - } - - /** Table lineage read with lineage meta query. */ - protected static class TableLineageRead implements InnerTableRead { - private final LineageMetaFactory lineageMetaFactory; - private final Options options; - private final BiFunction> - tableLineageQuery; - @Nullable private Predicate predicate; - private RowType readType; - - protected TableLineageRead( - LineageMetaFactory lineageMetaFactory, - Options options, - BiFunction> - tableLineageQuery) { - this.lineageMetaFactory = lineageMetaFactory; - this.options = options; - this.tableLineageQuery = tableLineageQuery; - this.predicate = null; - } - - @Override - public InnerTableRead withFilter(Predicate predicate) { - this.predicate = predicate; - return this; - } - - @Override - public InnerTableRead withReadType(RowType readType) { - this.readType = readType; - return this; - } - - @Override - public TableRead withIOManager(IOManager ioManager) { - return this; - } - - @Override - public RecordReader createReader(Split split) throws IOException { - try (LineageMeta lineageMeta = lineageMetaFactory.create(() -> options)) { - Iterator sourceTableLineages = - tableLineageQuery.apply(lineageMeta, predicate); - return new IteratorRecordReader<>( - Iterators.transform( - sourceTableLineages, - entity -> { - checkNotNull(entity); - GenericRow row = - GenericRow.of( - BinaryString.fromString(entity.getDatabase()), - BinaryString.fromString(entity.getTable()), - BinaryString.fromString(entity.getJob()), - entity.getCreateTime()); - if (readType != null) { - return ProjectedRow.from( - readType, TableLineageTable.TABLE_TYPE) - .replaceRow(row); - } else { - return row; - } - })); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - } -} diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java index 4d1b4e22ab189..9aafdb5983fd5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java @@ -95,17 +95,13 @@ public class TagsTable implements ReadonlyTable { private final Path location; private final String branch; - public TagsTable(FileStoreTable dataTable) { - this( - dataTable.fileIO(), - dataTable.location(), - CoreOptions.branch(dataTable.schema().options())); - } + private final FileStoreTable dataTable; - public TagsTable(FileIO fileIO, Path location, String branchName) { - this.fileIO = fileIO; - this.location = location; - this.branch = branchName; + public TagsTable(FileStoreTable dataTable) { + this.fileIO = dataTable.fileIO(); + this.location = dataTable.location(); + this.branch = CoreOptions.branch(dataTable.schema().options()); + this.dataTable = dataTable; } @Override @@ -135,7 +131,7 @@ public InnerTableRead newRead() { @Override public Table copy(Map dynamicOptions) { - return new TagsTable(fileIO, location, branch); + return new TagsTable(dataTable.copy(dynamicOptions)); } private class TagsScan extends ReadOnceTableScan { diff --git a/paimon-core/src/main/java/org/apache/paimon/tag/Tag.java b/paimon-core/src/main/java/org/apache/paimon/tag/Tag.java index f1ac879d33a79..53641a2eb69f0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/tag/Tag.java +++ b/paimon-core/src/main/java/org/apache/paimon/tag/Tag.java @@ -33,6 +33,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.io.UncheckedIOException; import java.time.Duration; import java.time.LocalDateTime; import java.util.Map; @@ -113,29 +114,6 @@ public String toJson() { return JsonSerdeUtil.toJson(this); } - public static Tag fromJson(String json) { - return JsonSerdeUtil.fromJson(json, Tag.class); - } - - public static Tag fromPath(FileIO fileIO, Path path) { - try { - String json = fileIO.readFileUtf8(path); - return Tag.fromJson(json); - } catch (IOException e) { - throw new RuntimeException("Fails to read tag from path " + path, e); - } - } - - @Nullable - public static Tag safelyFromPath(FileIO fileIO, Path path) throws IOException { - try { - String json = fileIO.readFileUtf8(path); - return Tag.fromJson(json); - } catch (FileNotFoundException e) { - return null; - } - } - public static Tag fromSnapshotAndTagTtl( Snapshot snapshot, Duration tagTimeRetained, LocalDateTime tagCreateTime) { return new Tag( @@ -201,4 +179,28 @@ public boolean equals(Object o) { return Objects.equals(tagCreateTime, that.tagCreateTime) && Objects.equals(tagTimeRetained, that.tagTimeRetained); } + + // =================== Utils for reading ========================= + + public static Tag fromJson(String json) { + return JsonSerdeUtil.fromJson(json, Tag.class); + } + + public static Tag fromPath(FileIO fileIO, Path path) { + try { + return tryFromPath(fileIO, path); + } catch (FileNotFoundException e) { + throw new RuntimeException(e.getMessage(), e); + } + } + + public static Tag tryFromPath(FileIO fileIO, Path path) throws FileNotFoundException { + try { + return fromJson(fileIO.readFileUtf8(path)); + } catch (FileNotFoundException e) { + throw e; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java b/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java index 58241033f5fb0..3989786bd2777 100644 --- a/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java +++ b/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoCreation.java @@ -150,22 +150,28 @@ public void run() { private void tryToCreateTags(Snapshot snapshot) { Optional timeOptional = timeExtractor.extract(snapshot.timeMillis(), snapshot.watermark()); + LOG.info("Starting to create a tag for snapshot {}.", snapshot.id()); if (!timeOptional.isPresent()) { return; } LocalDateTime time = timeOptional.get(); + LOG.info("The time of snapshot {} is {}.", snapshot.id(), time); + LOG.info("The next tag time is {}.", nextTag); if (nextTag == null || isAfterOrEqual(time.minus(delay), periodHandler.nextTagTime(nextTag))) { LocalDateTime thisTag = periodHandler.normalizeToPreviousTag(time); + LOG.info("Create tag for snapshot {} with time {}.", snapshot.id(), thisTag); if (automaticCompletion && nextTag != null) { thisTag = nextTag; } String tagName = periodHandler.timeToTag(thisTag); + LOG.info("The tag name is {}.", tagName); if (!tagManager.tagExists(tagName)) { tagManager.createTag(snapshot, tagName, defaultTimeRetained, callbacks); } nextTag = periodHandler.nextTagTime(thisTag); + LOG.info("The next tag time after this is {}.", nextTag); if (numRetainedMax != null) { // only handle auto-created tags here diff --git a/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoManager.java b/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoManager.java index 1ed1b3f2d4a27..817c20af46128 100644 --- a/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/tag/TagAutoManager.java @@ -52,6 +52,7 @@ public static TagAutoManager create( TagManager tagManager, TagDeletion tagDeletion, List callbacks) { + TagTimeExtractor extractor = TagTimeExtractor.createForAutoTag(options); return new TagAutoManager( diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java b/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java index bc353bb10d163..2ea5f542f4e52 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java @@ -23,6 +23,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.tag.Tag; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -178,7 +179,7 @@ public void fastForward(String branchName) { List deleteSchemaPaths = schemaManager.schemaPaths(id -> id >= earliestSchemaId); List deleteTagPaths = tagManager.tagPaths( - path -> Snapshot.fromPath(fileIO, path).id() >= earliestSnapshotId); + path -> Tag.fromPath(fileIO, path).id() >= earliestSnapshotId); List deletePaths = Stream.of(deleteSnapshotPaths, deleteSchemaPaths, deleteTagPaths) @@ -201,6 +202,7 @@ public void fastForward(String branchName) { tagManager.copyWithBranch(branchName).tagDirectory(), tagManager.tagDirectory(), true); + snapshotManager.invalidateCache(); } catch (IOException e) { throw new RuntimeException( String.format( diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java index fcdc4634d74ab..8896ec3286802 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java @@ -24,6 +24,7 @@ import org.apache.paimon.io.DataFilePathFactory; import org.apache.paimon.types.RowType; +import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; import java.util.List; @@ -46,6 +47,8 @@ public class FileStorePathFactory { private final boolean fileSuffixIncludeCompression; private final String fileCompression; + @Nullable private final String dataFilePathDirectory; + private final AtomicInteger manifestFileCount; private final AtomicInteger manifestListCount; private final AtomicInteger indexManifestCount; @@ -61,8 +64,10 @@ public FileStorePathFactory( String changelogFilePrefix, boolean legacyPartitionName, boolean fileSuffixIncludeCompression, - String fileCompression) { + String fileCompression, + @Nullable String dataFilePathDirectory) { this.root = root; + this.dataFilePathDirectory = dataFilePathDirectory; this.uuid = UUID.randomUUID().toString(); this.partitionComputer = @@ -125,7 +130,11 @@ public DataFilePathFactory createDataFilePathFactory(BinaryRow partition, int bu } public Path bucketPath(BinaryRow partition, int bucket) { - return new Path(root + "/" + relativePartitionAndBucketPath(partition, bucket)); + Path dataFileRoot = this.root; + if (dataFilePathDirectory != null) { + dataFileRoot = new Path(dataFileRoot, dataFilePathDirectory); + } + return new Path(dataFileRoot + "/" + relativePartitionAndBucketPath(partition, bucket)); } public Path relativePartitionAndBucketPath(BinaryRow partition, int bucket) { diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BulkFormatMapping.java b/paimon-core/src/main/java/org/apache/paimon/utils/FormatReaderMapping.java similarity index 58% rename from paimon-core/src/main/java/org/apache/paimon/utils/BulkFormatMapping.java rename to paimon-core/src/main/java/org/apache/paimon/utils/FormatReaderMapping.java index 037622f95f1e6..f6c6287f51b40 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/BulkFormatMapping.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FormatReaderMapping.java @@ -26,6 +26,7 @@ import org.apache.paimon.schema.IndexCastMapping; import org.apache.paimon.schema.SchemaEvolutionUtil; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; @@ -35,37 +36,66 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.function.Function; import static org.apache.paimon.predicate.PredicateBuilder.excludePredicateWithFields; +import static org.apache.paimon.table.SpecialFields.KEY_FIELD_ID_START; -/** Class with index mapping and bulk format. */ -public class BulkFormatMapping { +/** Class with index mapping and format reader. */ +public class FormatReaderMapping { + // Index mapping from data schema fields to table schema fields, this is used to realize paimon + // schema evolution. And it combines trimeedKeyMapping, which maps key fields to the value + // fields @Nullable private final int[] indexMapping; + // help indexMapping to cast different data type @Nullable private final CastFieldGetter[] castMapping; + // partition fields mapping, add partition fields to the read fields @Nullable private final Pair partitionPair; - private final FormatReaderFactory bulkFormat; + private final FormatReaderFactory readerFactory; private final TableSchema dataSchema; private final List dataFilters; - public BulkFormatMapping( + public FormatReaderMapping( @Nullable int[] indexMapping, @Nullable CastFieldGetter[] castMapping, + @Nullable int[] trimmedKeyMapping, @Nullable Pair partitionPair, - FormatReaderFactory bulkFormat, + FormatReaderFactory readerFactory, TableSchema dataSchema, List dataFilters) { - this.indexMapping = indexMapping; + this.indexMapping = combine(indexMapping, trimmedKeyMapping); this.castMapping = castMapping; - this.bulkFormat = bulkFormat; + this.readerFactory = readerFactory; this.partitionPair = partitionPair; this.dataSchema = dataSchema; this.dataFilters = dataFilters; } + private int[] combine(@Nullable int[] indexMapping, @Nullable int[] trimmedKeyMapping) { + if (indexMapping == null) { + return trimmedKeyMapping; + } + if (trimmedKeyMapping == null) { + return indexMapping; + } + + int[] combined = new int[indexMapping.length]; + + for (int i = 0; i < indexMapping.length; i++) { + if (indexMapping[i] < 0) { + combined[i] = indexMapping[i]; + } else { + combined[i] = trimmedKeyMapping[indexMapping[i]]; + } + } + return combined; + } + @Nullable public int[] getIndexMapping() { return indexMapping; @@ -82,7 +112,7 @@ public Pair getPartitionPair() { } public FormatReaderFactory getReaderFactory() { - return bulkFormat; + return readerFactory; } public TableSchema getDataSchema() { @@ -93,15 +123,15 @@ public List getDataFilters() { return dataFilters; } - /** Builder for {@link BulkFormatMapping}. */ - public static class BulkFormatMappingBuilder { + /** Builder for {@link FormatReaderMapping}. */ + public static class Builder { private final FileFormatDiscover formatDiscover; private final List readTableFields; private final Function> fieldsExtractor; @Nullable private final List filters; - public BulkFormatMappingBuilder( + public Builder( FileFormatDiscover formatDiscover, List readTableFields, Function> fieldsExtractor, @@ -112,44 +142,103 @@ public BulkFormatMappingBuilder( this.filters = filters; } - public BulkFormatMapping build( + /** + * There are three steps here to build {@link FormatReaderMapping}: + * + *

    1. Calculate the readDataFields, which is what we intend to read from the data schema. + * Meanwhile, generate the indexCastMapping, which is used to map the index of the + * readDataFields to the index of the data schema. + * + *

    2. Calculate the mapping to trim _KEY_ fields. For example: we want _KEY_a, _KEY_b, + * _FIELD_SEQUENCE, _ROW_KIND, a, b, c, d, e, f, g from the data, but actually we don't need + * to read _KEY_a and a, _KEY_b and b the same time, so we need to trim them. So we mapping + * it: read before: _KEY_a, _KEY_b, _FIELD_SEQUENCE, _ROW_KIND, a, b, c, d, e, f, g read + * after: a, b, _FIELD_SEQUENCE, _ROW_KIND, c, d, e, f, g and the mapping is + * [0,1,2,3,0,1,4,5,6,7,8], it converts the [read after] columns to [read before] columns. + * + *

    3. We want read much fewer fields than readDataFields, so we kick out the partition + * fields. We generate the partitionMappingAndFieldsWithoutPartitionPair which helps reduce + * the real read fields and tell us how to map it back. + */ + public FormatReaderMapping build( String formatIdentifier, TableSchema tableSchema, TableSchema dataSchema) { - List readDataFields = readDataFields(dataSchema); - + // extract the whole data fields in logic. + List allDataFields = fieldsExtractor.apply(dataSchema); + List readDataFields = readDataFields(allDataFields); // build index cast mapping IndexCastMapping indexCastMapping = SchemaEvolutionUtil.createIndexCastMapping(readTableFields, readDataFields); + // map from key fields reading to value fields reading + Pair trimmedKeyPair = trimKeyFields(readDataFields, allDataFields); + // build partition mapping and filter partition fields Pair, List> partitionMappingAndFieldsWithoutPartitionPair = - PartitionUtils.constructPartitionMapping(dataSchema, readDataFields); + PartitionUtils.constructPartitionMapping( + dataSchema, trimmedKeyPair.getRight().getFields()); Pair partitionMapping = partitionMappingAndFieldsWithoutPartitionPair.getLeft(); - // build read row type - RowType readDataRowType = + RowType readRowType = new RowType(partitionMappingAndFieldsWithoutPartitionPair.getRight()); // build read filters List readFilters = readFilters(filters, tableSchema, dataSchema); - return new BulkFormatMapping( + return new FormatReaderMapping( indexCastMapping.getIndexMapping(), indexCastMapping.getCastMapping(), + trimmedKeyPair.getLeft(), partitionMapping, formatDiscover .discover(formatIdentifier) - .createReaderFactory(readDataRowType, readFilters), + .createReaderFactory(readRowType, readFilters), dataSchema, readFilters); } - private List readDataFields(TableSchema dataSchema) { - List dataFields = fieldsExtractor.apply(dataSchema); + static Pair trimKeyFields( + List fieldsWithoutPartition, List fields) { + int[] map = new int[fieldsWithoutPartition.size()]; + List trimmedFields = new ArrayList<>(); + Map fieldMap = new HashMap<>(); + Map positionMap = new HashMap<>(); + + for (DataField field : fields) { + fieldMap.put(field.id(), field); + } + + for (int i = 0; i < fieldsWithoutPartition.size(); i++) { + DataField field = fieldsWithoutPartition.get(i); + boolean keyField = SpecialFields.isKeyField(field.name()); + int id = keyField ? field.id() - KEY_FIELD_ID_START : field.id(); + // field in data schema + DataField f = fieldMap.get(id); + + if (f != null) { + if (positionMap.containsKey(id)) { + map[i] = positionMap.get(id); + } else { + map[i] = positionMap.computeIfAbsent(id, k -> trimmedFields.size()); + // If the target field is not key field, we remain what it is, because it + // may be projected. Example: the target field is a row type, but only read + // the few fields in it. If we simply trimmedFields.add(f), we will read + // more fields than we need. + trimmedFields.add(keyField ? f : field); + } + } else { + throw new RuntimeException("Can't find field with id: " + id + " in fields."); + } + } + + return Pair.of(map, new RowType(trimmedFields)); + } + + private List readDataFields(List allDataFields) { List readDataFields = new ArrayList<>(); - for (DataField dataField : dataFields) { + for (DataField dataField : allDataFields) { readTableFields.stream() .filter(f -> f.id() == dataField.id()) .findFirst() diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ManifestReadThreadPool.java b/paimon-core/src/main/java/org/apache/paimon/utils/ManifestReadThreadPool.java index d967e778fe996..49fcfc8bd9094 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/ManifestReadThreadPool.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/ManifestReadThreadPool.java @@ -54,9 +54,9 @@ public static Iterable sequentialBatchedExecute( } /** This method aims to parallel process tasks with randomly but return values sequentially. */ - public static Iterator randomlyExecute( + public static Iterator randomlyExecuteSequentialReturn( Function> processor, List input, @Nullable Integer threadNum) { ThreadPoolExecutor executor = getExecutorService(threadNum); - return ThreadPoolUtils.randomlyExecute(executor, processor, input); + return ThreadPoolUtils.randomlyExecuteSequentialReturn(executor, processor, input); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java index ff1c76cb0a033..87d5ca3e54361 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsCache.java @@ -70,11 +70,12 @@ public List read( K key, @Nullable Long fileSize, Filter loadFilter, - Filter readFilter) + Filter readFilter, + Filter readVFilter) throws IOException { Segments segments = cache.getIfPresents(key); if (segments != null) { - return readFromSegments(segments, readFilter); + return readFromSegments(segments, readFilter, readVFilter); } else { if (LOG.isDebugEnabled()) { LOG.debug("not match cache key {}", key); @@ -85,15 +86,16 @@ public List read( if (fileSize <= cache.maxElementSize()) { segments = readSegments(key, fileSize, loadFilter); cache.put(key, segments); - return readFromSegments(segments, readFilter); + return readFromSegments(segments, readFilter, readVFilter); } else { return readFromIterator( - reader.apply(key, fileSize), projectedSerializer, readFilter); + reader.apply(key, fileSize), projectedSerializer, readFilter, readVFilter); } } } - private List readFromSegments(Segments segments, Filter readFilter) + private List readFromSegments( + Segments segments, Filter readFilter, Filter readVFilter) throws IOException { InternalRowSerializer formatSerializer = this.formatSerializer.get(); List entries = new ArrayList<>(); @@ -105,7 +107,10 @@ private List readFromSegments(Segments segments, Filter readFilt try { formatSerializer.mapFromPages(binaryRow, view); if (readFilter.test(binaryRow)) { - entries.add(projectedSerializer.fromRow(binaryRow)); + V v = projectedSerializer.fromRow(binaryRow); + if (readVFilter.test(v)) { + entries.add(v); + } } } catch (EOFException e) { return entries; diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsFile.java b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsFile.java index 3c261f4103246..b0bea8e66a82d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/ObjectsFile.java @@ -94,7 +94,8 @@ public List read(String fileName) { } public List read(String fileName, @Nullable Long fileSize) { - return read(fileName, fileSize, Filter.alwaysTrue(), Filter.alwaysTrue()); + return read( + fileName, fileSize, Filter.alwaysTrue(), Filter.alwaysTrue(), Filter.alwaysTrue()); } public List readWithIOException(String fileName) throws IOException { @@ -103,7 +104,8 @@ public List readWithIOException(String fileName) throws IOException { public List readWithIOException(String fileName, @Nullable Long fileSize) throws IOException { - return readWithIOException(fileName, fileSize, Filter.alwaysTrue(), Filter.alwaysTrue()); + return readWithIOException( + fileName, fileSize, Filter.alwaysTrue(), Filter.alwaysTrue(), Filter.alwaysTrue()); } public boolean exists(String fileName) { @@ -118,9 +120,10 @@ public List read( String fileName, @Nullable Long fileSize, Filter loadFilter, - Filter readFilter) { + Filter readFilter, + Filter readTFilter) { try { - return readWithIOException(fileName, fileSize, loadFilter, readFilter); + return readWithIOException(fileName, fileSize, loadFilter, readFilter, readTFilter); } catch (IOException e) { throw new RuntimeException("Failed to read " + fileName, e); } @@ -130,14 +133,16 @@ private List readWithIOException( String fileName, @Nullable Long fileSize, Filter loadFilter, - Filter readFilter) + Filter readFilter, + Filter readTFilter) throws IOException { Path path = pathFactory.toPath(fileName); if (cache != null) { - return cache.read(path, fileSize, loadFilter, readFilter); + return cache.read(path, fileSize, loadFilter, readFilter, readTFilter); } - return readFromIterator(createIterator(path, fileSize), serializer, readFilter); + return readFromIterator( + createIterator(path, fileSize), serializer, readFilter, readTFilter); } public String writeWithoutRolling(Collection records) { @@ -184,13 +189,17 @@ public void delete(String fileName) { public static List readFromIterator( CloseableIterator inputIterator, ObjectSerializer serializer, - Filter readFilter) { + Filter readFilter, + Filter readVFilter) { try (CloseableIterator iterator = inputIterator) { List result = new ArrayList<>(); while (iterator.hasNext()) { InternalRow row = iterator.next(); if (readFilter.test(row)) { - result.add(serializer.fromRow(row)); + V v = serializer.fromRow(row); + if (readVFilter.test(v)) { + result.add(v); + } } } return result; diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java b/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java index 5902d4c84cf59..49da83bfe48a9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java @@ -24,6 +24,8 @@ import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; +import org.apache.paimon.shade.caffeine2.com.github.benmanes.caffeine.cache.Cache; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,6 +45,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.function.BinaryOperator; import java.util.function.Consumer; @@ -74,16 +77,26 @@ public class SnapshotManager implements Serializable { private final FileIO fileIO; private final Path tablePath; private final String branch; + @Nullable private final Cache cache; public SnapshotManager(FileIO fileIO, Path tablePath) { this(fileIO, tablePath, DEFAULT_MAIN_BRANCH); } /** Specify the default branch for data writing. */ - public SnapshotManager(FileIO fileIO, Path tablePath, String branchName) { + public SnapshotManager(FileIO fileIO, Path tablePath, @Nullable String branchName) { + this(fileIO, tablePath, branchName, null); + } + + public SnapshotManager( + FileIO fileIO, + Path tablePath, + @Nullable String branchName, + @Nullable Cache cache) { this.fileIO = fileIO; this.tablePath = tablePath; this.branch = BranchManager.normalizeBranch(branchName); + this.cache = cache; } public SnapshotManager copyWithBranch(String branchName) { @@ -120,20 +133,34 @@ public Path snapshotDirectory() { return new Path(branchPath(tablePath, branch) + "/snapshot"); } + public void invalidateCache() { + if (cache != null) { + cache.invalidateAll(); + } + } + public Snapshot snapshot(long snapshotId) { - Path snapshotPath = snapshotPath(snapshotId); - return Snapshot.fromPath(fileIO, snapshotPath); + Path path = snapshotPath(snapshotId); + Snapshot snapshot = cache == null ? null : cache.getIfPresent(path); + if (snapshot == null) { + snapshot = Snapshot.fromPath(fileIO, path); + if (cache != null) { + cache.put(path, snapshot); + } + } + return snapshot; } public Snapshot tryGetSnapshot(long snapshotId) throws FileNotFoundException { - try { - Path snapshotPath = snapshotPath(snapshotId); - return Snapshot.fromJson(fileIO.readFileUtf8(snapshotPath)); - } catch (FileNotFoundException fileNotFoundException) { - throw fileNotFoundException; - } catch (IOException ioException) { - throw new RuntimeException(ioException); + Path path = snapshotPath(snapshotId); + Snapshot snapshot = cache == null ? null : cache.getIfPresent(path); + if (snapshot == null) { + snapshot = Snapshot.tryFromPath(fileIO, path); + if (cache != null) { + cache.put(path, snapshot); + } } + return snapshot; } public Changelog changelog(long snapshotId) { @@ -340,6 +367,65 @@ private Snapshot changelogOrSnapshot(long snapshotId) { return finalSnapshot; } + public @Nullable Snapshot earlierOrEqualWatermark(long watermark) { + Long earliest = earliestSnapshotId(); + Long latest = latestSnapshotId(); + // If latest == Long.MIN_VALUE don't need next binary search for watermark + // which can reduce IO cost with snapshot + if (earliest == null || latest == null || snapshot(latest).watermark() == Long.MIN_VALUE) { + return null; + } + Long earliestWatermark = null; + // find the first snapshot with watermark + if ((earliestWatermark = snapshot(earliest).watermark()) == null) { + while (earliest < latest) { + earliest++; + earliestWatermark = snapshot(earliest).watermark(); + if (earliestWatermark != null) { + break; + } + } + } + if (earliestWatermark == null) { + return null; + } + + if (earliestWatermark >= watermark) { + return snapshot(earliest); + } + Snapshot finalSnapshot = null; + + while (earliest <= latest) { + long mid = earliest + (latest - earliest) / 2; // Avoid overflow + Snapshot snapshot = snapshot(mid); + Long commitWatermark = snapshot.watermark(); + if (commitWatermark == null) { + // find the first snapshot with watermark + while (mid >= earliest) { + mid--; + commitWatermark = snapshot(mid).watermark(); + if (commitWatermark != null) { + break; + } + } + } + if (commitWatermark == null) { + earliest = mid + 1; + } else { + if (commitWatermark > watermark) { + latest = mid - 1; // Search in the left half + } else if (commitWatermark < watermark) { + earliest = mid + 1; // Search in the right half + finalSnapshot = snapshot; + } else { + finalSnapshot = snapshot; // Found the exact match + break; + } + } + } + return finalSnapshot; + } + public @Nullable Snapshot laterOrEqualWatermark(long watermark) { Long earliest = earliestSnapshotId(); Long latest = latestSnapshotId(); @@ -486,11 +572,9 @@ public List safelyGetAllSnapshots() throws IOException { collectSnapshots( path -> { try { - snapshots.add(Snapshot.fromJson(fileIO.readFileUtf8(path))); - } catch (IOException e) { - if (!(e instanceof FileNotFoundException)) { - throw new RuntimeException(e); - } + // do not pollution cache + snapshots.add(Snapshot.tryFromPath(fileIO, path)); + } catch (FileNotFoundException ignored) { } }, paths); @@ -539,15 +623,15 @@ private void collectSnapshots(Consumer pathConsumer, List paths) * Try to get non snapshot files. If any error occurred, just ignore it and return an empty * result. */ - public List tryGetNonSnapshotFiles(Predicate fileStatusFilter) { + public List> tryGetNonSnapshotFiles(Predicate fileStatusFilter) { return listPathWithFilter(snapshotDirectory(), fileStatusFilter, nonSnapshotFileFilter()); } - public List tryGetNonChangelogFiles(Predicate fileStatusFilter) { + public List> tryGetNonChangelogFiles(Predicate fileStatusFilter) { return listPathWithFilter(changelogDirectory(), fileStatusFilter, nonChangelogFileFilter()); } - private List listPathWithFilter( + private List> listPathWithFilter( Path directory, Predicate fileStatusFilter, Predicate fileFilter) { try { FileStatus[] statuses = fileIO.listStatus(directory); @@ -557,8 +641,8 @@ private List listPathWithFilter( return Arrays.stream(statuses) .filter(fileStatusFilter) - .map(FileStatus::getPath) - .filter(fileFilter) + .filter(status -> fileFilter.test(status.getPath())) + .map(status -> Pair.of(status.getPath(), status.getLen())) .collect(Collectors.toList()); } catch (IOException ignored) { return Collections.emptyList(); @@ -800,6 +884,23 @@ public void commitEarliestHint(long snapshotId) throws IOException { private void commitHint(long snapshotId, String fileName, Path dir) throws IOException { Path hintFile = new Path(dir, fileName); - fileIO.overwriteFileUtf8(hintFile, String.valueOf(snapshotId)); + int loopTime = 3; + while (loopTime-- > 0) { + try { + fileIO.overwriteFileUtf8(hintFile, String.valueOf(snapshotId)); + return; + } catch (IOException e) { + try { + Thread.sleep(ThreadLocalRandom.current().nextInt(1000) + 500); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + // throw root cause + throw new RuntimeException(e); + } + if (loopTime == 0) { + throw e; + } + } + } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/StatsCollectorFactories.java b/paimon-core/src/main/java/org/apache/paimon/utils/StatsCollectorFactories.java index de94b2e23eff5..abb1d686073fb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/StatsCollectorFactories.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/StatsCollectorFactories.java @@ -24,6 +24,7 @@ import org.apache.paimon.statistics.TruncateSimpleColStatsCollector; import org.apache.paimon.table.SpecialFields; +import java.util.Collections; import java.util.List; import static org.apache.paimon.CoreOptions.FIELDS_PREFIX; @@ -35,6 +36,11 @@ public class StatsCollectorFactories { public static SimpleColStatsCollector.Factory[] createStatsFactories( CoreOptions options, List fields) { + return createStatsFactories(options, fields, Collections.emptyList()); + } + + public static SimpleColStatsCollector.Factory[] createStatsFactories( + CoreOptions options, List fields, List keyNames) { Options cfg = options.toConfiguration(); SimpleColStatsCollector.Factory[] modes = new SimpleColStatsCollector.Factory[fields.size()]; @@ -47,7 +53,11 @@ public static SimpleColStatsCollector.Factory[] createStatsFactories( .noDefaultValue()); if (fieldMode != null) { modes[i] = SimpleColStatsCollector.from(fieldMode); - } else if (SpecialFields.isSystemField(field)) { + } else if (SpecialFields.isSystemField(field) + || + // If we config DATA_FILE_THIN_MODE to true, we need to maintain the + // stats for key fields. + keyNames.contains(SpecialFields.KEY_FIELD_PREFIX + field)) { modes[i] = () -> new TruncateSimpleColStatsCollector(128); } else { modes[i] = SimpleColStatsCollector.from(cfg.get(CoreOptions.METADATA_STATS_MODE)); diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java b/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java index 65963aafdf6ba..4019395d8d65a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java @@ -23,7 +23,7 @@ import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; -import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.manifest.ExpireFileEntry; import org.apache.paimon.operation.TagDeletion; import org.apache.paimon.table.sink.TagCallback; import org.apache.paimon.tag.Tag; @@ -255,7 +255,7 @@ private void doClean( skippedSnapshots.add(right); // delete data files and empty directories - Predicate dataFileSkipper = null; + Predicate dataFileSkipper = null; boolean success = true; try { dataFileSkipper = tagDeletion.dataFileSkipper(skippedSnapshots); @@ -353,7 +353,7 @@ public SortedMap> tags(Predicate filter) { // If the tag file is not found, it might be deleted by // other processes, so just skip this tag try { - Snapshot snapshot = Snapshot.fromJson(fileIO.readFileUtf8(path)); + Snapshot snapshot = Tag.tryFromPath(fileIO, path).trimToSnapshot(); tags.computeIfAbsent(snapshot, s -> new ArrayList<>()).add(tagName); } catch (FileNotFoundException ignored) { } @@ -371,9 +371,9 @@ public List> tagObjects() { List> tags = new ArrayList<>(); for (Path path : paths) { String tagName = path.getName().substring(TAG_PREFIX.length()); - Tag tag = Tag.safelyFromPath(fileIO, path); - if (tag != null) { - tags.add(Pair.of(tag, tagName)); + try { + tags.add(Pair.of(Tag.tryFromPath(fileIO, path), tagName)); + } catch (FileNotFoundException ignored) { } } return tags; diff --git a/paimon-core/src/main/resources/META-INF/NOTICE b/paimon-core/src/main/resources/META-INF/NOTICE new file mode 100644 index 0000000000000..dd2479b1d6e7c --- /dev/null +++ b/paimon-core/src/main/resources/META-INF/NOTICE @@ -0,0 +1,8 @@ +paimon-core +Copyright 2023-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) +- com.squareup.okhttp3:okhttp:4.12.0 \ No newline at end of file diff --git a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory index ac6cc98fed6b7..6416edd720f8d 100644 --- a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -36,3 +36,6 @@ org.apache.paimon.mergetree.compact.aggregate.factory.FieldRoaringBitmap32AggFac org.apache.paimon.mergetree.compact.aggregate.factory.FieldRoaringBitmap64AggFactory org.apache.paimon.mergetree.compact.aggregate.factory.FieldSumAggFactory org.apache.paimon.mergetree.compact.aggregate.factory.FieldThetaSketchAggFactory +org.apache.paimon.rest.RESTCatalogFactory +org.apache.paimon.rest.auth.BearTokenCredentialsProviderFactory +org.apache.paimon.rest.auth.BearTokenFileCredentialsProviderFactory diff --git a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java index 303879337780f..0d8ea5f4a49a5 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java @@ -26,10 +26,12 @@ import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.IndexIncrement; +import org.apache.paimon.manifest.FileEntry; import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.FileSource; import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.manifest.ManifestFileMeta; import org.apache.paimon.manifest.ManifestList; import org.apache.paimon.memory.HeapMemorySegmentPool; @@ -38,7 +40,6 @@ import org.apache.paimon.operation.AbstractFileStoreWrite; import org.apache.paimon.operation.FileStoreCommit; import org.apache.paimon.operation.FileStoreCommitImpl; -import org.apache.paimon.operation.FileStoreScan; import org.apache.paimon.operation.SplitRead; import org.apache.paimon.options.ExpireConfig; import org.apache.paimon.options.MemorySize; @@ -222,7 +223,8 @@ public List commitData( null, Collections.emptyList(), (commit, committable) -> { - logOffsets.forEach(committable::addLogOffset); + logOffsets.forEach( + (bucket, offset) -> committable.addLogOffset(bucket, offset, false)); commit.commit(committable, Collections.emptyMap()); }); } @@ -560,29 +562,41 @@ public Set getFilesInUse(long snapshotId) { return getFilesInUse( snapshotId, snapshotManager(), - newScan(), fileIO, pathFactory(), - manifestListFactory().create()); + manifestListFactory().create(), + manifestFileFactory().create()); } public static Set getFilesInUse( long snapshotId, SnapshotManager snapshotManager, - FileStoreScan scan, FileIO fileIO, FileStorePathFactory pathFactory, - ManifestList manifestList) { + ManifestList manifestList, + ManifestFile manifestFile) { Set result = new HashSet<>(); if (snapshotManager.snapshotExists(snapshotId)) { - result.addAll( + Set files = getSnapshotFileInUse( - snapshotId, snapshotManager, scan, fileIO, pathFactory, manifestList)); + snapshotId, + snapshotManager, + fileIO, + pathFactory, + manifestList, + manifestFile); + result.addAll(files); } else if (snapshotManager.longLivedChangelogExists(snapshotId)) { - result.addAll( + Set files = getChangelogFileInUse( - snapshotId, snapshotManager, scan, fileIO, pathFactory, manifestList)); + snapshotId, + snapshotManager, + fileIO, + pathFactory, + manifestList, + manifestFile); + result.addAll(files); } else { throw new RuntimeException( String.format("The snapshot %s does not exist.", snapshotId)); @@ -594,10 +608,10 @@ public static Set getFilesInUse( private static Set getSnapshotFileInUse( long snapshotId, SnapshotManager snapshotManager, - FileStoreScan scan, FileIO fileIO, FileStorePathFactory pathFactory, - ManifestList manifestList) { + ManifestList manifestList, + ManifestFile manifestFile) { Set result = new HashSet<>(); SchemaManager schemaManager = new SchemaManager(fileIO, snapshotManager.tablePath()); CoreOptions options = new CoreOptions(schemaManager.latest().get().options()); @@ -624,7 +638,11 @@ private static Set getSnapshotFileInUse( manifests.forEach(m -> result.add(pathFactory.toManifestFilePath(m.fileName()))); // data file - List entries = scan.withManifestList(manifests).plan().files(); + List entries = + manifests.stream() + .flatMap(m -> manifestFile.read(m.fileName()).stream()) + .collect(Collectors.toList()); + entries = new ArrayList<>(FileEntry.mergeEntries(entries)); for (ManifestEntry entry : entries) { result.add( new Path( @@ -640,7 +658,9 @@ private static Set getSnapshotFileInUse( // use list. if (changelogDecoupled && !produceChangelog) { entries = - scan.withManifestList(manifestList.readDeltaManifests(snapshot)).plan().files(); + manifestList.readDeltaManifests(snapshot).stream() + .flatMap(m -> manifestFile.read(m.fileName()).stream()) + .collect(Collectors.toList()); for (ManifestEntry entry : entries) { // append delete file are delayed to delete if (entry.kind() == FileKind.DELETE @@ -660,15 +680,13 @@ private static Set getSnapshotFileInUse( private static Set getChangelogFileInUse( long changelogId, SnapshotManager snapshotManager, - FileStoreScan scan, FileIO fileIO, FileStorePathFactory pathFactory, - ManifestList manifestList) { + ManifestList manifestList, + ManifestFile manifestFile) { Set result = new HashSet<>(); SchemaManager schemaManager = new SchemaManager(fileIO, snapshotManager.tablePath()); CoreOptions options = new CoreOptions(schemaManager.latest().get().options()); - boolean produceChangelog = - options.changelogProducer() != CoreOptions.ChangelogProducer.NONE; Path changelogPath = snapshotManager.longLivedChangelogPath(changelogId); Changelog changelog = Changelog.fromPath(fileIO, changelogPath); @@ -676,35 +694,27 @@ private static Set getChangelogFileInUse( // changelog file result.add(changelogPath); - // manifest lists - if (!produceChangelog) { - result.add(pathFactory.toManifestListPath(changelog.baseManifestList())); - result.add(pathFactory.toManifestListPath(changelog.deltaManifestList())); - } - if (changelog.changelogManifestList() != null) { - result.add(pathFactory.toManifestListPath(changelog.changelogManifestList())); - } - - // manifests - List manifests = - new ArrayList<>(manifestList.readChangelogManifests(changelog)); - if (!produceChangelog) { - manifests.addAll(manifestList.readDataManifests(changelog)); - } - - manifests.forEach(m -> result.add(pathFactory.toManifestFilePath(m.fileName()))); - // data file // not all manifests contains useful data file // (1) produceChangelog = 'true': data file in changelog manifests // (2) produceChangelog = 'false': 'APPEND' data file in delta manifests // delta file - if (!produceChangelog) { - for (ManifestEntry entry : - scan.withManifestList(manifestList.readDeltaManifests(changelog)) - .plan() - .files()) { + if (options.changelogProducer() == CoreOptions.ChangelogProducer.NONE) { + // TODO why we need to keep base manifests? + result.add(pathFactory.toManifestListPath(changelog.baseManifestList())); + manifestList + .readDataManifests(changelog) + .forEach(m -> result.add(pathFactory.toManifestFilePath(m.fileName()))); + + result.add(pathFactory.toManifestListPath(changelog.deltaManifestList())); + List manifests = manifestList.readDeltaManifests(changelog); + manifests.forEach(m -> result.add(pathFactory.toManifestFilePath(m.fileName()))); + List files = + manifests.stream() + .flatMap(m -> manifestFile.read(m.fileName()).stream()) + .collect(Collectors.toList()); + for (ManifestEntry entry : files) { if (entry.file().fileSource().orElse(FileSource.APPEND) == FileSource.APPEND) { result.add( new Path( @@ -712,12 +722,15 @@ private static Set getChangelogFileInUse( entry.file().fileName())); } } - } else { - // changelog - for (ManifestEntry entry : - scan.withManifestList(manifestList.readChangelogManifests(changelog)) - .plan() - .files()) { + } else if (changelog.changelogManifestList() != null) { + result.add(pathFactory.toManifestListPath(changelog.changelogManifestList())); + List manifests = manifestList.readChangelogManifests(changelog); + manifests.forEach(m -> result.add(pathFactory.toManifestFilePath(m.fileName()))); + List files = + manifests.stream() + .flatMap(m -> manifestFile.read(m.fileName()).stream()) + .collect(Collectors.toList()); + for (ManifestEntry entry : files) { result.add( new Path( pathFactory.bucketPath(entry.partition(), entry.bucket()), diff --git a/paimon-core/src/test/java/org/apache/paimon/TestKeyValueGenerator.java b/paimon-core/src/test/java/org/apache/paimon/TestKeyValueGenerator.java index 657c791351a4a..587204cd76160 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestKeyValueGenerator.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestKeyValueGenerator.java @@ -95,10 +95,12 @@ public class TestKeyValueGenerator { public static final RowType KEY_TYPE = RowType.of( new DataField( - 2 + SpecialFields.KEY_FIELD_ID_START, "key_shopId", new IntType(false)), + 2 + SpecialFields.KEY_FIELD_ID_START, + SpecialFields.KEY_FIELD_PREFIX + "shopId", + new IntType(false)), new DataField( 3 + SpecialFields.KEY_FIELD_ID_START, - "key_orderId", + SpecialFields.KEY_FIELD_PREFIX + "orderId", new BigIntType(false))); public static final InternalRowSerializer DEFAULT_ROW_SERIALIZER = @@ -281,7 +283,7 @@ public BinaryRow getPartition(KeyValue kv) { public static List getPrimaryKeys(GeneratorMode mode) { List trimmedPk = KEY_TYPE.getFieldNames().stream() - .map(f -> f.replaceFirst("key_", "")) + .map(f -> f.replaceFirst(SpecialFields.KEY_FIELD_PREFIX, "")) .collect(Collectors.toList()); if (mode != NON_PARTITIONED) { trimmedPk = new ArrayList<>(trimmedPk); @@ -394,7 +396,7 @@ public List keyFields(TableSchema schema) { f -> new DataField( f.id() + SpecialFields.KEY_FIELD_ID_START, - "key_" + f.name(), + SpecialFields.KEY_FIELD_PREFIX + f.name(), f.type(), f.description())) .collect(Collectors.toList()); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinatorTest.java b/paimon-core/src/test/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinatorTest.java index 95826c195ec72..9bb461ffe151c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinatorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/UnawareAppendTableCompactionCoordinatorTest.java @@ -29,6 +29,7 @@ import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.source.EndOfScanException; import org.apache.paimon.types.DataTypes; import org.junit.jupiter.api.BeforeEach; @@ -43,7 +44,9 @@ import static org.apache.paimon.mergetree.compact.MergeTreeCompactManagerTest.row; import static org.apache.paimon.stats.StatsTestUtils.newSimpleStats; +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; /** Tests for {@link UnawareAppendTableCompactionCoordinator}. */ public class UnawareAppendTableCompactionCoordinatorTest { @@ -135,6 +138,14 @@ public void testAgeGrowUp() { .isEqualTo(0); } + @Test + public void testBatchScanEmptyTable() { + compactionCoordinator = + new UnawareAppendTableCompactionCoordinator(appendOnlyFileStoreTable, false); + assertThatThrownBy(() -> compactionCoordinator.scan()) + .satisfies(anyCauseMatches(EndOfScanException.class)); + } + private void assertTasks(List files, int taskNum) { compactionCoordinator.notifyNewFiles(partition, files); List tasks = compactionCoordinator.compactPlan(); diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java index 65ed5ce0b7bf5..4792e33c932b7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CachingCatalogTest.java @@ -18,6 +18,7 @@ package org.apache.paimon.catalog; +import org.apache.paimon.Snapshot; import org.apache.paimon.data.GenericRow; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; @@ -47,10 +48,8 @@ import java.io.FileNotFoundException; import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -100,14 +99,49 @@ public void testInvalidateSystemTablesIfBaseTableIsModified() throws Exception { @Test public void testInvalidateSysTablesIfBaseTableIsDropped() throws Exception { - Catalog catalog = new CachingCatalog(this.catalog); + TestableCachingCatalog catalog = + new TestableCachingCatalog(this.catalog, EXPIRATION_TTL, ticker); Identifier tableIdent = new Identifier("db", "tbl"); catalog.createTable(new Identifier("db", "tbl"), DEFAULT_TABLE_SCHEMA, false); Identifier sysIdent = new Identifier("db", "tbl$files"); + // get system table will only cache the origin table catalog.getTable(sysIdent); + assertThat(catalog.tableCache.asMap()).containsKey(tableIdent); + assertThat(catalog.tableCache.asMap()).doesNotContainKey(sysIdent); + // test case sensitivity + Identifier sysIdent1 = new Identifier("db", "tbl$SNAPSHOTS"); + catalog.getTable(sysIdent1); + assertThat(catalog.tableCache.asMap()).doesNotContainKey(sysIdent1); + catalog.dropTable(tableIdent, false); + assertThat(catalog.tableCache.asMap()).doesNotContainKey(tableIdent); assertThatThrownBy(() -> catalog.getTable(sysIdent)) .hasMessage("Table db.tbl does not exist."); + assertThatThrownBy(() -> catalog.getTable(sysIdent1)) + .hasMessage("Table db.tbl does not exist."); + } + + @Test + public void testInvalidateBranchIfBaseTableIsDropped() throws Exception { + TestableCachingCatalog catalog = + new TestableCachingCatalog(this.catalog, EXPIRATION_TTL, ticker); + Identifier tableIdent = new Identifier("db", "tbl"); + catalog.createTable(new Identifier("db", "tbl"), DEFAULT_TABLE_SCHEMA, false); + catalog.getTable(tableIdent).createBranch("b1"); + + Identifier branchIdent = new Identifier("db", "tbl$branch_b1"); + Identifier branchSysIdent = new Identifier("db", "tbl$branch_b1$FILES"); + // get system table will only cache the origin table + catalog.getTable(branchSysIdent); + assertThat(catalog.tableCache.asMap()).containsKey(branchIdent); + assertThat(catalog.tableCache.asMap()).doesNotContainKey(branchSysIdent); + + catalog.dropTable(tableIdent, false); + assertThat(catalog.tableCache.asMap()).doesNotContainKey(branchIdent); + assertThatThrownBy(() -> catalog.getTable(branchIdent)) + .hasMessage("Table db.tbl$branch_b1 does not exist."); + assertThatThrownBy(() -> catalog.getTable(branchSysIdent)) + .hasMessage("Table db.tbl$branch_b1 does not exist."); } @Test @@ -174,59 +208,6 @@ public void testCatalogExpirationTtlRefreshesAfterAccessViaCatalog() throws Exce assertThat(catalog.remainingAgeFor(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); } - @Test - public void testCacheExpirationEagerlyRemovesSysTables() throws Exception { - TestableCachingCatalog catalog = - new TestableCachingCatalog(this.catalog, EXPIRATION_TTL, ticker); - - Identifier tableIdent = new Identifier("db", "tbl"); - catalog.createTable(tableIdent, DEFAULT_TABLE_SCHEMA, false); - Table table = catalog.getTable(tableIdent); - assertThat(catalog.tableCache().asMap()).containsKey(tableIdent); - assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(Duration.ZERO); - - ticker.advance(HALF_OF_EXPIRATION); - assertThat(catalog.tableCache().asMap()).containsKey(tableIdent); - assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); - - for (Identifier sysTable : sysTables(tableIdent)) { - catalog.getTable(sysTable); - } - assertThat(catalog.tableCache().asMap()).containsKeys(sysTables(tableIdent)); - assertThat(Arrays.stream(sysTables(tableIdent)).map(catalog::ageOf)) - .isNotEmpty() - .allMatch(age -> age.isPresent() && age.get().equals(Duration.ZERO)); - - assertThat(catalog.remainingAgeFor(tableIdent)) - .as("Loading a non-cached sys table should refresh the main table's age") - .isEqualTo(Optional.of(EXPIRATION_TTL)); - - // Move time forward and access already cached sys tables. - ticker.advance(HALF_OF_EXPIRATION); - for (Identifier sysTable : sysTables(tableIdent)) { - catalog.getTable(sysTable); - } - assertThat(Arrays.stream(sysTables(tableIdent)).map(catalog::ageOf)) - .isNotEmpty() - .allMatch(age -> age.isPresent() && age.get().equals(Duration.ZERO)); - - assertThat(catalog.remainingAgeFor(tableIdent)) - .as("Accessing a cached sys table should not affect the main table's age") - .isEqualTo(Optional.of(HALF_OF_EXPIRATION)); - - // Move time forward so the data table drops. - ticker.advance(HALF_OF_EXPIRATION); - assertThat(catalog.tableCache().asMap()).doesNotContainKey(tableIdent); - - Arrays.stream(sysTables(tableIdent)) - .forEach( - sysTable -> - assertThat(catalog.tableCache().asMap()) - .as( - "When a data table expires, its sys tables should expire regardless of age") - .doesNotContainKeys(sysTable)); - } - @Test public void testPartitionCache() throws Exception { TestableCachingCatalog catalog = @@ -332,6 +313,31 @@ public static Identifier[] sysTables(Identifier tableIdent) { .toArray(Identifier[]::new); } + @Test + public void testSnapshotCache() throws Exception { + TestableCachingCatalog wrappedCatalog = + new TestableCachingCatalog(this.catalog, EXPIRATION_TTL, ticker); + Identifier tableIdent = new Identifier("db", "tbl"); + wrappedCatalog.createTable(tableIdent, DEFAULT_TABLE_SCHEMA, false); + Table table = wrappedCatalog.getTable(tableIdent); + + // write + BatchWriteBuilder writeBuilder = table.newBatchWriteBuilder(); + try (BatchTableWrite write = writeBuilder.newWrite(); + BatchTableCommit commit = writeBuilder.newCommit()) { + write.write(GenericRow.of(1, fromString("1"), fromString("1"))); + write.write(GenericRow.of(2, fromString("2"), fromString("2"))); + commit.commit(write.prepareCommit()); + } + + Snapshot snapshot = table.snapshot(1); + assertThat(snapshot).isSameAs(table.snapshot(1)); + + // copy + Snapshot copied = table.copy(Collections.singletonMap("a", "b")).snapshot(1); + assertThat(copied).isSameAs(snapshot); + } + @Test public void testManifestCache() throws Exception { innerTestManifestCache(Long.MAX_VALUE); @@ -346,7 +352,8 @@ private void innerTestManifestCache(long manifestCacheThreshold) throws Exceptio Duration.ofSeconds(10), MemorySize.ofMebiBytes(1), manifestCacheThreshold, - 0L); + 0L, + 10); Identifier tableIdent = new Identifier("db", "tbl"); catalog.dropTable(tableIdent, true); catalog.createTable(tableIdent, DEFAULT_TABLE_SCHEMA, false); @@ -363,7 +370,8 @@ private void innerTestManifestCache(long manifestCacheThreshold) throws Exceptio // repeat read for (int i = 0; i < 5; i++) { - table = catalog.getTable(tableIdent); + // test copy too + table = catalog.getTable(tableIdent).copy(Collections.singletonMap("a", "b")); ReadBuilder readBuilder = table.newReadBuilder(); TableScan scan = readBuilder.newScan(); TableRead read = readBuilder.newRead(); diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java index f130920a7c0e4..98a9b92c5c38c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java @@ -948,4 +948,16 @@ public void testFormatTable() throws Exception { .isInstanceOf(Catalog.TableNotExistException.class); assertThat(catalog.getTable(newIdentifier)).isInstanceOf(FormatTable.class); } + + @Test + public void testTableUUID() throws Exception { + catalog.createDatabase("test_db", false); + Identifier identifier = Identifier.create("test_db", "test_table"); + catalog.createTable(identifier, DEFAULT_TABLE_SCHEMA, false); + Table table = catalog.getTable(identifier); + String uuid = table.uuid(); + assertThat(uuid).startsWith(identifier.getFullName() + "."); + assertThat(Long.parseLong(uuid.substring((identifier.getFullName() + ".").length()))) + .isGreaterThan(0); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java index 35a5c06ffbc68..303a9d8733d4b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/FileSystemCatalogTest.java @@ -19,8 +19,15 @@ package org.apache.paimon.catalog; import org.apache.paimon.fs.Path; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.types.DataTypes; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; /** Tests for {@link FileSystemCatalog}. */ public class FileSystemCatalogTest extends CatalogTestBase { @@ -28,6 +35,36 @@ public class FileSystemCatalogTest extends CatalogTestBase { @BeforeEach public void setUp() throws Exception { super.setUp(); - catalog = new FileSystemCatalog(fileIO, new Path(warehouse)); + Options catalogOptions = new Options(); + catalogOptions.set(CatalogOptions.ALLOW_UPPER_CASE, false); + catalog = new FileSystemCatalog(fileIO, new Path(warehouse), catalogOptions); + } + + @Test + public void testCreateTableAllowUpperCase() throws Exception { + catalog.createDatabase("test_db", false); + Identifier identifier = Identifier.create("test_db", "new_table"); + Schema schema = + Schema.newBuilder() + .column("Pk1", DataTypes.INT()) + .column("pk2", DataTypes.STRING()) + .column("pk3", DataTypes.STRING()) + .column( + "Col1", + DataTypes.ROW( + DataTypes.STRING(), + DataTypes.BIGINT(), + DataTypes.TIMESTAMP(), + DataTypes.ARRAY(DataTypes.STRING()))) + .column("col2", DataTypes.MAP(DataTypes.STRING(), DataTypes.BIGINT())) + .column("col3", DataTypes.ARRAY(DataTypes.ROW(DataTypes.STRING()))) + .partitionKeys("Pk1", "pk2") + .primaryKey("Pk1", "pk2", "pk3") + .build(); + + // Create table throws Exception if using uppercase when 'allow-upper-case' is false + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> catalog.createTable(identifier, schema, false)) + .withMessage("Field name [Pk1, Col1] cannot contain upper case in the catalog."); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/TestableCachingCatalog.java b/paimon-core/src/test/java/org/apache/paimon/catalog/TestableCachingCatalog.java index 4c70a0232c44f..1d4a9b0e8a58d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/catalog/TestableCachingCatalog.java +++ b/paimon-core/src/test/java/org/apache/paimon/catalog/TestableCachingCatalog.java @@ -38,7 +38,14 @@ public class TestableCachingCatalog extends CachingCatalog { private final Duration cacheExpirationInterval; public TestableCachingCatalog(Catalog catalog, Duration expirationInterval, Ticker ticker) { - super(catalog, expirationInterval, MemorySize.ZERO, Long.MAX_VALUE, Long.MAX_VALUE, ticker); + super( + catalog, + expirationInterval, + MemorySize.ZERO, + Long.MAX_VALUE, + Long.MAX_VALUE, + Integer.MAX_VALUE, + ticker); this.cacheExpirationInterval = expirationInterval; } diff --git a/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java b/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java index 6c674352b8d3c..a52819c805150 100644 --- a/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/deletionvectors/append/AppendDeletionFileMaintainerTest.java @@ -23,12 +23,12 @@ import org.apache.paimon.deletionvectors.DeletionVector; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.IndexManifestEntry; import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.table.source.DeletionFile; -import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.PathFactory; import org.junit.jupiter.api.Test; @@ -94,7 +94,7 @@ public void test() throws Exception { assertThat(res.size()).isEqualTo(3); IndexManifestEntry entry = res.stream().filter(file -> file.kind() == FileKind.ADD).findAny().get(); - assertThat(entry.indexFile().deletionVectorsRanges().containsKey("f2")).isTrue(); + assertThat(entry.indexFile().deletionVectorMetas().containsKey("f2")).isTrue(); entry = res.stream() .filter(file -> file.kind() == FileKind.DELETE) @@ -117,14 +117,15 @@ private Map createDeletionFileMapFromIndexFileMetas( PathFactory indexPathFactory, List fileMetas) { Map dataFileToDeletionFiles = new HashMap<>(); for (IndexFileMeta indexFileMeta : fileMetas) { - for (Map.Entry> range : - indexFileMeta.deletionVectorsRanges().entrySet()) { + for (Map.Entry dvMeta : + indexFileMeta.deletionVectorMetas().entrySet()) { dataFileToDeletionFiles.put( - range.getKey(), + dvMeta.getKey(), new DeletionFile( indexPathFactory.toPath(indexFileMeta.fileName()).toString(), - range.getValue().getLeft(), - range.getValue().getRight())); + dvMeta.getValue().offset(), + dvMeta.getValue().length(), + dvMeta.getValue().cardinality())); } } return dataFileToDeletionFiles; diff --git a/paimon-core/src/test/java/org/apache/paimon/format/ThinModeReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/format/ThinModeReadWriteTest.java new file mode 100644 index 0000000000000..3f8015b33b2dd --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/format/ThinModeReadWriteTest.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.format; + +import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.manifest.FileKind; +import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.Table; +import org.apache.paimon.table.TableTestBase; +import org.apache.paimon.types.DataTypes; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; + +/** This class test the compatibility and effectiveness of storage thin mode. */ +public class ThinModeReadWriteTest extends TableTestBase { + + private Table createTable(String format, Boolean thinMode) throws Exception { + catalog.createTable(identifier(), schema(format, thinMode), true); + return catalog.getTable(identifier()); + } + + private Schema schema(String format, Boolean thinMode) { + Schema.Builder schemaBuilder = Schema.newBuilder(); + schemaBuilder.column("f0", DataTypes.INT()); + schemaBuilder.column("f1", DataTypes.INT()); + schemaBuilder.column("f2", DataTypes.SMALLINT()); + schemaBuilder.column("f3", DataTypes.STRING()); + schemaBuilder.column("f4", DataTypes.DOUBLE()); + schemaBuilder.column("f5", DataTypes.CHAR(100)); + schemaBuilder.column("f6", DataTypes.VARCHAR(100)); + schemaBuilder.column("f7", DataTypes.BOOLEAN()); + schemaBuilder.column("f8", DataTypes.INT()); + schemaBuilder.column("f9", DataTypes.TIME()); + schemaBuilder.column("f10", DataTypes.TIMESTAMP()); + schemaBuilder.column("f11", DataTypes.DECIMAL(10, 2)); + schemaBuilder.column("f12", DataTypes.BYTES()); + schemaBuilder.column("f13", DataTypes.FLOAT()); + schemaBuilder.column("f14", DataTypes.BINARY(100)); + schemaBuilder.column("f15", DataTypes.VARBINARY(100)); + schemaBuilder.primaryKey( + "f0", "f1", "f2", "f3", "f4", "f5", "f6", "f7", "f8", "f9", "f10", "f11", "f12", + "f13"); + schemaBuilder.option("bucket", "1"); + schemaBuilder.option("bucket-key", "f1"); + schemaBuilder.option("file.format", format); + schemaBuilder.option("data-file.thin-mode", thinMode.toString()); + return schemaBuilder.build(); + } + + @Test + public void testThinModeWorks() throws Exception { + + InternalRow[] datas = datas(200000); + + Table table = createTable("orc", true); + write(table, datas); + + long size1 = tableSize(table); + dropTableDefault(); + + table = createTable("orc", false); + write(table, datas); + long size2 = tableSize(table); + dropTableDefault(); + + Assertions.assertThat(size2).isGreaterThan(size1); + } + + @Test + public void testAllFormatReadWrite() throws Exception { + testFormat("orc"); + testFormat("parquet"); + testFormat("avro"); + } + + private void testFormat(String format) throws Exception { + testReadWrite(format, true); + testReadWrite(format, true); + testReadWrite(format, false); + testReadWrite(format, false); + } + + private void testReadWrite(String format, boolean writeThin) throws Exception { + Table table = createTable(format, writeThin); + + InternalRow[] datas = datas(2000); + + write(table, datas); + + List readed = read(table); + + Assertions.assertThat(readed).containsExactlyInAnyOrder(datas); + dropTableDefault(); + } + + InternalRow[] datas(int i) { + InternalRow[] arrays = new InternalRow[i]; + for (int j = 0; j < i; j++) { + arrays[j] = data(); + } + return arrays; + } + + protected InternalRow data() { + return GenericRow.of( + RANDOM.nextInt(), + RANDOM.nextInt(), + (short) RANDOM.nextInt(), + randomString(), + RANDOM.nextDouble(), + randomString(), + randomString(), + RANDOM.nextBoolean(), + RANDOM.nextInt(), + RANDOM.nextInt(), + Timestamp.now(), + Decimal.zero(10, 2), + randomBytes(), + (float) RANDOM.nextDouble(), + randomBytes(), + randomBytes()); + } + + public static long tableSize(Table table) { + long count = 0; + List files = + ((FileStoreTable) table).store().newScan().plan().files(FileKind.ADD); + for (ManifestEntry file : files) { + count += file.file().fileSize(); + } + + return count; + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java index 9a27d5618459e..7258a1dd4170c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java @@ -25,11 +25,15 @@ import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.Decimal; +import org.apache.paimon.data.GenericArray; +import org.apache.paimon.data.GenericMap; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.Timestamp; import org.apache.paimon.disk.IOManagerImpl; +import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.iceberg.manifest.IcebergManifestFile; import org.apache.paimon.iceberg.manifest.IcebergManifestFileMeta; import org.apache.paimon.iceberg.manifest.IcebergManifestList; import org.apache.paimon.iceberg.metadata.IcebergMetadata; @@ -42,6 +46,7 @@ import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.TableCommitImpl; import org.apache.paimon.table.sink.TableWriteImpl; +import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeRoot; import org.apache.paimon.types.DataTypes; @@ -77,6 +82,7 @@ import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for Iceberg compatibility. */ public class IcebergCompatibilityTest { @@ -279,9 +285,10 @@ public void testIcebergSnapshotExpire() throws Exception { write.write(GenericRow.of(2, 20)); commit.commit(1, write.prepareCommit(false, 1)); assertThat(table.snapshotManager().latestSnapshotId()).isEqualTo(1L); + FileIO fileIO = table.fileIO(); IcebergMetadata metadata = IcebergMetadata.fromPath( - table.fileIO(), new Path(table.location(), "metadata/v1.metadata.json")); + fileIO, new Path(table.location(), "metadata/v1.metadata.json")); assertThat(metadata.snapshots()).hasSize(1); assertThat(metadata.currentSnapshotId()).isEqualTo(1); @@ -292,7 +299,7 @@ public void testIcebergSnapshotExpire() throws Exception { assertThat(table.snapshotManager().latestSnapshotId()).isEqualTo(3L); metadata = IcebergMetadata.fromPath( - table.fileIO(), new Path(table.location(), "metadata/v3.metadata.json")); + fileIO, new Path(table.location(), "metadata/v3.metadata.json")); assertThat(metadata.snapshots()).hasSize(3); assertThat(metadata.currentSnapshotId()).isEqualTo(3); @@ -302,12 +309,37 @@ public void testIcebergSnapshotExpire() throws Exception { IcebergPathFactory pathFactory = new IcebergPathFactory(new Path(table.location(), "metadata")); IcebergManifestList manifestList = IcebergManifestList.create(table, pathFactory); + assertThat(manifestList.compression()).isEqualTo("snappy"); + + IcebergManifestFile manifestFile = IcebergManifestFile.create(table, pathFactory); + assertThat(manifestFile.compression()).isEqualTo("snappy"); + Set usingManifests = new HashSet<>(); - for (IcebergManifestFileMeta fileMeta : - manifestList.read(new Path(metadata.currentSnapshot().manifestList()).getName())) { + String manifestListFile = new Path(metadata.currentSnapshot().manifestList()).getName(); + + assertThat(fileIO.readFileUtf8(new Path(pathFactory.metadataDirectory(), manifestListFile))) + .contains("snappy"); + + for (IcebergManifestFileMeta fileMeta : manifestList.read(manifestListFile)) { usingManifests.add(fileMeta.manifestPath()); + assertThat( + fileIO.readFileUtf8( + new Path( + pathFactory.metadataDirectory(), + fileMeta.manifestPath()))) + .contains("snappy"); } + IcebergManifestList legacyManifestList = + IcebergManifestList.create( + table.copy( + Collections.singletonMap( + IcebergOptions.MANIFEST_LEGACY_VERSION.key(), "true")), + pathFactory); + assertThatThrownBy(() -> legacyManifestList.read(manifestListFile)) + .rootCause() + .isInstanceOf(NullPointerException.class); + Set unusedFiles = new HashSet<>(); for (int i = 0; i < 2; i++) { unusedFiles.add(metadata.snapshots().get(i).manifestList()); @@ -328,7 +360,7 @@ public void testIcebergSnapshotExpire() throws Exception { assertThat(table.snapshotManager().latestSnapshotId()).isEqualTo(5L); metadata = IcebergMetadata.fromPath( - table.fileIO(), new Path(table.location(), "metadata/v5.metadata.json")); + fileIO, new Path(table.location(), "metadata/v5.metadata.json")); assertThat(metadata.snapshots()).hasSize(3); assertThat(metadata.currentSnapshotId()).isEqualTo(5); @@ -341,7 +373,7 @@ public void testIcebergSnapshotExpire() throws Exception { } for (String path : unusedFiles) { - assertThat(table.fileIO().exists(new Path(path))).isFalse(); + assertThat(fileIO.exists(new Path(path))).isFalse(); } // Test all existing Iceberg snapshots are valid. @@ -513,6 +545,56 @@ public void testAllTypeStatistics() throws Exception { } } + @Test + public void testNestedTypes() throws Exception { + RowType innerType = + RowType.of( + new DataField(2, "f1", DataTypes.STRING()), + new DataField(3, "f2", DataTypes.INT())); + RowType rowType = + RowType.of( + new DataField(0, "k", DataTypes.INT()), + new DataField( + 1, + "v", + DataTypes.MAP(DataTypes.INT(), DataTypes.ARRAY(innerType)))); + FileStoreTable table = + createPaimonTable(rowType, Collections.emptyList(), Collections.emptyList(), -1); + + String commitUser = UUID.randomUUID().toString(); + TableWriteImpl write = table.newWrite(commitUser); + TableCommitImpl commit = table.newCommit(commitUser); + + Map map1 = new HashMap<>(); + map1.put( + 10, + new GenericArray( + new GenericRow[] { + GenericRow.of(BinaryString.fromString("apple"), 100), + GenericRow.of(BinaryString.fromString("banana"), 101) + })); + write.write(GenericRow.of(1, new GenericMap(map1))); + + Map map2 = new HashMap<>(); + map2.put( + 20, + new GenericArray( + new GenericRow[] { + GenericRow.of(BinaryString.fromString("cherry"), 200), + GenericRow.of(BinaryString.fromString("pear"), 201) + })); + write.write(GenericRow.of(2, new GenericMap(map2))); + + commit.commit(1, write.prepareCommit(false, 1)); + write.close(); + commit.close(); + + assertThat(getIcebergResult()) + .containsExactlyInAnyOrder( + "Record(1, {10=[Record(apple, 100), Record(banana, 101)]})", + "Record(2, {20=[Record(cherry, 200), Record(pear, 201)]})"); + } + // ------------------------------------------------------------------------ // Random Tests // ------------------------------------------------------------------------ diff --git a/paimon-core/src/test/java/org/apache/paimon/index/IndexFileMetaSerializerTest.java b/paimon-core/src/test/java/org/apache/paimon/index/IndexFileMetaSerializerTest.java index 724d5b416359d..a7e692d2e554d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/index/IndexFileMetaSerializerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/index/IndexFileMetaSerializerTest.java @@ -21,7 +21,6 @@ import org.apache.paimon.deletionvectors.DeletionVectorsIndexFile; import org.apache.paimon.utils.ObjectSerializer; import org.apache.paimon.utils.ObjectSerializerTestBase; -import org.apache.paimon.utils.Pair; import java.util.LinkedHashMap; import java.util.Random; @@ -59,14 +58,20 @@ public static IndexFileMeta randomHashIndexFile() { public static IndexFileMeta randomDeletionVectorIndexFile() { Random rnd = new Random(); - LinkedHashMap> deletionVectorsRanges = new LinkedHashMap<>(); - deletionVectorsRanges.put("my_file_name1", Pair.of(rnd.nextInt(), rnd.nextInt())); - deletionVectorsRanges.put("my_file_name2", Pair.of(rnd.nextInt(), rnd.nextInt())); + LinkedHashMap deletionVectorMetas = new LinkedHashMap<>(); + deletionVectorMetas.put( + "my_file_name1", + new DeletionVectorMeta( + "my_file_name1", rnd.nextInt(), rnd.nextInt(), rnd.nextLong())); + deletionVectorMetas.put( + "my_file_name2", + new DeletionVectorMeta( + "my_file_name2", rnd.nextInt(), rnd.nextInt(), rnd.nextLong())); return new IndexFileMeta( DeletionVectorsIndexFile.DELETION_VECTORS_INDEX, "deletion_vectors_index_file_name" + rnd.nextLong(), rnd.nextInt(), rnd.nextInt(), - deletionVectorsRanges); + deletionVectorMetas); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java index 52d56afad8134..e43cd898dbc2a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java @@ -37,6 +37,7 @@ import org.apache.paimon.options.Options; import org.apache.paimon.reader.RecordReaderIterator; import org.apache.paimon.stats.StatsTestUtils; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.CloseableIterator; import org.apache.paimon.utils.FailingFileIO; @@ -158,7 +159,7 @@ public void testReadKeyType() throws Exception { List actualMetas = writer.result(); // projection: (shopId, orderId) -> (orderId) - RowType readKeyType = KEY_TYPE.project("key_orderId"); + RowType readKeyType = KEY_TYPE.project(SpecialFields.KEY_FIELD_PREFIX + "orderId"); KeyValueFileReaderFactory readerFactory = createReaderFactory(tempDir.toString(), "avro", readKeyType, null); InternalRowSerializer projectedKeySerializer = new InternalRowSerializer(readKeyType); @@ -231,7 +232,8 @@ protected KeyValueFileWriterFactory createWriterFactory(String pathStr, String f CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); Options options = new Options(); @@ -250,7 +252,8 @@ protected KeyValueFileWriterFactory createWriterFactory(String pathStr, String f CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue())); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null)); return KeyValueFileWriterFactory.builder( fileIO, diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java index bd272b745dc44..fbc02b2d73f2e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerCompatibilityTest.java @@ -19,6 +19,7 @@ package org.apache.paimon.manifest; import org.apache.paimon.data.Timestamp; +import org.apache.paimon.index.DeletionVectorMeta; import org.apache.paimon.index.IndexFileMeta; import org.apache.paimon.io.CompactIncrement; import org.apache.paimon.io.DataFileMeta; @@ -27,7 +28,6 @@ import org.apache.paimon.stats.SimpleStats; import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.utils.IOUtils; -import org.apache.paimon.utils.Pair; import org.junit.jupiter.api.Test; @@ -78,11 +78,11 @@ public void testProduction() throws IOException { Arrays.asList("field1", "field2", "field3")); List dataFiles = Collections.singletonList(dataFile); - LinkedHashMap> dvRanges = new LinkedHashMap<>(); - dvRanges.put("dv_key1", Pair.of(1, 2)); - dvRanges.put("dv_key2", Pair.of(3, 4)); + LinkedHashMap dvMetas = new LinkedHashMap<>(); + dvMetas.put("dv_key1", new DeletionVectorMeta("dv_key1", 1, 2, 3L)); + dvMetas.put("dv_key2", new DeletionVectorMeta("dv_key2", 3, 4, 5L)); IndexFileMeta indexFile = - new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvRanges); + new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvMetas); List indexFiles = Collections.singletonList(indexFile); CommitMessageImpl commitMessage = @@ -106,6 +106,76 @@ public void testProduction() throws IOException { assertThat(deserialized).isEqualTo(manifestCommittable); } + @Test + public void testCompatibilityToVersion4() throws IOException { + SimpleStats keyStats = + new SimpleStats( + singleColumn("min_key"), + singleColumn("max_key"), + fromLongArray(new Long[] {0L})); + SimpleStats valueStats = + new SimpleStats( + singleColumn("min_value"), + singleColumn("max_value"), + fromLongArray(new Long[] {0L})); + DataFileMeta dataFile = + new DataFileMeta( + "my_file", + 1024 * 1024, + 1024, + singleColumn("min_key"), + singleColumn("max_key"), + keyStats, + valueStats, + 15, + 200, + 5, + 3, + Arrays.asList("extra1", "extra2"), + Timestamp.fromLocalDateTime(LocalDateTime.parse("2022-03-02T20:20:12")), + 11L, + new byte[] {1, 2, 4}, + FileSource.COMPACT, + Arrays.asList("field1", "field2", "field3")); + List dataFiles = Collections.singletonList(dataFile); + + LinkedHashMap dvMetas = new LinkedHashMap<>(); + dvMetas.put("dv_key1", new DeletionVectorMeta("dv_key1", 1, 2, null)); + dvMetas.put("dv_key2", new DeletionVectorMeta("dv_key2", 3, 4, null)); + IndexFileMeta indexFile = + new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvMetas); + List indexFiles = Collections.singletonList(indexFile); + + CommitMessageImpl commitMessage = + new CommitMessageImpl( + singleColumn("my_partition"), + 11, + new DataIncrement(dataFiles, dataFiles, dataFiles), + new CompactIncrement(dataFiles, dataFiles, dataFiles), + new IndexIncrement(indexFiles)); + + ManifestCommittable manifestCommittable = + new ManifestCommittable( + 5, + 202020L, + Collections.singletonMap(5, 555L), + Collections.singletonList(commitMessage)); + + ManifestCommittableSerializer serializer = new ManifestCommittableSerializer(); + byte[] bytes = serializer.serialize(manifestCommittable); + ManifestCommittable deserialized = serializer.deserialize(3, bytes); + assertThat(deserialized).isEqualTo(manifestCommittable); + + byte[] v2Bytes = + IOUtils.readFully( + ManifestCommittableSerializerCompatibilityTest.class + .getClassLoader() + .getResourceAsStream("compatibility/manifest-committable-v4"), + true); + deserialized = serializer.deserialize(2, v2Bytes); + assertThat(deserialized).isEqualTo(manifestCommittable); + } + @Test public void testCompatibilityToVersion3() throws IOException { SimpleStats keyStats = @@ -139,11 +209,11 @@ public void testCompatibilityToVersion3() throws IOException { null); List dataFiles = Collections.singletonList(dataFile); - LinkedHashMap> dvRanges = new LinkedHashMap<>(); - dvRanges.put("dv_key1", Pair.of(1, 2)); - dvRanges.put("dv_key2", Pair.of(3, 4)); + LinkedHashMap dvMetas = new LinkedHashMap<>(); + dvMetas.put("dv_key1", new DeletionVectorMeta("dv_key1", 1, 2, null)); + dvMetas.put("dv_key2", new DeletionVectorMeta("dv_key2", 3, 4, null)); IndexFileMeta indexFile = - new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvRanges); + new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvMetas); List indexFiles = Collections.singletonList(indexFile); CommitMessageImpl commitMessage = @@ -209,11 +279,11 @@ public void testCompatibilityToVersion2() throws IOException { null); List dataFiles = Collections.singletonList(dataFile); - LinkedHashMap> dvRanges = new LinkedHashMap<>(); - dvRanges.put("dv_key1", Pair.of(1, 2)); - dvRanges.put("dv_key2", Pair.of(3, 4)); + LinkedHashMap dvMetas = new LinkedHashMap<>(); + dvMetas.put("dv_key1", new DeletionVectorMeta("dv_key1", 1, 2, null)); + dvMetas.put("dv_key2", new DeletionVectorMeta("dv_key2", 3, 4, null)); IndexFileMeta indexFile = - new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvRanges); + new IndexFileMeta("my_index_type", "my_index_file", 1024 * 100, 1002, dvMetas); List indexFiles = Collections.singletonList(indexFile); CommitMessageImpl commitMessage = diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java index c179a2c0a789f..8de8309bc8fbb 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestCommittableSerializerTest.java @@ -83,7 +83,7 @@ private static void addFileCommittables( if (!committable.logOffsets().containsKey(bucket)) { int offset = ID.incrementAndGet(); - committable.addLogOffset(bucket, offset); + committable.addLogOffset(bucket, offset, false); assertThat(committable.logOffsets().get(bucket)).isEqualTo(offset); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java index 9384eb88824d1..1be5993fb0d08 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTest.java @@ -452,7 +452,7 @@ public void testIdentifierAfterFullCompaction() throws Exception { containSameIdentifyEntryFile(fullCompacted, entryIdentifierExpected); } - @RepeatedTest(1000) + @RepeatedTest(10) public void testRandomFullCompaction() throws Exception { List input = new ArrayList<>(); Set manifestEntrySet = new HashSet<>(); diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java index 5e69035ca8348..52d82e76be2ab 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java @@ -149,7 +149,8 @@ protected ManifestFile createManifestFile(String pathStr) { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()), + CoreOptions.FILE_COMPRESSION.defaultValue(), + null), Long.MAX_VALUE, null) .create(); diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java index 34cca41e61a1f..089e11656a995 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java @@ -105,7 +105,8 @@ private ManifestFile createManifestFile(String pathStr) { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); return new ManifestFile.Factory( diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java index ce4f7b8072d2a..5bf01f32cb07d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java @@ -109,7 +109,8 @@ private ManifestList createManifestList(String pathStr) { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); return new ManifestList.Factory(FileIOFinder.find(path), avro, "zstd", pathFactory, null) .create(); } diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java index 0ab636c33aa3e..be49311427a05 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java @@ -41,6 +41,7 @@ import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.SchemaEvolutionTableTestBase; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowKind; @@ -77,7 +78,12 @@ public class ContainsLevelsTest { private final Comparator comparator = Comparator.comparingInt(o -> o.getInt(0)); - private final RowType keyType = DataTypes.ROW(DataTypes.FIELD(0, "_key", DataTypes.INT())); + private final RowType keyType = + DataTypes.ROW( + DataTypes.FIELD( + SpecialFields.KEY_FIELD_ID_START, + SpecialFields.KEY_FIELD_PREFIX + "key", + DataTypes.INT())); private final RowType rowType = DataTypes.ROW( DataTypes.FIELD(0, "key", DataTypes.INT()), diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java index 2dce81ce56b4e..a678534042eb9 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java @@ -41,6 +41,7 @@ import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.SchemaEvolutionTableTestBase; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowKind; @@ -79,7 +80,9 @@ public class LookupLevelsTest { private final Comparator comparator = Comparator.comparingInt(o -> o.getInt(0)); - private final RowType keyType = DataTypes.ROW(DataTypes.FIELD(0, "_key", DataTypes.INT())); + private final RowType keyType = + DataTypes.ROW( + DataTypes.FIELD(SpecialFields.KEY_FIELD_ID_START, "_KEY_key", DataTypes.INT())); private final RowType rowType = DataTypes.ROW( DataTypes.FIELD(0, "key", DataTypes.INT()), diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunctionTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunctionTest.java index a6e1b5f90fe23..93f634944e6e8 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunctionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/compact/PartialUpdateMergeFunctionTest.java @@ -95,6 +95,42 @@ public void testSequenceGroup() { validate(func, 1, null, null, 6, null, null, 6); } + @Test + public void testSequenceGroupPartialDelete() { + Options options = new Options(); + options.set("fields.f3.sequence-group", "f1,f2"); + options.set("fields.f6.sequence-group", "f4,f5"); + options.set("partial-update.remove-record-on-sequence-group", "f6"); + RowType rowType = + RowType.of( + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT()); + MergeFunction func = + PartialUpdateMergeFunction.factory(options, rowType, ImmutableList.of("f0")) + .create(); + func.reset(); + add(func, 1, 1, 1, 1, 1, 1, 1); + add(func, 1, 2, 2, 2, 2, 2, null); + validate(func, 1, 2, 2, 2, 1, 1, 1); + add(func, 1, 3, 3, 1, 3, 3, 3); + validate(func, 1, 2, 2, 2, 3, 3, 3); + + // delete + add(func, RowKind.DELETE, 1, 1, 1, 3, 1, 1, null); + validate(func, 1, null, null, 3, 3, 3, 3); + add(func, RowKind.DELETE, 1, 1, 1, 3, 1, 1, 4); + validate(func, null, null, null, null, null, null, null); + add(func, 1, 4, 4, 4, 5, 5, 5); + validate(func, 1, 4, 4, 4, 5, 5, 5); + add(func, RowKind.DELETE, 1, 1, 1, 6, 1, 1, 6); + validate(func, null, null, null, null, null, null, null); + } + @Test public void testMultiSequenceFields() { Options options = new Options(); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/ExpireSnapshotsTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/ExpireSnapshotsTest.java index 739d4b6bd6b32..9dc98343734b0 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/ExpireSnapshotsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/ExpireSnapshotsTest.java @@ -29,6 +29,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.io.DataFileMeta; +import org.apache.paimon.manifest.ExpireFileEntry; import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.FileSource; import org.apache.paimon.manifest.ManifestEntry; @@ -44,6 +45,7 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -217,7 +219,9 @@ public void testExpireExtraFiles() throws IOException { ManifestEntry delete = new ManifestEntry(FileKind.DELETE, partition, 0, 1, dataFile); // expire - expire.snapshotDeletion().cleanUnusedDataFile(Arrays.asList(add, delete)); + expire.snapshotDeletion() + .cleanUnusedDataFile( + Arrays.asList(ExpireFileEntry.from(add), ExpireFileEntry.from(delete))); // check assertThat(fileIO.exists(myDataFile)).isFalse(); @@ -451,7 +455,7 @@ public void testExpireWithUpgradedFile() throws Exception { store.assertCleaned(); } - @Test + @RepeatedTest(5) public void testChangelogOutLivedSnapshot() throws Exception { List allData = new ArrayList<>(); List snapshotPositions = new ArrayList<>(); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java index 67945df60ce83..9e4ba30eb8786 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileStoreCommitTest.java @@ -31,8 +31,11 @@ import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.index.IndexFileMeta; +import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.IndexManifestEntry; import org.apache.paimon.manifest.ManifestCommittable; +import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.manifest.ManifestFile; import org.apache.paimon.manifest.ManifestFileMeta; import org.apache.paimon.mergetree.compact.DeduplicateMergeFunction; import org.apache.paimon.predicate.PredicateBuilder; @@ -80,6 +83,7 @@ import static org.apache.paimon.index.HashIndexFile.HASH_INDEX; import static org.apache.paimon.partition.PartitionPredicate.createPartitionPredicate; +import static org.apache.paimon.stats.SimpleStats.EMPTY_STATS; 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; @@ -943,6 +947,32 @@ public void testManifestCompact() throws Exception { .isEqualTo(0); } + @Test + public void testDropStatsForOverwrite() throws Exception { + TestFileStore store = createStore(false); + store.options().toConfiguration().set(CoreOptions.MANIFEST_DELETE_FILE_DROP_STATS, true); + + List keyValues = generateDataList(1); + BinaryRow partition = gen.getPartition(keyValues.get(0)); + // commit 1 + Snapshot snapshot1 = + store.commitData(keyValues, s -> partition, kv -> 0, Collections.emptyMap()).get(0); + // overwrite commit 2 + Snapshot snapshot2 = + store.overwriteData(keyValues, s -> partition, kv -> 0, Collections.emptyMap()) + .get(0); + ManifestFile manifestFile = store.manifestFileFactory().create(); + List entries = + store.manifestListFactory().create().readDataManifests(snapshot2).stream() + .flatMap(meta -> manifestFile.read(meta.fileName()).stream()) + .collect(Collectors.toList()); + for (ManifestEntry manifestEntry : entries) { + if (manifestEntry.kind() == FileKind.DELETE) { + assertThat(manifestEntry.file().valueStats()).isEqualTo(EMPTY_STATS); + } + } + } + @Test public void testManifestCompactFull() throws Exception { // Disable full compaction by options. diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java index 07b2fc48e078c..f20fd06d31e06 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanTest.java @@ -26,8 +26,6 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.manifest.ManifestEntry; -import org.apache.paimon.manifest.ManifestFileMeta; -import org.apache.paimon.manifest.ManifestList; import org.apache.paimon.mergetree.compact.DeduplicateMergeFunction; import org.apache.paimon.predicate.PredicateBuilder; import org.apache.paimon.schema.Schema; @@ -51,6 +49,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; +import static org.apache.paimon.stats.SimpleStats.EMPTY_STATS; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link KeyValueFileStoreScan}. */ @@ -277,26 +276,24 @@ public void testWithSnapshot() throws Exception { } @Test - public void testWithManifestList() throws Exception { + public void testDropStatsInPlan() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); - int numCommits = random.nextInt(10) + 1; - for (int i = 0; i < numCommits; i++) { - List data = generateData(random.nextInt(100) + 1); - writeData(data); - } - - ManifestList manifestList = store.manifestListFactory().create(); - long wantedSnapshotId = random.nextLong(snapshotManager.latestSnapshotId()) + 1; - Snapshot wantedSnapshot = snapshotManager.snapshot(wantedSnapshotId); - List wantedManifests = manifestList.readDataManifests(wantedSnapshot); + List data = generateData(100, 0, (long) Math.abs(random.nextInt(1000))); + writeData(data, 0); + data = generateData(100, 1, (long) Math.abs(random.nextInt(1000)) + 1000); + writeData(data, 0); + data = generateData(100, 2, (long) Math.abs(random.nextInt(1000)) + 2000); + writeData(data, 0); + data = generateData(100, 3, (long) Math.abs(random.nextInt(1000)) + 3000); + Snapshot snapshot = writeData(data, 0); - FileStoreScan scan = store.newScan(); - scan.withManifestList(wantedManifests); + KeyValueFileStoreScan scan = store.newScan(); + scan.withSnapshot(snapshot.id()).dropStats(); + List files = scan.plan().files(); - List expectedKvs = store.readKvsFromSnapshot(wantedSnapshotId); - gen.sort(expectedKvs); - Map expected = store.toKvMap(expectedKvs); - runTestExactMatch(scan, null, expected); + for (ManifestEntry manifestEntry : files) { + assertThat(manifestEntry.file().valueStats()).isEqualTo(EMPTY_STATS); + } } private void runTestExactMatch( diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java index fdc68b34abb44..5139dd44957d7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java @@ -165,22 +165,20 @@ public void testNormallyRemoving() throws Throwable { // randomly delete tags List deleteTags = Collections.emptyList(); - if (!allTags.isEmpty()) { - deleteTags = randomlyPick(allTags); - for (String tagName : deleteTags) { - table.deleteTag(tagName); - } + deleteTags = randomlyPick(allTags); + for (String tagName : deleteTags) { + table.deleteTag(tagName); } // first check, nothing will be deleted because the default olderThan interval is 1 day LocalOrphanFilesClean orphanFilesClean = new LocalOrphanFilesClean(table); - assertThat(orphanFilesClean.clean().size()).isEqualTo(0); + assertThat(orphanFilesClean.clean().getDeletedFilesPath().size()).isEqualTo(0); // second check orphanFilesClean = new LocalOrphanFilesClean( table, System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(2)); - List deleted = orphanFilesClean.clean(); + List deleted = orphanFilesClean.clean().getDeletedFilesPath(); try { validate(deleted, snapshotData, new HashMap<>()); } catch (Throwable t) { @@ -363,13 +361,13 @@ public void testCleanOrphanFilesWithChangelogDecoupled(String changelogProducer) // first check, nothing will be deleted because the default olderThan interval is 1 day LocalOrphanFilesClean orphanFilesClean = new LocalOrphanFilesClean(table); - assertThat(orphanFilesClean.clean().size()).isEqualTo(0); + assertThat(orphanFilesClean.clean().getDeletedFilesPath().size()).isEqualTo(0); // second check orphanFilesClean = new LocalOrphanFilesClean( table, System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(2)); - List deleted = orphanFilesClean.clean(); + List deleted = orphanFilesClean.clean().getDeletedFilesPath(); validate(deleted, snapshotData, changelogData); } @@ -399,7 +397,7 @@ public void testAbnormallyRemoving() throws Exception { LocalOrphanFilesClean orphanFilesClean = new LocalOrphanFilesClean( table, System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(2)); - assertThat(orphanFilesClean.clean().size()).isGreaterThan(0); + assertThat(orphanFilesClean.clean().getDeletedFilesPath().size()).isGreaterThan(0); } private void writeData( diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/MergeFileSplitReadTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/MergeFileSplitReadTest.java index 46b64422fd9b2..59f848a296cf6 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/MergeFileSplitReadTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/MergeFileSplitReadTest.java @@ -37,6 +37,7 @@ import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.SpecialFields; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.DataField; @@ -284,7 +285,12 @@ private TestFileStore createStore( ? Collections.emptyList() : Stream.concat( keyType.getFieldNames().stream() - .map(field -> field.replace("key_", "")), + .map( + field -> + field.replace( + SpecialFields + .KEY_FIELD_PREFIX, + "")), partitionType.getFieldNames().stream()) .collect(Collectors.toList()), Collections.emptyMap(), diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/metrics/ScanMetricsTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/metrics/ScanMetricsTest.java index 2b9d0e0cb7288..a0427d95cab18 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/metrics/ScanMetricsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/metrics/ScanMetricsTest.java @@ -48,9 +48,7 @@ public void testGenericMetricsRegistration() { ScanMetrics.SCAN_DURATION, ScanMetrics.LAST_SCANNED_MANIFESTS, ScanMetrics.LAST_SCAN_SKIPPED_TABLE_FILES, - ScanMetrics.LAST_SCAN_RESULTED_TABLE_FILES, - ScanMetrics.LAST_SKIPPED_BY_PARTITION_AND_STATS, - ScanMetrics.LAST_SKIPPED_BY_WHOLE_BUCKET_FILES_FILTER); + ScanMetrics.LAST_SCAN_RESULTED_TABLE_FILES); } /** Tests that the metrics are updated properly. */ @@ -66,14 +64,6 @@ public void testMetricsAreUpdated() { (Histogram) registeredGenericMetrics.get(ScanMetrics.SCAN_DURATION); Gauge lastScannedManifests = (Gauge) registeredGenericMetrics.get(ScanMetrics.LAST_SCANNED_MANIFESTS); - Gauge lastSkippedByPartitionAndStats = - (Gauge) - registeredGenericMetrics.get( - ScanMetrics.LAST_SKIPPED_BY_PARTITION_AND_STATS); - Gauge lastSkippedByWholeBucketFilesFilter = - (Gauge) - registeredGenericMetrics.get( - ScanMetrics.LAST_SKIPPED_BY_WHOLE_BUCKET_FILES_FILTER); Gauge lastScanSkippedTableFiles = (Gauge) registeredGenericMetrics.get(ScanMetrics.LAST_SCAN_SKIPPED_TABLE_FILES); @@ -85,8 +75,6 @@ public void testMetricsAreUpdated() { assertThat(scanDuration.getCount()).isEqualTo(0); assertThat(scanDuration.getStatistics().size()).isEqualTo(0); assertThat(lastScannedManifests.getValue()).isEqualTo(0); - assertThat(lastSkippedByPartitionAndStats.getValue()).isEqualTo(0); - assertThat(lastSkippedByWholeBucketFilesFilter.getValue()).isEqualTo(0); assertThat(lastScanSkippedTableFiles.getValue()).isEqualTo(0); assertThat(lastScanResultedTableFiles.getValue()).isEqualTo(0); @@ -104,9 +92,7 @@ public void testMetricsAreUpdated() { assertThat(scanDuration.getStatistics().getMax()).isEqualTo(200); assertThat(scanDuration.getStatistics().getStdDev()).isEqualTo(0); assertThat(lastScannedManifests.getValue()).isEqualTo(20); - assertThat(lastSkippedByPartitionAndStats.getValue()).isEqualTo(25); - assertThat(lastSkippedByWholeBucketFilesFilter.getValue()).isEqualTo(32); - assertThat(lastScanSkippedTableFiles.getValue()).isEqualTo(57); + assertThat(lastScanSkippedTableFiles.getValue()).isEqualTo(25); assertThat(lastScanResultedTableFiles.getValue()).isEqualTo(10); // report again @@ -123,19 +109,17 @@ public void testMetricsAreUpdated() { assertThat(scanDuration.getStatistics().getMax()).isEqualTo(500); assertThat(scanDuration.getStatistics().getStdDev()).isCloseTo(212.132, offset(0.001)); assertThat(lastScannedManifests.getValue()).isEqualTo(22); - assertThat(lastSkippedByPartitionAndStats.getValue()).isEqualTo(30); - assertThat(lastSkippedByWholeBucketFilesFilter.getValue()).isEqualTo(33); - assertThat(lastScanSkippedTableFiles.getValue()).isEqualTo(63); + assertThat(lastScanSkippedTableFiles.getValue()).isEqualTo(30); assertThat(lastScanResultedTableFiles.getValue()).isEqualTo(8); } private void reportOnce(ScanMetrics scanMetrics) { - ScanStats scanStats = new ScanStats(200, 20, 25, 32, 10); + ScanStats scanStats = new ScanStats(200, 20, 25, 10); scanMetrics.reportScan(scanStats); } private void reportAgain(ScanMetrics scanMetrics) { - ScanStats scanStats = new ScanStats(500, 22, 30, 33, 8); + ScanStats scanStats = new ScanStats(500, 22, 30, 8); scanMetrics.reportScan(scanStats); } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.java new file mode 100644 index 0000000000000..340e38f6a7f8a --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/DefaultErrorHandlerTest.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.rest; + +import org.apache.paimon.rest.exceptions.AlreadyExistsException; +import org.apache.paimon.rest.exceptions.BadRequestException; +import org.apache.paimon.rest.exceptions.ForbiddenException; +import org.apache.paimon.rest.exceptions.NoSuchResourceException; +import org.apache.paimon.rest.exceptions.NotAuthorizedException; +import org.apache.paimon.rest.exceptions.RESTException; +import org.apache.paimon.rest.exceptions.ServiceFailureException; +import org.apache.paimon.rest.exceptions.ServiceUnavailableException; +import org.apache.paimon.rest.responses.ErrorResponse; + +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; + +import static org.junit.Assert.assertThrows; + +/** Test for {@link DefaultErrorHandler}. */ +public class DefaultErrorHandlerTest { + private ErrorHandler defaultErrorHandler; + + @Before + public void setUp() throws IOException { + defaultErrorHandler = DefaultErrorHandler.getInstance(); + } + + @Test + public void testHandleErrorResponse() { + assertThrows( + BadRequestException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(400))); + assertThrows( + NotAuthorizedException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(401))); + assertThrows( + ForbiddenException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(403))); + assertThrows( + NoSuchResourceException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(404))); + assertThrows( + RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(405))); + assertThrows( + RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(406))); + assertThrows( + AlreadyExistsException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(409))); + assertThrows( + ServiceFailureException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(500))); + assertThrows( + UnsupportedOperationException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(501))); + assertThrows( + RESTException.class, () -> defaultErrorHandler.accept(generateErrorResponse(502))); + assertThrows( + ServiceUnavailableException.class, + () -> defaultErrorHandler.accept(generateErrorResponse(503))); + } + + private ErrorResponse generateErrorResponse(int code) { + return new ErrorResponse("message", code, new ArrayList()); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java new file mode 100644 index 0000000000000..a3b06b8ce3a9f --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/HttpClientTest.java @@ -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.rest; + +import org.apache.paimon.rest.auth.BearTokenCredentialsProvider; +import org.apache.paimon.rest.auth.CredentialsProvider; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import okhttp3.mockwebserver.MockResponse; +import okhttp3.mockwebserver.MockWebServer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.time.Duration; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +/** Test for {@link HttpClient}. */ +public class HttpClientTest { + + private MockWebServer mockWebServer; + private HttpClient httpClient; + private ObjectMapper objectMapper = RESTObjectMapper.create(); + private ErrorHandler errorHandler; + private MockRESTData mockResponseData; + private String mockResponseDataStr; + private Map headers; + private static final String MOCK_PATH = "/v1/api/mock"; + private static final String TOKEN = "token"; + + @Before + public void setUp() throws IOException { + mockWebServer = new MockWebServer(); + mockWebServer.start(); + String baseUrl = mockWebServer.url("").toString(); + errorHandler = mock(ErrorHandler.class); + HttpClientOptions httpClientOptions = + new HttpClientOptions( + baseUrl, + Optional.of(Duration.ofSeconds(3)), + Optional.of(Duration.ofSeconds(3)), + objectMapper, + 1, + errorHandler); + mockResponseData = new MockRESTData(MOCK_PATH); + mockResponseDataStr = objectMapper.writeValueAsString(mockResponseData); + httpClient = new HttpClient(httpClientOptions); + CredentialsProvider credentialsProvider = new BearTokenCredentialsProvider(TOKEN); + headers = credentialsProvider.authHeader(); + } + + @After + public void tearDown() throws IOException { + mockWebServer.shutdown(); + } + + @Test + public void testGetSuccess() { + mockHttpCallWithCode(mockResponseDataStr, 200); + MockRESTData response = httpClient.get(MOCK_PATH, MockRESTData.class, headers); + verify(errorHandler, times(0)).accept(any()); + assertEquals(mockResponseData.data(), response.data()); + } + + @Test + public void testGetFail() { + mockHttpCallWithCode(mockResponseDataStr, 400); + httpClient.get(MOCK_PATH, MockRESTData.class, headers); + verify(errorHandler, times(1)).accept(any()); + } + + @Test + public void testPostSuccess() { + mockHttpCallWithCode(mockResponseDataStr, 200); + MockRESTData response = + httpClient.post(MOCK_PATH, mockResponseData, MockRESTData.class, headers); + verify(errorHandler, times(0)).accept(any()); + assertEquals(mockResponseData.data(), response.data()); + } + + @Test + public void testPostFail() { + mockHttpCallWithCode(mockResponseDataStr, 400); + httpClient.post(MOCK_PATH, mockResponseData, MockRESTData.class, headers); + verify(errorHandler, times(1)).accept(any()); + } + + @Test + public void testDeleteSuccess() { + mockHttpCallWithCode(mockResponseDataStr, 200); + MockRESTData response = httpClient.delete(MOCK_PATH, headers); + verify(errorHandler, times(0)).accept(any()); + } + + @Test + public void testDeleteFail() { + mockHttpCallWithCode(mockResponseDataStr, 400); + httpClient.delete(MOCK_PATH, headers); + verify(errorHandler, times(1)).accept(any()); + } + + private Map headers(String token) { + Map header = new HashMap<>(); + header.put("Authorization", "Bearer " + token); + return header; + } + + private void mockHttpCallWithCode(String body, Integer code) { + MockResponse mockResponseObj = generateMockResponse(body, code); + mockWebServer.enqueue(mockResponseObj); + } + + private MockResponse generateMockResponse(String data, Integer code) { + return new MockResponse() + .setResponseCode(code) + .setBody(data) + .addHeader("Content-Type", "application/json"); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java new file mode 100644 index 0000000000000..9b7f1003e76fd --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTData.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonGetter; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.annotation.JsonProperty; + +import java.beans.ConstructorProperties; + +/** Mock REST data. */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class MockRESTData implements RESTRequest, RESTResponse { + private static final String FIELD_DATA = "data"; + + @JsonProperty(FIELD_DATA) + private String data; + + @ConstructorProperties({FIELD_DATA}) + public MockRESTData(String data) { + this.data = data; + } + + @JsonGetter(FIELD_DATA) + public String data() { + return data; + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java new file mode 100644 index 0000000000000..a605e5e77c2a3 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.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.rest; + +import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.responses.CreateDatabaseResponse; +import org.apache.paimon.rest.responses.DatabaseName; +import org.apache.paimon.rest.responses.ErrorResponse; +import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.ListDatabasesResponse; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.rest.RESTCatalogInternalOptions.DATABASE_COMMENT; + +/** Mock REST message. */ +public class MockRESTMessage { + + public static String databaseName() { + return "database"; + } + + public static CreateDatabaseRequest createDatabaseRequest(String name) { + boolean ignoreIfExists = true; + Map options = new HashMap<>(); + options.put("a", "b"); + return new CreateDatabaseRequest(name, ignoreIfExists, options); + } + + public static CreateDatabaseResponse createDatabaseResponse(String name) { + Map options = new HashMap<>(); + options.put("a", "b"); + return new CreateDatabaseResponse(name, options); + } + + public static GetDatabaseResponse getDatabaseResponse(String name) { + Map options = new HashMap<>(); + options.put("a", "b"); + options.put(DATABASE_COMMENT.key(), "comment"); + return new GetDatabaseResponse(name, options); + } + + public static ListDatabasesResponse listDatabasesResponse(String name) { + DatabaseName databaseName = new DatabaseName(name); + List databaseNameList = new ArrayList<>(); + databaseNameList.add(databaseName); + return new ListDatabasesResponse(databaseNameList); + } + + public static ErrorResponse noSuchResourceExceptionErrorResponse() { + return new ErrorResponse("message", 404, new ArrayList<>()); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java new file mode 100644 index 0000000000000..0fff81afdcdec --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Database; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; +import org.apache.paimon.rest.responses.CreateDatabaseResponse; +import org.apache.paimon.rest.responses.ErrorResponse; +import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.ListDatabasesResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import okhttp3.mockwebserver.MockResponse; +import okhttp3.mockwebserver.MockWebServer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** Test for REST Catalog. */ +public class RESTCatalogTest { + + private final ObjectMapper mapper = RESTObjectMapper.create(); + private MockWebServer mockWebServer; + private RESTCatalog restCatalog; + private RESTCatalog mockRestCatalog; + + @Before + public void setUp() throws IOException { + mockWebServer = new MockWebServer(); + mockWebServer.start(); + String baseUrl = mockWebServer.url("").toString(); + Options options = new Options(); + options.set(RESTCatalogOptions.URI, baseUrl); + String initToken = "init_token"; + options.set(RESTCatalogOptions.TOKEN, initToken); + options.set(RESTCatalogOptions.THREAD_POOL_SIZE, 1); + String mockResponse = + String.format( + "{\"defaults\": {\"%s\": \"%s\"}}", + RESTCatalogInternalOptions.PREFIX.key(), "prefix"); + mockResponse(mockResponse, 200); + restCatalog = new RESTCatalog(options); + mockRestCatalog = spy(restCatalog); + } + + @After + public void tearDown() throws IOException { + mockWebServer.shutdown(); + } + + @Test + public void testInitFailWhenDefineWarehouse() { + Options options = new Options(); + options.set(CatalogOptions.WAREHOUSE, "/a/b/c"); + assertThrows(IllegalArgumentException.class, () -> new RESTCatalog(options)); + } + + @Test + public void testGetConfig() { + String key = "a"; + String value = "b"; + String mockResponse = String.format("{\"defaults\": {\"%s\": \"%s\"}}", key, value); + mockResponse(mockResponse, 200); + Map header = new HashMap<>(); + Map response = restCatalog.fetchOptionsFromServer(header, new HashMap<>()); + assertEquals(value, response.get(key)); + } + + @Test + public void testListDatabases() throws JsonProcessingException { + String name = MockRESTMessage.databaseName(); + ListDatabasesResponse response = MockRESTMessage.listDatabasesResponse(name); + mockResponse(mapper.writeValueAsString(response), 200); + List result = restCatalog.listDatabases(); + assertEquals(response.getDatabases().size(), result.size()); + assertEquals(name, result.get(0)); + } + + @Test + public void testCreateDatabase() throws Exception { + String name = MockRESTMessage.databaseName(); + CreateDatabaseResponse response = MockRESTMessage.createDatabaseResponse(name); + mockResponse(mapper.writeValueAsString(response), 200); + assertDoesNotThrow(() -> restCatalog.createDatabase(name, false, response.getOptions())); + } + + @Test + public void testGetDatabase() throws Exception { + String name = MockRESTMessage.databaseName(); + GetDatabaseResponse response = MockRESTMessage.getDatabaseResponse(name); + mockResponse(mapper.writeValueAsString(response), 200); + Database result = restCatalog.getDatabase(name); + assertEquals(name, result.name()); + assertEquals(response.getOptions().size(), result.options().size()); + assertEquals(response.comment().get(), result.comment().get()); + } + + @Test + public void testDropDatabase() throws Exception { + String name = MockRESTMessage.databaseName(); + mockResponse("", 200); + assertDoesNotThrow(() -> mockRestCatalog.dropDatabase(name, false, true)); + verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(false), eq(true)); + verify(mockRestCatalog, times(0)).listTables(eq(name)); + } + + @Test + public void testDropDatabaseWhenNoExistAndIgnoreIfNotExistsIsFalse() throws Exception { + String name = MockRESTMessage.databaseName(); + ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); + mockResponse(mapper.writeValueAsString(response), 404); + assertThrows( + Catalog.DatabaseNotExistException.class, + () -> mockRestCatalog.dropDatabase(name, false, true)); + } + + @Test + public void testDropDatabaseWhenNoExistAndIgnoreIfNotExistsIsTrue() throws Exception { + String name = MockRESTMessage.databaseName(); + ErrorResponse response = MockRESTMessage.noSuchResourceExceptionErrorResponse(); + mockResponse(mapper.writeValueAsString(response), 404); + assertDoesNotThrow(() -> mockRestCatalog.dropDatabase(name, true, true)); + verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(true), eq(true)); + verify(mockRestCatalog, times(0)).listTables(eq(name)); + } + + @Test + public void testDropDatabaseWhenCascadeIsFalseAndNoTables() throws Exception { + String name = MockRESTMessage.databaseName(); + boolean cascade = false; + mockResponse("", 200); + when(mockRestCatalog.listTables(name)).thenReturn(new ArrayList<>()); + assertDoesNotThrow(() -> mockRestCatalog.dropDatabase(name, false, cascade)); + verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(false), eq(cascade)); + verify(mockRestCatalog, times(1)).listTables(eq(name)); + } + + @Test + public void testDropDatabaseWhenCascadeIsFalseAndTablesExist() throws Exception { + String name = MockRESTMessage.databaseName(); + boolean cascade = false; + mockResponse("", 200); + List tables = new ArrayList<>(); + tables.add("t1"); + when(mockRestCatalog.listTables(name)).thenReturn(tables); + assertThrows( + Catalog.DatabaseNotEmptyException.class, + () -> mockRestCatalog.dropDatabase(name, false, cascade)); + verify(mockRestCatalog, times(1)).dropDatabase(eq(name), eq(false), eq(cascade)); + verify(mockRestCatalog, times(1)).listTables(eq(name)); + } + + private void mockResponse(String mockResponse, int httpCode) { + MockResponse mockResponseObj = + new MockResponse() + .setResponseCode(httpCode) + .setBody(mockResponse) + .addHeader("Content-Type", "application/json"); + mockWebServer.enqueue(mockResponseObj); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java new file mode 100644 index 0000000000000..7fee81ef10243 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTObjectMapperTest.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.rest; + +import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.rest.responses.CreateDatabaseResponse; +import org.apache.paimon.rest.responses.ErrorResponse; +import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.ListDatabasesResponse; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +/** Test for {@link RESTObjectMapper}. */ +public class RESTObjectMapperTest { + private ObjectMapper mapper = RESTObjectMapper.create(); + + @Test + public void configResponseParseTest() throws Exception { + String confKey = "a"; + Map conf = new HashMap<>(); + conf.put(confKey, "b"); + ConfigResponse response = new ConfigResponse(conf, conf); + String responseStr = mapper.writeValueAsString(response); + ConfigResponse parseData = mapper.readValue(responseStr, ConfigResponse.class); + assertEquals(conf.get(confKey), parseData.getDefaults().get(confKey)); + } + + @Test + public void errorResponseParseTest() throws Exception { + String message = "message"; + Integer code = 400; + ErrorResponse response = new ErrorResponse(message, code, new ArrayList()); + String responseStr = mapper.writeValueAsString(response); + ErrorResponse parseData = mapper.readValue(responseStr, ErrorResponse.class); + assertEquals(message, parseData.getMessage()); + assertEquals(code, parseData.getCode()); + } + + @Test + public void createDatabaseRequestParseTest() throws Exception { + String name = MockRESTMessage.databaseName(); + CreateDatabaseRequest request = MockRESTMessage.createDatabaseRequest(name); + String requestStr = mapper.writeValueAsString(request); + CreateDatabaseRequest parseData = mapper.readValue(requestStr, CreateDatabaseRequest.class); + assertEquals(request.getName(), parseData.getName()); + assertEquals(request.getIgnoreIfExists(), parseData.getIgnoreIfExists()); + assertEquals(request.getOptions().size(), parseData.getOptions().size()); + } + + @Test + public void createDatabaseResponseParseTest() throws Exception { + String name = MockRESTMessage.databaseName(); + CreateDatabaseResponse response = MockRESTMessage.createDatabaseResponse(name); + String responseStr = mapper.writeValueAsString(response); + CreateDatabaseResponse parseData = + mapper.readValue(responseStr, CreateDatabaseResponse.class); + assertEquals(name, parseData.getName()); + assertEquals(response.getOptions().size(), parseData.getOptions().size()); + } + + @Test + public void getDatabaseResponseParseTest() throws Exception { + String name = MockRESTMessage.databaseName(); + GetDatabaseResponse response = MockRESTMessage.getDatabaseResponse(name); + String responseStr = mapper.writeValueAsString(response); + GetDatabaseResponse parseData = mapper.readValue(responseStr, GetDatabaseResponse.class); + assertEquals(name, parseData.getName()); + assertEquals(response.getOptions().size(), parseData.getOptions().size()); + assertEquals(response.comment().get(), parseData.comment().get()); + } + + @Test + public void listDatabaseResponseParseTest() throws Exception { + String name = MockRESTMessage.databaseName(); + ListDatabasesResponse response = MockRESTMessage.listDatabasesResponse(name); + String responseStr = mapper.writeValueAsString(response); + ListDatabasesResponse parseData = + mapper.readValue(responseStr, ListDatabasesResponse.class); + assertEquals(response.getDatabases().size(), parseData.getDatabases().size()); + assertEquals(name, parseData.getDatabases().get(0).getName()); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java new file mode 100644 index 0000000000000..1f4a48fd5e8cb --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest.auth; + +import org.apache.paimon.utils.Pair; +import org.apache.paimon.utils.ThreadPoolUtils; + +import org.apache.commons.io.FileUtils; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.mockito.Mockito; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ScheduledExecutorService; + +import static org.apache.paimon.rest.auth.AuthSession.MAX_REFRESH_WINDOW_MILLIS; +import static org.apache.paimon.rest.auth.AuthSession.MIN_REFRESH_WAIT_MILLIS; +import static org.apache.paimon.rest.auth.AuthSession.TOKEN_REFRESH_NUM_RETRIES; +import static org.junit.Assert.assertEquals; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** Test for {@link AuthSession}. */ +public class AuthSessionTest { + + @Rule public TemporaryFolder folder = new TemporaryFolder(); + + @Test + public void testRefreshBearTokenFileCredentialsProvider() + throws IOException, InterruptedException { + String fileName = "token"; + Pair tokenFile2Token = generateTokenAndWriteToFile(fileName); + String token = tokenFile2Token.getRight(); + File tokenFile = tokenFile2Token.getLeft(); + Map initialHeaders = new HashMap<>(); + long expiresInMillis = 1000L; + CredentialsProvider credentialsProvider = + new BearTokenFileCredentialsProvider(tokenFile.getPath(), expiresInMillis); + ScheduledExecutorService executor = + ThreadPoolUtils.createScheduledThreadPool(1, "refresh-token"); + AuthSession session = + AuthSession.fromRefreshCredentialsProvider( + executor, initialHeaders, credentialsProvider); + Map header = session.getHeaders(); + assertEquals(header.get("Authorization"), "Bearer " + token); + tokenFile.delete(); + tokenFile2Token = generateTokenAndWriteToFile(fileName); + token = tokenFile2Token.getRight(); + Thread.sleep(expiresInMillis + 500L); + header = session.getHeaders(); + assertEquals(header.get("Authorization"), "Bearer " + token); + } + + @Test + public void testRefreshCredentialsProviderIsSoonExpire() + throws IOException, InterruptedException { + String fileName = "token"; + Pair tokenFile2Token = generateTokenAndWriteToFile(fileName); + String token = tokenFile2Token.getRight(); + File tokenFile = tokenFile2Token.getLeft(); + Map initialHeaders = new HashMap<>(); + long expiresInMillis = 1000L; + CredentialsProvider credentialsProvider = + new BearTokenFileCredentialsProvider(tokenFile.getPath(), expiresInMillis); + AuthSession session = + AuthSession.fromRefreshCredentialsProvider( + null, initialHeaders, credentialsProvider); + Map header = session.getHeaders(); + assertEquals(header.get("Authorization"), "Bearer " + token); + tokenFile.delete(); + tokenFile2Token = generateTokenAndWriteToFile(fileName); + token = tokenFile2Token.getRight(); + tokenFile = tokenFile2Token.getLeft(); + FileUtils.writeStringToFile(tokenFile, token); + Thread.sleep( + (long) (expiresInMillis * (1 - BearTokenFileCredentialsProvider.EXPIRED_FACTOR)) + + 10L); + header = session.getHeaders(); + assertEquals(header.get("Authorization"), "Bearer " + token); + } + + @Test + public void testRetryWhenRefreshFail() throws Exception { + Map initialHeaders = new HashMap<>(); + CredentialsProvider credentialsProvider = + Mockito.mock(BearTokenFileCredentialsProvider.class); + long expiresAtMillis = System.currentTimeMillis() - 1000L; + when(credentialsProvider.expiresAtMillis()).thenReturn(Optional.of(expiresAtMillis)); + when(credentialsProvider.expiresInMills()).thenReturn(Optional.of(50L)); + when(credentialsProvider.supportRefresh()).thenReturn(true); + when(credentialsProvider.keepRefreshed()).thenReturn(true); + when(credentialsProvider.refresh()).thenReturn(false); + AuthSession session = + AuthSession.fromRefreshCredentialsProvider( + null, initialHeaders, credentialsProvider); + AuthSession.scheduleTokenRefresh( + ThreadPoolUtils.createScheduledThreadPool(1, "refresh-token"), + session, + expiresAtMillis); + Thread.sleep(10_000L); + verify(credentialsProvider, Mockito.times(TOKEN_REFRESH_NUM_RETRIES + 1)).refresh(); + } + + @Test + public void testGetTimeToWaitByExpiresInMills() { + long expiresInMillis = -100L; + long timeToWait = AuthSession.getTimeToWaitByExpiresInMills(expiresInMillis); + assertEquals(MIN_REFRESH_WAIT_MILLIS, timeToWait); + expiresInMillis = (long) (MAX_REFRESH_WINDOW_MILLIS * 0.5); + timeToWait = AuthSession.getTimeToWaitByExpiresInMills(expiresInMillis); + assertEquals(MIN_REFRESH_WAIT_MILLIS, timeToWait); + expiresInMillis = MAX_REFRESH_WINDOW_MILLIS; + timeToWait = AuthSession.getTimeToWaitByExpiresInMills(expiresInMillis); + assertEquals(timeToWait, MIN_REFRESH_WAIT_MILLIS); + expiresInMillis = MAX_REFRESH_WINDOW_MILLIS * 2L; + timeToWait = AuthSession.getTimeToWaitByExpiresInMills(expiresInMillis); + assertEquals(timeToWait, MAX_REFRESH_WINDOW_MILLIS); + } + + private Pair generateTokenAndWriteToFile(String fileName) throws IOException { + File tokenFile = folder.newFile(fileName); + String token = UUID.randomUUID().toString(); + FileUtils.writeStringToFile(tokenFile, token); + return Pair.of(tokenFile, token); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/auth/CredentialsProviderFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/auth/CredentialsProviderFactoryTest.java new file mode 100644 index 0000000000000..e62a65a79aed6 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/rest/auth/CredentialsProviderFactoryTest.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.rest.auth; + +import org.apache.paimon.options.Options; +import org.apache.paimon.rest.RESTCatalogOptions; + +import org.apache.commons.io.FileUtils; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.time.Duration; +import java.util.UUID; + +import static org.apache.paimon.rest.RESTCatalogInternalOptions.CREDENTIALS_PROVIDER; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +/** Test for {@link CredentialsProviderFactory}. */ +public class CredentialsProviderFactoryTest { + + @Rule public TemporaryFolder folder = new TemporaryFolder(); + + @Test + public void testCreateBearTokenCredentialsProviderSuccess() { + Options options = new Options(); + String token = UUID.randomUUID().toString(); + options.set(RESTCatalogOptions.TOKEN, token); + BearTokenCredentialsProvider credentialsProvider = + (BearTokenCredentialsProvider) + CredentialsProviderFactory.createCredentialsProvider( + options, this.getClass().getClassLoader()); + assertEquals(token, credentialsProvider.token()); + } + + @Test + public void testCreateBearTokenCredentialsProviderFail() { + Options options = new Options(); + assertThrows( + IllegalArgumentException.class, + () -> + CredentialsProviderFactory.createCredentialsProvider( + options, this.getClass().getClassLoader())); + } + + @Test + public void testCreateBearTokenFileCredentialsProviderSuccess() throws Exception { + Options options = new Options(); + String fileName = "token"; + File tokenFile = folder.newFile(fileName); + String token = UUID.randomUUID().toString(); + FileUtils.writeStringToFile(tokenFile, token); + options.set(RESTCatalogOptions.TOKEN_PROVIDER_PATH, tokenFile.getPath()); + BearTokenFileCredentialsProvider credentialsProvider = + (BearTokenFileCredentialsProvider) + CredentialsProviderFactory.createCredentialsProvider( + options, this.getClass().getClassLoader()); + assertEquals(token, credentialsProvider.token()); + } + + @Test + public void testCreateBearTokenFileCredentialsProviderFail() throws Exception { + Options options = new Options(); + options.set(CREDENTIALS_PROVIDER, CredentialsProviderType.BEAR_TOKEN_FILE.name()); + assertThrows( + IllegalArgumentException.class, + () -> + CredentialsProviderFactory.createCredentialsProvider( + options, this.getClass().getClassLoader())); + } + + @Test + public void testCreateRefreshBearTokenFileCredentialsProviderSuccess() throws Exception { + Options options = new Options(); + String fileName = "token"; + File tokenFile = folder.newFile(fileName); + String token = UUID.randomUUID().toString(); + FileUtils.writeStringToFile(tokenFile, token); + options.set(RESTCatalogOptions.TOKEN_PROVIDER_PATH, tokenFile.getPath()); + options.set(RESTCatalogOptions.TOKEN_EXPIRATION_TIME, Duration.ofSeconds(10L)); + BearTokenFileCredentialsProvider credentialsProvider = + (BearTokenFileCredentialsProvider) + CredentialsProviderFactory.createCredentialsProvider( + options, this.getClass().getClassLoader()); + assertEquals(token, credentialsProvider.token()); + } + + @Test + public void getCredentialsProviderTypeByConfWhenDefineTokenPath() { + Options options = new Options(); + options.set(RESTCatalogOptions.TOKEN_PROVIDER_PATH, "/a/b/c"); + assertEquals( + CredentialsProviderType.BEAR_TOKEN_FILE, + CredentialsProviderFactory.getCredentialsProviderTypeByConf(options)); + } + + @Test + public void getCredentialsProviderTypeByConfWhenConfNotDefined() { + Options options = new Options(); + assertEquals( + CredentialsProviderType.BEAR_TOKEN, + CredentialsProviderFactory.getCredentialsProviderTypeByConf(options)); + } + + @Test + public void getCredentialsProviderTypeByConfWhenDefineProviderType() { + Options options = new Options(); + options.set(CREDENTIALS_PROVIDER, CredentialsProviderType.BEAR_TOKEN_FILE.name()); + assertEquals( + CredentialsProviderType.BEAR_TOKEN_FILE, + CredentialsProviderFactory.getCredentialsProviderTypeByConf(options)); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java index ac8d4cd91e1d8..c8b102b3584dc 100644 --- a/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/schema/SchemaManagerTest.java @@ -31,6 +31,7 @@ import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.table.sink.TableCommitImpl; import org.apache.paimon.table.sink.TableWriteImpl; +import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; @@ -555,7 +556,7 @@ public void testAddAndDropNestedColumns() throws Exception { SchemaChange addColumn = SchemaChange.addColumn( - Arrays.asList("v", "f2", "f3"), + new String[] {"v", "f2", "f3"}, DataTypes.STRING(), "", SchemaChange.Move.after("f3", "f1")); @@ -579,11 +580,11 @@ public void testAddAndDropNestedColumns() throws Exception { .hasMessageContaining("Column v.f2.f3 already exists"); SchemaChange middleColumnNotExistAddColumn = SchemaChange.addColumn( - Arrays.asList("v", "invalid", "f4"), DataTypes.STRING(), "", null); + new String[] {"v", "invalid", "f4"}, DataTypes.STRING(), "", null); assertThatCode(() -> manager.commitChanges(middleColumnNotExistAddColumn)) .hasMessageContaining("Column v.invalid does not exist"); - SchemaChange dropColumn = SchemaChange.dropColumn(Arrays.asList("v", "f2", "f1")); + SchemaChange dropColumn = SchemaChange.dropColumn(new String[] {"v", "f2", "f1"}); manager.commitChanges(dropColumn); innerType = @@ -602,7 +603,7 @@ public void testAddAndDropNestedColumns() throws Exception { assertThatCode(() -> manager.commitChanges(dropColumn)) .hasMessageContaining("Column v.f2.f1 does not exist"); SchemaChange middleColumnNotExistDropColumn = - SchemaChange.dropColumn(Arrays.asList("v", "invalid", "f2")); + SchemaChange.dropColumn(new String[] {"v", "invalid", "f2"}); assertThatCode(() -> manager.commitChanges(middleColumnNotExistDropColumn)) .hasMessageContaining("Column v.invalid does not exist"); } @@ -632,7 +633,7 @@ public void testRenameNestedColumns() throws Exception { manager.createTable(schema); SchemaChange renameColumn = - SchemaChange.renameColumn(Arrays.asList("v", "f2", "f1"), "f100"); + SchemaChange.renameColumn(new String[] {"v", "f2", "f1"}, "f100"); manager.commitChanges(renameColumn); innerType = @@ -649,17 +650,17 @@ public void testRenameNestedColumns() throws Exception { assertThat(manager.latest().get().logicalRowType()).isEqualTo(outerType); SchemaChange middleColumnNotExistRenameColumn = - SchemaChange.renameColumn(Arrays.asList("v", "invalid", "f2"), "f200"); + SchemaChange.renameColumn(new String[] {"v", "invalid", "f2"}, "f200"); assertThatCode(() -> manager.commitChanges(middleColumnNotExistRenameColumn)) .hasMessageContaining("Column v.invalid does not exist"); SchemaChange lastColumnNotExistRenameColumn = - SchemaChange.renameColumn(Arrays.asList("v", "f2", "invalid"), "new_invalid"); + SchemaChange.renameColumn(new String[] {"v", "f2", "invalid"}, "new_invalid"); assertThatCode(() -> manager.commitChanges(lastColumnNotExistRenameColumn)) .hasMessageContaining("Column v.f2.invalid does not exist"); SchemaChange newNameAlreadyExistRenameColumn = - SchemaChange.renameColumn(Arrays.asList("v", "f2", "f2"), "f100"); + SchemaChange.renameColumn(new String[] {"v", "f2", "f2"}, "f100"); assertThatCode(() -> manager.commitChanges(newNameAlreadyExistRenameColumn)) .hasMessageContaining("Column v.f2.f100 already exists"); } @@ -690,7 +691,7 @@ public void testUpdateNestedColumnType() throws Exception { SchemaChange updateColumnType = SchemaChange.updateColumnType( - Arrays.asList("v", "f2", "f1"), DataTypes.BIGINT(), true); + new String[] {"v", "f2", "f1"}, DataTypes.BIGINT(), false); manager.commitChanges(updateColumnType); innerType = @@ -708,8 +709,55 @@ public void testUpdateNestedColumnType() throws Exception { SchemaChange middleColumnNotExistUpdateColumnType = SchemaChange.updateColumnType( - Arrays.asList("v", "invalid", "f1"), DataTypes.BIGINT(), true); + new String[] {"v", "invalid", "f1"}, DataTypes.BIGINT(), false); assertThatCode(() -> manager.commitChanges(middleColumnNotExistUpdateColumnType)) .hasMessageContaining("Column v.invalid does not exist"); } + + @Test + public void testUpdateRowTypeInArrayAndMap() throws Exception { + RowType innerType = + RowType.of( + new DataField(2, "f1", DataTypes.INT()), + new DataField(3, "f2", DataTypes.BIGINT())); + RowType outerType = + RowType.of( + new DataField(0, "k", DataTypes.INT()), + new DataField( + 1, "v", new ArrayType(new MapType(DataTypes.INT(), innerType)))); + + Schema schema = + new Schema( + outerType.getFields(), + Collections.singletonList("k"), + Collections.emptyList(), + new HashMap<>(), + ""); + SchemaManager manager = new SchemaManager(LocalFileIO.create(), path); + manager.createTable(schema); + + SchemaChange addColumn = + SchemaChange.addColumn( + new String[] {"v", "element", "value", "f3"}, + DataTypes.STRING(), + null, + SchemaChange.Move.first("f3")); + SchemaChange dropColumn = + SchemaChange.dropColumn(new String[] {"v", "element", "value", "f2"}); + SchemaChange updateColumnType = + SchemaChange.updateColumnType( + new String[] {"v", "element", "value", "f1"}, DataTypes.BIGINT(), false); + manager.commitChanges(addColumn, dropColumn, updateColumnType); + + innerType = + RowType.of( + new DataField(4, "f3", DataTypes.STRING()), + new DataField(2, "f1", DataTypes.BIGINT())); + outerType = + RowType.of( + new DataField(0, "k", DataTypes.INT()), + new DataField( + 1, "v", new ArrayType(new MapType(DataTypes.INT(), innerType)))); + assertThat(manager.latest().get().logicalRowType()).isEqualTo(outerType); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java b/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java index 494b2e28e4593..ce8cfc9228adf 100644 --- a/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/stats/StatsTableTest.java @@ -33,20 +33,25 @@ import org.apache.paimon.table.TableTestBase; import org.apache.paimon.types.DataTypes; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import static org.apache.paimon.CoreOptions.METADATA_STATS_DENSE_STORE; import static org.apache.paimon.CoreOptions.METADATA_STATS_MODE; import static org.assertj.core.api.Assertions.assertThat; /** Test for table stats mode. */ public class StatsTableTest extends TableTestBase { - @Test - public void testPartitionStats() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testPartitionStatsNotDense(boolean thinMode) throws Exception { Identifier identifier = identifier("T"); Options options = new Options(); options.set(METADATA_STATS_MODE, "NONE"); + options.set(METADATA_STATS_DENSE_STORE, false); options.set(CoreOptions.BUCKET, 1); + options.set(CoreOptions.DATA_FILE_THIN_MODE, thinMode); Schema schema = Schema.newBuilder() .column("pt", DataTypes.INT()) @@ -84,10 +89,65 @@ public void testPartitionStats() throws Exception { manifestFile.read(manifest.fileName(), manifest.fileSize()).get(0).file(); SimpleStats recordStats = file.valueStats(); assertThat(recordStats.minValues().isNullAt(0)).isTrue(); - assertThat(recordStats.minValues().isNullAt(1)).isTrue(); + assertThat(recordStats.minValues().isNullAt(1)).isEqualTo(!thinMode); assertThat(recordStats.minValues().isNullAt(2)).isTrue(); assertThat(recordStats.maxValues().isNullAt(0)).isTrue(); - assertThat(recordStats.maxValues().isNullAt(1)).isTrue(); + assertThat(recordStats.maxValues().isNullAt(1)).isEqualTo(!thinMode); assertThat(recordStats.maxValues().isNullAt(2)).isTrue(); + + SimpleStats keyStats = file.keyStats(); + assertThat(keyStats.minValues().isNullAt(0)).isFalse(); + assertThat(keyStats.maxValues().isNullAt(0)).isFalse(); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testPartitionStatsDenseMode(boolean thinMode) throws Exception { + Identifier identifier = identifier("T"); + Options options = new Options(); + options.set(METADATA_STATS_MODE, "NONE"); + options.set(CoreOptions.BUCKET, 1); + options.set(CoreOptions.DATA_FILE_THIN_MODE, thinMode); + Schema schema = + Schema.newBuilder() + .column("pt", DataTypes.INT()) + .column("pk", DataTypes.INT()) + .column("col1", DataTypes.INT()) + .partitionKeys("pt") + .primaryKey("pk", "pt") + .options(options.toMap()) + .build(); + catalog.createTable(identifier, schema, true); + Table table = catalog.getTable(identifier); + + write( + table, + GenericRow.of(1, 1, 1), + GenericRow.of(1, 2, 1), + GenericRow.of(1, 3, 1), + GenericRow.of(2, 1, 1)); + + FileStoreTable storeTable = (FileStoreTable) table; + FileStore store = storeTable.store(); + String manifestListFile = storeTable.snapshotManager().latestSnapshot().deltaManifestList(); + + ManifestList manifestList = store.manifestListFactory().create(); + ManifestFileMeta manifest = manifestList.read(manifestListFile).get(0); + + // should have partition stats + SimpleStats partitionStats = manifest.partitionStats(); + assertThat(partitionStats.minValues().getInt(0)).isEqualTo(1); + assertThat(partitionStats.maxValues().getInt(0)).isEqualTo(2); + + // should not have record stats because of NONE mode + ManifestFile manifestFile = store.manifestFileFactory().create(); + DataFileMeta file = + manifestFile.read(manifest.fileName(), manifest.fileSize()).get(0).file(); + SimpleStats recordStats = file.valueStats(); + int count = thinMode ? 1 : 0; + assertThat(file.valueStatsCols().size()).isEqualTo(count); + assertThat(recordStats.minValues().getFieldCount()).isEqualTo(count); + assertThat(recordStats.maxValues().getFieldCount()).isEqualTo(count); + assertThat(recordStats.nullCounts().size()).isEqualTo(count); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileDataTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileDataTableTest.java index 93854e7661987..9ce3db0b1ada0 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileDataTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileDataTableTest.java @@ -33,7 +33,7 @@ protected FileStoreTable createFileStoreTable(Map tableSchema return new AppendOnlyFileStoreTable( FileIOFinder.find(tablePath), tablePath, schemaManager.latest().get()) { @Override - protected SchemaManager schemaManager() { + public SchemaManager schemaManager() { return schemaManager; } }; diff --git a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java index 81dd30262058a..01d4e89af95d6 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java @@ -78,7 +78,9 @@ import static org.apache.paimon.CoreOptions.BUCKET; import static org.apache.paimon.CoreOptions.BUCKET_KEY; +import static org.apache.paimon.CoreOptions.DATA_FILE_PATH_DIRECTORY; import static org.apache.paimon.CoreOptions.FILE_INDEX_IN_MANIFEST_THRESHOLD; +import static org.apache.paimon.CoreOptions.METADATA_STATS_MODE; import static org.apache.paimon.io.DataFileTestUtils.row; import static org.apache.paimon.table.sink.KeyAndBucketExtractor.bucket; import static org.apache.paimon.table.sink.KeyAndBucketExtractor.bucketKeyHashCode; @@ -142,6 +144,26 @@ public void testBatchReadWrite() throws Exception { "2|21|201|binary|varbinary|mapKey:mapVal|multiset")); } + @Test + public void testReadWriteWithDataDirectory() throws Exception { + Consumer optionsSetter = options -> options.set(DATA_FILE_PATH_DIRECTORY, "data"); + writeData(optionsSetter); + FileStoreTable table = createFileStoreTable(optionsSetter); + + assertThat(table.fileIO().exists(new Path(tablePath, "data/pt=1"))).isTrue(); + + List splits = toSplits(table.newSnapshotReader().read().dataSplits()); + TableRead read = table.newRead(); + assertThat(getResult(read, splits, binaryRow(1), 0, BATCH_ROW_TO_STRING)) + .hasSameElementsAs( + Arrays.asList( + "1|10|100|binary|varbinary|mapKey:mapVal|multiset", + "1|11|101|binary|varbinary|mapKey:mapVal|multiset", + "1|12|102|binary|varbinary|mapKey:mapVal|multiset", + "1|11|101|binary|varbinary|mapKey:mapVal|multiset", + "1|12|102|binary|varbinary|mapKey:mapVal|multiset")); + } + @Test public void testBatchRecordsWrite() throws Exception { FileStoreTable table = createFileStoreTable(); @@ -229,7 +251,6 @@ public void testBatchProjection() throws Exception { public void testBatchFilter(boolean statsDenseStore) throws Exception { Consumer optionsSetter = options -> { - options.set(CoreOptions.METADATA_STATS_DENSE_STORE, statsDenseStore); if (statsDenseStore) { options.set(CoreOptions.METADATA_STATS_MODE, "none"); options.set("fields.b.stats-mode", "full"); @@ -574,6 +595,7 @@ public void testBSIAndBitmapIndexInMemory() throws Exception { createUnawareBucketFileStoreTable( rowType, options -> { + options.set(METADATA_STATS_MODE, "NONE"); options.set( FileIndexOptions.FILE_INDEX + "." @@ -600,7 +622,11 @@ public void testBSIAndBitmapIndexInMemory() throws Exception { write.write(GenericRow.of(1, BinaryString.fromString("B"), 3L)); write.write(GenericRow.of(1, BinaryString.fromString("C"), 3L)); result.addAll(write.prepareCommit(true, 0)); + write.write(GenericRow.of(1, BinaryString.fromString("A"), 4L)); + write.write(GenericRow.of(1, BinaryString.fromString("B"), 3L)); write.write(GenericRow.of(1, BinaryString.fromString("C"), 4L)); + write.write(GenericRow.of(1, BinaryString.fromString("D"), 2L)); + write.write(GenericRow.of(1, BinaryString.fromString("D"), 4L)); result.addAll(write.prepareCommit(true, 0)); commit.commit(0, result); result.clear(); @@ -639,6 +665,7 @@ public void testBSIAndBitmapIndexInDisk() throws Exception { createUnawareBucketFileStoreTable( rowType, options -> { + options.set(METADATA_STATS_MODE, "NONE"); options.set( FileIndexOptions.FILE_INDEX + "." @@ -665,7 +692,11 @@ public void testBSIAndBitmapIndexInDisk() throws Exception { write.write(GenericRow.of(1, BinaryString.fromString("B"), 3L)); write.write(GenericRow.of(1, BinaryString.fromString("C"), 3L)); result.addAll(write.prepareCommit(true, 0)); + write.write(GenericRow.of(1, BinaryString.fromString("A"), 4L)); + write.write(GenericRow.of(1, BinaryString.fromString("B"), 3L)); write.write(GenericRow.of(1, BinaryString.fromString("C"), 4L)); + write.write(GenericRow.of(1, BinaryString.fromString("D"), 2L)); + write.write(GenericRow.of(1, BinaryString.fromString("D"), 4L)); result.addAll(write.prepareCommit(true, 0)); commit.commit(0, result); result.clear(); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableColumnTypeFileDataTest.java b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableColumnTypeFileDataTest.java index b4c16cef20a7a..64d0c728d10b2 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableColumnTypeFileDataTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableColumnTypeFileDataTest.java @@ -37,7 +37,7 @@ protected FileStoreTable createFileStoreTable(Map tableSchema SchemaManager schemaManager = new TestingSchemaManager(tablePath, tableSchemas); return new AppendOnlyFileStoreTable(fileIO, tablePath, schemaManager.latest().get()) { @Override - protected SchemaManager schemaManager() { + public SchemaManager schemaManager() { return schemaManager; } }; diff --git a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableColumnTypeFileMetaTest.java b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableColumnTypeFileMetaTest.java index f398d28cc524f..300483a9f34bf 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableColumnTypeFileMetaTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableColumnTypeFileMetaTest.java @@ -40,7 +40,7 @@ protected FileStoreTable createFileStoreTable(Map tableSchema SchemaManager schemaManager = new TestingSchemaManager(tablePath, tableSchemas); return new AppendOnlyFileStoreTable(fileIO, tablePath, schemaManager.latest().get()) { @Override - protected SchemaManager schemaManager() { + public SchemaManager schemaManager() { return schemaManager; } }; diff --git a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableFileMetaFilterTest.java b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableFileMetaFilterTest.java index f65546af75d84..85ed80299736e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableFileMetaFilterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyTableFileMetaFilterTest.java @@ -40,7 +40,7 @@ protected FileStoreTable createFileStoreTable(Map tableSchema SchemaManager schemaManager = new TestingSchemaManager(tablePath, tableSchemas); return new AppendOnlyFileStoreTable(fileIO, tablePath, schemaManager.latest().get()) { @Override - protected SchemaManager schemaManager() { + public SchemaManager schemaManager() { return schemaManager; } }; diff --git a/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java b/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java index f6343bfe437f0..75e284a68c3a9 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java @@ -1193,7 +1193,7 @@ public void testCreateBranch() throws Exception { SchemaManager schemaManager = new SchemaManager(new TraceableFileIO(), tablePath, "test-branch"); TableSchema branchSchema = - SchemaManager.fromPath(new TraceableFileIO(), schemaManager.toSchemaPath(0)); + TableSchema.fromPath(new TraceableFileIO(), schemaManager.toSchemaPath(0)); TableSchema schema0 = schemaManager.schema(0); assertThat(branchSchema.equals(schema0)).isTrue(); } @@ -1344,7 +1344,7 @@ public void testFastForward() throws Exception { // verify schema in branch1 and main branch is same SchemaManager schemaManager = new SchemaManager(new TraceableFileIO(), tablePath); TableSchema branchSchema = - SchemaManager.fromPath( + TableSchema.fromPath( new TraceableFileIO(), schemaManager.copyWithBranch(BRANCH_NAME).toSchemaPath(0)); TableSchema schema0 = schemaManager.schema(0); @@ -1473,10 +1473,10 @@ public void testAsyncExpireExecutionMode() throws Exception { TestFileStore.getFilesInUse( latestSnapshotId, snapshotManager, - store.newScan(), table.fileIO(), store.pathFactory(), - store.manifestListFactory().create()); + store.manifestListFactory().create(), + store.manifestFileFactory().create()); List unusedFileList = Files.walk(Paths.get(tempDir.toString())) diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyColumnTypeFileDataTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyColumnTypeFileDataTest.java index 8ba25c6617fe5..64bb5f21abbb4 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyColumnTypeFileDataTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyColumnTypeFileDataTest.java @@ -91,7 +91,7 @@ protected FileStoreTable createFileStoreTable(Map tableSchema SchemaManager schemaManager = new TestingSchemaManager(tablePath, tableSchemas); return new PrimaryKeyFileStoreTable(fileIO, tablePath, schemaManager.latest().get()) { @Override - protected SchemaManager schemaManager() { + public SchemaManager schemaManager() { return schemaManager; } }; diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileDataTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileDataTableTest.java index 1be321975466f..ba98138044985 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileDataTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileDataTableTest.java @@ -247,7 +247,7 @@ protected FileStoreTable createFileStoreTable(Map tableSchema return new PrimaryKeyFileStoreTable(fileIO, tablePath, schemaManager.latest().get()) { @Override - protected SchemaManager schemaManager() { + public SchemaManager schemaManager() { return schemaManager; } }; diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileMetaFilterTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileMetaFilterTest.java index 618e8691c65dc..88928fe991bcb 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileMetaFilterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileMetaFilterTest.java @@ -146,7 +146,7 @@ protected FileStoreTable createFileStoreTable(Map tableSchema SchemaManager schemaManager = new TestingSchemaManager(tablePath, tableSchemas); return new PrimaryKeyFileStoreTable(fileIO, tablePath, schemaManager.latest().get()) { @Override - protected SchemaManager schemaManager() { + public SchemaManager schemaManager() { return schemaManager; } }; diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java index dca86aa61ec28..fa635e2ab6669 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java @@ -84,6 +84,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; @@ -99,6 +100,7 @@ import static org.apache.paimon.CoreOptions.ChangelogProducer.LOOKUP; import static org.apache.paimon.CoreOptions.DELETION_VECTORS_ENABLED; import static org.apache.paimon.CoreOptions.FILE_FORMAT; +import static org.apache.paimon.CoreOptions.FILE_INDEX_IN_MANIFEST_THRESHOLD; import static org.apache.paimon.CoreOptions.LOOKUP_LOCAL_FILE_TYPE; import static org.apache.paimon.CoreOptions.MERGE_ENGINE; import static org.apache.paimon.CoreOptions.MergeEngine; @@ -120,17 +122,6 @@ /** Tests for {@link PrimaryKeyFileStoreTable}. */ public class PrimaryKeyFileStoreTableTest extends FileStoreTableTestBase { - protected static final RowType COMPATIBILITY_ROW_TYPE = - RowType.of( - new DataType[] { - DataTypes.INT(), - DataTypes.INT(), - DataTypes.BIGINT(), - DataTypes.BINARY(1), - DataTypes.VARBINARY(1) - }, - new String[] {"pt", "a", "b", "c", "d"}); - protected static final Function COMPATIBILITY_BATCH_ROW_TO_STRING = rowData -> rowData.getInt(0) @@ -143,12 +134,6 @@ public class PrimaryKeyFileStoreTableTest extends FileStoreTableTestBase { + "|" + new String(rowData.getBinary(4)); - protected static final Function COMPATIBILITY_CHANGELOG_ROW_TO_STRING = - rowData -> - rowData.getRowKind().shortString() - + " " - + COMPATIBILITY_BATCH_ROW_TO_STRING.apply(rowData); - @Test public void testMultipleWriters() throws Exception { WriteSelector selector = @@ -364,7 +349,6 @@ public void testBatchProjection() throws Exception { public void testBatchFilter(boolean statsDenseStore) throws Exception { Consumer optionsSetter = options -> { - options.set(CoreOptions.METADATA_STATS_DENSE_STORE, statsDenseStore); if (statsDenseStore) { // pk table doesn't need value stats options.set(CoreOptions.METADATA_STATS_MODE, "none"); @@ -826,6 +810,123 @@ public void testDeletionVectorsWithFileIndexInFile() throws Exception { "1|4|500|binary|varbinary|mapKey:mapVal|multiset")); } + @Test + public void testDeletionVectorsWithParquetFilter() throws Exception { + // RowGroup record range [pk] : + // + // RowGroup-0 : [0-93421) + // RowGroup-1 : [93421-187794) + // RowGroup-2 : [187794-200000) + // + // ColumnPage record count : + // + // col-0 : 300 + // col-1 : 200 + // col-2 : 300 + // col-3 : 300 + // col-4 : 300 + // col-5 : 200 + // col-6 : 100 + // col-7 : 100 + // col-8 : 100 + // col-9 : 100 + // col-10 : 100 + // col-11 : 300 + + FileStoreTable table = + createFileStoreTable( + conf -> { + conf.set(BUCKET, 1); + conf.set(DELETION_VECTORS_ENABLED, true); + conf.set(FILE_FORMAT, "parquet"); + conf.set("parquet.block.size", "1048576"); + conf.set("parquet.page.size", "1024"); + }); + + BatchWriteBuilder writeBuilder = table.newBatchWriteBuilder(); + + BatchTableWrite write = + (BatchTableWrite) + writeBuilder + .newWrite() + .withIOManager(new IOManagerImpl(tempDir.toString())); + + for (int i = 0; i < 200000; i++) { + write.write(rowData(1, i, i * 100L)); + } + + List messages = write.prepareCommit(); + BatchTableCommit commit = writeBuilder.newCommit(); + commit.commit(messages); + write = + (BatchTableWrite) + writeBuilder + .newWrite() + .withIOManager(new IOManagerImpl(tempDir.toString())); + for (int i = 110000; i < 115000; i++) { + write.write(rowDataWithKind(RowKind.DELETE, 1, i, i * 100L)); + } + + for (int i = 130000; i < 135000; i++) { + write.write(rowDataWithKind(RowKind.DELETE, 1, i, i * 100L)); + } + + messages = write.prepareCommit(); + commit = writeBuilder.newCommit(); + commit.commit(messages); + + PredicateBuilder builder = new PredicateBuilder(ROW_TYPE); + List splits = toSplits(table.newSnapshotReader().read().dataSplits()); + Random random = new Random(); + + // point filter + + for (int i = 0; i < 10; i++) { + int value = random.nextInt(110000); + TableRead read = table.newRead().withFilter(builder.equal(1, value)).executeFilter(); + assertThat(getResult(read, splits, BATCH_ROW_TO_STRING)) + .isEqualTo( + Arrays.asList( + String.format( + "%d|%d|%d|binary|varbinary|mapKey:mapVal|multiset", + 1, value, value * 100L))); + } + + for (int i = 0; i < 10; i++) { + int value = 130000 + random.nextInt(5000); + TableRead read = table.newRead().withFilter(builder.equal(1, value)).executeFilter(); + assertThat(getResult(read, splits, BATCH_ROW_TO_STRING)).isEmpty(); + } + + TableRead tableRead = + table.newRead() + .withFilter( + PredicateBuilder.and( + builder.greaterOrEqual(1, 100000), + builder.lessThan(1, 150000))) + .executeFilter(); + + List result = getResult(tableRead, splits, BATCH_ROW_TO_STRING); + + assertThat(result.size()).isEqualTo(40000); // filter 10000 + + assertThat(result) + .doesNotContain("1|110000|11000000|binary|varbinary|mapKey:mapVal|multiset"); + assertThat(result) + .doesNotContain("1|114999|11499900|binary|varbinary|mapKey:mapVal|multiset"); + assertThat(result) + .doesNotContain("1|130000|13000000|binary|varbinary|mapKey:mapVal|multiset"); + assertThat(result) + .doesNotContain("1|134999|13499900|binary|varbinary|mapKey:mapVal|multiset"); + assertThat(result).contains("1|100000|10000000|binary|varbinary|mapKey:mapVal|multiset"); + assertThat(result).contains("1|149999|14999900|binary|varbinary|mapKey:mapVal|multiset"); + + assertThat(result).contains("1|101099|10109900|binary|varbinary|mapKey:mapVal|multiset"); + assertThat(result).contains("1|115000|11500000|binary|varbinary|mapKey:mapVal|multiset"); + assertThat(result).contains("1|129999|12999900|binary|varbinary|mapKey:mapVal|multiset"); + assertThat(result).contains("1|135000|13500000|binary|varbinary|mapKey:mapVal|multiset"); + } + @Test public void testDeletionVectorsWithFileIndexInMeta() throws Exception { FileStoreTable table = @@ -862,6 +963,92 @@ public void testDeletionVectorsWithFileIndexInMeta() throws Exception { assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(1); } + @Test + public void testDeletionVectorsWithBitmapFileIndexInFile() throws Exception { + FileStoreTable table = + createFileStoreTable( + conf -> { + conf.set(BUCKET, 1); + conf.set(DELETION_VECTORS_ENABLED, true); + conf.set(TARGET_FILE_SIZE, MemorySize.ofBytes(1)); + conf.set(FILE_INDEX_IN_MANIFEST_THRESHOLD, MemorySize.ofBytes(1)); + conf.set("file-index.bitmap.columns", "b"); + }); + + StreamTableWrite write = + table.newWrite(commitUser).withIOManager(new IOManagerImpl(tempDir.toString())); + StreamTableCommit commit = table.newCommit(commitUser); + + write.write(rowData(1, 1, 300L)); + write.write(rowData(1, 2, 400L)); + write.write(rowData(1, 3, 100L)); + write.write(rowData(1, 4, 100L)); + commit.commit(0, write.prepareCommit(true, 0)); + + write.write(rowData(1, 1, 100L)); + write.write(rowData(1, 2, 100L)); + write.write(rowData(1, 3, 300L)); + write.write(rowData(1, 5, 100L)); + commit.commit(1, write.prepareCommit(true, 1)); + + write.write(rowData(1, 4, 200L)); + commit.commit(2, write.prepareCommit(true, 2)); + + PredicateBuilder builder = new PredicateBuilder(ROW_TYPE); + List splits = toSplits(table.newSnapshotReader().read().dataSplits()); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(2); + TableRead read = table.newRead().withFilter(builder.equal(2, 100L)); + assertThat(getResult(read, splits, BATCH_ROW_TO_STRING)) + .hasSameElementsAs( + Arrays.asList( + "1|1|100|binary|varbinary|mapKey:mapVal|multiset", + "1|2|100|binary|varbinary|mapKey:mapVal|multiset", + "1|5|100|binary|varbinary|mapKey:mapVal|multiset")); + } + + @Test + public void testDeletionVectorsWithBitmapFileIndexInMeta() throws Exception { + FileStoreTable table = + createFileStoreTable( + conf -> { + conf.set(BUCKET, 1); + conf.set(DELETION_VECTORS_ENABLED, true); + conf.set(TARGET_FILE_SIZE, MemorySize.ofBytes(1)); + conf.set(FILE_INDEX_IN_MANIFEST_THRESHOLD, MemorySize.ofMebiBytes(1)); + conf.set("file-index.bitmap.columns", "b"); + }); + + StreamTableWrite write = + table.newWrite(commitUser).withIOManager(new IOManagerImpl(tempDir.toString())); + StreamTableCommit commit = table.newCommit(commitUser); + + write.write(rowData(1, 1, 300L)); + write.write(rowData(1, 2, 400L)); + write.write(rowData(1, 3, 100L)); + write.write(rowData(1, 4, 100L)); + commit.commit(0, write.prepareCommit(true, 0)); + + write.write(rowData(1, 1, 100L)); + write.write(rowData(1, 2, 100L)); + write.write(rowData(1, 3, 300L)); + write.write(rowData(1, 5, 100L)); + commit.commit(1, write.prepareCommit(true, 1)); + + write.write(rowData(1, 4, 200L)); + commit.commit(2, write.prepareCommit(true, 2)); + + PredicateBuilder builder = new PredicateBuilder(ROW_TYPE); + List splits = toSplits(table.newSnapshotReader().read().dataSplits()); + assertThat(((DataSplit) splits.get(0)).dataFiles().size()).isEqualTo(2); + TableRead read = table.newRead().withFilter(builder.equal(2, 100L)); + assertThat(getResult(read, splits, BATCH_ROW_TO_STRING)) + .hasSameElementsAs( + Arrays.asList( + "1|1|100|binary|varbinary|mapKey:mapVal|multiset", + "1|2|100|binary|varbinary|mapKey:mapVal|multiset", + "1|5|100|binary|varbinary|mapKey:mapVal|multiset")); + } + @Test public void testWithShardFirstRow() throws Exception { FileStoreTable table = @@ -1126,6 +1313,90 @@ public void testPartialUpdateRemoveRecordOnDelete() throws Exception { commit.close(); } + @Test + public void testPartialUpdateRemoveRecordOnSequenceGroup() throws Exception { + RowType rowType = + RowType.of( + new DataType[] { + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT(), + DataTypes.INT() + }, + new String[] {"pt", "a", "b", "seq1", "c", "d", "seq2"}); + FileStoreTable table = + createFileStoreTable( + options -> { + options.set("merge-engine", "partial-update"); + options.set("fields.seq1.sequence-group", "b"); + options.set("fields.seq2.sequence-group", "c,d"); + options.set("partial-update.remove-record-on-sequence-group", "seq2"); + }, + rowType); + FileStoreTable wrongTable = + createFileStoreTable( + options -> { + options.set("merge-engine", "partial-update"); + options.set("fields.seq1.sequence-group", "b"); + options.set("fields.seq2.sequence-group", "c,d"); + options.set("partial-update.remove-record-on-sequence-group", "b"); + }, + rowType); + Function rowToString = row -> internalRowToString(row, rowType); + + assertThatThrownBy(() -> wrongTable.newWrite("")) + .hasMessageContaining( + "field 'b' defined in 'partial-update.remove-record-on-sequence-group' option must be part of sequence groups"); + + SnapshotReader snapshotReader = table.newSnapshotReader(); + TableRead read = table.newRead(); + StreamTableWrite write = table.newWrite(""); + StreamTableCommit commit = table.newCommit(""); + // 1. Inserts + write.write(GenericRow.of(1, 1, 10, 1, 20, 20, 1)); + write.write(GenericRow.of(1, 1, 11, 2, 25, 25, 0)); + write.write(GenericRow.of(1, 1, 12, 1, 29, 29, 2)); + commit.commit(0, write.prepareCommit(true, 0)); + List result = + getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).containsExactlyInAnyOrder("+I[1, 1, 11, 2, 29, 29, 2]"); + + // 2. Update Before + write.write(GenericRow.ofKind(RowKind.UPDATE_BEFORE, 1, 1, 11, 2, 29, 29, 2)); + commit.commit(1, write.prepareCommit(true, 1)); + result = getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).containsExactlyInAnyOrder("+I[1, 1, NULL, 2, NULL, NULL, 2]"); + + // 3. Update After + write.write(GenericRow.ofKind(RowKind.UPDATE_AFTER, 1, 1, 11, 2, 30, 30, 3)); + commit.commit(2, write.prepareCommit(true, 2)); + result = getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).containsExactlyInAnyOrder("+I[1, 1, 11, 2, 30, 30, 3]"); + + // 4. Retracts + write.write(GenericRow.ofKind(RowKind.DELETE, 1, 1, 12, 3, 30, 30, 2)); + commit.commit(3, write.prepareCommit(true, 3)); + result = getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).containsExactlyInAnyOrder("+I[1, 1, NULL, 3, 30, 30, 3]"); + + write.write(GenericRow.ofKind(RowKind.DELETE, 1, 1, 12, 2, 30, 31, 5)); + commit.commit(4, write.prepareCommit(true, 4)); + result = getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).isEmpty(); + + // 5. Inserts + write.write(GenericRow.of(1, 1, 11, 2, 30, 31, 6)); + commit.commit(5, write.prepareCommit(true, 5)); + result = getResult(read, toSplits(snapshotReader.read().dataSplits()), rowToString); + assertThat(result).containsExactlyInAnyOrder("+I[1, 1, 11, 2, 30, 31, 6]"); + + write.close(); + commit.close(); + } + @Test public void testPartialUpdateWithAgg() throws Exception { RowType rowType = @@ -1510,7 +1781,6 @@ public void testReadDeletionVectorTable(boolean statsDenseStore) throws Exceptio options.set(TARGET_FILE_SIZE, new MemorySize(1)); options.set(DELETION_VECTORS_ENABLED, true); - options.set(CoreOptions.METADATA_STATS_DENSE_STORE, statsDenseStore); if (statsDenseStore) { options.set(CoreOptions.METADATA_STATS_MODE, "none"); options.set("fields.b.stats-mode", "full"); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyTableColumnTypeFileMetaTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyTableColumnTypeFileMetaTest.java index 489c1ba052174..32a4138be564f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyTableColumnTypeFileMetaTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyTableColumnTypeFileMetaTest.java @@ -54,7 +54,7 @@ protected FileStoreTable createFileStoreTable(Map tableSchema SchemaManager schemaManager = new TestingSchemaManager(tablePath, tableSchemas); return new PrimaryKeyFileStoreTable(fileIO, tablePath, schemaManager.latest().get()) { @Override - protected SchemaManager schemaManager() { + public SchemaManager schemaManager() { return schemaManager; } }; diff --git a/paimon-core/src/test/java/org/apache/paimon/table/TableTestBase.java b/paimon-core/src/test/java/org/apache/paimon/table/TableTestBase.java index 7f850a7725b4c..7d7617cf8bd1a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/TableTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/TableTestBase.java @@ -160,6 +160,10 @@ protected void compact( } } + public void dropTableDefault() throws Exception { + catalog.dropTable(identifier(), true); + } + protected List read(Table table, Pair, String>... dynamicOptions) throws Exception { return read(table, null, dynamicOptions); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java b/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java index eb9105189b713..1f87838aea313 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/sink/CommitMessageSerializerTest.java @@ -48,7 +48,7 @@ public void test() throws IOException { CommitMessageImpl committable = new CommitMessageImpl(row(0), 1, dataIncrement, compactIncrement, indexIncrement); CommitMessageImpl newCommittable = - (CommitMessageImpl) serializer.deserialize(3, serializer.serialize(committable)); + (CommitMessageImpl) serializer.deserialize(5, serializer.serialize(committable)); assertThat(newCommittable.compactIncrement()).isEqualTo(committable.compactIncrement()); assertThat(newCommittable.newFilesIncrement()).isEqualTo(committable.newFilesIncrement()); assertThat(newCommittable.indexIncrement()).isEqualTo(committable.indexIncrement()); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java index a9e093dab124e..a1f7d69e28779 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitGeneratorTest.java @@ -43,10 +43,10 @@ public class SplitGeneratorTest { public static DataFileMeta newFileFromSequence( - String name, int rowCount, long minSequence, long maxSequence) { + String name, int fileSize, long minSequence, long maxSequence) { return new DataFileMeta( name, - rowCount, + fileSize, 1, EMPTY_ROW, EMPTY_ROW, diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java index c64a12ffae2c2..0219941a0ac0b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java @@ -49,6 +49,41 @@ /** Test for {@link DataSplit}. */ public class SplitTest { + @Test + public void testSplitMergedRowCount() { + // not rawConvertible + List dataFiles = + Arrays.asList(newDataFile(1000L), newDataFile(2000L), newDataFile(3000L)); + DataSplit split = newDataSplit(false, dataFiles, null); + assertThat(split.partialMergedRowCount()).isEqualTo(0L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(false); + + // rawConvertible without deletion files + split = newDataSplit(true, dataFiles, null); + assertThat(split.partialMergedRowCount()).isEqualTo(6000L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(true); + assertThat(split.mergedRowCount()).isEqualTo(6000L); + + // rawConvertible with deletion files without cardinality + ArrayList deletionFiles = new ArrayList<>(); + deletionFiles.add(null); + deletionFiles.add(new DeletionFile("p", 1, 2, null)); + deletionFiles.add(new DeletionFile("p", 1, 2, 100L)); + split = newDataSplit(true, dataFiles, deletionFiles); + assertThat(split.partialMergedRowCount()).isEqualTo(3900L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(false); + + // rawConvertible with deletion files with cardinality + deletionFiles = new ArrayList<>(); + deletionFiles.add(null); + deletionFiles.add(new DeletionFile("p", 1, 2, 200L)); + deletionFiles.add(new DeletionFile("p", 1, 2, 100L)); + split = newDataSplit(true, dataFiles, deletionFiles); + assertThat(split.partialMergedRowCount()).isEqualTo(5700L); + assertThat(split.mergedRowCountAvailable()).isEqualTo(true); + assertThat(split.mergedRowCount()).isEqualTo(5700L); + } + @Test public void testSerializer() throws IOException { DataFileTestDataGenerator gen = DataFileTestDataGenerator.builder().build(); @@ -107,6 +142,9 @@ public void testSerializerNormal() throws Exception { Arrays.asList("field1", "field2", "field3")); List dataFiles = Collections.singletonList(dataFile); + DeletionFile deletionFile = new DeletionFile("deletion_file", 100, 22, 33L); + List deletionFiles = Collections.singletonList(deletionFile); + BinaryRow partition = new BinaryRow(1); BinaryRowWriter binaryRowWriter = new BinaryRowWriter(partition); binaryRowWriter.writeString(0, BinaryString.fromString("aaaaa")); @@ -118,6 +156,7 @@ public void testSerializerNormal() throws Exception { .withPartition(partition) .withBucket(20) .withDataFiles(dataFiles) + .withDataDeletionFiles(deletionFiles) .withBucketPath("my path") .build(); @@ -243,4 +282,100 @@ public void testSerializerCompatibleV2() throws Exception { InstantiationUtil.deserializeObject(v2Bytes, DataSplit.class.getClassLoader()); assertThat(actual).isEqualTo(split); } + + @Test + public void testSerializerCompatibleV3() throws Exception { + SimpleStats keyStats = + new SimpleStats( + singleColumn("min_key"), + singleColumn("max_key"), + fromLongArray(new Long[] {0L})); + SimpleStats valueStats = + new SimpleStats( + singleColumn("min_value"), + singleColumn("max_value"), + fromLongArray(new Long[] {0L})); + + DataFileMeta dataFile = + new DataFileMeta( + "my_file", + 1024 * 1024, + 1024, + singleColumn("min_key"), + singleColumn("max_key"), + keyStats, + valueStats, + 15, + 200, + 5, + 3, + Arrays.asList("extra1", "extra2"), + Timestamp.fromLocalDateTime(LocalDateTime.parse("2022-03-02T20:20:12")), + 11L, + new byte[] {1, 2, 4}, + FileSource.COMPACT, + Arrays.asList("field1", "field2", "field3")); + List dataFiles = Collections.singletonList(dataFile); + + DeletionFile deletionFile = new DeletionFile("deletion_file", 100, 22, null); + List deletionFiles = Collections.singletonList(deletionFile); + + BinaryRow partition = new BinaryRow(1); + BinaryRowWriter binaryRowWriter = new BinaryRowWriter(partition); + binaryRowWriter.writeString(0, BinaryString.fromString("aaaaa")); + binaryRowWriter.complete(); + + DataSplit split = + DataSplit.builder() + .withSnapshot(18) + .withPartition(partition) + .withBucket(20) + .withDataFiles(dataFiles) + .withDataDeletionFiles(deletionFiles) + .withBucketPath("my path") + .build(); + + byte[] v2Bytes = + IOUtils.readFully( + SplitTest.class + .getClassLoader() + .getResourceAsStream("compatibility/datasplit-v3"), + true); + + DataSplit actual = + InstantiationUtil.deserializeObject(v2Bytes, DataSplit.class.getClassLoader()); + assertThat(actual).isEqualTo(split); + } + + private DataFileMeta newDataFile(long rowCount) { + return DataFileMeta.forAppend( + "my_data_file.parquet", + 1024 * 1024, + rowCount, + null, + 0L, + rowCount, + 1, + Collections.emptyList(), + null, + null, + null); + } + + private DataSplit newDataSplit( + boolean rawConvertible, + List dataFiles, + List deletionFiles) { + DataSplit.Builder builder = DataSplit.builder(); + builder.withSnapshot(1) + .withPartition(BinaryRow.EMPTY_ROW) + .withBucket(1) + .withBucketPath("my path") + .rawConvertible(rawConvertible) + .withDataFiles(dataFiles); + if (deletionFiles != null) { + builder.withDataDeletionFiles(deletionFiles); + } + return builder.build(); + } } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java index 1a692270ac5be..f0280560c2675 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java @@ -191,7 +191,16 @@ private List getExpectedResult(long snapshotId) { BinaryString.fromString( Arrays.toString(new String[] {partition1, partition2})), fileEntry.bucket(), - BinaryString.fromString(file.fileName()), + BinaryString.fromString( + table.location() + + "/pt1=" + + partition1 + + "/pt2=" + + partition2 + + "/bucket-" + + fileEntry.bucket() + + "/" + + file.fileName()), BinaryString.fromString(file.fileFormat()), file.schemaId(), file.level(), diff --git a/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java index a17dc75466a68..8d12dc707bf5d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/system/PartitionsTableTest.java @@ -40,6 +40,7 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import static org.assertj.core.api.Assertions.assertThat; @@ -76,7 +77,7 @@ public void before() throws Exception { partitionsTable = (PartitionsTable) catalog.getTable(filesTableId); // snapshot 1: append - write(table, GenericRow.of(1, 1, 1), GenericRow.of(1, 2, 5)); + write(table, GenericRow.of(1, 1, 1), GenericRow.of(1, 3, 5)); write(table, GenericRow.of(1, 1, 3), GenericRow.of(1, 2, 4)); } @@ -85,19 +86,36 @@ public void before() throws Exception { public void testPartitionRecordCount() throws Exception { List expectedRow = new ArrayList<>(); expectedRow.add(GenericRow.of(BinaryString.fromString("[1]"), 2L)); - expectedRow.add(GenericRow.of(BinaryString.fromString("[2]"), 2L)); + expectedRow.add(GenericRow.of(BinaryString.fromString("[2]"), 1L)); + expectedRow.add(GenericRow.of(BinaryString.fromString("[3]"), 1L)); // Only read partition and record count, record size may not stable. List result = read(partitionsTable, new int[][] {{0}, {1}}); assertThat(result).containsExactlyInAnyOrderElementsOf(expectedRow); } + @Test + public void testPartitionTimeTravel() throws Exception { + List expectedRow = new ArrayList<>(); + expectedRow.add(GenericRow.of(BinaryString.fromString("[1]"), 1L)); + expectedRow.add(GenericRow.of(BinaryString.fromString("[3]"), 1L)); + + // Only read partition and record count, record size may not stable. + List result = + read( + partitionsTable.copy( + Collections.singletonMap(CoreOptions.SCAN_VERSION.key(), "1")), + new int[][] {{0}, {1}}); + assertThat(result).containsExactlyInAnyOrderElementsOf(expectedRow); + } + @Test public void testPartitionValue() throws Exception { write(table, GenericRow.of(2, 1, 3), GenericRow.of(3, 1, 4)); List expectedRow = new ArrayList<>(); expectedRow.add(GenericRow.of(BinaryString.fromString("[1]"), 4L, 3L)); - expectedRow.add(GenericRow.of(BinaryString.fromString("[2]"), 2L, 2L)); + expectedRow.add(GenericRow.of(BinaryString.fromString("[2]"), 1L, 1L)); + expectedRow.add(GenericRow.of(BinaryString.fromString("[3]"), 1L, 1L)); List result = read(partitionsTable, new int[][] {{0}, {1}, {3}}); assertThat(result).containsExactlyInAnyOrderElementsOf(expectedRow); diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java index d4d45b3129614..6ca15cf1503d7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java @@ -90,7 +90,8 @@ public void testCreateDataFilePathFactoryWithPartition() { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); assertPartition("20211224", 16, pathFactory, "/dt=20211224/hr=16"); assertPartition("20211224", null, pathFactory, "/dt=20211224/hr=default"); @@ -130,6 +131,7 @@ public static FileStorePathFactory createNonPartFactory(Path root) { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/FormatReaderMappingTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/FormatReaderMappingTest.java new file mode 100644 index 0000000000000..dd00d142c83a3 --- /dev/null +++ b/paimon-core/src/test/java/org/apache/paimon/utils/FormatReaderMappingTest.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.utils; + +import org.apache.paimon.schema.IndexCastMapping; +import org.apache.paimon.schema.SchemaEvolutionUtil; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; + +/** Test for {@link FormatReaderMapping.Builder}. */ +public class FormatReaderMappingTest { + + @Test + public void testTrimKeyFields() { + List keyFields = new ArrayList<>(); + List allFields = new ArrayList<>(); + List testFields = new ArrayList<>(); + + for (int i = 0; i < 10; i++) { + keyFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + i, + SpecialFields.KEY_FIELD_PREFIX + i, + DataTypes.STRING())); + } + + allFields.addAll(keyFields); + for (int i = 0; i < 20; i++) { + allFields.add(new DataField(i, String.valueOf(i), DataTypes.STRING())); + } + + testFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 1, + SpecialFields.KEY_FIELD_PREFIX + 1, + DataTypes.STRING())); + testFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 3, + SpecialFields.KEY_FIELD_PREFIX + 3, + DataTypes.STRING())); + testFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 5, + SpecialFields.KEY_FIELD_PREFIX + 5, + DataTypes.STRING())); + testFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 7, + SpecialFields.KEY_FIELD_PREFIX + 7, + DataTypes.STRING())); + testFields.add(new DataField(3, String.valueOf(3), DataTypes.STRING())); + testFields.add(new DataField(4, String.valueOf(4), DataTypes.STRING())); + testFields.add(new DataField(5, String.valueOf(5), DataTypes.STRING())); + testFields.add(new DataField(1, String.valueOf(1), DataTypes.STRING())); + testFields.add(new DataField(6, String.valueOf(6), DataTypes.STRING())); + + Pair res = FormatReaderMapping.Builder.trimKeyFields(testFields, allFields); + + Assertions.assertThat(res.getKey()).containsExactly(0, 1, 2, 3, 1, 4, 2, 0, 5); + + List fields = res.getRight().getFields(); + Assertions.assertThat(fields.size()).isEqualTo(6); + Assertions.assertThat(fields.get(0).id()).isEqualTo(1); + Assertions.assertThat(fields.get(1).id()).isEqualTo(3); + Assertions.assertThat(fields.get(2).id()).isEqualTo(5); + Assertions.assertThat(fields.get(3).id()).isEqualTo(7); + Assertions.assertThat(fields.get(4).id()).isEqualTo(4); + Assertions.assertThat(fields.get(5).id()).isEqualTo(6); + } + + @Test + public void testTrimKeyWithIndexMapping() { + List readTableFields = new ArrayList<>(); + List readDataFields = new ArrayList<>(); + + readTableFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 1, + SpecialFields.KEY_FIELD_PREFIX + "a", + DataTypes.STRING())); + readTableFields.add(new DataField(0, "0", DataTypes.STRING())); + readTableFields.add(new DataField(1, "a", DataTypes.STRING())); + readTableFields.add(new DataField(2, "2", DataTypes.STRING())); + readTableFields.add(new DataField(3, "3", DataTypes.STRING())); + + readDataFields.add( + new DataField( + SpecialFields.KEY_FIELD_ID_START + 1, + SpecialFields.KEY_FIELD_PREFIX + "a", + DataTypes.STRING())); + readDataFields.add(new DataField(0, "0", DataTypes.STRING())); + readDataFields.add(new DataField(1, "a", DataTypes.STRING())); + readDataFields.add(new DataField(3, "3", DataTypes.STRING())); + + // build index cast mapping + IndexCastMapping indexCastMapping = + SchemaEvolutionUtil.createIndexCastMapping(readTableFields, readDataFields); + + // map from key fields reading to value fields reading + Pair trimmedKeyPair = + FormatReaderMapping.Builder.trimKeyFields(readDataFields, readDataFields); + + FormatReaderMapping formatReaderMapping = + new FormatReaderMapping( + indexCastMapping.getIndexMapping(), + indexCastMapping.getCastMapping(), + trimmedKeyPair.getLeft(), + null, + null, + null, + null); + + Assertions.assertThat(formatReaderMapping.getIndexMapping()) + .containsExactly(0, 1, 0, -1, 2); + List trimmed = trimmedKeyPair.getRight().getFields(); + Assertions.assertThat(trimmed.get(0).id()).isEqualTo(1); + Assertions.assertThat(trimmed.get(1).id()).isEqualTo(0); + Assertions.assertThat(trimmed.get(2).id()).isEqualTo(3); + Assertions.assertThat(trimmed.size()).isEqualTo(3); + } +} diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java index 8a4f0b0612e71..9d3275e3ab487 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/ObjectsCacheTest.java @@ -58,17 +58,23 @@ public void test() throws IOException { // test empty map.put("k1", Collections.emptyList()); - List values = cache.read("k1", null, Filter.alwaysTrue(), Filter.alwaysTrue()); + List values = + cache.read( + "k1", null, Filter.alwaysTrue(), Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).isEmpty(); // test values List expect = Arrays.asList("v1", "v2", "v3"); map.put("k2", expect); - values = cache.read("k2", null, Filter.alwaysTrue(), Filter.alwaysTrue()); + values = + cache.read( + "k2", null, Filter.alwaysTrue(), Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).containsExactlyElementsOf(expect); // test cache - values = cache.read("k2", null, Filter.alwaysTrue(), Filter.alwaysTrue()); + values = + cache.read( + "k2", null, Filter.alwaysTrue(), Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).containsExactlyElementsOf(expect); // test filter @@ -77,7 +83,8 @@ public void test() throws IOException { "k2", null, Filter.alwaysTrue(), - r -> r.getString(0).toString().endsWith("2")); + r -> r.getString(0).toString().endsWith("2"), + Filter.alwaysTrue()); assertThat(values).containsExactly("v2"); // test load filter @@ -88,6 +95,7 @@ public void test() throws IOException { "k3", null, r -> r.getString(0).toString().endsWith("2"), + Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).containsExactly("v2"); @@ -99,6 +107,7 @@ public void test() throws IOException { "k4", null, r -> r.getString(0).toString().endsWith("5"), + Filter.alwaysTrue(), Filter.alwaysTrue()); assertThat(values).isEmpty(); @@ -117,6 +126,7 @@ public void test() throws IOException { k, null, Filter.alwaysTrue(), + Filter.alwaysTrue(), Filter.alwaysTrue())) .containsExactly(k); } catch (IOException e) { diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java index 6b7b28263af02..26480cf411bb5 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/SnapshotManagerTest.java @@ -281,8 +281,8 @@ public void testLatestSnapshotOfUser() throws IOException, InterruptedException @Test public void testTraversalSnapshotsFromLatestSafely() throws IOException, InterruptedException { FileIO localFileIO = LocalFileIO.create(); - SnapshotManager snapshotManager = - new SnapshotManager(localFileIO, new Path(tempDir.toString())); + Path path = new Path(tempDir.toString()); + SnapshotManager snapshotManager = new SnapshotManager(localFileIO, path); // create 10 snapshots for (long i = 0; i < 10; i++) { Snapshot snapshot = diff --git a/paimon-core/src/test/resources/compatibility/datasplit-v3 b/paimon-core/src/test/resources/compatibility/datasplit-v3 new file mode 100644 index 0000000000000..6b19fe2d958d3 Binary files /dev/null and b/paimon-core/src/test/resources/compatibility/datasplit-v3 differ diff --git a/paimon-core/src/test/resources/compatibility/manifest-committable-v4 b/paimon-core/src/test/resources/compatibility/manifest-committable-v4 new file mode 100644 index 0000000000000..9c095669a34b8 Binary files /dev/null and b/paimon-core/src/test/resources/compatibility/manifest-committable-v4 differ diff --git a/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java b/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java index 6f700724ac019..1d35559af658e 100644 --- a/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java +++ b/paimon-docs/src/main/java/org/apache/paimon/docs/configuration/ConfigOptionsDocGenerator.java @@ -77,6 +77,7 @@ public class ConfigOptionsDocGenerator { new OptionsClassLocation("paimon-core", "org.apache.paimon.lookup"), new OptionsClassLocation("paimon-core", "org.apache.paimon.catalog"), new OptionsClassLocation("paimon-core", "org.apache.paimon.jdbc"), + new OptionsClassLocation("paimon-core", "org.apache.paimon.table"), new OptionsClassLocation("paimon-format", "org.apache.paimon.format"), new OptionsClassLocation( "paimon-flink/paimon-flink-common", "org.apache.paimon.flink"), diff --git a/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/E2eTestBase.java b/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/E2eTestBase.java index 55f9e984e5de3..0547ac4f27d1f 100644 --- a/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/E2eTestBase.java +++ b/paimon-e2e-tests/src/test/java/org/apache/paimon/tests/E2eTestBase.java @@ -111,7 +111,8 @@ public void before() throws Exception { for (String s : kafkaServices) { environment.withLogConsumer(s + "-1", new Slf4jLogConsumer(LOG)); } - environment.waitingFor("kafka-1", buildWaitStrategy(".*Recorded new controller.*", 2)); + environment.waitingFor( + "kafka-1", buildWaitStrategy(".*Recorded new ZK controller.*", 2)); } if (withHive) { List hiveServices = diff --git a/paimon-e2e-tests/src/test/resources-filtered/docker-compose.yaml b/paimon-e2e-tests/src/test/resources-filtered/docker-compose.yaml index 80d4a51fb1b52..b89420c8125a4 100644 --- a/paimon-e2e-tests/src/test/resources-filtered/docker-compose.yaml +++ b/paimon-e2e-tests/src/test/resources-filtered/docker-compose.yaml @@ -77,7 +77,7 @@ services: # ---------------------------------------- zookeeper: - image: confluentinc/cp-zookeeper:7.0.1 + image: confluentinc/cp-zookeeper:7.8.0 networks: testnetwork: aliases: @@ -89,7 +89,7 @@ services: - "2181" kafka: - image: confluentinc/cp-kafka:7.0.1 + image: confluentinc/cp-kafka:7.8.0 networks: testnetwork: aliases: diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java new file mode 100644 index 0000000000000..4ff5484b3b087 --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/functions/OpenContext.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.functions; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * The {@link OpenContext} interface provides necessary information required by the {@link + * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be + * used to add more methods without affecting the signature of {@code RichFunction#open}. + */ +@PublicEvolving +public interface OpenContext {} diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java new file mode 100644 index 0000000000000..16987469a948f --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.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.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public interface SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java new file mode 100644 index 0000000000000..374d33f6500d5 --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.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.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public class SerializerConfigImpl implements SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java new file mode 100644 index 0000000000000..563dbbe75e7e6 --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.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.api.connector.sink2; + +/** Placeholder class to resolve compatibility issues. */ +public interface WriterInitContext extends org.apache.flink.api.connector.sink2.Sink.InitContext {} diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java new file mode 100644 index 0000000000000..98aaf6418ff7f --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.v2; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; + +import java.io.IOException; + +/** + * A special sink that ignores all elements. + * + * @param The type of elements received by the sink. + */ +@PublicEvolving +public class DiscardingSink implements Sink { + private static final long serialVersionUID = 1L; + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return new DiscardingElementWriter(); + } + + private class DiscardingElementWriter implements SinkWriter { + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // discard it. + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + // this writer has no pending data. + } + + @Override + public void close() throws Exception { + // do nothing. + } + } +} diff --git a/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 0000000000000..460fea55ad7a3 --- /dev/null +++ b/paimon-flink/paimon-flink-1.15/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.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.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** Utility methods about Flink runtime context to resolve compatibility issues. */ +public class RuntimeContextUtils { + public static int getNumberOfParallelSubtasks(RuntimeContext context) { + return context.getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getIndexOfThisSubtask(); + } +} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java new file mode 100644 index 0000000000000..4ff5484b3b087 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/functions/OpenContext.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.functions; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * The {@link OpenContext} interface provides necessary information required by the {@link + * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be + * used to add more methods without affecting the signature of {@code RichFunction#open}. + */ +@PublicEvolving +public interface OpenContext {} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java new file mode 100644 index 0000000000000..16987469a948f --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.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.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public interface SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java new file mode 100644 index 0000000000000..374d33f6500d5 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.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.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public class SerializerConfigImpl implements SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java new file mode 100644 index 0000000000000..563dbbe75e7e6 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.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.api.connector.sink2; + +/** Placeholder class to resolve compatibility issues. */ +public interface WriterInitContext extends org.apache.flink.api.connector.sink2.Sink.InitContext {} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java new file mode 100644 index 0000000000000..98aaf6418ff7f --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.functions.sink.v2; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; + +import java.io.IOException; + +/** + * A special sink that ignores all elements. + * + * @param The type of elements received by the sink. + */ +@PublicEvolving +public class DiscardingSink implements Sink { + private static final long serialVersionUID = 1L; + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return new DiscardingElementWriter(); + } + + private class DiscardingElementWriter implements SinkWriter { + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // discard it. + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + // this writer has no pending data. + } + + @Override + public void close() throws Exception { + // do nothing. + } + } +} diff --git a/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 0000000000000..460fea55ad7a3 --- /dev/null +++ b/paimon-flink/paimon-flink-1.16/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.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.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** Utility methods about Flink runtime context to resolve compatibility issues. */ +public class RuntimeContextUtils { + public static int getNumberOfParallelSubtasks(RuntimeContext context) { + return context.getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getIndexOfThisSubtask(); + } +} diff --git a/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/iceberg/Flink116IcebergITCase.java b/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/iceberg/Flink116IcebergITCase.java index 0097700c70fad..3001fefe4bb3b 100644 --- a/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/iceberg/Flink116IcebergITCase.java +++ b/paimon-flink/paimon-flink-1.16/src/test/java/org/apache/paimon/flink/iceberg/Flink116IcebergITCase.java @@ -19,4 +19,11 @@ package org.apache.paimon.flink.iceberg; /** IT cases for Paimon Iceberg compatibility in Flink 1.16. */ -public class Flink116IcebergITCase extends FlinkIcebergITCaseBase {} +public class Flink116IcebergITCase extends FlinkIcebergITCaseBase { + + @Override + public void testNestedTypes(String format) { + // Flink 1.16 (or maybe Calcite?) will mistakenly cast the result to VARCHAR(5), + // so we skip this test in Flink 1.16. + } +} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java new file mode 100644 index 0000000000000..4ff5484b3b087 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/functions/OpenContext.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.functions; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * The {@link OpenContext} interface provides necessary information required by the {@link + * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be + * used to add more methods without affecting the signature of {@code RichFunction#open}. + */ +@PublicEvolving +public interface OpenContext {} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java new file mode 100644 index 0000000000000..16987469a948f --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.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.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public interface SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java new file mode 100644 index 0000000000000..374d33f6500d5 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.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.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public class SerializerConfigImpl implements SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java new file mode 100644 index 0000000000000..db45000425723 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.connector.sink2; + +import org.apache.flink.annotation.Public; + +/** Placeholder class to resolve compatibility issues. */ +@Public +public interface WriterInitContext extends org.apache.flink.api.connector.sink2.Sink.InitContext {} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java new file mode 100644 index 0000000000000..fc7eb0d483569 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.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.flink.streaming.api.functions.sink.v2; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; + +import java.io.IOException; + +/** + * A special sink that ignores all elements. + * + * @param The type of elements received by the sink. + */ +@PublicEvolving +public class DiscardingSink implements Sink, SupportsConcurrentExecutionAttempts { + private static final long serialVersionUID = 1L; + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return new DiscardingElementWriter(); + } + + private class DiscardingElementWriter implements SinkWriter { + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // discard it. + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + // this writer has no pending data. + } + + @Override + public void close() throws Exception { + // do nothing. + } + } +} diff --git a/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 0000000000000..460fea55ad7a3 --- /dev/null +++ b/paimon-flink/paimon-flink-1.17/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.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.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** Utility methods about Flink runtime context to resolve compatibility issues. */ +public class RuntimeContextUtils { + public static int getNumberOfParallelSubtasks(RuntimeContext context) { + return context.getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getIndexOfThisSubtask(); + } +} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java new file mode 100644 index 0000000000000..4ff5484b3b087 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/functions/OpenContext.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.functions; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * The {@link OpenContext} interface provides necessary information required by the {@link + * RichFunction} when it is opened. The {@link OpenContext} is currently empty because it can be + * used to add more methods without affecting the signature of {@code RichFunction#open}. + */ +@PublicEvolving +public interface OpenContext {} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.java new file mode 100644 index 0000000000000..16987469a948f --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfig.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.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public interface SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.java new file mode 100644 index 0000000000000..374d33f6500d5 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/common/serialization/SerializerConfigImpl.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.api.common.serialization; + +/** Placeholder class to resolve compatibility issues. */ +public class SerializerConfigImpl implements SerializerConfig {} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.java new file mode 100644 index 0000000000000..563dbbe75e7e6 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/api/connector/sink2/WriterInitContext.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.api.connector.sink2; + +/** Placeholder class to resolve compatibility issues. */ +public interface WriterInitContext extends org.apache.flink.api.connector.sink2.Sink.InitContext {} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.java new file mode 100644 index 0000000000000..fc7eb0d483569 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/flink/streaming/api/functions/sink/v2/DiscardingSink.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.flink.streaming.api.functions.sink.v2; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; + +import java.io.IOException; + +/** + * A special sink that ignores all elements. + * + * @param The type of elements received by the sink. + */ +@PublicEvolving +public class DiscardingSink implements Sink, SupportsConcurrentExecutionAttempts { + private static final long serialVersionUID = 1L; + + @Override + public SinkWriter createWriter(InitContext context) throws IOException { + return new DiscardingElementWriter(); + } + + private class DiscardingElementWriter implements SinkWriter { + + @Override + public void write(IN element, Context context) throws IOException, InterruptedException { + // discard it. + } + + @Override + public void flush(boolean endOfInput) throws IOException, InterruptedException { + // this writer has no pending data. + } + + @Override + public void close() throws Exception { + // do nothing. + } + } +} diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java index 99f205bacb58e..ac4340c113365 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java @@ -26,6 +26,8 @@ import java.util.Map; +import static org.apache.paimon.flink.action.ActionFactory.FULL; +import static org.apache.paimon.flink.action.CompactActionFactory.checkCompactStrategy; import static org.apache.paimon.utils.ParameterUtils.parseCommaSeparatedKeyValues; /** @@ -51,6 +53,7 @@ * * -- set table options ('k=v,...') * CALL sys.compact_database('includingDatabases', 'mode', 'includingTables', 'excludingTables', 'tableOptions') + * * */ public class CompactDatabaseProcedure extends ProcedureBase { @@ -106,7 +109,8 @@ public String[] call( includingTables, excludingTables, tableOptions, - ""); + "", + null); } public String[] call( @@ -116,7 +120,8 @@ public String[] call( String includingTables, String excludingTables, String tableOptions, - String partitionIdleTime) + String partitionIdleTime, + String compactStrategy) throws Exception { String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); @@ -133,6 +138,10 @@ public String[] call( action.withPartitionIdleTime(TimeUtils.parseDuration(partitionIdleTime)); } + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } + return execute(procedureContext, action, "Compact database job"); } diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java index 63aa6c906b947..560e532a6dbb3 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java @@ -31,6 +31,9 @@ import java.util.Collections; import java.util.Map; +import static org.apache.paimon.flink.action.ActionFactory.FULL; +import static org.apache.paimon.flink.action.CompactActionFactory.checkCompactStrategy; + /** * Stay compatible with 1.18 procedure which doesn't support named argument. Usage: * @@ -49,6 +52,9 @@ * -- compact specific partitions with sorting * CALL sys.compact('tableId', 'partitions', 'ORDER/ZORDER', 'col1,col2', 'sink.parallelism=6') * + * -- compact with specific compact strategy + * CALL sys.compact('tableId', 'partitions', 'ORDER/ZORDER', 'col1,col2', 'sink.parallelism=6', 'minor') + * * */ public class CompactProcedure extends ProcedureBase { @@ -118,7 +124,8 @@ public String[] call( orderByColumns, tableOptions, whereSql, - ""); + "", + null); } public String[] call( @@ -129,7 +136,8 @@ public String[] call( String orderByColumns, String tableOptions, String whereSql, - String partitionIdleTime) + String partitionIdleTime, + String compactStrategy) throws Exception { String warehouse = catalog.warehouse(); @@ -152,6 +160,10 @@ public String[] call( if (!(StringUtils.isNullOrWhitespaceOnly(partitionIdleTime))) { action.withPartitionIdleTime(TimeUtils.parseDuration(partitionIdleTime)); } + + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } jobName = "Compact Job"; } else if (!orderStrategy.isEmpty() && !orderByColumns.isEmpty()) { Preconditions.checkArgument( diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java index c0e5a65c49ef3..1c0d73cfbe386 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/ExpirePartitionsProcedure.java @@ -93,9 +93,10 @@ public String[] call( .catalogEnvironment() .metastoreClientFactory()) .map(MetastoreClient.Factory::create) - .orElse(null)); + .orElse(null), + fileStore.options().partitionExpireMaxNum()); if (maxExpires != null) { - partitionExpire.withMaxExpires(maxExpires); + partitionExpire.withMaxExpireNum(maxExpires); } List> expired = partitionExpire.expire(Long.MAX_VALUE); return expired == null || expired.isEmpty() diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java index c5fa7b7ba34ae..b4a3a6b359d91 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RemoveOrphanFilesProcedure.java @@ -20,9 +20,13 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.orphan.FlinkOrphanFilesClean; +import org.apache.paimon.operation.CleanOrphanFilesResult; +import org.apache.paimon.operation.LocalOrphanFilesClean; 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; @@ -66,20 +70,61 @@ public String[] call( boolean dryRun, Integer parallelism) throws Exception { + return call(procedureContext, tableId, olderThan, dryRun, parallelism, null); + } + + public String[] call( + ProcedureContext procedureContext, + String tableId, + String olderThan, + boolean dryRun, + Integer parallelism, + String mode) + throws Exception { Identifier identifier = Identifier.fromString(tableId); String databaseName = identifier.getDatabaseName(); String tableName = identifier.getObjectName(); + if (mode == null) { + mode = "DISTRIBUTED"; + } - long deleted = - FlinkOrphanFilesClean.executeDatabaseOrphanFiles( - procedureContext.getExecutionEnvironment(), - catalog, - olderThanMillis(olderThan), - createFileCleaner(catalog, dryRun), - parallelism, - databaseName, - tableName); - return new String[] {String.valueOf(deleted)}; + CleanOrphanFilesResult cleanOrphanFilesResult; + try { + switch (mode.toUpperCase(Locale.ROOT)) { + case "DISTRIBUTED": + cleanOrphanFilesResult = + FlinkOrphanFilesClean.executeDatabaseOrphanFiles( + procedureContext.getExecutionEnvironment(), + catalog, + olderThanMillis(olderThan), + createFileCleaner(catalog, dryRun), + parallelism, + databaseName, + tableName); + break; + case "LOCAL": + cleanOrphanFilesResult = + 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(cleanOrphanFilesResult.getDeletedFileCount()), + String.valueOf(cleanOrphanFilesResult.getDeletedFileTotalLenInBytes()) + }; + } catch (Exception e) { + throw new RuntimeException(e); + } } @Override diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RollbackToWatermarkProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RollbackToWatermarkProcedure.java new file mode 100644 index 0000000000000..da0b38f16b54b --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/RollbackToWatermarkProcedure.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.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.procedure.ProcedureContext; + +/** + * Rollback to watermark procedure. Usage: + * + *

    
    + *  -- rollback to the snapshot which earlier or equal than watermark.
    + *  CALL sys.rollback_to_watermark('tableId', watermark)
    + * 
    + */ +public class RollbackToWatermarkProcedure extends ProcedureBase { + + public static final String IDENTIFIER = "rollback_to_watermark"; + + public String[] call(ProcedureContext procedureContext, String tableId, long watermark) + throws Catalog.TableNotExistException { + Preconditions.checkNotNull(tableId, "table can not be empty"); + Table table = catalog.getTable(Identifier.fromString(tableId)); + FileStoreTable fileStoreTable = (FileStoreTable) table; + Snapshot snapshot = fileStoreTable.snapshotManager().earlierOrEqualWatermark(watermark); + Preconditions.checkNotNull( + snapshot, String.format("count not find snapshot earlier than %s", watermark)); + 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-1.18/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 0000000000000..460fea55ad7a3 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.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.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** Utility methods about Flink runtime context to resolve compatibility issues. */ +public class RuntimeContextUtils { + public static int getNumberOfParallelSubtasks(RuntimeContext context) { + return context.getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getIndexOfThisSubtask(); + } +} diff --git a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java new file mode 100644 index 0000000000000..a168c3785c7c2 --- /dev/null +++ b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java @@ -0,0 +1,299 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.CoreOptions; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.flink.action.ActionITCaseBase; +import org.apache.paimon.flink.action.RemoveOrphanFilesAction; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.options.Options; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.sink.StreamTableCommit; +import org.apache.paimon.table.sink.StreamTableWrite; +import org.apache.paimon.table.sink.StreamWriteBuilder; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; + +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.paimon.CoreOptions.SCAN_FALLBACK_BRANCH; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; + +/** IT cases base for {@link RemoveOrphanFilesAction} in Flink 1.18. */ +public class RemoveOrphanFilesActionITCase extends ActionITCaseBase { + private static final String ORPHAN_FILE_1 = "bucket-0/orphan_file1"; + private static final String ORPHAN_FILE_2 = "bucket-0/orphan_file2"; + + private FileStoreTable createTableAndWriteData(String tableName) throws Exception { + RowType rowType = + RowType.of( + new DataType[] {DataTypes.BIGINT(), DataTypes.STRING()}, + new String[] {"k", "v"}); + + FileStoreTable table = + createFileStoreTable( + tableName, + rowType, + 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"))); + + Path orphanFile1 = getOrphanFilePath(table, ORPHAN_FILE_1); + Path orphanFile2 = getOrphanFilePath(table, ORPHAN_FILE_2); + + FileIO fileIO = table.fileIO(); + fileIO.writeFile(orphanFile1, "a", true); + Thread.sleep(2000); + fileIO.writeFile(orphanFile2, "b", true); + + return table; + } + + private Path getOrphanFilePath(FileStoreTable table, String orphanFile) { + return new Path(table.location(), orphanFile); + } + + @Test + public void testRunWithoutException() 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("CALL sys.remove_orphan_files('%s.%s')", database, tableName); + + CloseableIterator withoutOlderThanCollect = executeSQL(withoutOlderThan); + assertThat(ImmutableList.copyOf(withoutOlderThanCollect)).containsOnly(Row.of("0")); + + String withDryRun = + String.format( + "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59', true)", + database, tableName); + ImmutableList actualDryRunDeleteFile = ImmutableList.copyOf(executeSQL(withDryRun)); + assertThat(actualDryRunDeleteFile).containsOnly(Row.of("2")); + + String withOlderThan = + String.format( + "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59')", + database, tableName); + ImmutableList actualDeleteFile = ImmutableList.copyOf(executeSQL(withOlderThan)); + + assertThat(actualDeleteFile).containsExactlyInAnyOrder(Row.of("2"), Row.of("2")); + } + + @Test + public void testRemoveDatabaseOrphanFilesITCase() throws Exception { + createTableAndWriteData("tableName1"); + createTableAndWriteData("tableName2"); + + List args = + new ArrayList<>( + Arrays.asList( + "remove_orphan_files", + "--warehouse", + warehouse, + "--database", + database, + "--table", + "*")); + 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(); + + args.add("--parallelism"); + args.add("5"); + RemoveOrphanFilesAction action3 = createAction(RemoveOrphanFilesAction.class, args); + assertThatCode(action3::run).doesNotThrowAnyException(); + + String withoutOlderThan = + String.format("CALL sys.remove_orphan_files('%s.%s')", database, "*"); + CloseableIterator withoutOlderThanCollect = executeSQL(withoutOlderThan); + assertThat(ImmutableList.copyOf(withoutOlderThanCollect)).containsOnly(Row.of("0")); + + String withParallelism = + String.format("CALL sys.remove_orphan_files('%s.%s','',true,5)", database, "*"); + CloseableIterator withParallelismCollect = executeSQL(withParallelism); + assertThat(ImmutableList.copyOf(withParallelismCollect)).containsOnly(Row.of("0")); + + String withDryRun = + String.format( + "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59', true)", + database, "*"); + ImmutableList actualDryRunDeleteFile = ImmutableList.copyOf(executeSQL(withDryRun)); + assertThat(actualDryRunDeleteFile).containsOnly(Row.of("4")); + + String withOlderThan = + String.format( + "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59')", + database, "*"); + ImmutableList actualDeleteFile = ImmutableList.copyOf(executeSQL(withOlderThan)); + + assertThat(actualDeleteFile).containsOnly(Row.of("4")); + } + + @Test + public void testCleanWithBranch() throws Exception { + // create main branch + FileStoreTable table = createTableAndWriteData(tableName); + + // create first branch and write some data + table.createBranch("br"); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location(), "br"); + TableSchema branchSchema = + schemaManager.commitChanges(SchemaChange.addColumn("v2", DataTypes.INT())); + Options branchOptions = new Options(branchSchema.options()); + branchOptions.set(CoreOptions.BRANCH, "br"); + branchSchema = branchSchema.copy(branchOptions.toMap()); + FileStoreTable branchTable = + FileStoreTableFactory.create(table.fileIO(), table.location(), branchSchema); + + String commitUser = UUID.randomUUID().toString(); + StreamTableWrite write = branchTable.newWrite(commitUser); + StreamTableCommit commit = branchTable.newCommit(commitUser); + write.write(GenericRow.of(2L, BinaryString.fromString("Hello"), 20)); + commit.commit(1, write.prepareCommit(false, 1)); + write.close(); + commit.close(); + + // create orphan file in snapshot directory of first branch + Path orphanFile3 = new Path(table.location(), "branch/branch-br/snapshot/orphan_file3"); + branchTable.fileIO().writeFile(orphanFile3, "x", true); + + // create second branch, which is empty + table.createBranch("br2"); + + // create orphan file in snapshot directory of second branch + Path orphanFile4 = new Path(table.location(), "branch/branch-br2/snapshot/orphan_file4"); + branchTable.fileIO().writeFile(orphanFile4, "y", true); + + if (ThreadLocalRandom.current().nextBoolean()) { + executeSQL( + String.format( + "ALTER TABLE `%s`.`%s` SET ('%s' = 'br')", + database, tableName, SCAN_FALLBACK_BRANCH.key()), + false, + true); + } + String procedure = + String.format( + "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59')", + database, "*"); + ImmutableList actualDeleteFile = ImmutableList.copyOf(executeSQL(procedure)); + assertThat(actualDeleteFile).containsOnly(Row.of("4")); + } + + @Test + public void testRunWithMode() 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("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( + "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( + "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( + "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-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java index f2385e66d2a1d..f79d6fb716b4c 100644 --- a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java +++ b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/procedure/ProcedurePositionalArgumentsITCase.java @@ -68,10 +68,16 @@ public void testCompactDatabaseAndTable() { sql( "CALL sys.compact('default.T', '', '', '', 'sink.parallelism=1','pt=1')")) .doesNotThrowAnyException(); - assertThatCode(() -> sql("CALL sys.compact('default.T', '', 'zorder', 'k', '','','5s')")) + assertThatCode( + () -> + sql( + "CALL sys.compact('default.T', '' ,'zorder', 'k', '','','5s', '')")) .message() .contains("sort compact do not support 'partition_idle_time'."); + assertThatCode(() -> sql("CALL sys.compact('default.T', '', '' ,'', '', '', '', 'full')")) + .doesNotThrowAnyException(); + assertThatCode(() -> sql("CALL sys.compact_database('default')")) .doesNotThrowAnyException(); } diff --git a/paimon-flink/paimon-flink-1.19/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java b/paimon-flink/paimon-flink-1.19/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java new file mode 100644 index 0000000000000..e1be410b8cb15 --- /dev/null +++ b/paimon-flink/paimon-flink-1.19/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink; + +import org.apache.paimon.flink.action.RemoveOrphanFilesAction; +import org.apache.paimon.flink.action.RemoveOrphanFilesActionITCaseBase; + +/** IT cases base for {@link RemoveOrphanFilesAction} in Flink 1.19. */ +public class RemoveOrphanFilesActionITCase extends RemoveOrphanFilesActionITCaseBase {} diff --git a/paimon-flink/paimon-flink-1.20/pom.xml b/paimon-flink/paimon-flink-1.20/pom.xml index 7cf1d8e98df71..f15792d2bea9e 100644 --- a/paimon-flink/paimon-flink-1.20/pom.xml +++ b/paimon-flink/paimon-flink-1.20/pom.xml @@ -55,6 +55,20 @@ under the License. + + + org.apache.flink + flink-streaming-java + ${flink.version} + provided + + + + org.apache.flink + flink-table-common + ${flink.version} + provided + diff --git a/paimon-flink/paimon-flink-cdc/pom.xml b/paimon-flink/paimon-flink-cdc/pom.xml index 7f5bb3e786da8..792c6c14378be 100644 --- a/paimon-flink/paimon-flink-cdc/pom.xml +++ b/paimon-flink/paimon-flink-cdc/pom.xml @@ -34,7 +34,7 @@ under the License. Paimon : Flink : CDC - 1.18.1 + 1.20.0 3.1.1 3.1.1 1.11.4 @@ -43,7 +43,7 @@ under the License. 1.19.1 4.0.0-1.17 7.5.0 - 3.0.1-1.18 + 3.3.0-1.20 @@ -167,6 +167,13 @@ under the License. + + commons-codec + commons-codec + 1.9 + test + + org.apache.paimon paimon-common 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 deleted file mode 100644 index efe5e12b12d70..0000000000000 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/ProcessRecordAttributesUtil.java +++ /dev/null @@ -1,31 +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; - -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/CdcActionCommonUtils.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java index 8f96022bde359..c8af6f91c4201 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/CdcActionCommonUtils.java @@ -56,6 +56,9 @@ public class CdcActionCommonUtils { public static final String PULSAR_CONF = "pulsar_conf"; public static final String TABLE_PREFIX = "table_prefix"; public static final String TABLE_SUFFIX = "table_suffix"; + public static final String TABLE_PREFIX_DB = "table_prefix_db"; + public static final String TABLE_SUFFIX_DB = "table_suffix_db"; + public static final String TABLE_MAPPING = "table_mapping"; public static final String INCLUDING_TABLES = "including_tables"; public static final String EXCLUDING_TABLES = "excluding_tables"; public static final String TYPE_MAPPING = "type_mapping"; diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java index 4ab56bdcf1189..4fb1339c51934 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionBase.java @@ -52,6 +52,9 @@ public abstract class SyncDatabaseActionBase extends SynchronizationActionBase { protected MultiTablesSinkMode mode = COMBINED; protected String tablePrefix = ""; protected String tableSuffix = ""; + protected Map tableMapping = new HashMap<>(); + protected Map dbPrefix = new HashMap<>(); + protected Map dbSuffix = new HashMap<>(); protected String includingTables = ".*"; protected List partitionKeys = new ArrayList<>(); protected List primaryKeys = new ArrayList<>(); @@ -97,6 +100,37 @@ public SyncDatabaseActionBase withTableSuffix(@Nullable String tableSuffix) { return this; } + public SyncDatabaseActionBase withDbPrefix(Map dbPrefix) { + if (dbPrefix != null) { + this.dbPrefix = + dbPrefix.entrySet().stream() + .collect( + HashMap::new, + (m, e) -> m.put(e.getKey().toLowerCase(), e.getValue()), + HashMap::putAll); + } + return this; + } + + public SyncDatabaseActionBase withDbSuffix(Map dbSuffix) { + if (dbSuffix != null) { + this.dbSuffix = + dbSuffix.entrySet().stream() + .collect( + HashMap::new, + (m, e) -> m.put(e.getKey().toLowerCase(), e.getValue()), + HashMap::putAll); + } + return this; + } + + public SyncDatabaseActionBase withTableMapping(Map tableMapping) { + if (tableMapping != null) { + this.tableMapping = tableMapping; + } + return this; + } + public SyncDatabaseActionBase includingTables(@Nullable String includingTables) { if (includingTables != null) { this.includingTables = includingTables; @@ -155,7 +189,14 @@ protected EventParser.Factory buildEventParserFactory() Pattern excludingPattern = excludingTables == null ? null : Pattern.compile(excludingTables); TableNameConverter tableNameConverter = - new TableNameConverter(allowUpperCase, mergeShards, tablePrefix, tableSuffix); + new TableNameConverter( + allowUpperCase, + mergeShards, + dbPrefix, + dbSuffix, + tablePrefix, + tableSuffix, + tableMapping); Set createdTables; try { createdTables = new HashSet<>(catalog.listTables(database)); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java index e7a386979d4e7..d497b588c2af6 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncDatabaseActionFactoryBase.java @@ -29,8 +29,11 @@ import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.MULTIPLE_TABLE_PARTITION_KEYS; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.PARTITION_KEYS; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.PRIMARY_KEYS; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_MAPPING; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_PREFIX; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_PREFIX_DB; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_SUFFIX; +import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TABLE_SUFFIX_DB; import static org.apache.paimon.flink.action.cdc.CdcActionCommonUtils.TYPE_MAPPING; /** Base {@link ActionFactory} for synchronizing into database. */ @@ -51,6 +54,9 @@ public Optional create(MultipleParameterToolAdapter params) { protected void withParams(MultipleParameterToolAdapter params, T action) { action.withTablePrefix(params.get(TABLE_PREFIX)) .withTableSuffix(params.get(TABLE_SUFFIX)) + .withDbPrefix(optionalConfigMap(params, TABLE_PREFIX_DB)) + .withDbSuffix(optionalConfigMap(params, TABLE_SUFFIX_DB)) + .withTableMapping(optionalConfigMap(params, TABLE_MAPPING)) .includingTables(params.get(INCLUDING_TABLES)) .excludingTables(params.get(EXCLUDING_TABLES)) .withPartitionKeyMultiple( 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 f103396389e5e..a7c770347410a 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 @@ -40,7 +40,6 @@ 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; import java.time.Duration; import java.util.HashMap; @@ -131,7 +130,7 @@ public void build() throws Exception { protected void beforeBuildingSourceSink() throws Exception {} - protected Object buildSource() { + protected Source buildSource() { return syncJobHandler.provideSource(); } @@ -147,41 +146,32 @@ protected void validateRuntimeExecutionMode() { "It's only support STREAMING mode for flink-cdc sync table action."); } - private DataStreamSource buildDataStreamSource(Object source) { - if (source instanceof Source) { - boolean isAutomaticWatermarkCreationEnabled = - tableConfig.containsKey(CoreOptions.TAG_AUTOMATIC_CREATION.key()) - && Objects.equals( - tableConfig.get(CoreOptions.TAG_AUTOMATIC_CREATION.key()), - WATERMARK.toString()); - - Options options = Options.fromMap(tableConfig); - Duration idleTimeout = options.get(SCAN_WATERMARK_IDLE_TIMEOUT); - String watermarkAlignGroup = options.get(SCAN_WATERMARK_ALIGNMENT_GROUP); - WatermarkStrategy watermarkStrategy = - isAutomaticWatermarkCreationEnabled - ? watermarkAlignGroup != null - ? new CdcWatermarkStrategy(createCdcTimestampExtractor()) - .withWatermarkAlignment( - watermarkAlignGroup, - options.get(SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT), - options.get( - SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL)) - : new CdcWatermarkStrategy(createCdcTimestampExtractor()) - : WatermarkStrategy.noWatermarks(); - if (idleTimeout != null) { - watermarkStrategy = watermarkStrategy.withIdleness(idleTimeout); - } - return env.fromSource( - (Source) source, - watermarkStrategy, - syncJobHandler.provideSourceName()); + private DataStreamSource buildDataStreamSource( + Source source) { + boolean isAutomaticWatermarkCreationEnabled = + tableConfig.containsKey(CoreOptions.TAG_AUTOMATIC_CREATION.key()) + && Objects.equals( + tableConfig.get(CoreOptions.TAG_AUTOMATIC_CREATION.key()), + WATERMARK.toString()); + + Options options = Options.fromMap(tableConfig); + Duration idleTimeout = options.get(SCAN_WATERMARK_IDLE_TIMEOUT); + String watermarkAlignGroup = options.get(SCAN_WATERMARK_ALIGNMENT_GROUP); + WatermarkStrategy watermarkStrategy = + isAutomaticWatermarkCreationEnabled + ? watermarkAlignGroup != null + ? new CdcWatermarkStrategy(createCdcTimestampExtractor()) + .withWatermarkAlignment( + watermarkAlignGroup, + options.get(SCAN_WATERMARK_ALIGNMENT_MAX_DRIFT), + options.get( + SCAN_WATERMARK_ALIGNMENT_UPDATE_INTERVAL)) + : new CdcWatermarkStrategy(createCdcTimestampExtractor()) + : WatermarkStrategy.noWatermarks(); + if (idleTimeout != null) { + watermarkStrategy = watermarkStrategy.withIdleness(idleTimeout); } - if (source instanceof SourceFunction) { - return env.addSource( - (SourceFunction) source, syncJobHandler.provideSourceName()); - } - throw new UnsupportedOperationException("Unrecognized source type"); + return env.fromSource(source, watermarkStrategy, syncJobHandler.provideSourceName()); } protected abstract FlatMapFunction recordParse(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java index 67c70aa58cdb0..15fc3507ce2db 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/TableNameConverter.java @@ -21,6 +21,8 @@ import org.apache.paimon.catalog.Identifier; import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; /** Used to convert a MySQL source table name to corresponding Paimon table name. */ public class TableNameConverter implements Serializable { @@ -29,24 +31,70 @@ public class TableNameConverter implements Serializable { private final boolean caseSensitive; private final boolean mergeShards; + private final Map dbPrefix; + private final Map dbSuffix; private final String prefix; private final String suffix; + private final Map tableMapping; public TableNameConverter(boolean caseSensitive) { - this(caseSensitive, true, "", ""); + this(caseSensitive, true, "", "", null); } public TableNameConverter( - boolean caseSensitive, boolean mergeShards, String prefix, String suffix) { + boolean caseSensitive, + boolean mergeShards, + String prefix, + String suffix, + Map tableMapping) { + this( + caseSensitive, + mergeShards, + new HashMap<>(), + new HashMap<>(), + prefix, + suffix, + tableMapping); + } + + public TableNameConverter( + boolean caseSensitive, + boolean mergeShards, + Map dbPrefix, + Map dbSuffix, + String prefix, + String suffix, + Map tableMapping) { this.caseSensitive = caseSensitive; this.mergeShards = mergeShards; + this.dbPrefix = dbPrefix; + this.dbSuffix = dbSuffix; this.prefix = prefix; this.suffix = suffix; + this.tableMapping = lowerMapKey(tableMapping); } - public String convert(String originName) { - String tableName = caseSensitive ? originName : originName.toLowerCase(); - return prefix + tableName + suffix; + public String convert(String originDbName, String originTblName) { + // top priority: table mapping + if (tableMapping.containsKey(originTblName.toLowerCase())) { + String mappedName = tableMapping.get(originTblName.toLowerCase()); + return caseSensitive ? mappedName : mappedName.toLowerCase(); + } + + String tblPrefix = prefix; + String tblSuffix = suffix; + + // second priority: prefix and postfix specified by db + if (dbPrefix.containsKey(originDbName.toLowerCase())) { + tblPrefix = dbPrefix.get(originDbName.toLowerCase()); + } + if (dbSuffix.containsKey(originDbName.toLowerCase())) { + tblSuffix = dbSuffix.get(originDbName.toLowerCase()); + } + + // third priority: normal prefix and suffix + String tableName = caseSensitive ? originTblName : originTblName.toLowerCase(); + return tblPrefix + tableName + tblSuffix; } public String convert(Identifier originIdentifier) { @@ -56,6 +104,20 @@ public String convert(Identifier originIdentifier) { : originIdentifier.getDatabaseName() + "_" + originIdentifier.getObjectName(); - return convert(rawName); + return convert(originIdentifier.getDatabaseName(), rawName); + } + + private Map lowerMapKey(Map map) { + int size = map == null ? 0 : map.size(); + Map lowerKeyMap = new HashMap<>(size); + if (size == 0) { + return lowerKeyMap; + } + + for (String key : map.keySet()) { + lowerKeyMap.put(key.toLowerCase(), map.get(key)); + } + + return lowerKeyMap; } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunDataFormat.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunDataFormat.java new file mode 100644 index 0000000000000..ccbacdc2af5e7 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunDataFormat.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.aliyun; + +import org.apache.paimon.flink.action.cdc.format.AbstractJsonDataFormat; +import org.apache.paimon.flink.action.cdc.format.RecordParserFactory; + +/** + * Supports the message queue's debezium json data format and provides definitions for the message + * queue's record json deserialization class and parsing class {@link AliyunRecordParser}. + */ +public class AliyunDataFormat extends AbstractJsonDataFormat { + + @Override + protected RecordParserFactory parser() { + return AliyunRecordParser::new; + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMetaFactory.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunDataFormatFactory.java similarity index 60% rename from paimon-common/src/main/java/org/apache/paimon/lineage/LineageMetaFactory.java rename to paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunDataFormatFactory.java index 11c6d3a1173c7..a07e2f205c904 100644 --- a/paimon-common/src/main/java/org/apache/paimon/lineage/LineageMetaFactory.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunDataFormatFactory.java @@ -16,22 +16,23 @@ * limitations under the License. */ -package org.apache.paimon.lineage; +package org.apache.paimon.flink.action.cdc.format.aliyun; -import org.apache.paimon.factories.Factory; -import org.apache.paimon.options.Options; +import org.apache.paimon.flink.action.cdc.format.DataFormat; +import org.apache.paimon.flink.action.cdc.format.DataFormatFactory; -import java.io.Serializable; +/** Factory to create {@link AliyunDataFormat}. */ +public class AliyunDataFormatFactory implements DataFormatFactory { -/** Factory to create {@link LineageMeta}. Each factory should have a unique identifier. */ -public interface LineageMetaFactory extends Factory, Serializable { + public static final String IDENTIFIER = "aliyun-json"; - LineageMeta create(LineageMetaContext context); + @Override + public String identifier() { + return IDENTIFIER; + } - /** - * Context has all options in a catalog and is used in factory to create {@link LineageMeta}. - */ - interface LineageMetaContext { - Options options(); + @Override + public DataFormat create() { + return new AliyunDataFormat(); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunFieldParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunFieldParser.java new file mode 100644 index 0000000000000..824ed91459430 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunFieldParser.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.format.aliyun; + +/** Converts some special types such as enum、set、geometry. */ +public class AliyunFieldParser { + + protected static byte[] convertGeoType2WkbArray(byte[] mysqlGeomBytes) { + int sridLength = 4; + boolean hasSrid = false; + for (int i = 0; i < sridLength; ++i) { + if (mysqlGeomBytes[i] != 0) { + hasSrid = true; + break; + } + } + byte[] wkb; + if (hasSrid) { + wkb = new byte[mysqlGeomBytes.length]; + // byteOrder + geometry + System.arraycopy(mysqlGeomBytes, 4, wkb, 0, 5); + // srid + System.arraycopy(mysqlGeomBytes, 0, wkb, 5, 4); + // geometry + System.arraycopy(mysqlGeomBytes, 9, wkb, 9, wkb.length - 9); + + // set srid flag + if (wkb[0] == 0) { + // big endian + wkb[1] = (byte) (wkb[1] + 32); + } else { + wkb[4] = (byte) (wkb[4] + 32); + } + } else { + wkb = new byte[mysqlGeomBytes.length - 4]; + System.arraycopy(mysqlGeomBytes, 4, wkb, 0, wkb.length); + } + return wkb; + } + + protected static String convertSet(String value, String mysqlType) { + // mysql set type value can be filled with more than one, value is a bit string conversion + // from the long + int indexes = Integer.parseInt(value); + return getSetValuesByIndex(mysqlType, indexes); + } + + protected static String convertEnum(String value, String mysqlType) { + int elementIndex = Integer.parseInt(value); + // enum('a','b','c') + return getEnumValueByIndex(mysqlType, elementIndex); + } + + protected static String getEnumValueByIndex(String mysqlType, int elementIndex) { + String[] options = extractEnumValueByIndex(mysqlType); + + return options[elementIndex - 1]; + } + + protected static String getSetValuesByIndex(String mysqlType, int indexes) { + String[] options = extractSetValuesByIndex(mysqlType); + + StringBuilder sb = new StringBuilder(); + sb.append("["); + int index = 0; + boolean first = true; + int optionLen = options.length; + + while (indexes != 0L) { + if (indexes % 2L != 0) { + if (first) { + first = false; + } else { + sb.append(','); + } + if (index < optionLen) { + sb.append(options[index]); + } else { + throw new RuntimeException( + String.format( + "extractSetValues from mysqlType[%s],index:%d failed", + mysqlType, indexes)); + } + } + ++index; + indexes = indexes >>> 1; + } + sb.append("]"); + return sb.toString(); + } + + private static String[] extractSetValuesByIndex(String mysqlType) { + // set('x','y') + return mysqlType.substring(5, mysqlType.length() - 2).split("'\\s*,\\s*'"); + } + + private static String[] extractEnumValueByIndex(String mysqlType) { + // enum('x','y') + return mysqlType.substring(6, mysqlType.length() - 2).split("'\\s*,\\s*'"); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunRecordParser.java new file mode 100644 index 0000000000000..e31b282a76cb3 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunRecordParser.java @@ -0,0 +1,260 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.aliyun; + +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.action.cdc.mysql.MySqlTypeUtils; +import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; +import org.apache.paimon.types.RowKind; +import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.JsonSerdeUtil; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.node.ArrayNode; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.nio.charset.StandardCharsets; +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.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.paimon.utils.JsonSerdeUtil.getNodeAs; +import static org.apache.paimon.utils.JsonSerdeUtil.isNull; + +/** + * The {@code CanalRecordParser} class is responsible for parsing records from the Canal-JSON + * format. Canal is a database binlog multi-platform consumer, which is used to synchronize data + * across databases. This parser extracts relevant information from the Canal-JSON format and + * transforms it into a list of {@link RichCdcMultiplexRecord} objects, which represent the changes + * captured in the database. + * + *

    The class handles different types of database operations such as INSERT, UPDATE, and DELETE, + * and generates corresponding {@link RichCdcMultiplexRecord} objects for each operation. + * + *

    Additionally, the parser supports schema extraction, which can be used to understand the + * structure of the incoming data and its corresponding field types. + */ +public class AliyunRecordParser extends AbstractJsonRecordParser { + + private static final Logger LOG = LoggerFactory.getLogger(AliyunRecordParser.class); + + private static final String FIELD_IS_DDL = "isDdl"; + private static final String FIELD_TYPE = "op"; + + private static final String OP_UPDATE_BEFORE = "UPDATE_BEFORE"; + private static final String OP_UPDATE_AFTER = "UPDATE_AFTER"; + private static final String OP_INSERT = "INSERT"; + private static final String OP_DELETE = "DELETE"; + + private static final String FIELD_PAYLOAD = "payload"; + private static final String FIELD_BEFORE = "before"; + private static final String FIELD_AFTER = "after"; + private static final String FIELD_COLUMN = "dataColumn"; + + private static final String FIELD_SCHEMA = "schema"; + private static final String FIELD_PK = "primaryKey"; + + @Override + protected boolean isDDL() { + JsonNode node = root.get(FIELD_IS_DDL); + return !isNull(node) && node.asBoolean(); + } + + public AliyunRecordParser(TypeMapping typeMapping, List computedColumns) { + super(typeMapping, computedColumns); + } + + @Override + protected String primaryField() { + return "schema.primaryKey"; + } + + @Override + protected String dataField() { + return "payload.dataColumn"; + } + + @Override + protected List extractPrimaryKeys() { + JsonNode schemaNode = root.get(FIELD_SCHEMA); + checkNotNull(schemaNode, FIELD_SCHEMA); + ArrayNode pkNode = getNodeAs(schemaNode, FIELD_PK, ArrayNode.class); + List pkFields = new ArrayList<>(); + pkNode.forEach( + pk -> { + if (isNull(pk)) { + throw new IllegalArgumentException( + String.format("Primary key cannot be null: %s", pk)); + } + + pkFields.add(pk.asText()); + }); + return pkFields; + } + + @Override + public List extractRecords() { + if (isDDL()) { + return Collections.emptyList(); + } + + List records = new ArrayList<>(); + + JsonNode payload = root.get(FIELD_PAYLOAD); + checkNotNull(payload, FIELD_PAYLOAD); + + String type = payload.get(FIELD_TYPE).asText(); + + RowKind rowKind = null; + String field = null; + switch (type) { + case OP_UPDATE_BEFORE: + rowKind = RowKind.UPDATE_BEFORE; + field = FIELD_BEFORE; + break; + case OP_UPDATE_AFTER: + rowKind = RowKind.UPDATE_AFTER; + field = FIELD_AFTER; + break; + case OP_INSERT: + rowKind = RowKind.INSERT; + field = FIELD_AFTER; + break; + case OP_DELETE: + rowKind = RowKind.DELETE; + field = FIELD_BEFORE; + break; + default: + throw new UnsupportedOperationException("Unknown record operation: " + type); + } + + JsonNode container = payload.get(field); + checkNotNull(container, String.format("%s.%s", FIELD_PAYLOAD, field)); + + JsonNode data = getNodeAs(container, FIELD_COLUMN, JsonNode.class); + checkNotNull(data, String.format("%s.%s.%s", FIELD_PAYLOAD, field, FIELD_COLUMN)); + + processRecord(data, rowKind, records); + + return records; + } + + @Override + protected Map extractRowData(JsonNode record, RowType.Builder rowTypeBuilder) { + + Map recordMap = + JsonSerdeUtil.convertValue(record, new TypeReference>() {}); + Map rowData = new HashMap<>(); + + fillDefaultTypes(record, rowTypeBuilder); + for (Map.Entry entry : recordMap.entrySet()) { + rowData.put(entry.getKey(), Objects.toString(entry.getValue(), null)); + } + + evalComputedColumns(rowData, rowTypeBuilder); + return rowData; + } + + @Override + protected String format() { + return "aliyun-json"; + } + + @Nullable + @Override + protected String getTableName() { + JsonNode schemaNode = root.get(FIELD_SCHEMA); + if (isNull(schemaNode)) { + return null; + } + JsonNode sourceNode = schemaNode.get("source"); + if (isNull(sourceNode)) { + return null; + } + + JsonNode tableNode = sourceNode.get("tableName"); + if (isNull(tableNode)) { + return null; + } + return tableNode.asText(); + } + + @Nullable + @Override + protected String getDatabaseName() { + JsonNode schemaNode = root.get(FIELD_SCHEMA); + if (isNull(schemaNode)) { + return null; + } + JsonNode sourceNode = schemaNode.get("source"); + if (isNull(sourceNode)) { + return null; + } + JsonNode databaseNode = sourceNode.get("dbName"); + if (isNull(databaseNode)) { + return null; + } + return databaseNode.asText(); + } + + private Map matchOldRecords(ArrayNode newData, ArrayNode oldData) { + return IntStream.range(0, newData.size()) + .boxed() + .collect(Collectors.toMap(newData::get, oldData::get)); + } + + private String transformValue(@Nullable String oldValue, String shortType, String mySqlType) { + if (oldValue == null) { + return null; + } + + if (MySqlTypeUtils.isSetType(shortType)) { + return AliyunFieldParser.convertSet(oldValue, mySqlType); + } + + if (MySqlTypeUtils.isEnumType(shortType)) { + return AliyunFieldParser.convertEnum(oldValue, mySqlType); + } + + if (MySqlTypeUtils.isGeoType(shortType)) { + try { + byte[] wkb = + AliyunFieldParser.convertGeoType2WkbArray( + oldValue.getBytes(StandardCharsets.ISO_8859_1)); + return MySqlTypeUtils.convertWkbArray(wkb); + } catch (Exception e) { + throw new IllegalArgumentException( + String.format("Failed to convert %s to geometry JSON.", oldValue), e); + } + } + return oldValue; + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java index fc672b9dc0ab6..eea364d460dee 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumAvroDeserializationSchema.java @@ -48,7 +48,7 @@ public class KafkaDebeziumAvroDeserializationSchema public KafkaDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { this.topic = KafkaActionUtils.findOneTopic(cdcSourceConfig); - this.schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); + this.schemaRegistryUrl = cdcSourceConfig.get(SCHEMA_REGISTRY_URL); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java index 64f127571134b..df288a4150e63 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mongodb/strategy/MongoVersionStrategy.java @@ -83,7 +83,7 @@ default Map getExtractRow( Configuration mongodbConfig) throws JsonProcessingException { SchemaAcquisitionMode mode = - SchemaAcquisitionMode.valueOf(mongodbConfig.getString(START_MODE).toUpperCase()); + SchemaAcquisitionMode.valueOf(mongodbConfig.get(START_MODE).toUpperCase()); ObjectNode objectNode = JsonSerdeUtil.asSpecificNodeType(jsonNode.asText(), ObjectNode.class); JsonNode idNode = objectNode.get(ID_FIELD); @@ -92,7 +92,7 @@ default Map getExtractRow( "The provided MongoDB JSON document does not contain an _id field."); } JsonNode document = - mongodbConfig.getBoolean(DEFAULT_ID_GENERATION) + mongodbConfig.get(DEFAULT_ID_GENERATION) ? objectNode.set( ID_FIELD, idNode.get(OID_FIELD) == null ? idNode : idNode.get(OID_FIELD)) @@ -101,8 +101,8 @@ default Map getExtractRow( case SPECIFIED: return parseFieldsFromJsonRecord( document.toString(), - mongodbConfig.getString(PARSER_PATH), - mongodbConfig.getString(FIELD_NAME), + mongodbConfig.get(PARSER_PATH), + mongodbConfig.get(FIELD_NAME), computedColumns, rowTypeBuilder); case DYNAMIC: diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java index 502e6237a477e..26579e718f564 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/mysql/MySqlRecordParser.java @@ -45,6 +45,8 @@ import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.cdc.connectors.mysql.source.config.MySqlSourceOptions; import org.apache.flink.cdc.debezium.table.DebeziumOptions; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import org.slf4j.Logger; @@ -99,11 +101,14 @@ public MySqlRecordParser( .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); String stringifyServerTimeZone = mySqlConfig.get(MySqlSourceOptions.SERVER_TIME_ZONE); - this.isDebeziumSchemaCommentsEnabled = - mySqlConfig.getBoolean( - DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX - + RelationalDatabaseConnectorConfig.INCLUDE_SCHEMA_COMMENTS.name(), - false); + ConfigOption includeSchemaCommentsConfig = + ConfigOptions.key( + DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX + + RelationalDatabaseConnectorConfig.INCLUDE_SCHEMA_COMMENTS + .name()) + .booleanType() + .defaultValue(false); + this.isDebeziumSchemaCommentsEnabled = mySqlConfig.get(includeSchemaCommentsConfig); this.serverTimeZone = stringifyServerTimeZone == null ? ZoneId.systemDefault() 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 f8ea8cdc44386..ce2e9124a664a 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 @@ -138,11 +138,13 @@ protected void beforeBuildingSourceSink() throws Exception { + ", or MySQL database does not exist."); TableNameConverter tableNameConverter = - new TableNameConverter(allowUpperCase, mergeShards, tablePrefix, tableSuffix); + new TableNameConverter( + allowUpperCase, mergeShards, tablePrefix, tableSuffix, tableMapping); for (JdbcTableInfo tableInfo : jdbcTableInfos) { Identifier identifier = Identifier.create( - database, tableNameConverter.convert(tableInfo.toPaimonTableName())); + database, + tableNameConverter.convert("", tableInfo.toPaimonTableName())); FileStoreTable table; Schema fromMySql = CdcActionCommonUtils.buildPaimonSchema( diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java index b0d1d1bf620f5..f45ee034bec84 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/pulsar/PulsarDebeziumAvroDeserializationSchema.java @@ -46,7 +46,7 @@ public class PulsarDebeziumAvroDeserializationSchema public PulsarDebeziumAvroDeserializationSchema(Configuration cdcSourceConfig) { this.topic = PulsarActionUtils.findOneTopic(cdcSourceConfig); - this.schemaRegistryUrl = cdcSourceConfig.getString(SCHEMA_REGISTRY_URL); + this.schemaRegistryUrl = cdcSourceConfig.get(SCHEMA_REGISTRY_URL); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/watermark/MessageQueueCdcTimestampExtractor.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/watermark/MessageQueueCdcTimestampExtractor.java index 8a9a28453badf..5bf2fefc1b78a 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/watermark/MessageQueueCdcTimestampExtractor.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/watermark/MessageQueueCdcTimestampExtractor.java @@ -54,6 +54,10 @@ public long extractTimestamp(CdcSourceRecord cdcSourceRecord) throws JsonProcess } else if (JsonSerdeUtil.isNodeExists(record, "source", "connector")) { // Dbz json return JsonSerdeUtil.extractValue(record, Long.class, "ts_ms"); + } else if (JsonSerdeUtil.isNodeExists(record, "payload", "timestamp")) { + // Aliyun json + return JsonSerdeUtil.extractValue( + record, Long.class, "payload", "timestamp", "systemTime"); } throw new RuntimeException( String.format( diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java index 72a177adceaf3..41e7141cf48a8 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/kafka/KafkaSinkFunction.java @@ -21,6 +21,7 @@ import org.apache.paimon.flink.sink.LogSinkFunction; import org.apache.paimon.table.sink.SinkRecord; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaException; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer; @@ -65,7 +66,16 @@ public void setWriteCallback(WriteCallback writeCallback) { this.writeCallback = writeCallback; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration configuration) throws Exception { super.open(configuration); Callback baseCallback = requireNonNull(callback); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java index 574ff685f3fa4..6d9e3a4a7c82f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java @@ -25,7 +25,7 @@ import org.apache.paimon.table.sink.KeyAndBucketExtractor; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; /** {@link CdcDynamicBucketSinkBase} for {@link CdcRecord}. */ public class CdcDynamicBucketSink extends CdcDynamicBucketSinkBase { @@ -42,8 +42,8 @@ protected KeyAndBucketExtractor createExtractor(TableSchema schema) { } @Override - protected OneInputStreamOperator, Committable> createWriteOperator( - StoreSinkWrite.Provider writeProvider, String commitUser) { - return new CdcDynamicBucketWriteOperator(table, writeProvider, commitUser); + protected OneInputStreamOperatorFactory, Committable> + createWriteOperatorFactory(StoreSinkWrite.Provider writeProvider, String commitUser) { + return new CdcDynamicBucketWriteOperator.Factory(table, writeProvider, commitUser); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java index b2fbdc3e93eef..b0b135b3610bb 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.data.GenericRow; +import org.apache.paimon.flink.sink.Committable; import org.apache.paimon.flink.sink.PrepareCommitOperator; import org.apache.paimon.flink.sink.StoreSinkWrite; import org.apache.paimon.flink.sink.TableWriteOperator; @@ -26,6 +27,9 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import java.io.IOException; @@ -43,11 +47,12 @@ public class CdcDynamicBucketWriteOperator extends TableWriteOperator parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(table, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); this.retrySleepMillis = table.coreOptions().toConfiguration().get(RETRY_SLEEP_TIME).toMillis(); } @@ -85,4 +90,30 @@ public void processElement(StreamRecord> element) thr throw new IOException(e); } } + + /** {@link StreamOperatorFactory} of {@link CdcDynamicBucketWriteOperator}. */ + public static class Factory extends TableWriteOperator.Factory> { + + public Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CdcDynamicBucketWriteOperator( + parameters, table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CdcDynamicBucketWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java index 0961ff160048d..886e33e2046aa 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicTableParsingProcessFunction.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.types.DataField; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -74,7 +75,16 @@ public CdcDynamicTableParsingProcessFunction( this.parserFactory = parserFactory; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { parser = parserFactory.create(); catalog = catalogLoader.load(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java index 59bdb192beea7..bec9508888b4c 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java @@ -24,7 +24,7 @@ import org.apache.paimon.flink.sink.StoreSinkWrite; import org.apache.paimon.table.FileStoreTable; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; /** * A {@link FlinkSink} for fixed-bucket table which accepts {@link CdcRecord} and waits for a schema @@ -39,8 +39,8 @@ public CdcFixedBucketSink(FileStoreTable table) { } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new CdcRecordStoreWriteOperator(table, writeProvider, commitUser); + return new CdcRecordStoreWriteOperator.Factory(table, writeProvider, commitUser); } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java index b18a05c280cbf..4c5e0600bb47e 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcMultiTableParsingProcessFunction.java @@ -20,6 +20,7 @@ import org.apache.paimon.types.DataField; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.configuration.Configuration; @@ -51,7 +52,16 @@ public CdcMultiTableParsingProcessFunction(EventParser.Factory parserFactory) this.parserFactory = parserFactory; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { parser = parserFactory.create(); updatedDataFieldsOutputTags = new HashMap<>(); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java index 3456634942c8d..eec228f3c09bb 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcParsingProcessFunction.java @@ -20,6 +20,7 @@ import org.apache.paimon.types.DataField; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; @@ -50,7 +51,16 @@ public CdcParsingProcessFunction(EventParser.Factory parserFactory) { this.parserFactory = parserFactory; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { parser = parserFactory.create(); } 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 7d72fe3e801f9..5db111a300474 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 @@ -38,6 +38,9 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import java.io.IOException; @@ -74,12 +77,13 @@ public class CdcRecordStoreMultiWriteOperator private String commitUser; private ExecutorService compactExecutor; - public CdcRecordStoreMultiWriteOperator( + private CdcRecordStoreMultiWriteOperator( + StreamOperatorParameters parameters, Catalog.Loader catalogLoader, StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider, String initialCommitUser, Options options) { - super(options); + super(parameters, options); this.catalogLoader = catalogLoader; this.storeSinkWriteProvider = storeSinkWriteProvider; this.initialCommitUser = initialCommitUser; @@ -254,4 +258,42 @@ public Map writes() { public String commitUser() { return commitUser; } + + /** {@link StreamOperatorFactory} of {@link CdcRecordStoreMultiWriteOperator}. */ + public static class Factory + extends PrepareCommitOperator.Factory { + private final StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider; + private final String initialCommitUser; + private final Catalog.Loader catalogLoader; + + public Factory( + Catalog.Loader catalogLoader, + StoreSinkWrite.WithWriteBufferProvider storeSinkWriteProvider, + String initialCommitUser, + Options options) { + super(options); + this.catalogLoader = catalogLoader; + this.storeSinkWriteProvider = storeSinkWriteProvider; + this.initialCommitUser = initialCommitUser; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CdcRecordStoreMultiWriteOperator( + parameters, + catalogLoader, + storeSinkWriteProvider, + initialCommitUser, + options); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CdcRecordStoreMultiWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java index dd0aa2e5622c2..195e683daaf6f 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink.cdc; import org.apache.paimon.data.GenericRow; +import org.apache.paimon.flink.sink.Committable; import org.apache.paimon.flink.sink.PrepareCommitOperator; import org.apache.paimon.flink.sink.StoreSinkWrite; import org.apache.paimon.flink.sink.TableWriteOperator; @@ -27,6 +28,9 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import java.io.IOException; @@ -50,11 +54,12 @@ public class CdcRecordStoreWriteOperator extends TableWriteOperator { private final long retrySleepMillis; - public CdcRecordStoreWriteOperator( + protected CdcRecordStoreWriteOperator( + StreamOperatorParameters parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(table, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); this.retrySleepMillis = table.coreOptions().toConfiguration().get(RETRY_SLEEP_TIME).toMillis(); } @@ -92,4 +97,30 @@ public void processElement(StreamRecord element) throws Exception { throw new IOException(e); } } + + /** {@link StreamOperatorFactory} of {@link CdcRecordStoreWriteOperator}. */ + public static class Factory extends TableWriteOperator.Factory { + + public Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CdcRecordStoreWriteOperator( + parameters, table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CdcRecordStoreWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java index 313f4d013ef89..820ef7728f8ce 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java @@ -24,7 +24,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -42,9 +42,9 @@ public CdcUnawareBucketSink(FileStoreTable table, Integer parallelism) { } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new CdcUnawareBucketWriteOperator(table, writeProvider, commitUser); + return new CdcUnawareBucketWriteOperator.Factory(table, writeProvider, commitUser); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketWriteOperator.java index c57a40f3f71da..26f65fdd09ce2 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketWriteOperator.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketWriteOperator.java @@ -18,21 +18,26 @@ package org.apache.paimon.flink.sink.cdc; +import org.apache.paimon.flink.sink.Committable; import org.apache.paimon.flink.sink.PrepareCommitOperator; import org.apache.paimon.flink.sink.StoreSinkWrite; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.types.RowKind; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** A {@link PrepareCommitOperator} to write {@link CdcRecord} to unaware-bucket mode table. */ public class CdcUnawareBucketWriteOperator extends CdcRecordStoreWriteOperator { - public CdcUnawareBucketWriteOperator( + private CdcUnawareBucketWriteOperator( + StreamOperatorParameters parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(table, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); } @Override @@ -42,4 +47,30 @@ public void processElement(StreamRecord element) throws Exception { super.processElement(element); } } + + /** {@link StreamOperatorFactory} of {@link CdcUnawareBucketWriteOperator}. */ + public static class Factory extends CdcRecordStoreWriteOperator.Factory { + + public Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CdcUnawareBucketWriteOperator( + parameters, table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CdcUnawareBucketWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java index 55e987c6055fa..1688d4deb0884 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java @@ -21,7 +21,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.sink.CommittableStateManager; import org.apache.paimon.flink.sink.Committer; -import org.apache.paimon.flink.sink.CommitterOperator; +import org.apache.paimon.flink.sink.CommitterOperatorFactory; import org.apache.paimon.flink.sink.FlinkSink; import org.apache.paimon.flink.sink.FlinkStreamPartitioner; import org.apache.paimon.flink.sink.MultiTableCommittable; @@ -40,8 +40,8 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -63,19 +63,16 @@ public class FlinkCdcMultiTableSink implements Serializable { private final Catalog.Loader catalogLoader; private final double commitCpuCores; @Nullable private final MemorySize commitHeapMemory; - private final boolean commitChaining; private final String commitUser; public FlinkCdcMultiTableSink( Catalog.Loader catalogLoader, double commitCpuCores, @Nullable MemorySize commitHeapMemory, - boolean commitChaining, String commitUser) { this.catalogLoader = catalogLoader; this.commitCpuCores = commitCpuCores; this.commitHeapMemory = commitHeapMemory; - this.commitChaining = commitChaining; this.commitUser = commitUser; } @@ -129,21 +126,21 @@ public DataStreamSink sinkFrom( .transform( GLOBAL_COMMITTER_NAME, typeInfo, - new CommitterOperator<>( + new CommitterOperatorFactory<>( true, false, - commitChaining, commitUser, createCommitterFactory(), createCommittableStateManager())) .setParallelism(input.getParallelism()); configureGlobalCommitter(committed, commitCpuCores, commitHeapMemory); - return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + return committed.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } - protected OneInputStreamOperator createWriteOperator( - StoreSinkWrite.WithWriteBufferProvider writeProvider, String commitUser) { - return new CdcRecordStoreMultiWriteOperator( + protected OneInputStreamOperatorFactory + createWriteOperator( + StoreSinkWrite.WithWriteBufferProvider writeProvider, String commitUser) { + return new CdcRecordStoreMultiWriteOperator.Factory( catalogLoader, writeProvider, commitUser, new Options()); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java index ed8fdd113389a..a9ad66847b4b5 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java @@ -66,7 +66,6 @@ public class FlinkCdcSyncDatabaseSinkBuilder { @Nullable private Integer parallelism; private double committerCpu; @Nullable private MemorySize committerMemory; - private boolean commitChaining; // Paimon catalog used to check and create tables. There will be two // places where this catalog is used. 1) in processing function, @@ -103,7 +102,6 @@ public FlinkCdcSyncDatabaseSinkBuilder withTableOptions(Options options) { this.parallelism = options.get(FlinkConnectorOptions.SINK_PARALLELISM); this.committerCpu = options.get(FlinkConnectorOptions.SINK_COMMITTER_CPU); this.committerMemory = options.get(FlinkConnectorOptions.SINK_COMMITTER_MEMORY); - this.commitChaining = options.get(FlinkConnectorOptions.SINK_COMMITTER_OPERATOR_CHAINING); this.commitUser = createCommitUser(options); return this; } @@ -169,7 +167,7 @@ private void buildCombinedCdcSink() { FlinkCdcMultiTableSink sink = new FlinkCdcMultiTableSink( - catalogLoader, committerCpu, committerMemory, commitChaining, commitUser); + catalogLoader, committerCpu, committerMemory, commitUser); sink.sinkFrom(partitioned); } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java index 4a33eb1b7ec94..64f00d96b0f51 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java @@ -23,11 +23,18 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.types.DataField; +import org.apache.paimon.types.FieldIdentifier; +import org.apache.paimon.types.RowType; +import org.apache.commons.collections.CollectionUtils; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.util.Collector; +import java.util.HashSet; import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; /** * A {@link ProcessFunction} to handle schema changes. New schema is represented by a list of {@link @@ -43,19 +50,51 @@ public class UpdatedDataFieldsProcessFunction private final Identifier identifier; + private Set latestFields; + public UpdatedDataFieldsProcessFunction( SchemaManager schemaManager, Identifier identifier, Catalog.Loader catalogLoader) { super(catalogLoader); this.schemaManager = schemaManager; this.identifier = identifier; + this.latestFields = new HashSet<>(); } @Override public void processElement( List updatedDataFields, Context context, Collector collector) throws Exception { - for (SchemaChange schemaChange : extractSchemaChanges(schemaManager, updatedDataFields)) { + List actualUpdatedDataFields = + updatedDataFields.stream() + .filter( + dataField -> + !latestDataFieldContain(new FieldIdentifier(dataField))) + .collect(Collectors.toList()); + if (CollectionUtils.isEmpty(actualUpdatedDataFields)) { + return; + } + for (SchemaChange schemaChange : + extractSchemaChanges(schemaManager, actualUpdatedDataFields)) { applySchemaChange(schemaManager, schemaChange, identifier); } + /** + * Here, actualUpdatedDataFields cannot be used to update latestFields because there is a + * non-SchemaChange.AddColumn scenario. Otherwise, the previously existing fields cannot be + * modified again. + */ + updateLatestFields(); + } + + private boolean latestDataFieldContain(FieldIdentifier dataField) { + return latestFields.stream().anyMatch(previous -> Objects.equals(previous, dataField)); + } + + private void updateLatestFields() { + RowType oldRowType = schemaManager.latest().get().logicalRowType(); + Set fieldIdentifiers = + oldRowType.getFields().stream() + .map(item -> new FieldIdentifier(item)) + .collect(Collectors.toSet()); + latestFields = fieldIdentifiers; } } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java index 0e93fdb073244..4f02b784c2ba6 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java @@ -31,6 +31,7 @@ import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.StringUtils; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.slf4j.Logger; @@ -73,7 +74,16 @@ protected UpdatedDataFieldsProcessFunctionBase(Catalog.Loader catalogLoader) { this.catalogLoader = catalogLoader; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public void open(Configuration parameters) { this.catalog = catalogLoader.load(); this.allowUpperCase = this.catalog.allowUpperCase(); @@ -101,7 +111,7 @@ protected void applySchemaChange( SchemaChange.UpdateColumnType updateColumnType = (SchemaChange.UpdateColumnType) schemaChange; Preconditions.checkState( - updateColumnType.fieldNames().size() == 1, + updateColumnType.fieldNames().length == 1, "Paimon CDC currently does not support nested type schema evolution."); TableSchema schema = schemaManager @@ -110,11 +120,11 @@ protected void applySchemaChange( () -> new RuntimeException( "Table does not exist. This is unexpected.")); - int idx = schema.fieldNames().indexOf(updateColumnType.fieldNames().get(0)); + int idx = schema.fieldNames().indexOf(updateColumnType.fieldNames()[0]); Preconditions.checkState( idx >= 0, "Field name " - + updateColumnType.fieldNames().get(0) + + updateColumnType.fieldNames()[0] + " does not exist in table. This is unexpected."); DataType oldType = schema.fields().get(idx).type(); DataType newType = updateColumnType.newDataType(); @@ -126,7 +136,7 @@ protected void applySchemaChange( throw new UnsupportedOperationException( String.format( "Cannot convert field %s from type %s to %s of Paimon table %s.", - updateColumnType.fieldNames().get(0), + updateColumnType.fieldNames()[0], oldType, newType, identifier.getFullName())); 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 17b8b29a20099..1b30c7ab63960 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 @@ -27,6 +27,7 @@ org.apache.paimon.flink.action.cdc.mongodb.MongoDBSyncDatabaseActionFactory org.apache.paimon.flink.action.cdc.postgres.PostgresSyncTableActionFactory ### message queue data format factories +org.apache.paimon.flink.action.cdc.format.aliyun.AliyunDataFormatFactory org.apache.paimon.flink.action.cdc.format.canal.CanalDataFormatFactory org.apache.paimon.flink.action.cdc.format.debezium.DebeziumAvroDataFormatFactory org.apache.paimon.flink.action.cdc.format.debezium.DebeziumJsonDataFormatFactory diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java new file mode 100644 index 0000000000000..9ba18376867ff --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.flink.FlinkCatalogFactory; +import org.apache.paimon.flink.sink.cdc.UpdatedDataFieldsProcessFunction; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.options.Options; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.SchemaUtils; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.TableTestBase; +import org.apache.paimon.types.BigIntType; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.DecimalType; +import org.apache.paimon.types.DoubleType; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.VarCharType; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; + +/** Used to test schema evolution related logic. */ +public class SchemaEvolutionTest extends TableTestBase { + + private static List> prepareData() { + List upField1 = + Arrays.asList( + new DataField(0, "col_0", new VarCharType(), "test description."), + new DataField(1, "col_1", new IntType(), "test description."), + new DataField(2, "col_2", new IntType(), "test description."), + new DataField(3, "col_3", new VarCharType(), "Someone's desc."), + new DataField(4, "col_4", new VarCharType(), "Someone's desc."), + new DataField(5, "col_5", new VarCharType(), "Someone's desc."), + new DataField(6, "col_6", new DecimalType(), "Someone's desc."), + new DataField(7, "col_7", new VarCharType(), "Someone's desc."), + new DataField(8, "col_8", new VarCharType(), "Someone's desc."), + new DataField(9, "col_9", new VarCharType(), "Someone's desc."), + new DataField(10, "col_10", new VarCharType(), "Someone's desc."), + new DataField(11, "col_11", new VarCharType(), "Someone's desc."), + new DataField(12, "col_12", new DoubleType(), "Someone's desc."), + new DataField(13, "col_13", new VarCharType(), "Someone's desc."), + new DataField(14, "col_14", new VarCharType(), "Someone's desc."), + new DataField(15, "col_15", new VarCharType(), "Someone's desc."), + new DataField(16, "col_16", new VarCharType(), "Someone's desc."), + new DataField(17, "col_17", new VarCharType(), "Someone's desc."), + new DataField(18, "col_18", new VarCharType(), "Someone's desc."), + new DataField(19, "col_19", new VarCharType(), "Someone's desc."), + new DataField(20, "col_20", new VarCharType(), "Someone's desc.")); + List upField2 = + Arrays.asList( + new DataField(0, "col_0", new VarCharType(), "test description."), + new DataField(1, "col_1", new BigIntType(), "test description."), + new DataField(2, "col_2", new IntType(), "test description."), + new DataField(3, "col_3", new VarCharType(), "Someone's desc."), + new DataField(4, "col_4", new VarCharType(), "Someone's desc."), + new DataField(5, "col_5", new VarCharType(), "Someone's desc."), + new DataField(6, "col_6", new DecimalType(), "Someone's desc."), + new DataField(7, "col_7", new VarCharType(), "Someone's desc."), + new DataField(8, "col_8", new VarCharType(), "Someone's desc."), + new DataField(9, "col_9", new VarCharType(), "Someone's desc."), + new DataField(10, "col_10", new VarCharType(), "Someone's desc."), + new DataField(11, "col_11", new VarCharType(), "Someone's desc."), + new DataField(12, "col_12", new DoubleType(), "Someone's desc."), + new DataField(13, "col_13", new VarCharType(), "Someone's desc."), + new DataField(14, "col_14", new VarCharType(), "Someone's desc."), + new DataField(15, "col_15", new VarCharType(), "Someone's desc."), + new DataField(16, "col_16", new VarCharType(), "Someone's desc."), + new DataField(17, "col_17", new VarCharType(), "Someone's desc."), + new DataField(18, "col_18", new VarCharType(), "Someone's desc."), + new DataField(19, "col_19", new VarCharType(), "Someone's desc."), + new DataField(20, "col_20", new VarCharType(), "Someone's desc.")); + List upField3 = + Arrays.asList( + new DataField(0, "col_0", new VarCharType(), "test description."), + new DataField(1, "col_1", new BigIntType(), "test description."), + new DataField(2, "col_2", new IntType(), "test description 2."), + new DataField(3, "col_3", new VarCharType(), "Someone's desc."), + new DataField(4, "col_4", new VarCharType(), "Someone's desc."), + new DataField(5, "col_5", new VarCharType(), "Someone's desc."), + new DataField(6, "col_6", new DecimalType(), "Someone's desc."), + new DataField(7, "col_7", new VarCharType(), "Someone's desc."), + new DataField(8, "col_8", new VarCharType(), "Someone's desc."), + new DataField(9, "col_9", new VarCharType(), "Someone's desc."), + new DataField(10, "col_10", new VarCharType(), "Someone's desc."), + new DataField(11, "col_11", new VarCharType(), "Someone's desc."), + new DataField(12, "col_12", new DoubleType(), "Someone's desc."), + new DataField(13, "col_13", new VarCharType(), "Someone's desc."), + new DataField(14, "col_14", new VarCharType(), "Someone's desc."), + new DataField(15, "col_15", new VarCharType(), "Someone's desc."), + new DataField(16, "col_16", new VarCharType(), "Someone's desc."), + new DataField(17, "col_17", new VarCharType(), "Someone's desc."), + new DataField(18, "col_18", new VarCharType(), "Someone's desc."), + new DataField(19, "col_19", new VarCharType(), "Someone's desc."), + new DataField(20, "col_20", new VarCharType(), "Someone's desc.")); + List upField4 = + Arrays.asList( + new DataField(0, "col_0", new VarCharType(), "test description."), + new DataField(1, "col_1", new BigIntType(), "test description."), + new DataField(2, "col_2", new IntType(), "test description."), + new DataField(3, "col_3_1", new VarCharType(), "Someone's desc."), + new DataField(4, "col_4", new VarCharType(), "Someone's desc."), + new DataField(5, "col_5", new VarCharType(), "Someone's desc."), + new DataField(6, "col_6", new DecimalType(), "Someone's desc."), + new DataField(7, "col_7", new VarCharType(), "Someone's desc."), + new DataField(8, "col_8", new VarCharType(), "Someone's desc."), + new DataField(9, "col_9", new VarCharType(), "Someone's desc."), + new DataField(10, "col_10", new VarCharType(), "Someone's desc."), + new DataField(11, "col_11", new VarCharType(), "Someone's desc."), + new DataField(12, "col_12", new DoubleType(), "Someone's desc."), + new DataField(13, "col_13", new VarCharType(), "Someone's desc."), + new DataField(14, "col_14", new VarCharType(), "Someone's desc."), + new DataField(15, "col_15", new VarCharType(), "Someone's desc."), + new DataField(16, "col_16", new VarCharType(), "Someone's desc."), + new DataField(17, "col_17", new VarCharType(), "Someone's desc."), + new DataField(18, "col_18", new VarCharType(), "Someone's desc."), + new DataField(19, "col_19", new VarCharType(), "Someone's desc."), + new DataField(20, "col_20", new VarCharType(), "Someone's desc.")); + List upField5 = + Arrays.asList( + new DataField(0, "col_0", new VarCharType(), "test description."), + new DataField(1, "col_1", new BigIntType(), "test description."), + new DataField(2, "col_2_1", new BigIntType(), "test description 2."), + new DataField(3, "col_3", new VarCharType(), "Someone's desc."), + new DataField(4, "col_4", new VarCharType(), "Someone's desc."), + new DataField(5, "col_5", new VarCharType(), "Someone's desc."), + new DataField(6, "col_6", new DecimalType(), "Someone's desc."), + new DataField(7, "col_7", new VarCharType(), "Someone's desc."), + new DataField(8, "col_8", new VarCharType(), "Someone's desc."), + new DataField(9, "col_9", new VarCharType(), "Someone's desc."), + new DataField(10, "col_10", new VarCharType(), "Someone's desc."), + new DataField(11, "col_11", new VarCharType(), "Someone's desc."), + new DataField(12, "col_12", new DoubleType(), "Someone's desc."), + new DataField(13, "col_13", new VarCharType(), "Someone's desc."), + new DataField(14, "col_14", new VarCharType(), "Someone's desc."), + new DataField(15, "col_15", new VarCharType(), "Someone's desc."), + new DataField(16, "col_16", new VarCharType(), "Someone's desc."), + new DataField(17, "col_17", new VarCharType(), "Someone's desc."), + new DataField(18, "col_18", new VarCharType(), "Someone's desc."), + new DataField(19, "col_19", new VarCharType(), "Someone's desc."), + new DataField(20, "col_20", new VarCharType(), "Someone's desc.")); + return Arrays.asList(upField1, upField2, upField3, upField4, upField5); + } + + private FileStoreTable table; + private String tableName = "MyTable"; + + @BeforeEach + public void before() throws Exception { + FileIO fileIO = LocalFileIO.create(); + Path tablePath = new Path(String.format("%s/%s.db/%s", warehouse, database, tableName)); + Schema schema = + Schema.newBuilder() + .column("pk", DataTypes.INT()) + .column("pt1", DataTypes.INT()) + .column("pt2", DataTypes.INT()) + .column("col1", DataTypes.INT()) + .partitionKeys("pt1", "pt2") + .primaryKey("pk", "pt1", "pt2") + .option(CoreOptions.CHANGELOG_PRODUCER.key(), "input") + .option(CoreOptions.BUCKET.key(), "2") + .option(CoreOptions.SEQUENCE_FIELD.key(), "col1") + .build(); + TableSchema tableSchema = + SchemaUtils.forceCommit(new SchemaManager(fileIO, tablePath), schema); + table = FileStoreTableFactory.create(LocalFileIO.create(), tablePath, tableSchema); + } + + @Test + public void testSchemaEvolution() throws Exception { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + DataStream> upDataFieldStream = env.fromCollection(prepareData()); + Options options = new Options(); + options.set("warehouse", tempPath.toString()); + final Catalog.Loader catalogLoader = () -> FlinkCatalogFactory.createPaimonCatalog(options); + Identifier identifier = Identifier.create(database, tableName); + DataStream schemaChangeProcessFunction = + upDataFieldStream + .process( + new UpdatedDataFieldsProcessFunction( + new SchemaManager(table.fileIO(), table.location()), + identifier, + catalogLoader)) + .name("Schema Evolution"); + schemaChangeProcessFunction.getTransformation().setParallelism(1); + schemaChangeProcessFunction.getTransformation().setMaxParallelism(1); + env.execute(); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/TableNameConverterTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/TableNameConverterTest.java new file mode 100644 index 0000000000000..89bbadfeb8c83 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/TableNameConverterTest.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +/** Tests for {@link TableNameConverter}. */ +public class TableNameConverterTest { + + @Test + public void testConvertTableName() { + Map tableMapping = new HashMap<>(1); + tableMapping.put("mapped_src", "mapped_TGT"); + TableNameConverter caseConverter = + new TableNameConverter(true, true, "pre_", "_pos", tableMapping); + Assert.assertEquals(caseConverter.convert("", "mapped_SRC"), "mapped_TGT"); + + Assert.assertEquals(caseConverter.convert("", "unmapped_src"), "pre_unmapped_src_pos"); + + TableNameConverter noCaseConverter = + new TableNameConverter(false, true, "pre_", "_pos", tableMapping); + Assert.assertEquals(noCaseConverter.convert("", "mapped_src"), "mapped_tgt"); + Assert.assertEquals(noCaseConverter.convert("", "unmapped_src"), "pre_unmapped_src_pos"); + } + + @Test + public void testConvertTableNameByDBPrefix_Suffix() { + Map dbPrefix = new HashMap<>(2); + dbPrefix.put("db_with_prefix", "db_pref_"); + dbPrefix.put("db_with_prefix_suffix", "db_pref_"); + + Map dbSuffix = new HashMap<>(2); + dbSuffix.put("db_with_suffix", "_db_suff"); + dbSuffix.put("db_with_prefix_suffix", "_db_suff"); + + TableNameConverter tblNameConverter = + new TableNameConverter(false, true, dbPrefix, dbSuffix, "pre_", "_suf", null); + + // Tables in the specified db should have the specified prefix and suffix. + + // db prefix + normal suffix + Assert.assertEquals( + "db_pref_table_name_suf", tblNameConverter.convert("db_with_prefix", "table_name")); + + // normal prefix + db suffix + Assert.assertEquals( + "pre_table_name_db_suff", tblNameConverter.convert("db_with_suffix", "table_name")); + + // db prefix + db suffix + Assert.assertEquals( + "db_pref_table_name_db_suff", + tblNameConverter.convert("db_with_prefix_suffix", "table_name")); + + // only normal prefix and suffix + Assert.assertEquals( + "pre_table_name_suf", + tblNameConverter.convert("db_without_prefix_suffix", "table_name")); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunJsonRecordParserTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunJsonRecordParserTest.java new file mode 100644 index 0000000000000..f06268d700e5e --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/format/aliyun/AliyunJsonRecordParserTest.java @@ -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.flink.action.cdc.format.aliyun; + +import org.apache.paimon.flink.action.cdc.CdcSourceRecord; +import org.apache.paimon.flink.action.cdc.TypeMapping; +import org.apache.paimon.flink.action.cdc.kafka.KafkaActionITCaseBase; +import org.apache.paimon.flink.action.cdc.watermark.MessageQueueCdcTimestampExtractor; +import org.apache.paimon.flink.sink.cdc.CdcRecord; +import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecord; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.types.RowKind; + +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** Test for AliyunJsonRecordParser. */ +public class AliyunJsonRecordParserTest extends KafkaActionITCaseBase { + + private static final Logger log = LoggerFactory.getLogger(AliyunJsonRecordParserTest.class); + private static List insertList = new ArrayList<>(); + private static List updateList = new ArrayList<>(); + private static List deleteList = new ArrayList<>(); + + private static ObjectMapper objMapper = new ObjectMapper(); + + @Before + public void setup() { + String insertRes = "kafka/aliyun/table/event/event-insert.txt"; + String updateRes = "kafka/aliyun/table/event/event-update-in-one.txt"; + String deleteRes = "kafka/aliyun/table/event/event-delete.txt"; + URL url; + try { + url = AliyunJsonRecordParserTest.class.getClassLoader().getResource(insertRes); + Files.readAllLines(Paths.get(url.toURI())).stream() + .filter(this::isRecordLine) + .forEach(e -> insertList.add(e)); + + url = AliyunJsonRecordParserTest.class.getClassLoader().getResource(updateRes); + Files.readAllLines(Paths.get(url.toURI())).stream() + .filter(this::isRecordLine) + .forEach(e -> updateList.add(e)); + + url = AliyunJsonRecordParserTest.class.getClassLoader().getResource(deleteRes); + Files.readAllLines(Paths.get(url.toURI())).stream() + .filter(this::isRecordLine) + .forEach(e -> deleteList.add(e)); + + } catch (Exception e) { + log.error("Fail to init aliyun-json cases", e); + } + } + + @Test + public void extractInsertRecord() throws Exception { + AliyunRecordParser parser = + new AliyunRecordParser(TypeMapping.defaultMapping(), Collections.emptyList()); + for (String json : insertList) { + // 将json解析为JsonNode对象 + JsonNode rootNode = objMapper.readValue(json, JsonNode.class); + CdcSourceRecord cdcRecord = new CdcSourceRecord(rootNode); + Schema schema = parser.buildSchema(cdcRecord); + Assert.assertEquals(schema.primaryKeys(), Arrays.asList("id")); + + List records = parser.extractRecords(); + Assert.assertEquals(records.size(), 1); + + CdcRecord result = records.get(0).toRichCdcRecord().toCdcRecord(); + Assert.assertEquals(result.kind(), RowKind.INSERT); + + String dbName = parser.getDatabaseName(); + Assert.assertEquals(dbName, "bigdata_test"); + + String tableName = parser.getTableName(); + Assert.assertEquals(tableName, "sync_test_table"); + + MessageQueueCdcTimestampExtractor extractor = new MessageQueueCdcTimestampExtractor(); + Assert.assertTrue(extractor.extractTimestamp(cdcRecord) > 0); + } + } + + @Test + public void extractUpdateRecord() throws Exception { + AliyunRecordParser parser = + new AliyunRecordParser(TypeMapping.defaultMapping(), Collections.emptyList()); + for (String json : updateList) { + // 将json解析为JsonNode对象 + JsonNode jsonNode = objMapper.readValue(json, JsonNode.class); + CdcSourceRecord cdcRecord = new CdcSourceRecord(jsonNode); + Schema schema = parser.buildSchema(cdcRecord); + Assert.assertEquals(schema.primaryKeys(), Arrays.asList("id")); + + List records = parser.extractRecords(); + Assert.assertEquals(records.size(), 1); + + CdcRecord result = records.get(0).toRichCdcRecord().toCdcRecord(); + Assert.assertEquals(result.kind(), RowKind.UPDATE_AFTER); + + String dbName = parser.getDatabaseName(); + Assert.assertEquals(dbName, "bigdata_test"); + + String tableName = parser.getTableName(); + Assert.assertEquals(tableName, "sync_test_table"); + + MessageQueueCdcTimestampExtractor extractor = new MessageQueueCdcTimestampExtractor(); + Assert.assertTrue(extractor.extractTimestamp(cdcRecord) > 0); + } + } + + @Test + public void extractDeleteRecord() throws Exception { + AliyunRecordParser parser = + new AliyunRecordParser(TypeMapping.defaultMapping(), Collections.emptyList()); + for (String json : deleteList) { + // 将json解析为JsonNode对象 + JsonNode jsonNode = objMapper.readValue(json, JsonNode.class); + CdcSourceRecord cdcRecord = new CdcSourceRecord(jsonNode); + Schema schema = parser.buildSchema(cdcRecord); + Assert.assertEquals(schema.primaryKeys(), Arrays.asList("id")); + + List records = parser.extractRecords(); + Assert.assertEquals(records.size(), 1); + + CdcRecord result = records.get(0).toRichCdcRecord().toCdcRecord(); + Assert.assertEquals(result.kind(), RowKind.DELETE); + + String dbName = parser.getDatabaseName(); + Assert.assertEquals(dbName, "bigdata_test"); + + String tableName = parser.getTableName(); + Assert.assertEquals(tableName, "sync_test_table"); + + MessageQueueCdcTimestampExtractor extractor = new MessageQueueCdcTimestampExtractor(); + Assert.assertTrue(extractor.extractTimestamp(cdcRecord) > 0); + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java index 394cdd1f149bc..f0328b5663246 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mongodb/MongodbSchemaITCase.java @@ -76,13 +76,12 @@ public static void initMongoDB() { @Test public void testCreateSchemaFromValidConfig() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); Schema schema = MongodbSchemaUtils.getMongodbSchema(mongodbConfig); assertNotNull(schema); } @@ -90,13 +89,12 @@ public void testCreateSchemaFromValidConfig() { @Test public void testCreateSchemaFromInvalidHost() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, "127.0.0.1:12345"); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, "127.0.0.1:12345"); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); assertThrows( RuntimeException.class, () -> MongodbSchemaUtils.getMongodbSchema(mongodbConfig)); @@ -106,7 +104,7 @@ public void testCreateSchemaFromInvalidHost() { public void testCreateSchemaFromIncompleteConfig() { // Create a Configuration object with missing necessary settings Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); // Expect an exception to be thrown due to missing necessary settings assertThrows( NullPointerException.class, @@ -117,13 +115,12 @@ public void testCreateSchemaFromIncompleteConfig() { public void testCreateSchemaFromDynamicConfig() { // Create a Configuration object with the necessary settings Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); // Call the method and check the results Schema schema = MongodbSchemaUtils.getMongodbSchema(mongodbConfig); @@ -142,13 +139,12 @@ public void testCreateSchemaFromDynamicConfig() { @Test public void testCreateSchemaFromInvalidDatabase() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "invalidDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "testCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "invalidDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "testCollection"); assertThrows( RuntimeException.class, () -> MongodbSchemaUtils.getMongodbSchema(mongodbConfig)); @@ -157,13 +153,12 @@ public void testCreateSchemaFromInvalidDatabase() { @Test public void testCreateSchemaFromInvalidCollection() { Configuration mongodbConfig = new Configuration(); - mongodbConfig.setString(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); - mongodbConfig.setString(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); - mongodbConfig.setString( - MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); - mongodbConfig.setString(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); - mongodbConfig.setString(MongoDBSourceOptions.DATABASE, "testDatabase"); - mongodbConfig.setString(MongoDBSourceOptions.COLLECTION, "invalidCollection"); + mongodbConfig.set(MongoDBSourceOptions.HOSTS, MONGODB_CONTAINER.getHostAndPort()); + mongodbConfig.set(MongoDBSourceOptions.USERNAME, MongoDBContainer.PAIMON_USER); + mongodbConfig.set(MongoDBSourceOptions.PASSWORD, MongoDBContainer.PAIMON_USER_PASSWORD); + mongodbConfig.set(MongoDBSourceOptions.CONNECTION_OPTIONS, "authSource=admin"); + mongodbConfig.set(MongoDBSourceOptions.DATABASE, "testDatabase"); + mongodbConfig.set(MongoDBSourceOptions.COLLECTION, "invalidCollection"); assertThrows( RuntimeException.class, () -> MongodbSchemaUtils.getMongodbSchema(mongodbConfig)); 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 bdeab07a746cc..febbe4e1deaad 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 @@ -31,7 +31,8 @@ import org.apache.paimon.utils.CommonTestUtils; import org.apache.paimon.utils.JsonSerdeUtil; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.BeforeAll; @@ -1285,8 +1286,11 @@ public void testDefaultCheckpointInterval() throws Exception { mySqlConfig.put("database-name", "default_checkpoint"); mySqlConfig.put("table-name", "t"); - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setRestartStrategy(RestartStrategies.noRestart()); + // Using `none` to avoid compatibility issues with Flink 1.18-. + Configuration configuration = new Configuration(); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY, "none"); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build(); action.withStreamExecutionEnvironment(env); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java index 698900436e8d5..b202ca53c9cc9 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordSerializeITCase.java @@ -25,6 +25,8 @@ import org.apache.paimon.types.VarCharType; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; +import org.apache.flink.api.common.serialization.SerializerConfigImpl; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -35,6 +37,8 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -49,7 +53,7 @@ public class CdcRecordSerializeITCase { @Test - public void testCdcRecordKryoSerialize() throws IOException { + public void testCdcRecordKryoSerialize() throws Exception { KryoSerializer kr = createFlinkKryoSerializer(RichCdcMultiplexRecord.class); RowType.Builder rowType = RowType.builder(); @@ -78,7 +82,7 @@ public void testCdcRecordKryoSerialize() throws IOException { } @Test - public void testUnmodifiableListKryoSerialize() throws IOException { + public void testUnmodifiableListKryoSerialize() throws Exception { KryoSerializer kryoSerializer = createFlinkKryoSerializer(List.class); RowType.Builder rowType = RowType.builder(); rowType.field("id", new BigIntType()); @@ -101,8 +105,24 @@ public void testUnmodifiableListKryoSerialize() throws IOException { assertThat(deserializeRecord).isEqualTo(fields); } - public static KryoSerializer createFlinkKryoSerializer(Class type) { - return new KryoSerializer<>(type, new ExecutionConfig()); + @SuppressWarnings({"unchecked", "rawtypes"}) + public static KryoSerializer createFlinkKryoSerializer(Class type) + throws NoSuchMethodException, InvocationTargetException, InstantiationException, + IllegalAccessException { + try { + Constructor constructor = + KryoSerializer.class.getConstructor(Class.class, SerializerConfig.class); + return (KryoSerializer) constructor.newInstance(type, new SerializerConfigImpl()); + } catch (NoSuchMethodException + | InvocationTargetException + | IllegalAccessException + | InstantiationException e) { + // to stay compatible with Flink 1.18- + } + + Constructor constructor = + KryoSerializer.class.getConstructor(Class.class, ExecutionConfig.class); + return (KryoSerializer) constructor.newInstance(type, new ExecutionConfig()); } private static final class TestOutputView extends DataOutputStream implements DataOutputView { 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 8c78ab853a60a..9f35b25026bbb 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 @@ -689,8 +689,8 @@ public void testUsingTheSameCompactExecutor() throws Exception { private OneInputStreamOperatorTestHarness createTestHarness(Catalog.Loader catalogLoader) throws Exception { - CdcRecordStoreMultiWriteOperator operator = - new CdcRecordStoreMultiWriteOperator( + CdcRecordStoreMultiWriteOperator.Factory operatorFactory = + new CdcRecordStoreMultiWriteOperator.Factory( catalogLoader, (t, commitUser, state, ioManager, memoryPoolFactory, metricGroup) -> new StoreSinkWriteImpl( @@ -709,7 +709,7 @@ public void testUsingTheSameCompactExecutor() throws Exception { TypeSerializer outputSerializer = new MultiTableCommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator, inputSerializer); + new OneInputStreamOperatorTestHarness<>(operatorFactory, inputSerializer); harness.setup(outputSerializer); return harness; } 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 f3693fe405ded..f00229d99890e 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 @@ -253,8 +253,8 @@ public void testUpdateColumnType() throws Exception { private OneInputStreamOperatorTestHarness createTestHarness( FileStoreTable table) throws Exception { - CdcRecordStoreWriteOperator operator = - new CdcRecordStoreWriteOperator( + CdcRecordStoreWriteOperator.Factory operatorFactory = + new CdcRecordStoreWriteOperator.Factory( table, (t, commitUser, state, ioManager, memoryPool, metricGroup) -> new StoreSinkWriteImpl( @@ -272,7 +272,7 @@ private OneInputStreamOperatorTestHarness createTestHarn TypeSerializer outputSerializer = new CommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator, inputSerializer); + new OneInputStreamOperatorTestHarness<>(operatorFactory, inputSerializer); harness.setup(outputSerializer); return harness; } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java index fd23e500d5e55..ab81e37c7d04b 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSinkTest.java @@ -22,11 +22,10 @@ import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.options.Options; +import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; -import org.apache.flink.streaming.api.transformations.LegacySinkTransformation; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.junit.jupiter.api.Test; @@ -45,14 +44,7 @@ public void testTransformationParallelism() { env.setParallelism(8); int inputParallelism = ThreadLocalRandom.current().nextInt(8) + 1; DataStreamSource input = - env.addSource( - new ParallelSourceFunction() { - @Override - public void run(SourceContext ctx) {} - - @Override - public void cancel() {} - }) + env.fromData(CdcMultiplexRecord.class, new CdcMultiplexRecord("", "", null)) .setParallelism(inputParallelism); FlinkCdcMultiTableSink sink = @@ -60,13 +52,11 @@ public void cancel() {} () -> FlinkCatalogFactory.createPaimonCatalog(new Options()), FlinkConnectorOptions.SINK_COMMITTER_CPU.defaultValue(), null, - true, UUID.randomUUID().toString()); DataStreamSink dataStreamSink = sink.sinkFrom(input); // check the transformation graph - LegacySinkTransformation end = - (LegacySinkTransformation) dataStreamSink.getTransformation(); + Transformation end = dataStreamSink.getTransformation(); assertThat(end.getName()).isEqualTo("end"); OneInputTransformation committer = diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java index a7c6b2cb63238..28b137a93ed9c 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java @@ -42,6 +42,7 @@ import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.Test; @@ -154,8 +155,9 @@ private void innerTestRandomCdcEvents(Supplier bucket, boolean unawareB .allowRestart(enableFailure) .build(); - TestCdcSourceFunction sourceFunction = new TestCdcSourceFunction(events); - DataStreamSource source = env.addSource(sourceFunction); + TestCdcSource testCdcSource = new TestCdcSource(events); + DataStreamSource source = + env.fromSource(testCdcSource, WatermarkStrategy.noWatermarks(), "TestCdcSource"); source.setParallelism(2); Options catalogOptions = new Options(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java index 081bd7d073d74..8b19391f3edab 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java @@ -43,6 +43,7 @@ import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.Disabled; @@ -151,8 +152,9 @@ private void innerTestRandomCdcEvents( .allowRestart(enableFailure) .build(); - TestCdcSourceFunction sourceFunction = new TestCdcSourceFunction(testTable.events()); - DataStreamSource source = env.addSource(sourceFunction); + TestCdcSource testCdcSource = new TestCdcSource(testTable.events()); + DataStreamSource source = + env.fromSource(testCdcSource, WatermarkStrategy.noWatermarks(), "TestCdcSource"); source.setParallelism(2); Options catalogOptions = new Options(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java new file mode 100644 index 0000000000000..b45983000a232 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSource.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.cdc; + +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.SplitListState; + +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; + +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Testing parallel {@link org.apache.flink.api.connector.source.Source} to produce {@link + * TestCdcEvent}. {@link TestCdcEvent}s with the same key will be produced by the same parallelism. + */ +public class TestCdcSource extends AbstractNonCoordinatedSource { + + private static final long serialVersionUID = 1L; + private final LinkedList events; + + public TestCdcSource(Collection events) { + this.events = new LinkedList<>(events); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext context) { + return new Reader( + context.getIndexOfSubtask(), + context.currentParallelism(), + new LinkedList<>(events)); + } + + private static class Reader extends AbstractNonCoordinatedSourceReader { + private final int subtaskId; + private final int totalSubtasks; + + private final LinkedList events; + private final SplitListState remainingEventsCount = + new SplitListState<>("events", x -> Integer.toString(x), Integer::parseInt); + + private final int numRecordsPerCheckpoint; + private final AtomicInteger recordsThisCheckpoint; + + private Reader(int subtaskId, int totalSubtasks, LinkedList events) { + this.subtaskId = subtaskId; + this.totalSubtasks = totalSubtasks; + this.events = events; + numRecordsPerCheckpoint = + events.size() / ThreadLocalRandom.current().nextInt(10, 20) + 1; + recordsThisCheckpoint = new AtomicInteger(0); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + if (events.isEmpty()) { + return InputStatus.END_OF_INPUT; + } + + if (recordsThisCheckpoint.get() >= numRecordsPerCheckpoint) { + Thread.sleep(10); + return InputStatus.MORE_AVAILABLE; + } + + TestCdcEvent event = events.poll(); + if (event.records() != null) { + if (Math.abs(event.hashCode()) % totalSubtasks != subtaskId) { + return InputStatus.MORE_AVAILABLE; + } + } + readerOutput.collect(event); + recordsThisCheckpoint.incrementAndGet(); + return InputStatus.MORE_AVAILABLE; + } + + @Override + public List snapshotState(long l) { + recordsThisCheckpoint.set(0); + remainingEventsCount.clear(); + remainingEventsCount.add(events.size()); + return remainingEventsCount.snapshotState(); + } + + @Override + public void addSplits(List list) { + remainingEventsCount.restoreState(list); + int count = 0; + for (int c : remainingEventsCount.get()) { + count += c; + } + while (events.size() > count) { + events.poll(); + } + } + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java deleted file mode 100644 index 4e03256a52532..0000000000000 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/TestCdcSourceFunction.java +++ /dev/null @@ -1,107 +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.sink.cdc; - -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; - -import java.util.Collection; -import java.util.LinkedList; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * Testing {@link RichParallelSourceFunction} to produce {@link TestCdcEvent}. {@link TestCdcEvent}s - * with the same key will be produced by the same parallelism. - */ -public class TestCdcSourceFunction extends RichParallelSourceFunction - implements CheckpointedFunction { - - private static final long serialVersionUID = 1L; - - private final LinkedList events; - - private volatile boolean isRunning = true; - private transient int numRecordsPerCheckpoint; - private transient AtomicInteger recordsThisCheckpoint; - private transient ListState remainingEventsCount; - - public TestCdcSourceFunction(Collection events) { - this.events = new LinkedList<>(events); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - numRecordsPerCheckpoint = events.size() / ThreadLocalRandom.current().nextInt(10, 20) + 1; - recordsThisCheckpoint = new AtomicInteger(0); - - remainingEventsCount = - context.getOperatorStateStore() - .getListState(new ListStateDescriptor<>("count", Integer.class)); - - if (context.isRestored()) { - int count = 0; - for (int c : remainingEventsCount.get()) { - count += c; - } - while (events.size() > count) { - events.poll(); - } - } - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - recordsThisCheckpoint.set(0); - remainingEventsCount.clear(); - remainingEventsCount.add(events.size()); - } - - @Override - public void run(SourceContext ctx) throws Exception { - while (isRunning && !events.isEmpty()) { - if (recordsThisCheckpoint.get() >= numRecordsPerCheckpoint) { - Thread.sleep(10); - continue; - } - - synchronized (ctx.getCheckpointLock()) { - TestCdcEvent event = events.poll(); - if (event.records() != null) { - int subtaskId = getRuntimeContext().getIndexOfThisSubtask(); - int totalSubtasks = getRuntimeContext().getNumberOfParallelSubtasks(); - if (Math.abs(event.hashCode()) % totalSubtasks != subtaskId) { - continue; - } - } - ctx.collect(event); - recordsThisCheckpoint.incrementAndGet(); - } - } - } - - @Override - public void cancel() { - isRunning = false; - } -} diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aliyun/table/event/event-delete.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aliyun/table/event/event-delete.txt new file mode 100644 index 0000000000000..ebae6608a7556 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aliyun/table/event/event-delete.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":{"dataColumn":[{"name":"id","type":"LONG"},{"name":"val","type":"DOUBLE"},{"name":"name","type":"STRING"},{"name":"create_time","type":"DATE"}],"primaryKey":["id"],"source":{"dbType":"MySQL","dbName":"bigdata_test","tableName":"sync_test_table"}},"payload":{"before":{"dataColumn":{"id":1,"val":"1.100000","name":"a","create_time":1731661114000}},"after":null,"sequenceId":"1731663842292000000","timestamp":{"eventTime":1731662085000,"systemTime":1731663848953,"checkpointTime":1731662085000},"op":"DELETE","ddl":null},"version":"0.0.1"} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aliyun/table/event/event-insert.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aliyun/table/event/event-insert.txt new file mode 100644 index 0000000000000..d1cd34e5e6ac2 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aliyun/table/event/event-insert.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. + */ + +{"payload":{"after":{"dataColumn":{"create_time":1731661114000,"id":2,"name":"a","val":"1.100000"}},"before":null,"ddl":null,"op":"INSERT","sequenceId":"-1","timestamp":{"checkpointTime":-1,"eventTime":-1,"systemTime":1731661820245}},"schema":{"dataColumn":[{"name":"id","type":"LONG"},{"name":"val","type":"DOUBLE"},{"name":"name","type":"STRING"},{"name":"create_time","type":"DATE"}],"primaryKey":["id"],"source":{"dbName":"bigdata_test","dbType":"MySQL","tableName":"sync_test_table"}},"version":"0.0.1"} \ No newline at end of file diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aliyun/table/event/event-update-in-one.txt b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aliyun/table/event/event-update-in-one.txt new file mode 100644 index 0000000000000..9acf6309cc481 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/kafka/aliyun/table/event/event-update-in-one.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":{"dataColumn":[{"name":"id","type":"LONG"},{"name":"val","type":"DOUBLE"},{"name":"name","type":"STRING"},{"name":"create_time","type":"DATE"}],"primaryKey":["id"],"source":{"dbType":"MySQL","dbName":"bigdata_test","tableName":"sync_test_table"}},"payload":{"before":{"dataColumn":{"id":2,"val":"1.100000","name":"a","create_time":1731661114000}},"after":{"dataColumn":{"id":2,"val":"2.200000","name":"a","create_time":1731661114000}},"sequenceId":"1731663842292000001","timestamp":{"eventTime":1731662097000,"systemTime":1731663848979,"checkpointTime":1731662097000},"op":"UPDATE_AFTER","ddl":null},"version":"0.0.1"} \ 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 4452af266e5e6..91222983bf6ba 100644 --- a/paimon-flink/paimon-flink-common/pom.xml +++ b/paimon-flink/paimon-flink-common/pom.xml @@ -162,6 +162,12 @@ under the License. iceberg-data ${iceberg.version} test + + + parquet-avro + org.apache.parquet + + 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 9f90a2cd01304..6b10dbb84bf46 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 @@ -25,15 +25,10 @@ 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.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; @@ -47,7 +42,6 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ExecutionOptions; -import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.ValidationException; @@ -71,7 +65,6 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.function.BiConsumer; import java.util.regex.Pattern; import static org.apache.paimon.CoreOptions.LOG_CHANGELOG_MODE; @@ -109,23 +102,9 @@ public DynamicTableSource createDynamicTableSource(Context context) { isStreamingMode, context.getObjectIdentifier()); } else { - Table table = buildPaimonTable(context); - if (table instanceof FileStoreTable) { - storeTableLineage( - ((FileStoreTable) table).catalogEnvironment().lineageMetaFactory(), - context, - (entity, lineageFactory) -> { - try (LineageMeta lineage = - lineageFactory.create(() -> Options.fromMap(table.options()))) { - lineage.saveSourceTableLineage(entity); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); - } return new DataTableSource( context.getObjectIdentifier(), - table, + buildPaimonTable(context), isStreamingMode, context, createOptionalLogStoreFactory(context).orElse(null)); @@ -134,46 +113,13 @@ public DynamicTableSource createDynamicTableSource(Context context) { @Override public DynamicTableSink createDynamicTableSink(Context context) { - Table table = buildPaimonTable(context); - if (table instanceof FileStoreTable) { - storeTableLineage( - ((FileStoreTable) table).catalogEnvironment().lineageMetaFactory(), - context, - (entity, lineageFactory) -> { - try (LineageMeta lineage = - lineageFactory.create(() -> Options.fromMap(table.options()))) { - lineage.saveSinkTableLineage(entity); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); - } return new FlinkTableSink( context.getObjectIdentifier(), - table, + buildPaimonTable(context), context, createOptionalLogStoreFactory(context).orElse(null)); } - private void storeTableLineage( - @Nullable LineageMetaFactory lineageMetaFactory, - Context context, - BiConsumer tableLineage) { - if (lineageMetaFactory != null) { - String pipelineName = context.getConfiguration().get(PipelineOptions.NAME); - if (pipelineName == null) { - throw new ValidationException("Cannot get pipeline name for lineage meta."); - } - tableLineage.accept( - new TableLineageEntityImpl( - context.getObjectIdentifier().getDatabaseName(), - context.getObjectIdentifier().getObjectName(), - pipelineName, - Timestamp.fromEpochMillis(System.currentTimeMillis())), - lineageMetaFactory); - } - } - @Override public Set> requiredOptions() { return Collections.emptySet(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java index 019d7bd6892fe..e141581b476ba 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/DataCatalogTable.java @@ -23,33 +23,55 @@ import org.apache.paimon.types.DataField; import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.constraints.UniqueConstraint; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.CatalogTableImpl; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.stream.Collectors; -/** A {@link CatalogTableImpl} to wrap {@link FileStoreTable}. */ -public class DataCatalogTable extends CatalogTableImpl { +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** A {@link CatalogTable} to wrap {@link FileStoreTable}. */ +public class DataCatalogTable implements CatalogTable { + // Schema of the table (column names and types) + private final Schema schema; + + // Partition keys if this is a partitioned table. It's an empty set if the table is not + // partitioned + private final List partitionKeys; + + // Properties of the table + private final Map options; + + // Comment of the table + private final String comment; private final Table table; private final Map nonPhysicalColumnComments; public DataCatalogTable( Table table, - TableSchema tableSchema, + Schema resolvedSchema, List partitionKeys, - Map properties, + Map options, String comment, Map nonPhysicalColumnComments) { - super(tableSchema, partitionKeys, properties, comment); + this.schema = resolvedSchema; + this.partitionKeys = checkNotNull(partitionKeys, "partitionKeys cannot be null"); + this.options = checkNotNull(options, "options cannot be null"); + + checkArgument( + options.entrySet().stream() + .allMatch(e -> e.getKey() != null && e.getValue() != null), + "properties cannot have null keys or values"); + + this.comment = comment; + this.table = table; this.nonPhysicalColumnComments = nonPhysicalColumnComments; } @@ -66,32 +88,30 @@ public Schema getUnresolvedSchema() { .filter(dataField -> dataField.description() != null) .collect(Collectors.toMap(DataField::name, DataField::description)); - return toSchema(getSchema(), columnComments); + return toSchema(schema, columnComments); } - /** Copied from {@link TableSchema#toSchema(Map)} to support versions lower than 1.17. */ - private Schema toSchema(TableSchema tableSchema, Map comments) { + private Schema toSchema(Schema tableSchema, Map comments) { final Schema.Builder builder = Schema.newBuilder(); - tableSchema - .getTableColumns() + .getColumns() .forEach( column -> { - if (column instanceof TableColumn.PhysicalColumn) { - final TableColumn.PhysicalColumn c = - (TableColumn.PhysicalColumn) column; - builder.column(c.getName(), c.getType()); - } else if (column instanceof TableColumn.MetadataColumn) { - final TableColumn.MetadataColumn c = - (TableColumn.MetadataColumn) column; + if (column instanceof Schema.UnresolvedPhysicalColumn) { + final Schema.UnresolvedPhysicalColumn c = + (Schema.UnresolvedPhysicalColumn) column; + builder.column(c.getName(), c.getDataType()); + } else if (column instanceof Schema.UnresolvedMetadataColumn) { + final Schema.UnresolvedMetadataColumn c = + (Schema.UnresolvedMetadataColumn) column; builder.columnByMetadata( c.getName(), - c.getType(), - c.getMetadataAlias().orElse(null), + c.getDataType(), + c.getMetadataKey(), c.isVirtual()); - } else if (column instanceof TableColumn.ComputedColumn) { - final TableColumn.ComputedColumn c = - (TableColumn.ComputedColumn) column; + } else if (column instanceof Schema.UnresolvedComputedColumn) { + final Schema.UnresolvedComputedColumn c = + (Schema.UnresolvedComputedColumn) column; builder.columnByExpression(c.getName(), c.getExpression()); } else { throw new IllegalArgumentException( @@ -104,19 +124,16 @@ private Schema toSchema(TableSchema tableSchema, Map comments) { builder.withComment(nonPhysicalColumnComments.get(colName)); } }); - tableSchema .getWatermarkSpecs() .forEach( spec -> builder.watermark( - spec.getRowtimeAttribute(), spec.getWatermarkExpr())); - + spec.getColumnName(), spec.getWatermarkExpression())); if (tableSchema.getPrimaryKey().isPresent()) { - UniqueConstraint primaryKey = tableSchema.getPrimaryKey().get(); - builder.primaryKeyNamed(primaryKey.getName(), primaryKey.getColumns()); + Schema.UnresolvedPrimaryKey primaryKey = tableSchema.getPrimaryKey().get(); + builder.primaryKeyNamed(primaryKey.getConstraintName(), primaryKey.getColumnNames()); } - return builder.build(); } @@ -124,7 +141,7 @@ private Schema toSchema(TableSchema tableSchema, Map comments) { public CatalogBaseTable copy() { return new DataCatalogTable( table, - getSchema().copy(), + schema, new ArrayList<>(getPartitionKeys()), new HashMap<>(getOptions()), getComment(), @@ -135,10 +152,40 @@ public CatalogBaseTable copy() { public CatalogTable copy(Map options) { return new DataCatalogTable( table, - getSchema(), + schema, getPartitionKeys(), options, getComment(), nonPhysicalColumnComments); } + + @Override + public Optional getDescription() { + return Optional.of(getComment()); + } + + @Override + public Optional getDetailedDescription() { + return Optional.of("This is a catalog table in an im-memory catalog"); + } + + @Override + public boolean isPartitioned() { + return !partitionKeys.isEmpty(); + } + + @Override + public List getPartitionKeys() { + return partitionKeys; + } + + @Override + public Map getOptions() { + return options; + } + + @Override + public String getComment() { + return comment != null ? comment : ""; + } } 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 cae6e6f0e3672..3a7f9790ccca1 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 @@ -24,6 +24,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.procedure.ProcedureUtil; import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; +import org.apache.paimon.flink.utils.FlinkDescriptorProperties; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.operation.FileStoreCommit; @@ -37,6 +38,8 @@ import org.apache.paimon.table.Table; import org.apache.paimon.table.sink.BatchWriteBuilder; import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypeRoot; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.InternalRowPartitionComputer; import org.apache.paimon.utils.Preconditions; @@ -44,7 +47,6 @@ 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; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogDatabase; @@ -94,11 +96,9 @@ import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; import org.apache.flink.table.catalog.stats.CatalogTableStatistics; -import org.apache.flink.table.descriptors.DescriptorProperties; import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.factories.Factory; import org.apache.flink.table.procedures.Procedure; -import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -110,21 +110,16 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; -import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; -import static org.apache.flink.table.descriptors.DescriptorProperties.NAME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -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; @@ -149,11 +144,18 @@ import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; import static org.apache.paimon.flink.log.LogStoreRegister.registerLogSystem; import static org.apache.paimon.flink.log.LogStoreRegister.unRegisterLogSystem; +import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn; 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.FlinkDescriptorProperties.COMMENT; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.NAME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_EXPR; 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; @@ -595,17 +597,12 @@ private List toSchemaChange( if (!oldTableNonPhysicalColumnIndex.containsKey( ((ModifyPhysicalColumnType) change).getOldColumn().getName())) { ModifyPhysicalColumnType modify = (ModifyPhysicalColumnType) change; - LogicalType newColumnType = modify.getNewType().getLogicalType(); - LogicalType oldColumnType = modify.getOldColumn().getDataType().getLogicalType(); - if (newColumnType.isNullable() != oldColumnType.isNullable()) { - schemaChanges.add( - SchemaChange.updateColumnNullability( - modify.getNewColumn().getName(), newColumnType.isNullable())); - } - schemaChanges.add( - SchemaChange.updateColumnType( - modify.getOldColumn().getName(), - LogicalTypeConversion.toDataType(newColumnType))); + generateNestedColumnUpdates( + Collections.singletonList(modify.getOldColumn().getName()), + LogicalTypeConversion.toDataType( + modify.getOldColumn().getDataType().getLogicalType()), + LogicalTypeConversion.toDataType(modify.getNewType().getLogicalType()), + schemaChanges); } return schemaChanges; } else if (change instanceof ModifyColumnPosition) { @@ -670,6 +667,139 @@ && handleMaterializedTableChange(change, schemaChanges)) { throw new UnsupportedOperationException("Change is not supported: " + change.getClass()); } + private void generateNestedColumnUpdates( + List fieldNames, + org.apache.paimon.types.DataType oldType, + org.apache.paimon.types.DataType newType, + List schemaChanges) { + String joinedNames = String.join(".", fieldNames); + if (oldType.getTypeRoot() == DataTypeRoot.ROW) { + Preconditions.checkArgument( + newType.getTypeRoot() == DataTypeRoot.ROW, + "Column %s can only be updated to row type, and cannot be updated to %s type", + joinedNames, + newType.getTypeRoot()); + org.apache.paimon.types.RowType oldRowType = (org.apache.paimon.types.RowType) oldType; + org.apache.paimon.types.RowType newRowType = (org.apache.paimon.types.RowType) newType; + + // check that existing fields have same order + Map oldFieldOrders = new HashMap<>(); + for (int i = 0; i < oldRowType.getFieldCount(); i++) { + oldFieldOrders.put(oldRowType.getFields().get(i).name(), i); + } + int lastIdx = -1; + String lastFieldName = ""; + for (DataField newField : newRowType.getFields()) { + String name = newField.name(); + if (oldFieldOrders.containsKey(name)) { + int idx = oldFieldOrders.get(name); + Preconditions.checkState( + lastIdx < idx, + "Order of existing fields in column %s must be kept the same. " + + "However, field %s and %s have changed their orders.", + joinedNames, + lastFieldName, + name); + lastIdx = idx; + lastFieldName = name; + } + } + + // drop fields + Set newFieldNames = new HashSet<>(newRowType.getFieldNames()); + for (String name : oldRowType.getFieldNames()) { + if (!newFieldNames.contains(name)) { + List dropColumnNames = new ArrayList<>(fieldNames); + dropColumnNames.add(name); + schemaChanges.add( + SchemaChange.dropColumn(dropColumnNames.toArray(new String[0]))); + } + } + + for (int i = 0; i < newRowType.getFieldCount(); i++) { + DataField field = newRowType.getFields().get(i); + String name = field.name(); + List fullFieldNames = new ArrayList<>(fieldNames); + fullFieldNames.add(name); + if (!oldFieldOrders.containsKey(name)) { + // add fields + SchemaChange.Move move; + if (i == 0) { + move = SchemaChange.Move.first(name); + } else { + String lastName = newRowType.getFields().get(i - 1).name(); + move = SchemaChange.Move.after(name, lastName); + } + schemaChanges.add( + SchemaChange.addColumn( + fullFieldNames.toArray(new String[0]), + field.type(), + field.description(), + move)); + } else { + // update existing fields + DataField oldField = oldRowType.getFields().get(oldFieldOrders.get(name)); + if (!Objects.equals(oldField.description(), field.description())) { + schemaChanges.add( + SchemaChange.updateColumnComment( + fullFieldNames.toArray(new String[0]), + field.description())); + } + generateNestedColumnUpdates( + fullFieldNames, oldField.type(), field.type(), schemaChanges); + } + } + } else if (oldType.getTypeRoot() == DataTypeRoot.ARRAY) { + Preconditions.checkArgument( + newType.getTypeRoot() == DataTypeRoot.ARRAY, + "Column %s can only be updated to array type, and cannot be updated to %s type", + joinedNames, + newType); + List fullFieldNames = new ArrayList<>(fieldNames); + // add a dummy column name indicating the element of array + fullFieldNames.add("element"); + generateNestedColumnUpdates( + fullFieldNames, + ((org.apache.paimon.types.ArrayType) oldType).getElementType(), + ((org.apache.paimon.types.ArrayType) newType).getElementType(), + schemaChanges); + } else if (oldType.getTypeRoot() == DataTypeRoot.MAP) { + Preconditions.checkArgument( + newType.getTypeRoot() == DataTypeRoot.MAP, + "Column %s can only be updated to map type, and cannot be updated to %s type", + joinedNames, + newType); + org.apache.paimon.types.MapType oldMapType = (org.apache.paimon.types.MapType) oldType; + org.apache.paimon.types.MapType newMapType = (org.apache.paimon.types.MapType) newType; + Preconditions.checkArgument( + oldMapType.getKeyType().equals(newMapType.getKeyType()), + "Cannot update key type of column %s from %s type to %s type", + joinedNames, + oldMapType.getKeyType(), + newMapType.getKeyType()); + List fullFieldNames = new ArrayList<>(fieldNames); + // add a dummy column name indicating the value of map + fullFieldNames.add("value"); + generateNestedColumnUpdates( + fullFieldNames, + oldMapType.getValueType(), + newMapType.getValueType(), + schemaChanges); + } else { + if (!oldType.equalsIgnoreNullable(newType)) { + schemaChanges.add( + SchemaChange.updateColumnType( + fieldNames.toArray(new String[0]), newType, false)); + } + } + + if (oldType.isNullable() != newType.isNullable()) { + schemaChanges.add( + SchemaChange.updateColumnNullability( + fieldNames.toArray(new String[0]), newType.isNullable())); + } + } + /** * Try handle change related to materialized table. * @@ -877,18 +1007,18 @@ private static void validateAlterTable(CatalogBaseTable ct1, CatalogBaseTable ct } // 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(); + org.apache.flink.table.api.Schema ts1 = ct1.getUnresolvedSchema(); + org.apache.flink.table.api.Schema ts2 = ct2.getUnresolvedSchema(); boolean pkEquality = false; if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { pkEquality = Objects.equals( - ts1.getPrimaryKey().get().getType(), - ts2.getPrimaryKey().get().getType()) + ts1.getPrimaryKey().get().getConstraintName(), + ts2.getPrimaryKey().get().getConstraintName()) && Objects.equals( - ts1.getPrimaryKey().get().getColumns(), - ts2.getPrimaryKey().get().getColumns()); + ts1.getPrimaryKey().get().getColumnNames(), + ts2.getPrimaryKey().get().getColumnNames()); } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { pkEquality = true; } @@ -932,7 +1062,8 @@ public final void close() throws CatalogException { private CatalogBaseTable toCatalogTable(Table table) { Map newOptions = new HashMap<>(table.options()); - TableSchema.Builder builder = TableSchema.builder(); + org.apache.flink.table.api.Schema.Builder builder = + org.apache.flink.table.api.Schema.newBuilder(); Map nonPhysicalColumnComments = new HashMap<>(); // add columns @@ -947,10 +1078,10 @@ private CatalogBaseTable toCatalogTable(Table table) { if (optionalName == null || physicalColumns.contains(optionalName)) { // build physical column from table row field RowType.RowField field = physicalRowFields.get(physicalColumnIndex++); - builder.field(field.getName(), fromLogicalToDataType(field.getType())); + builder.column(field.getName(), fromLogicalToDataType(field.getType())); } else { // build non-physical column from options - builder.add(deserializeNonPhysicalColumn(newOptions, i)); + deserializeNonPhysicalColumn(newOptions, i, builder); if (newOptions.containsKey(compoundKey(SCHEMA, i, COMMENT))) { nonPhysicalColumnComments.put( optionalName, newOptions.get(compoundKey(SCHEMA, i, COMMENT))); @@ -962,22 +1093,18 @@ private CatalogBaseTable toCatalogTable(Table table) { // extract watermark information if (newOptions.keySet().stream() .anyMatch(key -> key.startsWith(compoundKey(SCHEMA, WATERMARK)))) { - builder.watermark(deserializeWatermarkSpec(newOptions)); + deserializeWatermarkSpec(newOptions, builder); } // add primary keys if (table.primaryKeys().size() > 0) { - builder.primaryKey( - table.primaryKeys().stream().collect(Collectors.joining("_", "PK_", "")), - table.primaryKeys().toArray(new String[0])); + builder.primaryKey(table.primaryKeys()); } - TableSchema schema = builder.build(); + org.apache.flink.table.api.Schema schema = builder.build(); // remove schema from options - DescriptorProperties removeProperties = new DescriptorProperties(false); - removeProperties.putTableSchema(SCHEMA, schema); - removeProperties.asMap().keySet().forEach(newOptions::remove); + FlinkDescriptorProperties.removeSchemaKeys(SCHEMA, schema, newOptions); Options options = Options.fromMap(newOptions); if (TableType.MATERIALIZED_TABLE == options.get(CoreOptions.TYPE)) { @@ -993,7 +1120,10 @@ private CatalogBaseTable toCatalogTable(Table table) { } private CatalogMaterializedTable buildMaterializedTable( - Table table, Map newOptions, TableSchema schema, Options options) { + Table table, + Map newOptions, + org.apache.flink.table.api.Schema schema, + Options options) { String definitionQuery = options.get(MATERIALIZED_TABLE_DEFINITION_QUERY); IntervalFreshness freshness = IntervalFreshness.of( @@ -1017,7 +1147,7 @@ private CatalogMaterializedTable buildMaterializedTable( // remove materialized table related options allMaterializedTableAttributes().forEach(newOptions::remove); return CatalogMaterializedTable.newBuilder() - .schema(schema.toSchema()) + .schema(schema) .comment(table.comment().orElse("")) .partitionKeys(table.partitionKeys()) .options(newOptions) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java index 74512409bfc80..617d25125f371 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkFileIO.java @@ -29,10 +29,10 @@ import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.FileSystem.WriteMode; -import org.apache.flink.core.fs.FileSystemKind; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Locale; /** Flink {@link FileIO} to use {@link FileSystem}. */ public class FlinkFileIO implements FileIO { @@ -48,7 +48,27 @@ public FlinkFileIO(Path path) { @Override public boolean isObjectStore() { try { - return path.getFileSystem().getKind() != FileSystemKind.FILE_SYSTEM; + FileSystem fs = path.getFileSystem(); + String scheme = fs.getUri().getScheme().toLowerCase(Locale.US); + + if (scheme.startsWith("s3") + || scheme.startsWith("emr") + || scheme.startsWith("oss") + || scheme.startsWith("wasb") + || scheme.startsWith("gs")) { + // the Amazon S3 storage or Aliyun OSS storage or Azure Blob Storage + // or Google Cloud Storage + return true; + } else if (scheme.startsWith("http") || scheme.startsWith("ftp")) { + // file servers instead of file systems + // they might actually be consistent, but we have no hard guarantees + // currently to rely on that + return true; + } else { + // the remainder should include hdfs, kosmos, ceph, ... + // this also includes federated HDFS (viewfs). + return false; + } } catch (IOException e) { throw new UncheckedIOException(e); } 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 37bed2d0480f5..75af5917bb492 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 @@ -48,7 +48,6 @@ import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.factories.Factory; import org.apache.flink.table.factories.FunctionDefinitionFactory; -import org.apache.flink.table.factories.TableFactory; import org.apache.flink.table.procedures.Procedure; import java.util.List; @@ -86,11 +85,6 @@ public Optional getFactory() { new FlinkGenericTableFactory(paimon.getFactory().get(), flink.getFactory().get())); } - @Override - public Optional getTableFactory() { - return flink.getTableFactory(); - } - @Override public Optional getFunctionDefinitionFactory() { return flink.getFunctionDefinitionFactory(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java index d5d843d91bb10..f88a808713c24 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/SystemCatalogTable.java @@ -22,7 +22,6 @@ import org.apache.paimon.table.system.AuditLogTable; import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.WatermarkSpec; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.types.utils.TypeConversions; @@ -32,11 +31,11 @@ import java.util.Map; import java.util.Optional; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; import static org.apache.paimon.flink.LogicalTypeConversion.toLogicalType; +import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.deserializeWatermarkSpec; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; /** A {@link CatalogTable} to represent system table. */ public class SystemCatalogTable implements CatalogTable { @@ -60,11 +59,8 @@ public Schema getUnresolvedSchema() { Map newOptions = new HashMap<>(table.options()); if (newOptions.keySet().stream() .anyMatch(key -> key.startsWith(compoundKey(SCHEMA, WATERMARK)))) { - WatermarkSpec watermarkSpec = deserializeWatermarkSpec(newOptions); - return builder.watermark( - watermarkSpec.getRowtimeAttribute(), - watermarkSpec.getWatermarkExpr()) - .build(); + deserializeWatermarkSpec(newOptions, builder); + return builder.build(); } } return builder.build(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java index 43719f715d9de..fbf8f12f49eb6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionFactory.java @@ -58,6 +58,10 @@ public interface ActionFactory extends Factory { String TIMESTAMPFORMATTER = "timestamp_formatter"; String EXPIRE_STRATEGY = "expire_strategy"; String TIMESTAMP_PATTERN = "timestamp_pattern"; + // Supports `full` and `minor`. + String COMPACT_STRATEGY = "compact_strategy"; + String MINOR = "minor"; + String FULL = "full"; Optional create(MultipleParameterToolAdapter params); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java index 2f90147eeb2a3..bac030dd04966 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CloneAction.java @@ -32,7 +32,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import java.util.HashMap; import java.util.Map; @@ -141,7 +141,7 @@ copyFiles, new SnapshotHintChannelComputer(), parallelism) new SnapshotHintOperator(targetCatalogConfig)) .setParallelism(parallelism); - snapshotHintOperator.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + snapshotHintOperator.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } @Override 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 8ea120015609c..84e37a5b10f9c 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 @@ -59,6 +59,8 @@ public class CompactAction extends TableActionBase { @Nullable private Duration partitionIdleTime = null; + private Boolean fullCompaction; + public CompactAction(String warehouse, String database, String tableName) { this(warehouse, database, tableName, Collections.emptyMap(), Collections.emptyMap()); } @@ -100,6 +102,11 @@ public CompactAction withPartitionIdleTime(@Nullable Duration partitionIdleTime) return this; } + public CompactAction withFullCompaction(Boolean fullCompaction) { + this.fullCompaction = fullCompaction; + return this; + } + @Override public void build() throws Exception { ReadableConfig conf = env.getConfiguration(); @@ -124,6 +131,13 @@ public void build() throws Exception { private void buildForTraditionalCompaction( StreamExecutionEnvironment env, FileStoreTable table, boolean isStreaming) throws Exception { + if (fullCompaction == null) { + fullCompaction = !isStreaming; + } else { + Preconditions.checkArgument( + !(fullCompaction && isStreaming), + "The full compact strategy is only supported in batch mode. Please add -Dexecution.runtime-mode=BATCH."); + } if (isStreaming) { // for completely asynchronous compaction HashMap dynamicOptions = @@ -138,7 +152,7 @@ private void buildForTraditionalCompaction( } CompactorSourceBuilder sourceBuilder = new CompactorSourceBuilder(identifier.getFullName(), table); - CompactorSinkBuilder sinkBuilder = new CompactorSinkBuilder(table); + CompactorSinkBuilder sinkBuilder = new CompactorSinkBuilder(table, fullCompaction); sourceBuilder.withPartitionPredicate(getPredicate()); DataStreamSource source = diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java index f43c7a747c990..fc60a870eabe1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactActionFactory.java @@ -76,6 +76,10 @@ public Optional create(MultipleParameterToolAdapter params) { action.withPartitionIdleTime( TimeUtils.parseDuration(params.get(PARTITION_IDLE_TIME))); } + String compactStrategy = params.get(COMPACT_STRATEGY); + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } } if (params.has(PARTITION)) { @@ -88,6 +92,19 @@ public Optional create(MultipleParameterToolAdapter params) { return Optional.of(action); } + public static boolean checkCompactStrategy(String compactStrategy) { + if (compactStrategy != null) { + Preconditions.checkArgument( + compactStrategy.equalsIgnoreCase(MINOR) + || compactStrategy.equalsIgnoreCase(FULL), + String.format( + "The compact strategy only supports 'full' or 'minor', but '%s' is configured.", + compactStrategy)); + return true; + } + return false; + } + @Override public void printHelp() { System.out.println( @@ -101,7 +118,8 @@ public void printHelp() { + "[--order_strategy ]" + "[--table_conf =]" + "[--order_by ]" - + "[--partition_idle_time ]"); + + "[--partition_idle_time ]" + + "[--compact_strategy ]"); System.out.println( " compact --warehouse s3://path/to/warehouse --database " + "--table [--catalog_conf [--catalog_conf ...]]"); @@ -132,6 +150,10 @@ public void printHelp() { System.out.println( " compact --warehouse hdfs:///path/to/warehouse --database test_db --table test_table " + "--partition_idle_time 10s"); + System.out.println( + "--compact_strategy determines how to pick files to be merged, the default is determined by the runtime execution mode. " + + "`full` : Only supports batch mode. All files will be selected for merging." + + "`minor`: Pick the set of files that need to be merged based on specified conditions."); System.out.println( " compact --warehouse s3:///path/to/warehouse " + "--database test_db " 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 fda9ff695e1e3..124d3ca687761 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 @@ -72,6 +72,10 @@ public class CompactDatabaseAction extends ActionBase { @Nullable private Duration partitionIdleTime = null; + private Boolean fullCompaction; + + private boolean isStreaming; + public CompactDatabaseAction(String warehouse, Map catalogConfig) { super(warehouse, catalogConfig); } @@ -110,6 +114,11 @@ public CompactDatabaseAction withPartitionIdleTime(@Nullable Duration partitionI return this; } + public CompactDatabaseAction withFullCompaction(boolean fullCompaction) { + this.fullCompaction = fullCompaction; + return this; + } + private boolean shouldCompactionTable(String paimonFullTableName) { boolean shouldCompaction = includingPattern.matcher(paimonFullTableName).matches(); if (excludingPattern != null) { @@ -124,6 +133,12 @@ private boolean shouldCompactionTable(String paimonFullTableName) { @Override public void build() { + ReadableConfig conf = env.getConfiguration(); + isStreaming = conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; + + if (fullCompaction == null) { + fullCompaction = !isStreaming; + } if (databaseCompactMode == MultiTablesSinkMode.DIVIDED) { buildForDividedMode(); } else { @@ -170,24 +185,19 @@ private void buildForDividedMode() { !tableMap.isEmpty(), "no tables to be compacted. possible cause is that there are no tables detected after pattern matching"); - ReadableConfig conf = env.getConfiguration(); - boolean isStreaming = - conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; for (Map.Entry entry : tableMap.entrySet()) { FileStoreTable fileStoreTable = entry.getValue(); switch (fileStoreTable.bucketMode()) { case BUCKET_UNAWARE: { - buildForUnawareBucketCompaction( - env, entry.getKey(), fileStoreTable, isStreaming); + buildForUnawareBucketCompaction(env, entry.getKey(), fileStoreTable); break; } case HASH_FIXED: case HASH_DYNAMIC: default: { - buildForTraditionalCompaction( - env, entry.getKey(), fileStoreTable, isStreaming); + buildForTraditionalCompaction(env, entry.getKey(), fileStoreTable); } } } @@ -195,9 +205,6 @@ private void buildForDividedMode() { private void buildForCombinedMode() { - ReadableConfig conf = env.getConfiguration(); - boolean isStreaming = - conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.STREAMING; CombinedTableCompactorSourceBuilder sourceBuilder = new CombinedTableCompactorSourceBuilder( catalogLoader(), @@ -234,15 +241,17 @@ private void buildForCombinedMode() { .buildForUnawareBucketsTableSource(), parallelism); - new CombinedTableCompactorSink(catalogLoader(), tableOptions) + new CombinedTableCompactorSink(catalogLoader(), tableOptions, fullCompaction) .sinkFrom(awareBucketTableSource, unawareBucketTableSource); } private void buildForTraditionalCompaction( - StreamExecutionEnvironment env, - String fullName, - FileStoreTable table, - boolean isStreaming) { + StreamExecutionEnvironment env, String fullName, FileStoreTable table) { + + Preconditions.checkArgument( + !(fullCompaction && isStreaming), + "The full compact strategy is only supported in batch mode. Please add -Dexecution.runtime-mode=BATCH."); + if (isStreaming) { // for completely asynchronous compaction HashMap dynamicOptions = @@ -259,7 +268,7 @@ private void buildForTraditionalCompaction( CompactorSourceBuilder sourceBuilder = new CompactorSourceBuilder(fullName, table) .withPartitionIdleTime(partitionIdleTime); - CompactorSinkBuilder sinkBuilder = new CompactorSinkBuilder(table); + CompactorSinkBuilder sinkBuilder = new CompactorSinkBuilder(table, fullCompaction); DataStreamSource source = sourceBuilder.withEnv(env).withContinuousMode(isStreaming).build(); @@ -267,10 +276,7 @@ private void buildForTraditionalCompaction( } private void buildForUnawareBucketCompaction( - StreamExecutionEnvironment env, - String fullName, - FileStoreTable table, - boolean isStreaming) { + StreamExecutionEnvironment env, String fullName, FileStoreTable table) { UnawareBucketCompactionTopoBuilder unawareBucketCompactionTopoBuilder = new UnawareBucketCompactionTopoBuilder(env, fullName, table); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java index b268709078093..5672f99dc30f3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/CompactDatabaseActionFactory.java @@ -22,6 +22,8 @@ import java.util.Optional; +import static org.apache.paimon.flink.action.CompactActionFactory.checkCompactStrategy; + /** Factory to create {@link CompactDatabaseAction}. */ public class CompactDatabaseActionFactory implements ActionFactory { @@ -55,6 +57,11 @@ public Optional create(MultipleParameterToolAdapter params) { action.withPartitionIdleTime(TimeUtils.parseDuration(partitionIdleTime)); } + String compactStrategy = params.get(COMPACT_STRATEGY); + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } + return Optional.of(action); } @@ -70,7 +77,8 @@ public void printHelp() { + "[--including_tables ] " + "[--excluding_tables ] " + "[--mode ]" - + "[--partition_idle_time ]"); + + "[--partition_idle_time ]" + + "[--compact_strategy ]"); System.out.println( " compact_database --warehouse s3://path/to/warehouse --including_databases " + "[--catalog_conf [--catalog_conf ...]]"); @@ -93,6 +101,11 @@ public void printHelp() { System.out.println( "--partition_idle_time is used to do a full compaction for partition which had not receive any new data for 'partition_idle_time' time. And only these partitions will be compacted."); System.out.println("--partition_idle_time is only supported in batch mode. "); + System.out.println( + "--compact_strategy determines how to pick files to be merged, the default is determined by the runtime execution mode. " + + "`full` : Only supports batch mode. All files will be selected for merging." + + "`minor`: Pick the set of files that need to be merged based on specified conditions."); + System.out.println(); System.out.println("Examples:"); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsAction.java index 9528bc137d6f2..0fa17e1a8ddbe 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ExpirePartitionsAction.java @@ -72,7 +72,8 @@ public ExpirePartitionsAction( .catalogEnvironment() .metastoreClientFactory()) .map(MetastoreClient.Factory::create) - .orElse(null)); + .orElse(null), + fileStore.options().partitionExpireMaxNum()); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneSourceBuilder.java index a0f4ef33dee28..585c73cb952cc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneSourceBuilder.java @@ -34,6 +34,7 @@ import java.util.Map; import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.utils.Preconditions.checkState; /** * Pick the tables to be cloned based on the user input parameters. The record type of the build @@ -114,6 +115,8 @@ private DataStream> build(Catalog sourceCatalog) throws E database + "." + tableName, targetDatabase + "." + targetTableName)); } + checkState(!result.isEmpty(), "Didn't find any table in source catalog."); + if (LOG.isDebugEnabled()) { LOG.debug("The clone identifiers of source table and target table are: {}", result); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java index 883d7b06ab5f8..67eecbc6f2aea 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java @@ -18,15 +18,21 @@ package org.apache.paimon.flink.clone; +import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.utils.Preconditions; +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; +import org.apache.paimon.shade.guava30.com.google.common.collect.Iterables; + import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; @@ -37,6 +43,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; /** * Pick the files to be cloned of a table based on the input record. The record type it produce is @@ -77,7 +84,7 @@ public void processElement(StreamRecord> streamRecord) th FileStoreTable sourceTable = (FileStoreTable) sourceCatalog.getTable(sourceIdentifier); targetCatalog.createDatabase(targetIdentifier.getDatabaseName(), true); targetCatalog.createTable( - targetIdentifier, Schema.fromTableSchema(sourceTable.schema()), true); + targetIdentifier, newSchemaFromTableSchema(sourceTable.schema()), true); List result = toCloneFileInfos( @@ -95,6 +102,18 @@ public void processElement(StreamRecord> streamRecord) th } } + private static Schema newSchemaFromTableSchema(TableSchema tableSchema) { + return new Schema( + ImmutableList.copyOf(tableSchema.fields()), + ImmutableList.copyOf(tableSchema.partitionKeys()), + ImmutableList.copyOf(tableSchema.primaryKeys()), + ImmutableMap.copyOf( + Iterables.filter( + tableSchema.options().entrySet(), + entry -> !Objects.equals(entry.getKey(), CoreOptions.PATH.key()))), + tableSchema.comment()); + } + private List toCloneFileInfos( List files, Path sourceTableRoot, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java index 747995d20d675..88730132ef68c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiAwareBucketTableScan.java @@ -32,7 +32,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -52,15 +51,8 @@ public MultiAwareBucketTableScan( Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { - super( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + boolean isStreaming) { + super(catalogLoader, includingPattern, excludingPattern, databasePattern, isStreaming); tablesMap = new HashMap<>(); scansMap = new HashMap<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java index bd4ffe83a4ca0..f5940740b691c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiTableScanBase.java @@ -26,12 +26,11 @@ import org.apache.paimon.table.source.EndOfScanException; import org.apache.paimon.table.source.Split; -import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.api.connector.source.ReaderOutput; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import static org.apache.paimon.flink.utils.MultiTablesCompactorUtil.shouldCompactTable; @@ -57,7 +56,6 @@ public abstract class MultiTableScanBase implements AutoCloseable { protected transient Catalog catalog; - protected AtomicBoolean isRunning; protected boolean isStreaming; public MultiTableScanBase( @@ -65,14 +63,12 @@ public MultiTableScanBase( Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { + boolean isStreaming) { catalog = catalogLoader.load(); this.includingPattern = includingPattern; this.excludingPattern = excludingPattern; this.databasePattern = databasePattern; - this.isRunning = isRunning; this.isStreaming = isStreaming; } @@ -104,13 +100,9 @@ protected void updateTableMap() } } - public ScanResult scanTable(SourceFunction.SourceContext ctx) + public ScanResult scanTable(ReaderOutput ctx) throws Catalog.TableNotExistException, Catalog.DatabaseNotExistException { try { - if (!isRunning.get()) { - return ScanResult.FINISHED; - } - updateTableMap(); List tasks = doScan(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java index 56bf971240e73..da86b93af5120 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/MultiUnawareBucketTableScan.java @@ -29,7 +29,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; /** @@ -46,15 +45,8 @@ public MultiUnawareBucketTableScan( Pattern includingPattern, Pattern excludingPattern, Pattern databasePattern, - boolean isStreaming, - AtomicBoolean isRunning) { - super( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + boolean isStreaming) { + super(catalogLoader, includingPattern, excludingPattern, databasePattern, isStreaming); tablesMap = new HashMap<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java index 8c6ed4c9f59e1..a572354e89845 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/UnawareBucketCompactionTopoBuilder.java @@ -126,7 +126,7 @@ private DataStreamSource buildSource() { new BucketUnawareCompactSource( table, isContinuous, scanInterval, partitionPredicate); - return BucketUnawareCompactSource.buildSource(env, source, isContinuous, tableIdentifier); + return BucketUnawareCompactSource.buildSource(env, source, tableIdentifier); } private void sinkFromSource(DataStreamSource input) { 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 index 5cae899a07040..a529e6764faea 100644 --- 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 @@ -21,6 +21,7 @@ import org.apache.paimon.flink.sink.NoneCopyVersionedSerializerTypeSerializerProxy; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -56,7 +57,17 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer( + SerializerConfig serializerConfig) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // we don't need copy for task return new NoneCopyVersionedSerializerTypeSerializerProxy( 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 index e17566f302cdc..e0aed448db93c 100644 --- 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 @@ -27,7 +27,7 @@ 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 org.apache.paimon.reader.FileRecordReader; import java.io.EOFException; import java.io.IOException; @@ -60,7 +60,7 @@ public CompactedChangelogFormatReaderFactory(FormatReaderFactory wrapped) { } @Override - public RecordReader createReader(Context context) throws IOException { + public FileRecordReader createReader(Context context) throws IOException { OffsetReadOnlyFileIO fileIO = new OffsetReadOnlyFileIO(context.fileIO()); long length = decodePath(context.filePath()).length; 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 9503960fbe17b..e3f2fe110c6c3 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 @@ -249,7 +249,7 @@ public Collection lookup(RowData keyRow) { rows.add(new FlinkRowData(matchedRow)); } return rows; - } catch (OutOfRangeException e) { + } catch (OutOfRangeException | ReopenException e) { reopen(); return lookup(keyRow); } catch (Exception e) { 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 index 908884a573c0f..f43d80321ecc9 100644 --- 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 @@ -19,6 +19,7 @@ package org.apache.paimon.flink.lookup; import org.apache.paimon.CoreOptions; +import org.apache.paimon.Snapshot; import org.apache.paimon.operation.DefaultValueAssigner; import org.apache.paimon.table.source.DataTableStreamScan; import org.apache.paimon.table.source.snapshot.AllDeltaFollowUpScanner; @@ -29,6 +30,8 @@ import org.apache.paimon.table.source.snapshot.StartingScanner; import org.apache.paimon.utils.SnapshotManager; +import javax.annotation.Nullable; + import static org.apache.paimon.CoreOptions.StartupMode; import static org.apache.paimon.flink.lookup.LookupFileStoreTable.LookupStreamScanMode; @@ -56,6 +59,17 @@ public LookupDataTableScan( defaultValueAssigner); this.startupMode = options.startupMode(); this.lookupScanMode = lookupScanMode; + dropStats(); + } + + @Override + @Nullable + protected SnapshotReader.Plan handleOverwriteSnapshot(Snapshot snapshot) { + SnapshotReader.Plan plan = super.handleOverwriteSnapshot(snapshot); + if (plan != null) { + return plan; + } + throw new ReopenException(); } @Override 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 ef5543ac9b7cf..7bd7a652b56e4 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 @@ -207,6 +207,7 @@ private LocalQueryExecutor( this.scan = table.newReadBuilder() + .dropStats() .withFilter(filter) .withBucketFilter( requireCachedBucketIds == null diff --git a/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/spark/paimon/Utils.scala b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/ReopenException.java similarity index 74% rename from paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/spark/paimon/Utils.scala rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/ReopenException.java index 1a899f5001536..7149d591f8dfd 100644 --- a/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/spark/paimon/Utils.scala +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/ReopenException.java @@ -16,17 +16,14 @@ * limitations under the License. */ -package org.apache.spark.paimon +package org.apache.paimon.flink.lookup; -import org.apache.spark.util.{Utils => SparkUtils} +/** Signals that dim table source need to reopen. */ +public class ReopenException extends RuntimeException { -import java.io.File - -/** - * A wrapper that some Objects or Classes is limited to access beyond [[org.apache.spark]] package. - */ -object Utils { - - def createTempDir: File = SparkUtils.createTempDir() + private static final long serialVersionUID = 1L; + public ReopenException() { + super(); + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/orphan/FlinkOrphanFilesClean.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/orphan/FlinkOrphanFilesClean.java index f50414620551d..23bbbc9b609ca 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/orphan/FlinkOrphanFilesClean.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/orphan/FlinkOrphanFilesClean.java @@ -27,12 +27,15 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.manifest.ManifestFile; +import org.apache.paimon.operation.CleanOrphanFilesResult; import org.apache.paimon.operation.OrphanFilesClean; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; +import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.SerializableConsumer; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; @@ -61,7 +64,6 @@ import java.util.function.Consumer; import java.util.stream.Collectors; -import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.LONG_TYPE_INFO; import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO; import static org.apache.flink.util.Preconditions.checkState; import static org.apache.paimon.utils.Preconditions.checkArgument; @@ -81,7 +83,7 @@ public FlinkOrphanFilesClean( } @Nullable - public DataStream doOrphanClean(StreamExecutionEnvironment env) { + public DataStream doOrphanClean(StreamExecutionEnvironment env) { Configuration flinkConf = new Configuration(); flinkConf.set(ExecutionOptions.RUNTIME_MODE, RuntimeExecutionMode.BATCH); flinkConf.set(ExecutionOptions.SORT_INPUTS, false); @@ -97,8 +99,12 @@ public DataStream doOrphanClean(StreamExecutionEnvironment env) { // snapshot and changelog files are the root of everything, so they are handled specially // here, and subsequently, we will not count their orphan files. - AtomicLong deletedInLocal = new AtomicLong(0); - cleanSnapshotDir(branches, p -> deletedInLocal.incrementAndGet()); + AtomicLong deletedFilesCountInLocal = new AtomicLong(0); + AtomicLong deletedFilesLenInBytesInLocal = new AtomicLong(0); + cleanSnapshotDir( + branches, + path -> deletedFilesCountInLocal.incrementAndGet(), + deletedFilesLenInBytesInLocal::addAndGet); // branch and manifest file final OutputTag> manifestOutputTag = @@ -203,36 +209,45 @@ public void endInput() throws IOException { .map(Path::toUri) .map(Object::toString) .collect(Collectors.toList()); - DataStream candidates = + DataStream> candidates = env.fromCollection(fileDirs) .process( - new ProcessFunction() { + new ProcessFunction>() { @Override public void processElement( String dir, - ProcessFunction.Context ctx, - Collector out) { + ProcessFunction>.Context ctx, + Collector> out) { for (FileStatus fileStatus : tryBestListingDirs(new Path(dir))) { if (oldEnough(fileStatus)) { out.collect( - fileStatus.getPath().toUri().toString()); + Pair.of( + fileStatus + .getPath() + .toUri() + .toString(), + fileStatus.getLen())); } } } }); - DataStream deleted = + DataStream deleted = usedFiles .keyBy(f -> f) - .connect(candidates.keyBy(path -> new Path(path).getName())) + .connect( + candidates.keyBy( + pathAndSize -> new Path(pathAndSize.getKey()).getName())) .transform( "files_join", - LONG_TYPE_INFO, - new BoundedTwoInputOperator() { + TypeInformation.of(CleanOrphanFilesResult.class), + new BoundedTwoInputOperator< + String, Pair, CleanOrphanFilesResult>() { private boolean buildEnd; - private long emitted; + private long emittedFilesCount; + private long emittedFilesLen; private final Set used = new HashSet<>(); @@ -254,8 +269,15 @@ public void endInput(int inputId) { case 2: checkState(buildEnd, "Should build ended."); LOG.info("Finish probe phase."); - LOG.info("Clean files: {}", emitted); - output.collect(new StreamRecord<>(emitted)); + LOG.info( + "Clean files count : {}", + emittedFilesCount); + LOG.info("Clean files size : {}", emittedFilesLen); + output.collect( + new StreamRecord<>( + new CleanOrphanFilesResult( + emittedFilesCount, + emittedFilesLen))); break; } } @@ -266,25 +288,34 @@ public void processElement1(StreamRecord element) { } @Override - public void processElement2(StreamRecord element) { + public void processElement2( + StreamRecord> element) { checkState(buildEnd, "Should build ended."); - String value = element.getValue(); + Pair fileInfo = element.getValue(); + String value = fileInfo.getLeft(); Path path = new Path(value); if (!used.contains(path.getName())) { + emittedFilesCount++; + emittedFilesLen += fileInfo.getRight(); fileCleaner.accept(path); LOG.info("Dry clean: {}", path); - emitted++; } } }); - if (deletedInLocal.get() != 0) { - deleted = deleted.union(env.fromData(deletedInLocal.get())); + if (deletedFilesCountInLocal.get() != 0 || deletedFilesLenInBytesInLocal.get() != 0) { + deleted = + deleted.union( + env.fromElements( + new CleanOrphanFilesResult( + deletedFilesCountInLocal.get(), + deletedFilesLenInBytesInLocal.get()))); } + return deleted; } - public static long executeDatabaseOrphanFiles( + public static CleanOrphanFilesResult executeDatabaseOrphanFiles( StreamExecutionEnvironment env, Catalog catalog, long olderThanMillis, @@ -293,12 +324,13 @@ public static long executeDatabaseOrphanFiles( String databaseName, @Nullable String tableName) throws Catalog.DatabaseNotExistException, Catalog.TableNotExistException { - List> orphanFilesCleans = new ArrayList<>(); List tableNames = Collections.singletonList(tableName); if (tableName == null || "*".equals(tableName)) { tableNames = catalog.listTables(databaseName); } + List> orphanFilesCleans = + new ArrayList<>(tableNames.size()); for (String t : tableNames) { Identifier identifier = new Identifier(databaseName, t); Table table = catalog.getTable(identifier); @@ -307,7 +339,7 @@ public static long executeDatabaseOrphanFiles( "Only FileStoreTable supports remove-orphan-files action. The table type is '%s'.", table.getClass().getName()); - DataStream clean = + DataStream clean = new FlinkOrphanFilesClean( (FileStoreTable) table, olderThanMillis, @@ -319,8 +351,8 @@ public static long executeDatabaseOrphanFiles( } } - DataStream result = null; - for (DataStream clean : orphanFilesCleans) { + DataStream result = null; + for (DataStream clean : orphanFilesCleans) { if (result == null) { result = clean; } else { @@ -331,20 +363,24 @@ public static long executeDatabaseOrphanFiles( return sum(result); } - private static long sum(DataStream deleted) { - long deleteCount = 0; + private static CleanOrphanFilesResult sum(DataStream deleted) { + long deletedFilesCount = 0; + long deletedFilesLenInBytes = 0; if (deleted != null) { try { - CloseableIterator iterator = + CloseableIterator iterator = deleted.global().executeAndCollect("OrphanFilesClean"); while (iterator.hasNext()) { - deleteCount += iterator.next(); + CleanOrphanFilesResult cleanOrphanFilesResult = iterator.next(); + deletedFilesCount += cleanOrphanFilesResult.getDeletedFileCount(); + deletedFilesLenInBytes += + cleanOrphanFilesResult.getDeletedFileTotalLenInBytes(); } iterator.close(); } catch (Exception e) { throw new RuntimeException(e); } } - return deleteCount; + return new CleanOrphanFilesResult(deletedFilesCount, deletedFilesLenInBytes); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java index dd71e974c7b16..80602b755aa59 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactDatabaseProcedure.java @@ -29,6 +29,8 @@ import java.util.Map; +import static org.apache.paimon.flink.action.ActionFactory.FULL; +import static org.apache.paimon.flink.action.CompactActionFactory.checkCompactStrategy; import static org.apache.paimon.utils.ParameterUtils.parseCommaSeparatedKeyValues; /** @@ -82,6 +84,10 @@ public class CompactDatabaseProcedure extends ProcedureBase { @ArgumentHint( name = "partition_idle_time", type = @DataTypeHint("STRING"), + isOptional = true), + @ArgumentHint( + name = "compact_strategy", + type = @DataTypeHint("STRING"), isOptional = true) }) public String[] call( @@ -91,7 +97,8 @@ public String[] call( String includingTables, String excludingTables, String tableOptions, - String partitionIdleTime) + String partitionIdleTime, + String compactStrategy) throws Exception { partitionIdleTime = notnull(partitionIdleTime); String warehouse = catalog.warehouse(); @@ -109,6 +116,10 @@ public String[] call( action.withPartitionIdleTime(TimeUtils.parseDuration(partitionIdleTime)); } + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } + return execute(procedureContext, action, "Compact database job"); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java index 8589069126988..282f5af340432 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/CompactProcedure.java @@ -32,6 +32,8 @@ import java.util.Collections; import java.util.Map; +import static org.apache.paimon.flink.action.ActionFactory.FULL; +import static org.apache.paimon.flink.action.CompactActionFactory.checkCompactStrategy; import static org.apache.paimon.utils.ParameterUtils.getPartitions; import static org.apache.paimon.utils.ParameterUtils.parseCommaSeparatedKeyValues; import static org.apache.paimon.utils.StringUtils.isNullOrWhitespaceOnly; @@ -58,6 +60,10 @@ public class CompactProcedure extends ProcedureBase { @ArgumentHint( name = "partition_idle_time", type = @DataTypeHint("STRING"), + isOptional = true), + @ArgumentHint( + name = "compact_strategy", + type = @DataTypeHint("STRING"), isOptional = true) }) public String[] call( @@ -68,7 +74,8 @@ public String[] call( String orderByColumns, String tableOptions, String where, - String partitionIdleTime) + String partitionIdleTime, + String compactStrategy) throws Exception { String warehouse = catalog.warehouse(); Map catalogOptions = catalog.options(); @@ -90,6 +97,10 @@ public String[] call( if (!isNullOrWhitespaceOnly(partitionIdleTime)) { action.withPartitionIdleTime(TimeUtils.parseDuration(partitionIdleTime)); } + + if (checkCompactStrategy(compactStrategy)) { + action.withFullCompaction(compactStrategy.trim().equalsIgnoreCase(FULL)); + } jobName = "Compact Job"; } else if (!isNullOrWhitespaceOnly(orderStrategy) && !isNullOrWhitespaceOnly(orderByColumns)) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/DeleteBranchProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/DeleteBranchProcedure.java index c95fd62bee40e..56c6490286502 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/DeleteBranchProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/DeleteBranchProcedure.java @@ -49,7 +49,10 @@ public String identifier() { }) public String[] call(ProcedureContext procedureContext, String tableId, String branchStr) throws Catalog.TableNotExistException { - catalog.getTable(Identifier.fromString(tableId)).deleteBranches(branchStr); + Identifier identifier = Identifier.fromString(tableId); + catalog.getTable(identifier).deleteBranches(branchStr); + catalog.invalidateTable( + new Identifier(identifier.getDatabaseName(), identifier.getTableName(), branchStr)); return new String[] {"Success"}; } } 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 ee6075a927d3f..ce282c6800cc1 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 @@ -97,9 +97,10 @@ public String identifier() { .catalogEnvironment() .metastoreClientFactory()) .map(MetastoreClient.Factory::create) - .orElse(null)); + .orElse(null), + fileStore.options().partitionExpireMaxNum()); if (maxExpires != null) { - partitionExpire.withMaxExpires(maxExpires); + partitionExpire.withMaxExpireNum(maxExpires); } List> expired = partitionExpire.expire(Long.MAX_VALUE); return expired == null || expired.isEmpty() 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 10ad878e0ccb5..4cd1b3e003038 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,6 +20,7 @@ import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.orphan.FlinkOrphanFilesClean; +import org.apache.paimon.operation.CleanOrphanFilesResult; import org.apache.paimon.operation.LocalOrphanFilesClean; import org.apache.flink.table.annotation.ArgumentHint; @@ -75,11 +76,11 @@ public String[] call( if (mode == null) { mode = "DISTRIBUTED"; } - long deletedFiles; + CleanOrphanFilesResult cleanOrphanFilesResult; try { switch (mode.toUpperCase(Locale.ROOT)) { case "DISTRIBUTED": - deletedFiles = + cleanOrphanFilesResult = FlinkOrphanFilesClean.executeDatabaseOrphanFiles( procedureContext.getExecutionEnvironment(), catalog, @@ -90,7 +91,7 @@ public String[] call( tableName); break; case "LOCAL": - deletedFiles = + cleanOrphanFilesResult = LocalOrphanFilesClean.executeDatabaseOrphanFiles( catalog, databaseName, @@ -105,7 +106,10 @@ public String[] call( + mode + ". Only 'DISTRIBUTED' and 'LOCAL' are supported."); } - return new String[] {String.valueOf(deletedFiles)}; + return new String[] { + String.valueOf(cleanOrphanFilesResult.getDeletedFileCount()), + String.valueOf(cleanOrphanFilesResult.getDeletedFileTotalLenInBytes()) + }; } catch (Exception e) { throw new RuntimeException(e); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RollbackToWatermarkProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RollbackToWatermarkProcedure.java new file mode 100644 index 0000000000000..ab1ea8080de97 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/RollbackToWatermarkProcedure.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 watermark procedure. Usage: + * + *

    
    + *  -- rollback to the snapshot which earlier or equal than watermark.
    + *  CALL sys.rollback_to_watermark(`table` => 'tableId', watermark => watermark)
    + * 
    + */ +public class RollbackToWatermarkProcedure extends ProcedureBase { + + public static final String IDENTIFIER = "rollback_to_watermark"; + + @ProcedureHint( + argument = { + @ArgumentHint(name = "table", type = @DataTypeHint("STRING")), + @ArgumentHint(name = "watermark", type = @DataTypeHint("BIGINT")) + }) + public String[] call(ProcedureContext procedureContext, String tableId, Long watermark) + throws Catalog.TableNotExistException { + Table table = catalog.getTable(Identifier.fromString(tableId)); + FileStoreTable fileStoreTable = (FileStoreTable) table; + Snapshot snapshot = fileStoreTable.snapshotManager().earlierOrEqualWatermark(watermark); + Preconditions.checkNotNull( + snapshot, String.format("count not find snapshot earlier than %s", watermark)); + 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/service/QueryAddressRegister.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java index df3cf7abf2a5f..00d527506cfef 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryAddressRegister.java @@ -23,9 +23,9 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; -import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.WriterInitContext; import java.net.InetSocketAddress; import java.util.TreeMap; @@ -33,48 +33,68 @@ import static org.apache.paimon.service.ServiceManager.PRIMARY_KEY_LOOKUP; /** Operator for address server to register addresses to {@link ServiceManager}. */ -public class QueryAddressRegister extends RichSinkFunction { - +public class QueryAddressRegister implements Sink { private final ServiceManager serviceManager; - private transient int numberExecutors; - private transient TreeMap executors; - public QueryAddressRegister(Table table) { this.serviceManager = ((FileStoreTable) table).store().newServiceManager(); } - @Override - public void open(Configuration parameters) throws Exception { - this.executors = new TreeMap<>(); + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ + public SinkWriter createWriter(InitContext context) { + return new QueryAddressRegisterSinkWriter(serviceManager); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public SinkWriter createWriter(WriterInitContext context) { + return new QueryAddressRegisterSinkWriter(serviceManager); } - @Override - public void invoke(InternalRow row, SinkFunction.Context context) { - int numberExecutors = row.getInt(0); - if (this.numberExecutors != 0 && this.numberExecutors != numberExecutors) { - throw new IllegalArgumentException( - String.format( - "Number Executors can not be changed! Old %s , New %s .", - this.numberExecutors, numberExecutors)); + private static class QueryAddressRegisterSinkWriter implements SinkWriter { + private final ServiceManager serviceManager; + + private final TreeMap executors; + + private int numberExecutors; + + private QueryAddressRegisterSinkWriter(ServiceManager serviceManager) { + this.serviceManager = serviceManager; + this.executors = new TreeMap<>(); } - this.numberExecutors = numberExecutors; - int executorId = row.getInt(1); - String hostname = row.getString(2).toString(); - int port = row.getInt(3); + @Override + public void write(InternalRow row, Context context) { + int numberExecutors = row.getInt(0); + if (this.numberExecutors != 0 && this.numberExecutors != numberExecutors) { + throw new IllegalArgumentException( + String.format( + "Number Executors can not be changed! Old %s , New %s .", + this.numberExecutors, numberExecutors)); + } + this.numberExecutors = numberExecutors; + + int executorId = row.getInt(1); + String hostname = row.getString(2).toString(); + int port = row.getInt(3); - executors.put(executorId, new InetSocketAddress(hostname, port)); + executors.put(executorId, new InetSocketAddress(hostname, port)); - if (executors.size() == numberExecutors) { - serviceManager.resetService( - PRIMARY_KEY_LOOKUP, executors.values().toArray(new InetSocketAddress[0])); + if (executors.size() == numberExecutors) { + serviceManager.resetService( + PRIMARY_KEY_LOOKUP, executors.values().toArray(new InetSocketAddress[0])); + } } - } - @Override - public void close() throws Exception { - super.close(); - serviceManager.deleteService(PRIMARY_KEY_LOOKUP); + @Override + public void flush(boolean endOfInput) {} + + @Override + public void close() { + serviceManager.deleteService(PRIMARY_KEY_LOOKUP); + } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java index 556c308396884..bf0521d55049c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryExecutorOperator.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; import org.apache.paimon.service.network.NetworkUtils; @@ -77,8 +78,8 @@ public void initializeState(StateInitializationContext context) throws Exception this.query = ((FileStoreTable) table).newLocalTableQuery().withIOManager(ioManager); KvQueryServer server = new KvQueryServer( - getRuntimeContext().getIndexOfThisSubtask(), - getRuntimeContext().getNumberOfParallelSubtasks(), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()), NetworkUtils.findHostAddress(), Collections.singletonList(0).iterator(), 1, @@ -96,8 +97,9 @@ public void initializeState(StateInitializationContext context) throws Exception this.output.collect( new StreamRecord<>( GenericRow.of( - getRuntimeContext().getNumberOfParallelSubtasks(), - getRuntimeContext().getIndexOfThisSubtask(), + RuntimeContextUtils.getNumberOfParallelSubtasks( + getRuntimeContext()), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), BinaryString.fromString(address.getHostName()), address.getPort()))); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java index 43cf654e91fe6..6688503778a0f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryFileMonitor.java @@ -21,6 +21,9 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.InternalTypeInfo; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; @@ -31,10 +34,14 @@ import org.apache.paimon.table.source.TableRead; import org.apache.paimon.table.system.FileMonitorTable; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import java.util.ArrayList; import java.util.List; @@ -49,19 +56,13 @@ *
  • Assigning them to downstream tasks for further processing. * */ -public class QueryFileMonitor extends RichSourceFunction { +public class QueryFileMonitor extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 1L; private final Table table; private final long monitorInterval; - private transient SourceContext ctx; - private transient StreamTableScan scan; - private transient TableRead read; - - private volatile boolean isRunning = true; - public QueryFileMonitor(Table table) { this.table = table; this.monitorInterval = @@ -71,55 +72,53 @@ public QueryFileMonitor(Table table) { } @Override - public void open(Configuration parameters) throws Exception { - FileMonitorTable monitorTable = new FileMonitorTable((FileStoreTable) table); - ReadBuilder readBuilder = monitorTable.newReadBuilder(); - this.scan = readBuilder.newStreamScan(); - this.read = readBuilder.newRead(); + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; } @Override - public void run(SourceContext ctx) throws Exception { - this.ctx = ctx; - while (isRunning) { - boolean isEmpty; - synchronized (ctx.getCheckpointLock()) { - if (!isRunning) { - return; - } - isEmpty = doScan(); - } + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); + } + + private class Reader extends AbstractNonCoordinatedSourceReader { + private transient StreamTableScan scan; + private transient TableRead read; + + @Override + public void start() { + FileMonitorTable monitorTable = new FileMonitorTable((FileStoreTable) table); + ReadBuilder readBuilder = monitorTable.newReadBuilder().dropStats(); + this.scan = readBuilder.newStreamScan(); + this.read = readBuilder.newRead(); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + boolean isEmpty = doScan(readerOutput); if (isEmpty) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; } - } - private boolean doScan() throws Exception { - List records = new ArrayList<>(); - read.createReader(scan.plan()).forEachRemaining(records::add); - records.forEach(ctx::collect); - return records.isEmpty(); - } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning = false; - } - } else { - isRunning = false; + private boolean doScan(ReaderOutput readerOutput) throws Exception { + List records = new ArrayList<>(); + read.createReader(scan.plan()).forEachRemaining(records::add); + records.forEach(readerOutput::collect); + return records.isEmpty(); } } public static DataStream build(StreamExecutionEnvironment env, Table table) { - return env.addSource( - new QueryFileMonitor(table), - "FileMonitor-" + table.name(), - InternalTypeInfo.fromRowType(FileMonitorTable.getRowType())); + return env.fromSource( + new QueryFileMonitor(table), + WatermarkStrategy.noWatermarks(), + "FileMonitor-" + table.name(), + InternalTypeInfo.fromRowType(FileMonitorTable.getRowType())) + .setParallelism(1); } public static ChannelComputer createChannelComputer() { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java index bd433fe0f00d0..752d54cff5a0f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/service/QueryService.java @@ -62,7 +62,7 @@ public static void build(StreamExecutionEnvironment env, Table table, int parall InternalTypeInfo.fromRowType(QueryExecutorOperator.outputType()), executorOperator) .setParallelism(parallelism) - .addSink(new QueryAddressRegister(table)) + .sinkTo(new QueryAddressRegister(table)) .setParallelism(1); sink.getTransformation().setMaxParallelism(1); 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 54104130438bf..8760f1dc5f804 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 @@ -27,6 +27,7 @@ import org.apache.paimon.utils.SerializableSupplier; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; @@ -182,9 +183,19 @@ public KeyAndSizeExtractor(RowType rowType, boolean isSortBySize) { this.isSortBySize = isSortBySize; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink + * 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink + * 2.0+. + */ public void open(Configuration parameters) throws Exception { - super.open(parameters); InternalRowToSizeVisitor internalRowToSizeVisitor = new InternalRowToSizeVisitor(); fieldSizeCalculator = rowType.getFieldTypes().stream() diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java index 92cd31ea8aa26..977511920a061 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendBypassCompactWorkerOperator.java @@ -21,7 +21,9 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.table.FileStoreTable; -import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.types.Either; @@ -29,9 +31,11 @@ public class AppendBypassCompactWorkerOperator extends AppendCompactWorkerOperator> { - public AppendBypassCompactWorkerOperator(FileStoreTable table, String commitUser) { - super(table, commitUser); - this.chainingStrategy = ChainingStrategy.HEAD; + private AppendBypassCompactWorkerOperator( + StreamOperatorParameters parameters, + FileStoreTable table, + String commitUser) { + super(parameters, table, commitUser); } @Override @@ -49,4 +53,27 @@ public void processElement( unawareBucketCompactor.processElement(element.getValue().right()); } } + + /** {@link StreamOperatorFactory} of {@link AppendBypassCompactWorkerOperator}. */ + public static class Factory + extends AppendCompactWorkerOperator.Factory< + Either> { + + public Factory(FileStoreTable table, String initialCommitUser) { + super(table, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) new AppendBypassCompactWorkerOperator(parameters, table, commitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return AppendBypassCompactWorkerOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendCompactWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendCompactWorkerOperator.java index 52ab75de6b2c3..7a3c0231eb65d 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendCompactWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendCompactWorkerOperator.java @@ -27,6 +27,8 @@ import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.utils.ExecutorThreadFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,8 +55,11 @@ public abstract class AppendCompactWorkerOperator private transient ExecutorService lazyCompactExecutor; - public AppendCompactWorkerOperator(FileStoreTable table, String commitUser) { - super(Options.fromMap(table.options())); + public AppendCompactWorkerOperator( + StreamOperatorParameters parameters, + FileStoreTable table, + String commitUser) { + super(parameters, Options.fromMap(table.options())); this.table = table; this.commitUser = commitUser; } @@ -101,4 +106,17 @@ public void close() throws Exception { this.unawareBucketCompactor.close(); } } + + /** {@link StreamOperatorFactory} of {@link AppendCompactWorkerOperator}. */ + protected abstract static class Factory + extends PrepareCommitOperator.Factory { + protected final FileStoreTable table; + protected final String commitUser; + + protected Factory(FileStoreTable table, String commitUser) { + super(Options.fromMap(table.options())); + this.table = table; + this.commitUser = commitUser; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java index 15e7b9746fe6c..83d51f302e51f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperator.java @@ -28,6 +28,9 @@ import org.apache.paimon.utils.ExceptionUtils; import org.apache.paimon.utils.ExecutorThreadFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,9 +65,12 @@ public class AppendOnlyMultiTableCompactionWorkerOperator private transient Catalog catalog; - public AppendOnlyMultiTableCompactionWorkerOperator( - Catalog.Loader catalogLoader, String commitUser, Options options) { - super(options); + private AppendOnlyMultiTableCompactionWorkerOperator( + StreamOperatorParameters parameters, + Catalog.Loader catalogLoader, + String commitUser, + Options options) { + super(parameters, options); this.commitUser = commitUser; this.catalogLoader = catalogLoader; } @@ -175,4 +181,34 @@ public void close() throws Exception { ExceptionUtils.throwMultiException(exceptions); } + + /** {@link StreamOperatorFactory} of {@link AppendOnlyMultiTableCompactionWorkerOperator}. */ + public static class Factory + extends PrepareCommitOperator.Factory< + MultiTableUnawareAppendCompactionTask, MultiTableCommittable> { + + private final String commitUser; + private final Catalog.Loader catalogLoader; + + public Factory(Catalog.Loader catalogLoader, String commitUser, Options options) { + super(options); + this.commitUser = commitUser; + this.catalogLoader = catalogLoader; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new AppendOnlyMultiTableCompactionWorkerOperator( + parameters, catalogLoader, commitUser, options); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return AppendOnlyMultiTableCompactionWorkerOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperator.java index 4d0201d324615..917a7f64f1a04 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperator.java @@ -22,6 +22,9 @@ import org.apache.paimon.flink.source.BucketUnawareCompactSource; import org.apache.paimon.table.FileStoreTable; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** @@ -31,12 +34,39 @@ public class AppendOnlySingleTableCompactionWorkerOperator extends AppendCompactWorkerOperator { - public AppendOnlySingleTableCompactionWorkerOperator(FileStoreTable table, String commitUser) { - super(table, commitUser); + private AppendOnlySingleTableCompactionWorkerOperator( + StreamOperatorParameters parameters, + FileStoreTable table, + String commitUser) { + super(parameters, table, commitUser); } @Override public void processElement(StreamRecord element) throws Exception { this.unawareBucketCompactor.processElement(element.getValue()); } + + /** {@link StreamOperatorFactory} of {@link AppendOnlySingleTableCompactionWorkerOperator}. */ + public static class Factory + extends AppendCompactWorkerOperator.Factory { + + public Factory(FileStoreTable table, String initialCommitUser) { + super(table, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new AppendOnlySingleTableCompactionWorkerOperator( + parameters, table, commitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return AppendOnlySingleTableCompactionWorkerOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java index 6d27c60194837..0822f04612413 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java @@ -32,18 +32,13 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CheckpointStreamFactory; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; -import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.operators.SetupableStreamOperator; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import java.time.Duration; @@ -58,9 +53,7 @@ * time, tags are automatically created for each flink savepoint. */ public class AutoTagForSavepointCommitterOperator - implements OneInputStreamOperator, - SetupableStreamOperator, - BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { public static final String SAVEPOINT_TAG_PREFIX = "savepoint-"; private static final long serialVersionUID = 1L; @@ -256,19 +249,4 @@ public void setKeyContextElement(StreamRecord record) throws Exception public void endInput() throws Exception { commitOperator.endInput(); } - - @Override - public void setup(StreamTask containingTask, StreamConfig config, Output output) { - commitOperator.setup(containingTask, config, output); - } - - @Override - public ChainingStrategy getChainingStrategy() { - return commitOperator.getChainingStrategy(); - } - - @Override - public void setChainingStrategy(ChainingStrategy strategy) { - commitOperator.setChainingStrategy(strategy); - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.java new file mode 100644 index 0000000000000..1787f8e7adce5 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorFactory.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.sink; + +import org.apache.paimon.operation.TagDeletion; +import org.apache.paimon.table.sink.TagCallback; +import org.apache.paimon.utils.SerializableSupplier; +import org.apache.paimon.utils.SnapshotManager; +import org.apache.paimon.utils.TagManager; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +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 java.time.Duration; +import java.util.List; +import java.util.NavigableSet; +import java.util.TreeSet; + +/** + * {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for {@link + * AutoTagForSavepointCommitterOperator}. + */ +public class AutoTagForSavepointCommitterOperatorFactory + extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + + private final CommitterOperatorFactory commitOperatorFactory; + + private final SerializableSupplier snapshotManagerFactory; + + private final SerializableSupplier tagManagerFactory; + + private final SerializableSupplier tagDeletionFactory; + + private final SerializableSupplier> callbacksSupplier; + + private final NavigableSet identifiersForTags; + + private final Duration tagTimeRetained; + + public AutoTagForSavepointCommitterOperatorFactory( + CommitterOperatorFactory commitOperatorFactory, + SerializableSupplier snapshotManagerFactory, + SerializableSupplier tagManagerFactory, + SerializableSupplier tagDeletionFactory, + SerializableSupplier> callbacksSupplier, + Duration tagTimeRetained) { + this.commitOperatorFactory = commitOperatorFactory; + this.tagManagerFactory = tagManagerFactory; + this.snapshotManagerFactory = snapshotManagerFactory; + this.tagDeletionFactory = tagDeletionFactory; + this.callbacksSupplier = callbacksSupplier; + this.identifiersForTags = new TreeSet<>(); + this.tagTimeRetained = tagTimeRetained; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new AutoTagForSavepointCommitterOperator<>( + commitOperatorFactory.createStreamOperator(parameters), + snapshotManagerFactory, + tagManagerFactory, + tagDeletionFactory, + callbacksSupplier, + tagTimeRetained); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return AutoTagForSavepointCommitterOperator.class; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java index 23202b45077ff..1cbcc4b2262fd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java @@ -28,18 +28,13 @@ import org.apache.flink.runtime.checkpoint.CheckpointOptions; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.CheckpointStreamFactory; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; -import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.operators.SetupableStreamOperator; import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import java.time.Instant; @@ -53,9 +48,7 @@ * completed, the corresponding tag is generated. */ public class BatchWriteGeneratorTagOperator - implements OneInputStreamOperator, - SetupableStreamOperator, - BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { private static final String BATCH_WRITE_TAG_PREFIX = "batch-write-"; @@ -250,19 +243,4 @@ public void setKeyContextElement(StreamRecord record) throws Exception public void endInput() throws Exception { commitOperator.endInput(); } - - @Override - public void setup(StreamTask containingTask, StreamConfig config, Output output) { - commitOperator.setup(containingTask, config, output); - } - - @Override - public ChainingStrategy getChainingStrategy() { - return commitOperator.getChainingStrategy(); - } - - @Override - public void setChainingStrategy(ChainingStrategy strategy) { - commitOperator.setChainingStrategy(strategy); - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.java new file mode 100644 index 0000000000000..e3c0e5c491680 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorFactory.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.sink; + +import org.apache.paimon.table.FileStoreTable; + +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; + +/** + * {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for {@link + * BatchWriteGeneratorTagOperator}. + */ +public class BatchWriteGeneratorTagOperatorFactory + extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final CommitterOperatorFactory commitOperatorFactory; + + protected final FileStoreTable table; + + public BatchWriteGeneratorTagOperatorFactory( + CommitterOperatorFactory commitOperatorFactory, + FileStoreTable table) { + this.table = table; + this.commitOperatorFactory = commitOperatorFactory; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new BatchWriteGeneratorTagOperator<>( + commitOperatorFactory.createStreamOperator(parameters), table); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return BatchWriteGeneratorTagOperator.class; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java index 87a28091fa302..25f76ce976833 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CombinedTableCompactorSink.java @@ -32,8 +32,8 @@ import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.data.RowData; import java.io.Serializable; @@ -57,11 +57,15 @@ public class CombinedTableCompactorSink implements Serializable { private final Catalog.Loader catalogLoader; private final boolean ignorePreviousFiles; + private final boolean fullCompaction; + private final Options options; - public CombinedTableCompactorSink(Catalog.Loader catalogLoader, Options options) { + public CombinedTableCompactorSink( + Catalog.Loader catalogLoader, Options options, boolean fullCompaction) { this.catalogLoader = catalogLoader; this.ignorePreviousFiles = false; + this.fullCompaction = fullCompaction; this.options = options; } @@ -104,7 +108,10 @@ public DataStream doWrite( String.format("%s-%s", "Multi-Bucket-Table", WRITER_NAME), new MultiTableCommittableTypeInfo(), combinedMultiComacptionWriteOperator( - env.getCheckpointConfig(), isStreaming, commitUser)) + env.getCheckpointConfig(), + isStreaming, + fullCompaction, + commitUser)) .setParallelism(awareBucketTableSource.getParallelism()); SingleOutputStreamOperator unawareBucketTableRewriter = @@ -112,7 +119,7 @@ public DataStream doWrite( .transform( String.format("%s-%s", "Unaware-Bucket-Table", WRITER_NAME), new MultiTableCommittableTypeInfo(), - new AppendOnlyMultiTableCompactionWorkerOperator( + new AppendOnlyMultiTableCompactionWorkerOperator.Factory( catalogLoader, commitUser, options)) .setParallelism(unawareBucketTableSource.getParallelism()); @@ -153,28 +160,34 @@ protected DataStreamSink doCommit( .transform( GLOBAL_COMMITTER_NAME, new MultiTableCommittableTypeInfo(), - new CommitterOperator<>( + new CommitterOperatorFactory<>( streamingCheckpointEnabled, false, - options.get(SINK_COMMITTER_OPERATOR_CHAINING), commitUser, createCommitterFactory(isStreaming), createCommittableStateManager(), options.get(END_INPUT_WATERMARK))) .setParallelism(written.getParallelism()); - return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + if (!options.get(SINK_COMMITTER_OPERATOR_CHAINING)) { + committed = committed.startNewChain(); + } + return committed.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } // TODO:refactor FlinkSink to adopt this sink - protected OneInputStreamOperator + protected OneInputStreamOperatorFactory combinedMultiComacptionWriteOperator( - CheckpointConfig checkpointConfig, boolean isStreaming, String commitUser) { - return new MultiTablesStoreCompactOperator( + CheckpointConfig checkpointConfig, + boolean isStreaming, + boolean fullCompaction, + String commitUser) { + return new MultiTablesStoreCompactOperator.Factory( catalogLoader, commitUser, checkpointConfig, isStreaming, ignorePreviousFiles, + fullCompaction, options); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java index dcb87238b8337..92e826a913792 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommittableTypeInfo.java @@ -21,6 +21,7 @@ import org.apache.paimon.table.sink.CommitMessageSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -57,7 +58,16 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // no copy, so that data from writer is directly going into committer while chaining return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java index 2ec90b8c6c40c..383cbcd6ebf71 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperator.java @@ -18,14 +18,15 @@ package org.apache.paimon.flink.sink; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.utils.Preconditions; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -90,26 +91,9 @@ public class CommitterOperator extends AbstractStreamOpe private final Long endInputWatermark; public CommitterOperator( + StreamOperatorParameters parameters, boolean streamingCheckpointEnabled, boolean forceSingleParallelism, - boolean chaining, - String initialCommitUser, - Committer.Factory committerFactory, - CommittableStateManager committableStateManager) { - this( - streamingCheckpointEnabled, - forceSingleParallelism, - chaining, - initialCommitUser, - committerFactory, - committableStateManager, - null); - } - - public CommitterOperator( - boolean streamingCheckpointEnabled, - boolean forceSingleParallelism, - boolean chaining, String initialCommitUser, Committer.Factory committerFactory, CommittableStateManager committableStateManager, @@ -121,7 +105,10 @@ public CommitterOperator( this.committerFactory = checkNotNull(committerFactory); this.committableStateManager = committableStateManager; this.endInputWatermark = endInputWatermark; - setChainingStrategy(chaining ? ChainingStrategy.ALWAYS : ChainingStrategy.HEAD); + this.setup( + parameters.getContainingTask(), + parameters.getStreamConfig(), + parameters.getOutput()); } @Override @@ -129,7 +116,9 @@ public void initializeState(StateInitializationContext context) throws Exception super.initializeState(context); Preconditions.checkArgument( - !forceSingleParallelism || getRuntimeContext().getNumberOfParallelSubtasks() == 1, + !forceSingleParallelism + || RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) + == 1, "Committer Operator parallelism in paimon MUST be one."); this.currentWatermark = Long.MIN_VALUE; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java new file mode 100644 index 0000000000000..cce3d4e176bf8 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CommitterOperatorFactory.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.flink.streaming.api.operators.AbstractStreamOperatorFactory; +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 java.util.NavigableMap; +import java.util.TreeMap; + +import static org.apache.paimon.utils.Preconditions.checkNotNull; + +/** + * {@link org.apache.flink.streaming.api.operators.StreamOperatorFactory} for {@link + * CommitterOperator}. + */ +public class CommitterOperatorFactory + extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + protected final boolean streamingCheckpointEnabled; + + /** Whether to check the parallelism while runtime. */ + protected final boolean forceSingleParallelism; + /** + * This commitUser is valid only for new jobs. After the job starts, this commitUser will be + * recorded into the states of write and commit operators. When the job restarts, commitUser + * will be recovered from states and this value is ignored. + */ + protected final String initialCommitUser; + + /** Group the committable by the checkpoint id. */ + protected final NavigableMap committablesPerCheckpoint; + + protected final Committer.Factory committerFactory; + + protected final CommittableStateManager committableStateManager; + + /** + * Aggregate committables to global committables and commit the global committables to the + * external system. + */ + protected Committer committer; + + protected final Long endInputWatermark; + + public CommitterOperatorFactory( + boolean streamingCheckpointEnabled, + boolean forceSingleParallelism, + String initialCommitUser, + Committer.Factory committerFactory, + CommittableStateManager committableStateManager) { + this( + streamingCheckpointEnabled, + forceSingleParallelism, + initialCommitUser, + committerFactory, + committableStateManager, + null); + } + + public CommitterOperatorFactory( + boolean streamingCheckpointEnabled, + boolean forceSingleParallelism, + String initialCommitUser, + Committer.Factory committerFactory, + CommittableStateManager committableStateManager, + Long endInputWatermark) { + this.streamingCheckpointEnabled = streamingCheckpointEnabled; + this.forceSingleParallelism = forceSingleParallelism; + this.initialCommitUser = initialCommitUser; + this.committablesPerCheckpoint = new TreeMap<>(); + this.committerFactory = checkNotNull(committerFactory); + this.committableStateManager = committableStateManager; + this.endInputWatermark = endInputWatermark; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CommitterOperator<>( + parameters, + streamingCheckpointEnabled, + forceSingleParallelism, + initialCommitUser, + committerFactory, + committableStateManager, + endInputWatermark); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return CommitterOperator.class; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java index 47defa61a9713..6510a85b800af 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactionTaskTypeInfo.java @@ -22,6 +22,7 @@ import org.apache.paimon.table.sink.CompactionTaskSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -58,7 +59,16 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // we don't need copy for task return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java index 7dc3ab1150b07..a9c6031dfa346 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java @@ -21,7 +21,7 @@ import org.apache.paimon.manifest.ManifestCommittable; import org.apache.paimon.table.FileStoreTable; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.data.RowData; /** {@link FlinkSink} for dedicated compact jobs. */ @@ -29,14 +29,17 @@ public class CompactorSink extends FlinkSink { private static final long serialVersionUID = 1L; - public CompactorSink(FileStoreTable table) { + private final boolean fullCompaction; + + public CompactorSink(FileStoreTable table, boolean fullCompaction) { super(table, false); + this.fullCompaction = fullCompaction; } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new StoreCompactOperator(table, writeProvider, commitUser); + return new StoreCompactOperator.Factory(table, writeProvider, commitUser, fullCompaction); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSinkBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSinkBuilder.java index 926155cabf291..2d84ae6726fd2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSinkBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSinkBuilder.java @@ -37,8 +37,11 @@ public class CompactorSinkBuilder { private DataStream input; - public CompactorSinkBuilder(FileStoreTable table) { + private final boolean fullCompaction; + + public CompactorSinkBuilder(FileStoreTable table, boolean fullCompaction) { this.table = table; + this.fullCompaction = fullCompaction; } public CompactorSinkBuilder withInput(DataStream input) { @@ -66,6 +69,6 @@ private DataStreamSink buildForBucketAware() { .orElse(null); DataStream partitioned = partition(input, new BucketsRowChannelComputer(), parallelism); - return new CompactorSink(table).sinkFrom(partitioned); + return new CompactorSink(table, fullCompaction).sinkFrom(partitioned); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java index 53b9be457c3d8..b31a1af05224f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/DynamicBucketRowWriteOperator.java @@ -22,6 +22,9 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** @@ -32,11 +35,12 @@ public class DynamicBucketRowWriteOperator private static final long serialVersionUID = 1L; - public DynamicBucketRowWriteOperator( + private DynamicBucketRowWriteOperator( + StreamOperatorParameters parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(table, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); } @Override @@ -49,4 +53,30 @@ public void processElement(StreamRecord> element) throws Exception { write.write(element.getValue().f0, element.getValue().f1); } + + /** {@link StreamOperatorFactory} of {@link DynamicBucketRowWriteOperator}. */ + public static class Factory extends TableWriteOperator.Factory> { + + public Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new DynamicBucketRowWriteOperator( + parameters, table, storeSinkWriteProvider, initialCommitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return DynamicBucketRowWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java index 613bf369b0524..402abb4d5aac0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java @@ -21,7 +21,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.table.FileStoreTable; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -43,8 +43,9 @@ public FixedBucketSink( } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new RowDataStoreWriteOperator(table, logSinkFunction, writeProvider, commitUser); + return new RowDataStoreWriteOperator.Factory( + table, logSinkFunction, writeProvider, commitUser); } } 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 59f2f4b1035f0..002f5887b5f08 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 @@ -42,10 +42,9 @@ import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.CheckpointConfig; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; import javax.annotation.Nullable; @@ -221,7 +220,7 @@ public DataStream doWrite( + " : " + table.name(), new CommittableTypeInfo(), - createWriteOperator( + createWriteOperatorFactory( createWriteProvider( env.getCheckpointConfig(), isStreaming, @@ -269,11 +268,10 @@ protected DataStreamSink doCommit(DataStream written, String com } Options options = Options.fromMap(table.options()); - OneInputStreamOperator committerOperator = - new CommitterOperator<>( + OneInputStreamOperatorFactory committerOperator = + new CommitterOperatorFactory<>( streamingCheckpointEnabled, true, - options.get(SINK_COMMITTER_OPERATOR_CHAINING), commitUser, createCommitterFactory(), createCommittableStateManager(), @@ -281,8 +279,9 @@ protected DataStreamSink doCommit(DataStream written, String com if (options.get(SINK_AUTO_TAG_FOR_SAVEPOINT)) { committerOperator = - new AutoTagForSavepointCommitterOperator<>( - (CommitterOperator) committerOperator, + new AutoTagForSavepointCommitterOperatorFactory<>( + (CommitterOperatorFactory) + committerOperator, table::snapshotManager, table::tagManager, () -> table.store().newTagDeletion(), @@ -292,8 +291,9 @@ protected DataStreamSink doCommit(DataStream written, String com if (conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.BATCH && table.coreOptions().tagCreationMode() == TagCreationMode.BATCH) { committerOperator = - new BatchWriteGeneratorTagOperator<>( - (CommitterOperator) committerOperator, + new BatchWriteGeneratorTagOperatorFactory<>( + (CommitterOperatorFactory) + committerOperator, table); } SingleOutputStreamOperator committed = @@ -311,9 +311,12 @@ protected DataStreamSink doCommit(DataStream written, String com table.name(), options.get(SINK_OPERATOR_UID_SUFFIX))); } + if (!options.get(SINK_COMMITTER_OPERATOR_CHAINING)) { + committed = committed.startNewChain(); + } configureGlobalCommitter( committed, options.get(SINK_COMMITTER_CPU), options.get(SINK_COMMITTER_MEMORY)); - return committed.addSink(new DiscardingSink<>()).name("end").setParallelism(1); + return committed.sinkTo(new DiscardingSink<>()).name("end").setParallelism(1); } public static void configureGlobalCommitter( @@ -338,13 +341,11 @@ public static void assertStreamingConfiguration(StreamExecutionEnvironment env) checkArgument( !env.getCheckpointConfig().isUnalignedCheckpointsEnabled(), "Paimon sink currently does not support unaligned checkpoints. Please set " - + ExecutionCheckpointingOptions.ENABLE_UNALIGNED.key() - + " to false."); + + "execution.checkpointing.unaligned.enabled to false."); checkArgument( env.getCheckpointConfig().getCheckpointingMode() == CheckpointingMode.EXACTLY_ONCE, "Paimon sink currently only supports EXACTLY_ONCE checkpoint mode. Please set " - + ExecutionCheckpointingOptions.CHECKPOINTING_MODE.key() - + " to exactly-once"); + + "execution.checkpointing.mode to exactly-once"); } public static void assertBatchAdaptiveParallelism( @@ -365,7 +366,7 @@ public static void assertBatchAdaptiveParallelism( } } - protected abstract OneInputStreamOperator createWriteOperator( + protected abstract OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser); protected abstract Committer.Factory createCommitterFactory(); 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 dcccd0a1a988b..ecaa5678dd0b1 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 @@ -222,7 +222,7 @@ public DataStreamSink build() { .transform( "local merge", input.getType(), - new LocalMergeOperator(table.schema())) + new LocalMergeOperator.Factory(table.schema())) .setParallelism(input.getParallelism()); } @@ -265,6 +265,16 @@ protected DataStreamSink buildDynamicBucketSink( } protected DataStreamSink buildForFixedBucket(DataStream input) { + int bucketNums = table.bucketSpec().getNumBuckets(); + if (parallelism == null + && bucketNums < input.getParallelism() + && table.partitionKeys().isEmpty()) { + // For non-partitioned table, if the bucketNums is less than job parallelism. + LOG.warn( + "For non-partitioned table, if bucketNums is less than the parallelism of inputOperator," + + " then the parallelism of writerOperator will be set to bucketNums."); + parallelism = bucketNums; + } DataStream partitioned = partition( input, 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 70fac7a83e93c..0c101c6d1e01f 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 @@ -19,6 +19,7 @@ package org.apache.paimon.flink.sink; import org.apache.paimon.flink.ProcessRecordAttributesUtil; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.index.BucketAssigner; import org.apache.paimon.index.HashBucketAssigner; import org.apache.paimon.index.SimpleHashBucketAssigner; @@ -76,8 +77,8 @@ public void initializeState(StateInitializationContext context) throws Exception StateUtils.getSingleValueFromState( context, "commit_user_state", String.class, initialCommitUser); - int numberTasks = getRuntimeContext().getNumberOfParallelSubtasks(); - int taskId = getRuntimeContext().getIndexOfThisSubtask(); + int numberTasks = RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()); + int taskId = RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); long targetRowNum = table.coreOptions().dynamicBucketTargetRowNum(); this.assigner = overwrite 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 6931fe9072180..070262147643f 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 @@ -44,10 +44,15 @@ import org.apache.paimon.utils.UserDefinedSeqComparator; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -76,13 +81,14 @@ public class LocalMergeOperator extends AbstractStreamOperator private transient boolean endOfInput; - public LocalMergeOperator(TableSchema schema) { + private LocalMergeOperator( + StreamOperatorParameters parameters, TableSchema schema) { Preconditions.checkArgument( schema.primaryKeys().size() > 0, "LocalMergeOperator currently only support tables with primary keys"); this.schema = schema; this.ignoreDelete = CoreOptions.fromMap(schema.options()).ignoreDelete(); - setChainingStrategy(ChainingStrategy.ALWAYS); + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); } @Override @@ -235,4 +241,28 @@ LocalMerger merger() { void setOutput(Output> output) { this.output = output; } + + /** {@link StreamOperatorFactory} of {@link LocalMergeOperator}. */ + public static class Factory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final TableSchema schema; + + public Factory(TableSchema schema) { + this.chainingStrategy = ChainingStrategy.ALWAYS; + this.schema = schema; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) new LocalMergeOperator(parameters, schema); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return LocalMergeOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java index f82f082098671..7da0ae0e20788 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCommittableTypeInfo.java @@ -21,6 +21,7 @@ import org.apache.paimon.table.sink.CommitMessageSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -57,7 +58,16 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { // no copy, so that data from writer is directly going into committer while chaining return new NoneCopyVersionedSerializerTypeSerializerProxy( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java index f27f29f87fe74..0116ff198811e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTableCompactionTaskTypeInfo.java @@ -23,6 +23,7 @@ import org.apache.paimon.table.sink.MultiTableCompactionTaskSerializer; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.io.SimpleVersionedSerializerTypeSerializerProxy; @@ -60,7 +61,17 @@ public boolean isKeyType() { return false; } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer( + SerializerConfig serializerConfig) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer( ExecutionConfig executionConfig) { return new SimpleVersionedSerializerTypeSerializerProxy< 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 7cb5d30c2f8ed..58f6a3834096c 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 @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; import org.apache.paimon.options.Options; @@ -32,6 +33,9 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.environment.CheckpointConfig; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; @@ -62,6 +66,7 @@ public class MultiTablesStoreCompactOperator private final CheckpointConfig checkpointConfig; private final boolean isStreaming; private final boolean ignorePreviousFiles; + private final boolean fullCompaction; private final String initialCommitUser; private transient StoreSinkWriteState state; @@ -74,19 +79,22 @@ public class MultiTablesStoreCompactOperator protected Map writes; protected String commitUser; - public MultiTablesStoreCompactOperator( + private MultiTablesStoreCompactOperator( + StreamOperatorParameters parameters, Catalog.Loader catalogLoader, String initialCommitUser, CheckpointConfig checkpointConfig, boolean isStreaming, boolean ignorePreviousFiles, + boolean fullCompaction, Options options) { - super(options); + super(parameters, options); this.catalogLoader = catalogLoader; this.initialCommitUser = initialCommitUser; this.checkpointConfig = checkpointConfig; this.isStreaming = isStreaming; this.ignorePreviousFiles = ignorePreviousFiles; + this.fullCompaction = fullCompaction; } @Override @@ -109,8 +117,10 @@ public void initializeState(StateInitializationContext context) throws Exception ChannelComputer.select( partition, bucket, - getRuntimeContext().getNumberOfParallelSubtasks()) - == getRuntimeContext().getIndexOfThisSubtask()); + RuntimeContextUtils.getNumberOfParallelSubtasks( + getRuntimeContext())) + == RuntimeContextUtils.getIndexOfThisSubtask( + getRuntimeContext())); tables = new HashMap<>(); writes = new HashMap<>(); @@ -159,13 +169,14 @@ public void processElement(StreamRecord element) throws Exception { if (write.streamingMode()) { write.notifyNewFiles(snapshotId, partition, bucket, files); + // The full compact is not supported in streaming mode. write.compact(partition, bucket, false); } else { Preconditions.checkArgument( files.isEmpty(), "Batch compact job does not concern what files are compacted. " + "They only need to know what buckets are compacted."); - write.compact(partition, bucket, true); + write.compact(partition, bucket, fullCompaction); } } @@ -309,4 +320,54 @@ private StoreSinkWrite.Provider createWriteProvider( memoryPool, metricGroup); } + + /** {@link StreamOperatorFactory} of {@link MultiTablesStoreCompactOperator}. */ + public static class Factory + extends PrepareCommitOperator.Factory { + private final Catalog.Loader catalogLoader; + private final CheckpointConfig checkpointConfig; + private final boolean isStreaming; + private final boolean ignorePreviousFiles; + private final boolean fullCompaction; + private final String initialCommitUser; + + public Factory( + Catalog.Loader catalogLoader, + String initialCommitUser, + CheckpointConfig checkpointConfig, + boolean isStreaming, + boolean ignorePreviousFiles, + boolean fullCompaction, + Options options) { + super(options); + this.catalogLoader = catalogLoader; + this.initialCommitUser = initialCommitUser; + this.checkpointConfig = checkpointConfig; + this.isStreaming = isStreaming; + this.ignorePreviousFiles = ignorePreviousFiles; + this.fullCompaction = fullCompaction; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new MultiTablesStoreCompactOperator( + parameters, + catalogLoader, + initialCommitUser, + checkpointConfig, + isStreaming, + ignorePreviousFiles, + fullCompaction, + options); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return MultiTablesStoreCompactOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java index 3668386ddc2db..8b114d3e492ff 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java @@ -26,10 +26,14 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTask; @@ -52,9 +56,9 @@ public abstract class PrepareCommitOperator extends AbstractStreamOpera private final Options options; private boolean endOfInput = false; - public PrepareCommitOperator(Options options) { + public PrepareCommitOperator(StreamOperatorParameters parameters, Options options) { this.options = options; - setChainingStrategy(ChainingStrategy.ALWAYS); + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); } @Override @@ -103,4 +107,15 @@ private void emitCommittables(boolean waitCompaction, long checkpointId) throws protected abstract List prepareCommit(boolean waitCompaction, long checkpointId) throws IOException; + + /** {@link StreamOperatorFactory} of {@link PrepareCommitOperator}. */ + protected abstract static class Factory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + protected final Options options; + + protected Factory(Options options) { + this.options = options; + this.chainingStrategy = ChainingStrategy.ALWAYS; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java index 39dcca03c6aab..d9f863c6b9195 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java @@ -45,11 +45,10 @@ import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.Pair; -import org.apache.flink.streaming.api.graph.StreamConfig; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.api.operators.Output; +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.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import javax.annotation.Nullable; @@ -76,34 +75,49 @@ public RewriteFileIndexSink(FileStoreTable table) { } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new FileIndexModificationOperator(table.coreOptions().toConfiguration(), table); + return new FileIndexModificationOperatorFactory( + table.coreOptions().toConfiguration(), table); } - /** File index modification operator to rewrite file index. */ - private static class FileIndexModificationOperator - extends PrepareCommitOperator { - - private static final long serialVersionUID = 1L; - + private static class FileIndexModificationOperatorFactory + extends PrepareCommitOperator.Factory { private final FileStoreTable table; - private transient FileIndexProcessor fileIndexProcessor; - private transient List messages; - - public FileIndexModificationOperator(Options options, FileStoreTable table) { + public FileIndexModificationOperatorFactory(Options options, FileStoreTable table) { super(options); this.table = table; } @Override - public void setup( - StreamTask containingTask, - StreamConfig config, - Output> output) { - super.setup(containingTask, config, output); + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) new FileIndexModificationOperator(parameters, options, table); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return FileIndexModificationOperator.class; + } + } + + /** File index modification operator to rewrite file index. */ + private static class FileIndexModificationOperator + extends PrepareCommitOperator { + + private static final long serialVersionUID = 1L; + + private final transient FileIndexProcessor fileIndexProcessor; + private final transient List messages; + private FileIndexModificationOperator( + StreamOperatorParameters parameters, + Options options, + FileStoreTable table) { + super(parameters, options); this.fileIndexProcessor = new FileIndexProcessor(table); this.messages = new ArrayList<>(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java index 07fe275543a18..8009bec9677f7 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDataStoreWriteOperator.java @@ -23,6 +23,8 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.SinkRecord; +import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.state.CheckpointListener; @@ -30,18 +32,20 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.api.graph.StreamConfig; import org.apache.flink.streaming.api.operators.InternalTimerService; -import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.streaming.util.functions.StreamingFunctionUtils; import javax.annotation.Nullable; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.util.List; import java.util.Objects; @@ -57,21 +61,14 @@ public class RowDataStoreWriteOperator extends TableWriteOperator { /** We listen to this ourselves because we don't have an {@link InternalTimerService}. */ private long currentWatermark = Long.MIN_VALUE; - public RowDataStoreWriteOperator( + protected RowDataStoreWriteOperator( + StreamOperatorParameters parameters, FileStoreTable table, @Nullable LogSinkFunction logSinkFunction, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(table, storeSinkWriteProvider, initialCommitUser); + super(parameters, table, storeSinkWriteProvider, initialCommitUser); this.logSinkFunction = logSinkFunction; - } - - @Override - public void setup( - StreamTask containingTask, - StreamConfig config, - Output> output) { - super.setup(containingTask, config, output); if (logSinkFunction != null) { FunctionUtils.setFunctionRuntimeContext(logSinkFunction, getRuntimeContext()); } @@ -97,17 +94,29 @@ public void open() throws Exception { this.sinkContext = new SimpleContext(getProcessingTimeService()); if (logSinkFunction != null) { - // to stay compatible with Flink 1.18- - if (logSinkFunction instanceof RichFunction) { - RichFunction richFunction = (RichFunction) logSinkFunction; - richFunction.open(new Configuration()); - } - + openFunction(logSinkFunction); logCallback = new LogWriteCallback(); logSinkFunction.setWriteCallback(logCallback); } } + private static void openFunction(Function function) throws Exception { + if (function instanceof RichFunction) { + RichFunction richFunction = (RichFunction) function; + + try { + Method method = RichFunction.class.getDeclaredMethod("open", OpenContext.class); + method.invoke(richFunction, new OpenContext() {}); + return; + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + // to stay compatible with Flink 1.18- + } + + Method method = RichFunction.class.getDeclaredMethod("open", Configuration.class); + method.invoke(richFunction, new Configuration()); + } + } + @Override public void processWatermark(Watermark mark) throws Exception { super.processWatermark(mark); @@ -233,4 +242,38 @@ public Long timestamp() { return timestamp; } } + + /** {@link StreamOperatorFactory} of {@link RowDataStoreWriteOperator}. */ + public static class Factory extends TableWriteOperator.Factory { + + @Nullable private final LogSinkFunction logSinkFunction; + + public Factory( + FileStoreTable table, + @Nullable LogSinkFunction logSinkFunction, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(table, storeSinkWriteProvider, initialCommitUser); + this.logSinkFunction = logSinkFunction; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new RowDataStoreWriteOperator( + parameters, + table, + logSinkFunction, + storeSinkWriteProvider, + initialCommitUser); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return RowDataStoreWriteOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java index bf6c70f0aa295..1f7e62d74916e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java @@ -27,7 +27,7 @@ import org.apache.paimon.utils.SerializableFunction; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -60,8 +60,8 @@ protected ChannelComputer> channelComputer2() { } @Override - protected OneInputStreamOperator, Committable> createWriteOperator( - StoreSinkWrite.Provider writeProvider, String commitUser) { - return new DynamicBucketRowWriteOperator(table, writeProvider, commitUser); + protected OneInputStreamOperatorFactory, Committable> + createWriteOperatorFactory(StoreSinkWrite.Provider writeProvider, String commitUser) { + return new DynamicBucketRowWriteOperator.Factory(table, writeProvider, commitUser); } } 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 1cd10390c1a0a..fea8a382a954c 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 @@ -22,7 +22,9 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +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 java.util.Map; @@ -38,25 +40,35 @@ public RowUnawareBucketSink( } @Override - protected OneInputStreamOperator createWriteOperator( + protected OneInputStreamOperatorFactory createWriteOperatorFactory( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new RowDataStoreWriteOperator(table, logSinkFunction, writeProvider, commitUser) { - + return new RowDataStoreWriteOperator.Factory( + 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); - } + public StreamOperator createStreamOperator(StreamOperatorParameters parameters) { + return new RowDataStoreWriteOperator( + parameters, table, logSinkFunction, writeProvider, commitUser) { - @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; + @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 d237f4da56cf7..4908b99317bae 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 @@ -23,6 +23,7 @@ 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.BucketMode; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.CommitMessageImpl; @@ -44,6 +45,7 @@ public class StoreCommitter implements Committer committables) { if (committerMetrics == null) { return; 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 bc7bb350df217..1870a0493c2f1 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 @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.io.DataFileMeta; import org.apache.paimon.io.DataFileMetaSerializer; import org.apache.paimon.options.Options; @@ -30,6 +31,9 @@ import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; @@ -52,23 +56,27 @@ public class StoreCompactOperator extends PrepareCommitOperator> waitToCompact; - public StoreCompactOperator( + private StoreCompactOperator( + StreamOperatorParameters parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, - String initialCommitUser) { - super(Options.fromMap(table.options())); + String initialCommitUser, + boolean fullCompaction) { + super(parameters, Options.fromMap(table.options())); Preconditions.checkArgument( !table.coreOptions().writeOnly(), CoreOptions.WRITE_ONLY.key() + " should not be true for StoreCompactOperator."); this.table = table; this.storeSinkWriteProvider = storeSinkWriteProvider; this.initialCommitUser = initialCommitUser; + this.fullCompaction = fullCompaction; } @Override @@ -89,8 +97,10 @@ public void initializeState(StateInitializationContext context) throws Exception ChannelComputer.select( partition, bucket, - getRuntimeContext().getNumberOfParallelSubtasks()) - == getRuntimeContext().getIndexOfThisSubtask()); + RuntimeContextUtils.getNumberOfParallelSubtasks( + getRuntimeContext())) + == RuntimeContextUtils.getIndexOfThisSubtask( + getRuntimeContext())); write = storeSinkWriteProvider.provide( table, @@ -136,10 +146,7 @@ protected List prepareCommit(boolean waitCompaction, long checkpoin try { for (Pair partitionBucket : waitToCompact) { - write.compact( - partitionBucket.getKey(), - partitionBucket.getRight(), - !write.streamingMode()); + write.compact(partitionBucket.getKey(), partitionBucket.getRight(), fullCompaction); } } catch (Exception e) { throw new RuntimeException("Exception happens while executing compaction.", e); @@ -160,4 +167,46 @@ public void close() throws Exception { super.close(); write.close(); } + + /** {@link StreamOperatorFactory} of {@link StoreCompactOperator}. */ + public static class Factory extends PrepareCommitOperator.Factory { + private final FileStoreTable table; + private final StoreSinkWrite.Provider storeSinkWriteProvider; + private final String initialCommitUser; + private final boolean fullCompaction; + + public Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser, + boolean fullCompaction) { + super(Options.fromMap(table.options())); + Preconditions.checkArgument( + !table.coreOptions().writeOnly(), + CoreOptions.WRITE_ONLY.key() + " should not be true for StoreCompactOperator."); + this.table = table; + this.storeSinkWriteProvider = storeSinkWriteProvider; + this.initialCommitUser = initialCommitUser; + this.fullCompaction = fullCompaction; + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new StoreCompactOperator( + parameters, + table, + storeSinkWriteProvider, + initialCommitUser, + fullCompaction); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return StoreCompactOperator.class; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java index aeb3e1857b9b7..537a98f97fb03 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java @@ -92,11 +92,11 @@ public WrappedManifestCommittable combine( WrappedManifestCommittable wrappedManifestCommittable, List committables) { for (MultiTableCommittable committable : committables) { + Identifier identifier = + Identifier.create(committable.getDatabase(), committable.getTable()); ManifestCommittable manifestCommittable = wrappedManifestCommittable.computeCommittableIfAbsent( - Identifier.create(committable.getDatabase(), committable.getTable()), - checkpointId, - watermark); + identifier, checkpointId, watermark); switch (committable.kind()) { case FILE: @@ -106,7 +106,9 @@ public WrappedManifestCommittable combine( case LOG_OFFSET: LogOffsetCommittable offset = (LogOffsetCommittable) committable.wrappedCommittable(); - manifestCommittable.addLogOffset(offset.bucket(), offset.offset()); + StoreCommitter committer = tableCommitters.get(identifier); + manifestCommittable.addLogOffset( + offset.bucket(), offset.offset(), committer.allowLogOffsetDuplicate()); break; } } 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 67b4720e29645..fd876698c094a 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 @@ -21,12 +21,15 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.flink.ProcessRecordAttributesUtil; import org.apache.paimon.flink.sink.StoreSinkWriteState.StateValueFilter; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.ChannelComputer; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import java.io.IOException; @@ -44,10 +47,11 @@ public abstract class TableWriteOperator extends PrepareCommitOperator parameters, FileStoreTable table, StoreSinkWrite.Provider storeSinkWriteProvider, String initialCommitUser) { - super(Options.fromMap(table.options())); + super(parameters, Options.fromMap(table.options())); this.table = table; this.storeSinkWriteProvider = storeSinkWriteProvider; this.initialCommitUser = initialCommitUser; @@ -58,14 +62,14 @@ public void initializeState(StateInitializationContext context) throws Exception super.initializeState(context); boolean containLogSystem = containLogSystem(); - int numTasks = getRuntimeContext().getNumberOfParallelSubtasks(); + int numTasks = RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()); StateValueFilter stateFilter = (tableName, partition, bucket) -> { int task = containLogSystem ? ChannelComputer.select(bucket, numTasks) : ChannelComputer.select(partition, bucket, numTasks); - return task == getRuntimeContext().getIndexOfThisSubtask(); + return task == RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()); }; state = createState(context, stateFilter); @@ -127,4 +131,22 @@ protected List prepareCommit(boolean waitCompaction, long checkpoin public StoreSinkWrite getWrite() { return write; } + + /** {@link StreamOperatorFactory} of {@link TableWriteOperator}. */ + protected abstract static class Factory + extends PrepareCommitOperator.Factory { + protected final FileStoreTable table; + protected final StoreSinkWrite.Provider storeSinkWriteProvider; + protected final String initialCommitUser; + + protected Factory( + FileStoreTable table, + StoreSinkWrite.Provider storeSinkWriteProvider, + String initialCommitUser) { + super(Options.fromMap(table.options())); + this.table = table; + this.storeSinkWriteProvider = storeSinkWriteProvider; + this.initialCommitUser = initialCommitUser; + } + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java index da966d5e51566..7a4095f896cc3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java @@ -24,7 +24,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; /** Compaction Sink for unaware-bucket table. */ public class UnawareBucketCompactionSink extends FlinkSink { @@ -42,9 +42,9 @@ public static DataStreamSink sink( } @Override - protected OneInputStreamOperator createWriteOperator( - StoreSinkWrite.Provider writeProvider, String commitUser) { - return new AppendOnlySingleTableCompactionWorkerOperator(table, commitUser); + protected OneInputStreamOperatorFactory + createWriteOperatorFactory(StoreSinkWrite.Provider writeProvider, String commitUser) { + return new AppendOnlySingleTableCompactionWorkerOperator.Factory(table, commitUser); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java index 98b58aa8e96d3..7bc40d4c2080b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketSink.java @@ -74,11 +74,14 @@ public DataStream doWrite( new CommittableTypeInfo(), new CompactionTaskTypeInfo()), new AppendBypassCoordinateOperatorFactory<>(table)) + .startNewChain() .forceNonParallel() .transform( "Compact Worker: " + table.name(), new CommittableTypeInfo(), - new AppendBypassCompactWorkerOperator(table, initialCommitUser)) + new AppendBypassCompactWorkerOperator.Factory( + table, initialCommitUser)) + .startNewChain() .setParallelism(written.getParallelism()); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java index 26e080c32e834..7022002a43ba3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java @@ -39,7 +39,7 @@ 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.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import javax.annotation.Nullable; @@ -63,9 +63,9 @@ public GlobalDynamicBucketSink( } @Override - protected OneInputStreamOperator, Committable> createWriteOperator( - StoreSinkWrite.Provider writeProvider, String commitUser) { - return new DynamicBucketRowWriteOperator(table, writeProvider, commitUser); + protected OneInputStreamOperatorFactory, Committable> + createWriteOperatorFactory(StoreSinkWrite.Provider writeProvider, String commitUser) { + return new DynamicBucketRowWriteOperator.Factory(table, writeProvider, commitUser); } public DataStreamSink build(DataStream input, @Nullable Integer parallelism) { @@ -89,7 +89,8 @@ public DataStreamSink build(DataStream input, @Nullable Integer new InternalTypeInfo<>( new KeyWithRowSerializer<>( bootstrapSerializer, rowSerializer)), - new IndexBootstrapOperator<>(new IndexBootstrap(table), r -> r)) + new IndexBootstrapOperator.Factory<>( + new IndexBootstrap(table), r -> r)) .setParallelism(input.getParallelism()); // 1. shuffle by key hash diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java index 7fee3f45f3db7..99cce07fdc574 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalIndexAssignerOperator.java @@ -22,6 +22,7 @@ import org.apache.paimon.crosspartition.KeyPartOrRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.table.Table; import org.apache.flink.api.java.tuple.Tuple2; @@ -59,8 +60,8 @@ public void initializeState(StateInitializationContext context) throws Exception assigner.open( computeManagedMemory(this), ioManager, - getRuntimeContext().getNumberOfParallelSubtasks(), - getRuntimeContext().getIndexOfThisSubtask(), + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), this::collect); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java index 501e35dff46cb..8136565f98cf8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/IndexBootstrapOperator.java @@ -21,13 +21,19 @@ import org.apache.paimon.crosspartition.IndexBootstrap; import org.apache.paimon.crosspartition.KeyPartOrRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.utils.SerializableFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; /** Operator for {@link IndexBootstrap}. */ @@ -39,19 +45,21 @@ public class IndexBootstrapOperator extends AbstractStreamOperator converter; - public IndexBootstrapOperator( - IndexBootstrap bootstrap, SerializableFunction converter) { + private IndexBootstrapOperator( + StreamOperatorParameters> parameters, + IndexBootstrap bootstrap, + SerializableFunction converter) { this.bootstrap = bootstrap; this.converter = converter; - setChainingStrategy(ChainingStrategy.ALWAYS); + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); } @Override public void initializeState(StateInitializationContext context) throws Exception { super.initializeState(context); bootstrap.bootstrap( - getRuntimeContext().getNumberOfParallelSubtasks(), - getRuntimeContext().getIndexOfThisSubtask(), + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()), + RuntimeContextUtils.getIndexOfThisSubtask(getRuntimeContext()), this::collect); } @@ -64,4 +72,30 @@ private void collect(InternalRow row) { output.collect( new StreamRecord<>(new Tuple2<>(KeyPartOrRow.KEY_PART, converter.apply(row)))); } + + /** {@link StreamOperatorFactory} of {@link IndexBootstrapOperator}. */ + public static class Factory extends AbstractStreamOperatorFactory> + implements OneInputStreamOperatorFactory> { + private final IndexBootstrap bootstrap; + private final SerializableFunction converter; + + public Factory(IndexBootstrap bootstrap, SerializableFunction converter) { + this.chainingStrategy = ChainingStrategy.ALWAYS; + this.bootstrap = bootstrap; + this.converter = converter; + } + + @Override + @SuppressWarnings("unchecked") + public >> OP createStreamOperator( + StreamOperatorParameters> parameters) { + return (OP) new IndexBootstrapOperator<>(parameters, bootstrap, converter); + } + + @Override + @SuppressWarnings("rawtypes") + public Class getStreamOperatorClass(ClassLoader classLoader) { + return IndexBootstrapOperator.class; + } + } } 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 index eb965aa3a3188..853dc52c20bfc 100644 --- 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 @@ -93,7 +93,7 @@ public void report(String partition, long modifyTime) throws Exception { statistic.put(HIVE_LAST_UPDATE_TIME_PROP, String.valueOf(modifyTime / 1000)); LOG.info("alter partition {} with statistic {}.", partitionSpec, statistic); - metastoreClient.alterPartition(partitionSpec, statistic, modifyTime); + metastoreClient.alterPartition(partitionSpec, statistic, modifyTime, true); } } 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 d4d5dd7416816..b6847125fbc66 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 @@ -23,6 +23,7 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.options.MemorySize; import org.apache.paimon.sort.BinaryExternalSortBuffer; import org.apache.paimon.types.RowType; @@ -79,7 +80,8 @@ public SortOperator( public void open() throws Exception { super.open(); initBuffer(); - if (sinkParallelism != getRuntimeContext().getNumberOfParallelSubtasks()) { + if (sinkParallelism + != RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext())) { throw new IllegalArgumentException( "Please ensure that the runtime parallelism of the sink matches the initial configuration " + "to avoid potential issues with skewed range partitioning."); 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 f590c2fb7fff0..b30e145512966 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 @@ -31,6 +31,7 @@ import org.apache.paimon.utils.KeyProjectedRow; import org.apache.paimon.utils.SerializableSupplier; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; @@ -119,9 +120,19 @@ public static DataStream sortStreamByKey( .map( new RichMapFunction>() { - @Override + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) throws Exception { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 2.0+. + */ public void open(Configuration parameters) throws Exception { - super.open(parameters); shuffleKeyAbstract.open(); } @@ -172,7 +183,18 @@ public Tuple2 map(RowData value) { private transient KeyProjectedRow keyProjectedRow; - @Override + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 1.18-. + */ + public void open(OpenContext openContext) { + open(new Configuration()); + } + + /** + * Do not annotate with @override here to maintain + * compatibility with Flink 2.0+. + */ public void open(Configuration parameters) { keyProjectedRow = new KeyProjectedRow(valueProjectionMap); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java new file mode 100644 index 0000000000000..a9a389e837a21 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSource.java @@ -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.flink.source; + +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +/** {@link Source} that does not require coordination between JobManager and TaskManagers. */ +public abstract class AbstractNonCoordinatedSource + implements Source { + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new NoOpEnumerator<>(); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, NoOpEnumState checkpoint) { + return new NoOpEnumerator<>(); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new SimpleSourceSplitSerializer(); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new NoOpEnumStateSerializer(); + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntityImpl.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java similarity index 52% rename from paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntityImpl.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java index ef11ee87f15c9..18c278868ffae 100644 --- a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntityImpl.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/AbstractNonCoordinatedSourceReader.java @@ -16,41 +16,36 @@ * limitations under the License. */ -package org.apache.paimon.lineage; - -import org.apache.paimon.data.Timestamp; - -/** Default implementation for {@link TableLineageEntity}. */ -public class TableLineageEntityImpl implements TableLineageEntity { - private final String database; - private final String table; - private final String job; - private final Timestamp timestamp; - - public TableLineageEntityImpl(String database, String table, String job, Timestamp timestamp) { - this.database = database; - this.table = table; - this.job = job; - this.timestamp = timestamp; - } +package org.apache.paimon.flink.source; + +import org.apache.flink.api.connector.source.SourceReader; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +/** Abstract {@link SourceReader} for {@link AbstractNonCoordinatedSource}. */ +public abstract class AbstractNonCoordinatedSourceReader + implements SourceReader { @Override - public String getDatabase() { - return database; - } + public void start() {} @Override - public String getTable() { - return table; + public List snapshotState(long l) { + return Collections.emptyList(); } @Override - public String getJob() { - return job; + public CompletableFuture isAvailable() { + return CompletableFuture.completedFuture(null); } @Override - public Timestamp getCreateTime() { - return timestamp; - } + public void addSplits(List list) {} + + @Override + public void notifyNoMoreSplits() {} + + @Override + public void close() throws Exception {} } 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 668aa24c145d7..b8b0d61e10a96 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 @@ -20,13 +20,14 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; +import org.apache.paimon.flink.utils.RuntimeContextUtils; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.utils.ExecutorUtils; 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.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.types.Either; @@ -57,17 +58,19 @@ public class AppendBypassCoordinateOperator private transient LinkedBlockingQueue compactTasks; public AppendBypassCoordinateOperator( - FileStoreTable table, ProcessingTimeService processingTimeService) { + StreamOperatorParameters> parameters, + FileStoreTable table, + ProcessingTimeService processingTimeService) { this.table = table; this.processingTimeService = processingTimeService; - this.chainingStrategy = ChainingStrategy.HEAD; + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); } @Override public void open() throws Exception { super.open(); checkArgument( - getRuntimeContext().getNumberOfParallelSubtasks() == 1, + RuntimeContextUtils.getNumberOfParallelSubtasks(getRuntimeContext()) == 1, "Compaction Coordinator parallelism in paimon MUST be one."); long intervalMs = table.coreOptions().continuousDiscoveryInterval().toMillis(); this.compactTasks = new LinkedBlockingQueue<>(); 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 7c53e01b47e65..a4c51e5b5a9be 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 @@ -45,11 +45,7 @@ T createStreamOperator( StreamOperatorParameters> parameters) { AppendBypassCoordinateOperator operator = - new AppendBypassCoordinateOperator<>(table, processingTimeService); - operator.setup( - parameters.getContainingTask(), - parameters.getStreamConfig(), - parameters.getOutput()); + new AppendBypassCoordinateOperator<>(parameters, table, processingTimeService); return (T) operator; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java index d306c7d8e1e5f..7954aad2df0a4 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/BucketUnawareCompactSource.java @@ -24,14 +24,16 @@ import org.apache.paimon.predicate.Predicate; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.source.EndOfScanException; -import org.apache.paimon.utils.Preconditions; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,15 +42,16 @@ import java.util.List; /** - * Source Function for unaware-bucket Compaction. + * Source for unaware-bucket Compaction. * - *

    Note: The function is the source function of unaware-bucket compactor coordinator. It will - * read the latest snapshot continuously by compactionCoordinator, and generate new compaction - * tasks. The source function is used in unaware-bucket compaction job (both stand-alone and - * write-combined). Besides, we don't need to save state in this function, it will invoke a full - * scan when starting up, and scan continuously for the following snapshot. + *

    Note: The function is the source of unaware-bucket compactor coordinator. It will read the + * latest snapshot continuously by compactionCoordinator, and generate new compaction tasks. The + * source is used in unaware-bucket compaction job (both stand-alone and write-combined). Besides, + * we don't need to save state in this source, it will invoke a full scan when starting up, and scan + * continuously for the following snapshot. */ -public class BucketUnawareCompactSource extends RichSourceFunction { +public class BucketUnawareCompactSource + extends AbstractNonCoordinatedSource { private static final Logger LOG = LoggerFactory.getLogger(BucketUnawareCompactSource.class); private static final String COMPACTION_COORDINATOR_NAME = "Compaction Coordinator"; @@ -57,9 +60,6 @@ public class BucketUnawareCompactSource extends RichSourceFunction ctx; - private volatile boolean isRunning = true; public BucketUnawareCompactSource( FileStoreTable table, @@ -73,66 +73,63 @@ public BucketUnawareCompactSource( } @Override - public void open(Configuration parameters) throws Exception { - compactionCoordinator = - new UnawareAppendTableCompactionCoordinator(table, streaming, filter); + public Boundedness getBoundedness() { + return streaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader( + SourceReaderContext readerContext) throws Exception { Preconditions.checkArgument( - this.getRuntimeContext().getNumberOfParallelSubtasks() == 1, + readerContext.currentParallelism() == 1, "Compaction Operator parallelism in paimon MUST be one."); + return new BucketUnawareCompactSourceReader(table, streaming, filter, scanInterval); } - @Override - public void run(SourceContext sourceContext) throws Exception { - this.ctx = sourceContext; - while (isRunning) { + /** BucketUnawareCompactSourceReader. */ + public static class BucketUnawareCompactSourceReader + extends AbstractNonCoordinatedSourceReader { + private final UnawareAppendTableCompactionCoordinator compactionCoordinator; + private final long scanInterval; + + public BucketUnawareCompactSourceReader( + FileStoreTable table, boolean streaming, Predicate filter, long scanInterval) { + this.scanInterval = scanInterval; + compactionCoordinator = + new UnawareAppendTableCompactionCoordinator(table, streaming, filter); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) + throws Exception { boolean isEmpty; - synchronized (ctx.getCheckpointLock()) { - if (!isRunning) { - return; - } - try { - // do scan and plan action, emit append-only compaction tasks. - List tasks = compactionCoordinator.run(); - isEmpty = tasks.isEmpty(); - tasks.forEach(ctx::collect); - } catch (EndOfScanException esf) { - LOG.info("Catching EndOfStreamException, the stream is finished."); - return; - } + try { + // do scan and plan action, emit append-only compaction tasks. + List tasks = compactionCoordinator.run(); + isEmpty = tasks.isEmpty(); + tasks.forEach(readerOutput::collect); + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return InputStatus.END_OF_INPUT; } if (isEmpty) { Thread.sleep(scanInterval); } - } - } - - @Override - public void cancel() { - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning = false; - } - } else { - isRunning = false; + return InputStatus.MORE_AVAILABLE; } } public static DataStreamSource buildSource( StreamExecutionEnvironment env, BucketUnawareCompactSource source, - boolean streaming, String tableIdentifier) { - final StreamSource sourceOperator = - new StreamSource<>(source); return (DataStreamSource) - new DataStreamSource<>( - env, - new CompactionTaskTypeInfo(), - sourceOperator, - false, + env.fromSource( + source, + WatermarkStrategy.noWatermarks(), COMPACTION_COORDINATOR_NAME + " : " + tableIdentifier, - streaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED) + new CompactionTaskTypeInfo()) .setParallelism(1) .setMaxParallelism(1); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java index e5cbbe845ceb0..415eddb037dfe 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/CombinedTableCompactorSourceBuilder.java @@ -21,10 +21,10 @@ import org.apache.paimon.append.MultiTableUnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.LogicalTypeConversion; -import org.apache.paimon.flink.source.operator.CombinedAwareBatchSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedAwareStreamingSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedUnawareBatchSourceFunction; -import org.apache.paimon.flink.source.operator.CombinedUnawareStreamingSourceFunction; +import org.apache.paimon.flink.source.operator.CombinedAwareBatchSource; +import org.apache.paimon.flink.source.operator.CombinedAwareStreamingSource; +import org.apache.paimon.flink.source.operator.CombinedUnawareBatchSource; +import org.apache.paimon.flink.source.operator.CombinedUnawareStreamingSource; import org.apache.paimon.table.system.CompactBucketsTable; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.Preconditions; @@ -87,7 +87,7 @@ public DataStream buildAwareBucketTableSource() { Preconditions.checkArgument(env != null, "StreamExecutionEnvironment should not be null."); RowType produceType = CompactBucketsTable.getRowType(); if (isContinuous) { - return CombinedAwareStreamingSourceFunction.buildSource( + return CombinedAwareStreamingSource.buildSource( env, "Combine-MultiBucketTables--StreamingCompactorSource", InternalTypeInfo.of(LogicalTypeConversion.toLogicalType(produceType)), @@ -97,7 +97,7 @@ public DataStream buildAwareBucketTableSource() { databasePattern, monitorInterval); } else { - return CombinedAwareBatchSourceFunction.buildSource( + return CombinedAwareBatchSource.buildSource( env, "Combine-MultiBucketTables-BatchCompactorSource", InternalTypeInfo.of(LogicalTypeConversion.toLogicalType(produceType)), @@ -112,7 +112,7 @@ public DataStream buildAwareBucketTableSource() { public DataStream buildForUnawareBucketsTableSource() { Preconditions.checkArgument(env != null, "StreamExecutionEnvironment should not be null."); if (isContinuous) { - return CombinedUnawareStreamingSourceFunction.buildSource( + return CombinedUnawareStreamingSource.buildSource( env, "Combined-UnawareBucketTables-StreamingCompactorSource", catalogLoader, @@ -121,7 +121,7 @@ public DataStream buildForUnawareBucketsT databasePattern, monitorInterval); } else { - return CombinedUnawareBatchSourceFunction.buildSource( + return CombinedUnawareBatchSource.buildSource( env, "Combined-UnawareBucketTables-BatchCompactorSource", catalogLoader, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java index 92adf5e04998e..8fc78c868ba57 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FileStoreSourceReader.java @@ -25,9 +25,7 @@ import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; -import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.file.src.reader.BulkFormat.RecordIterator; import org.apache.flink.table.data.RowData; @@ -64,27 +62,6 @@ public FileStoreSourceReader( this.ioManager = ioManager; } - public FileStoreSourceReader( - SourceReaderContext readerContext, - TableRead tableRead, - FileStoreSourceReaderMetrics metrics, - IOManager ioManager, - @Nullable Long limit, - FutureCompletingBlockingQueue>> - elementsQueue) { - super( - elementsQueue, - () -> - new FileStoreSourceSplitReader( - tableRead, RecordLimiter.create(limit), metrics), - (element, output, state) -> - FlinkRecordsWithSplitIds.emitRecord( - readerContext, element, output, state, metrics), - readerContext.getConfiguration(), - readerContext); - this.ioManager = ioManager; - } - @Override public void start() { // we request a split only if we did not get splits during the checkpoint restore 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 ed94043c035d8..b85d5274b241c 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 @@ -26,7 +26,7 @@ import org.apache.paimon.flink.log.LogSourceProvider; import org.apache.paimon.flink.sink.FlinkSink; import org.apache.paimon.flink.source.align.AlignedContinuousFileStoreSource; -import org.apache.paimon.flink.source.operator.MonitorFunction; +import org.apache.paimon.flink.source.operator.MonitorSource; import org.apache.paimon.flink.utils.TableScanUtils; import org.apache.paimon.options.Options; import org.apache.paimon.predicate.Predicate; @@ -46,7 +46,6 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.CheckpointConfig; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; @@ -178,7 +177,7 @@ private ReadBuilder createReadBuilder() { if (limit != null) { readBuilder.withLimit(limit.intValue()); } - return readBuilder; + return readBuilder.dropStats(); } private DataStream buildStaticFileSource() { @@ -259,7 +258,9 @@ public DataStream build() { if (conf.contains(CoreOptions.CONSUMER_ID) && !conf.contains(CoreOptions.CONSUMER_EXPIRATION_TIME)) { throw new IllegalArgumentException( - "consumer.expiration-time should be specified when using consumer-id."); + "You need to configure 'consumer.expiration-time' (ALTER TABLE) and restart your write job for it" + + " to take effect, when you need consumer-id feature. This is to prevent consumers from leaving" + + " too many snapshots that could pose a risk to the file system."); } if (sourceBounded) { @@ -305,7 +306,7 @@ private DataStream buildContinuousStreamOperator() { "Cannot limit streaming source, please use batch execution mode."); } dataStream = - MonitorFunction.buildSource( + MonitorSource.buildSource( env, sourceName, produceTypeInfo(), @@ -329,30 +330,25 @@ private void assertStreamingConfigurationForAlignMode(StreamExecutionEnvironment checkArgument( checkpointConfig.isCheckpointingEnabled(), "The align mode of paimon source is only supported when checkpoint enabled. Please set " - + ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL.key() - + "larger than 0"); + + "execution.checkpointing.interval larger than 0"); checkArgument( checkpointConfig.getMaxConcurrentCheckpoints() == 1, "The align mode of paimon source supports at most one ongoing checkpoint at the same time. Please set " - + ExecutionCheckpointingOptions.MAX_CONCURRENT_CHECKPOINTS.key() - + " to 1"); + + "execution.checkpointing.max-concurrent-checkpoints to 1"); checkArgument( checkpointConfig.getCheckpointTimeout() > conf.get(FlinkConnectorOptions.SOURCE_CHECKPOINT_ALIGN_TIMEOUT) .toMillis(), "The align mode of paimon source requires that the timeout of checkpoint is greater than the timeout of the source's snapshot alignment. Please increase " - + ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT.key() - + " or decrease " + + "execution.checkpointing.timeout or decrease " + FlinkConnectorOptions.SOURCE_CHECKPOINT_ALIGN_TIMEOUT.key()); checkArgument( !env.getCheckpointConfig().isUnalignedCheckpointsEnabled(), "The align mode of paimon source currently does not support unaligned checkpoints. Please set " - + ExecutionCheckpointingOptions.ENABLE_UNALIGNED.key() - + " to false."); + + "execution.checkpointing.unaligned.enabled to false."); checkArgument( env.getCheckpointConfig().getCheckpointingMode() == CheckpointingMode.EXACTLY_ONCE, "The align mode of paimon source currently only supports EXACTLY_ONCE checkpoint mode. Please set " - + ExecutionCheckpointingOptions.CHECKPOINTING_MODE.key() - + " to exactly-once"); + + "execution.checkpointing.mode to exactly-once"); } } 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 2be0248f3ce8c..12b579589d0fe 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 @@ -31,6 +31,7 @@ import org.apache.paimon.table.DataTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.Split; +import org.apache.paimon.table.source.TableScan; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -172,11 +173,7 @@ protected Integer inferSourceParallelism(StreamExecutionEnvironment env) { protected void scanSplitsForInference() { if (splitStatistics == null) { if (table instanceof DataTable) { - List partitionEntries = - table.newReadBuilder() - .withFilter(predicate) - .newScan() - .listPartitionEntries(); + List partitionEntries = newTableScan().listPartitionEntries(); long totalSize = 0; long rowCount = 0; for (PartitionEntry entry : partitionEntries) { @@ -187,8 +184,7 @@ protected void scanSplitsForInference() { splitStatistics = new SplitStatistics((int) (totalSize / splitTargetSize + 1), rowCount); } else { - List splits = - table.newReadBuilder().withFilter(predicate).newScan().plan().splits(); + List splits = newTableScan().plan().splits(); splitStatistics = new SplitStatistics( splits.size(), splits.stream().mapToLong(Split::rowCount).sum()); @@ -196,6 +192,10 @@ protected void scanSplitsForInference() { } } + private TableScan newTableScan() { + return table.newReadBuilder().dropStats().withFilter(predicate).newScan(); + } + /** Split statistics for inferring row count and parallelism size. */ protected static class SplitStatistics { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.java new file mode 100644 index 0000000000000..f07317c155aa5 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumState.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.source; + +/** The enumerator state class for {@link NoOpEnumerator}. */ +public class NoOpEnumState {} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java new file mode 100644 index 0000000000000..89c0ad6ac1f10 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumStateSerializer.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.source; + +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.IOException; + +/** {@link SimpleVersionedSerializer} for {@link NoOpEnumState}. */ +public class NoOpEnumStateSerializer implements SimpleVersionedSerializer { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(NoOpEnumState obj) throws IOException { + return new byte[0]; + } + + @Override + public NoOpEnumState deserialize(int version, byte[] serialized) throws IOException { + return new NoOpEnumState(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.java new file mode 100644 index 0000000000000..f29c6d6db76dd --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/NoOpEnumerator.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.source; + +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumerator; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; + +/** + * A {@link SplitEnumerator} that provides no functionality. It is basically used for sources that + * does not require a coordinator. + */ +public class NoOpEnumerator + implements SplitEnumerator { + @Override + public void start() {} + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {} + + @Override + public void addSplitsBack(List splits, int subtaskId) {} + + @Override + public void addReader(int subtaskId) {} + + @Override + public NoOpEnumState snapshotState(long checkpointId) throws Exception { + return new NoOpEnumState(); + } + + @Override + public void close() throws IOException {} +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.java new file mode 100644 index 0000000000000..2db0868f8e34a --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplit.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.source; + +import org.apache.flink.api.connector.source.SourceSplit; + +import java.util.UUID; + +/** A {@link SourceSplit} that provides basic information through splitId. */ +public class SimpleSourceSplit implements SourceSplit { + private final String splitId; + private final String value; + + public SimpleSourceSplit() { + this(""); + } + + public SimpleSourceSplit(String value) { + this(UUID.randomUUID().toString(), value); + } + + public SimpleSourceSplit(String splitId, String value) { + this.splitId = splitId; + this.value = value; + } + + @Override + public String splitId() { + return splitId; + } + + public String value() { + return value; + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.java new file mode 100644 index 0000000000000..3387afed1c2a8 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SimpleSourceSplitSerializer.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.source; + +import org.apache.flink.core.io.SimpleVersionedSerializer; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +/** {@link SimpleVersionedSerializer} for {@link SimpleSourceSplit}. */ +public class SimpleSourceSplitSerializer implements SimpleVersionedSerializer { + + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(SimpleSourceSplit split) throws IOException { + if (split.splitId() == null) { + return new byte[0]; + } + + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputStream out = new DataOutputStream(baos)) { + writeString(out, split.splitId()); + writeString(out, split.value()); + return baos.toByteArray(); + } + } + + @Override + public SimpleSourceSplit deserialize(int version, byte[] serialized) throws IOException { + if (serialized.length == 0) { + return new SimpleSourceSplit(); + } + + try (final ByteArrayInputStream bais = new ByteArrayInputStream(serialized); + final DataInputStream in = new DataInputStream(bais)) { + String splitId = readString(in); + String value = readString(in); + return new SimpleSourceSplit(splitId, value); + } + } + + private void writeString(DataOutputStream out, String str) throws IOException { + byte[] bytes = str.getBytes(); + out.writeInt(bytes.length); + out.write(str.getBytes()); + } + + private String readString(DataInputStream in) throws IOException { + int length = in.readInt(); + byte[] bytes = new byte[length]; + in.readFully(bytes); + return new String(bytes); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SplitListState.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SplitListState.java new file mode 100644 index 0000000000000..0049bdf284e35 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/SplitListState.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.utils.Preconditions; + +import org.apache.flink.api.common.state.ListState; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Utility class to provide {@link ListState}-like experience for sources that use {@link + * SimpleSourceSplit}. + */ +public class SplitListState implements ListState { + private final String splitPrefix; + private final List values; + private final Function serializer; + private final Function deserializer; + + public SplitListState( + String identifier, Function serializer, Function deserializer) { + Preconditions.checkArgument( + !Character.isDigit(identifier.charAt(0)), + String.format("Identifier %s should not start with digits.", identifier)); + this.splitPrefix = identifier.length() + identifier; + this.serializer = serializer; + this.deserializer = deserializer; + this.values = new ArrayList<>(); + } + + @Override + public void add(T value) { + values.add(value); + } + + @Override + public List get() { + return new ArrayList<>(values); + } + + @Override + public void update(List values) { + this.values.clear(); + this.values.addAll(values); + } + + @Override + public void addAll(List values) throws Exception { + this.values.addAll(values); + } + + @Override + public void clear() { + values.clear(); + } + + public List snapshotState() { + return values.stream() + .map(x -> new SimpleSourceSplit(splitPrefix + serializer.apply(x))) + .collect(Collectors.toList()); + } + + public void restoreState(List splits) { + values.clear(); + splits.stream() + .map(SimpleSourceSplit::value) + .filter(x -> x.startsWith(splitPrefix)) + .map(x -> x.substring(splitPrefix.length())) + .map(this.deserializer) + .forEach(values::add); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java index d6b7060763acf..705e1d9a7a4c5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedContinuousFileStoreSource.java @@ -73,7 +73,7 @@ public SourceReader createReader(SourceReaderCont limit, new FutureCompletingBlockingQueue<>( context.getConfiguration() - .getInteger(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY))); + .get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY))); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java index 1f0bbca314b63..a8ffe3de561f9 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/align/AlignedSourceReader.java @@ -58,7 +58,7 @@ public AlignedSourceReader( @Nullable Long limit, FutureCompletingBlockingQueue>> elementsQueue) { - super(readerContext, tableRead, metrics, ioManager, limit, elementsQueue); + super(readerContext, tableRead, metrics, ioManager, limit); this.elementsQueue = elementsQueue; this.nextCheckpointId = null; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/metrics/FileStoreSourceReaderMetrics.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/metrics/FileStoreSourceReaderMetrics.java index 2e1e947779491..a270e0eceecd0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/metrics/FileStoreSourceReaderMetrics.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/metrics/FileStoreSourceReaderMetrics.java @@ -29,6 +29,7 @@ public class FileStoreSourceReaderMetrics { private long lastSplitUpdateTime = UNDEFINED; public static final long UNDEFINED = -1L; + public static final long ACTIVE = Long.MAX_VALUE; public FileStoreSourceReaderMetrics(MetricGroup sourceReaderMetricGroup) { sourceReaderMetricGroup.gauge( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java similarity index 66% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java index cee6081aa29f1..c3a1258bb176e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareBatchSource.java @@ -21,20 +21,23 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.compact.MultiAwareBucketTableScan; import org.apache.paimon.flink.compact.MultiTableScanBase; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.JavaTypeInfo; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputStatus; 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.StreamSource; import org.apache.flink.table.data.RowData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,15 +49,11 @@ import static org.apache.paimon.flink.compact.MultiTableScanBase.ScanResult.IS_EMPTY; /** It is responsible for monitoring compactor source of aware bucket table in batch mode. */ -public class CombinedAwareBatchSourceFunction - extends CombinedCompactorSourceFunction> { +public class CombinedAwareBatchSource extends CombinedCompactorSource> { - private static final Logger LOGGER = - LoggerFactory.getLogger(CombinedAwareBatchSourceFunction.class); + private static final Logger LOGGER = LoggerFactory.getLogger(CombinedAwareBatchSource.class); - private MultiTableScanBase> tableScan; - - public CombinedAwareBatchSourceFunction( + public CombinedAwareBatchSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -63,24 +62,32 @@ public CombinedAwareBatchSourceFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiAwareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + public SourceReader, SimpleSourceSplit> createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - @Override - void scanTable() throws Exception { - if (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + private class Reader extends AbstractNonCoordinatedSourceReader> { + private MultiTableScanBase> tableScan; + + @Override + public void start() { + super.start(); + tableScan = + new MultiAwareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext(ReaderOutput> readerOutput) + throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { // Currently, in the combined mode, there are two scan tasks for the table of two @@ -89,6 +96,15 @@ void scanTable() throws Exception { // should not be thrown exception here. LOGGER.info("No file were collected for the table of aware-bucket"); } + return InputStatus.END_OF_INPUT; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -101,15 +117,14 @@ public static DataStream buildSource( Pattern excludingPattern, Pattern databasePattern, Duration partitionIdleTime) { - CombinedAwareBatchSourceFunction function = - new CombinedAwareBatchSourceFunction( + CombinedAwareBatchSource source = + new CombinedAwareBatchSource( catalogLoader, includingPattern, excludingPattern, databasePattern); - StreamSource, ?> sourceOperator = new StreamSource<>(function); TupleTypeInfo> tupleTypeInfo = new TupleTypeInfo<>( new JavaTypeInfo<>(Split.class), BasicTypeInfo.STRING_TYPE_INFO); - return new DataStreamSource<>( - env, tupleTypeInfo, sourceOperator, false, name, Boundedness.BOUNDED) + + return env.fromSource(source, WatermarkStrategy.noWatermarks(), name, tupleTypeInfo) .forceNonParallel() .partitionCustom( (key, numPartitions) -> key % numPartitions, @@ -119,12 +134,4 @@ public static DataStream buildSource( typeInfo, new MultiTablesReadOperator(catalogLoader, false, partitionIdleTime)); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java similarity index 65% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java index bff690ea30c23..9bd4a84f571c3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedAwareStreamingSource.java @@ -21,20 +21,23 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.flink.compact.MultiAwareBucketTableScan; import org.apache.paimon.flink.compact.MultiTableScanBase; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.flink.utils.JavaTypeInfo; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.Split; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputStatus; 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.StreamSource; import org.apache.flink.table.data.RowData; import java.util.regex.Pattern; @@ -43,13 +46,11 @@ import static org.apache.paimon.flink.compact.MultiTableScanBase.ScanResult.IS_EMPTY; /** It is responsible for monitoring compactor source of multi bucket table in stream mode. */ -public class CombinedAwareStreamingSourceFunction - extends CombinedCompactorSourceFunction> { +public class CombinedAwareStreamingSource extends CombinedCompactorSource> { private final long monitorInterval; - private transient MultiTableScanBase> tableScan; - public CombinedAwareStreamingSourceFunction( + public CombinedAwareStreamingSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -60,29 +61,45 @@ public CombinedAwareStreamingSourceFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiAwareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + public SourceReader, SimpleSourceSplit> createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - @SuppressWarnings("BusyWait") - @Override - void scanTable() throws Exception { - while (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + private class Reader extends AbstractNonCoordinatedSourceReader> { + private transient MultiTableScanBase> tableScan; + + @Override + public void start() { + super.start(); + tableScan = + new MultiAwareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext(ReaderOutput> readerOutput) + throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -96,37 +113,22 @@ public static DataStream buildSource( Pattern databasePattern, long monitorInterval) { - CombinedAwareStreamingSourceFunction function = - new CombinedAwareStreamingSourceFunction( + CombinedAwareStreamingSource source = + new CombinedAwareStreamingSource( catalogLoader, includingPattern, excludingPattern, databasePattern, monitorInterval); - StreamSource, ?> sourceOperator = new StreamSource<>(function); - boolean isParallel = false; TupleTypeInfo> tupleTypeInfo = new TupleTypeInfo<>( new JavaTypeInfo<>(Split.class), BasicTypeInfo.STRING_TYPE_INFO); - return new DataStreamSource<>( - env, - tupleTypeInfo, - sourceOperator, - isParallel, - name, - Boundedness.CONTINUOUS_UNBOUNDED) + + return env.fromSource(source, WatermarkStrategy.noWatermarks(), name, tupleTypeInfo) .forceNonParallel() .partitionCustom( (key, numPartitions) -> key % numPartitions, split -> ((DataSplit) split.f0).bucket()) .transform(name, typeInfo, new MultiTablesReadOperator(catalogLoader, true)); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java similarity index 70% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java index 1964927b5cddc..f58d86cdd65eb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSourceFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedCompactorSource.java @@ -20,12 +20,11 @@ import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; import org.apache.paimon.table.source.Split; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.api.connector.source.Boundedness; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; /** @@ -44,8 +43,7 @@ *

    Currently, only dedicated compaction job for multi-tables rely on this monitor. This is the * single (non-parallel) monitoring task, it is responsible for the new Paimon table. */ -public abstract class CombinedCompactorSourceFunction extends RichSourceFunction { - +public abstract class CombinedCompactorSource extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 2L; protected final Catalog.Loader catalogLoader; @@ -54,10 +52,7 @@ public abstract class CombinedCompactorSourceFunction extends RichSourceFunct protected final Pattern databasePattern; protected final boolean isStreaming; - protected transient AtomicBoolean isRunning; - protected transient SourceContext ctx; - - public CombinedCompactorSourceFunction( + public CombinedCompactorSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -71,27 +66,7 @@ public CombinedCompactorSourceFunction( } @Override - public void open(Configuration parameters) throws Exception { - isRunning = new AtomicBoolean(true); - } - - @Override - public void run(SourceContext sourceContext) throws Exception { - this.ctx = sourceContext; - scanTable(); + public Boundedness getBoundedness() { + return isStreaming ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning.set(false); - } - } else { - isRunning.set(false); - } - } - - abstract void scanTable() throws Exception; } 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/CombinedUnawareBatchSource.java similarity index 71% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareBatchSource.java index 8ec8d5f2c1a2e..64f0c38f5a11d 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/CombinedUnawareBatchSource.java @@ -25,17 +25,20 @@ import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; 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.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,14 +57,12 @@ * It is responsible for the batch compactor source of the table with unaware bucket in combined * mode. */ -public class CombinedUnawareBatchSourceFunction - extends CombinedCompactorSourceFunction { +public class CombinedUnawareBatchSource + extends CombinedCompactorSource { - private static final Logger LOGGER = - LoggerFactory.getLogger(CombinedUnawareBatchSourceFunction.class); - private transient MultiTableScanBase tableScan; + private static final Logger LOGGER = LoggerFactory.getLogger(CombinedUnawareBatchSource.class); - public CombinedUnawareBatchSourceFunction( + public CombinedUnawareBatchSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -70,24 +71,33 @@ public CombinedUnawareBatchSourceFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiUnawareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - @Override - void scanTable() throws Exception { - if (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + private class Reader + extends AbstractNonCoordinatedSourceReader { + private transient MultiTableScanBase tableScan; + + @Override + public void start() { + super.start(); + tableScan = + new MultiUnawareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext( + ReaderOutput readerOutput) throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { // Currently, in the combined mode, there are two scan tasks for the table of two @@ -96,6 +106,15 @@ void scanTable() throws Exception { // should not be thrown exception here. LOGGER.info("No file were collected for the table of unaware-bucket"); } + return InputStatus.END_OF_INPUT; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -107,22 +126,18 @@ public static DataStream buildSource( Pattern excludingPattern, Pattern databasePattern, @Nullable Duration partitionIdleTime) { - CombinedUnawareBatchSourceFunction function = - new CombinedUnawareBatchSourceFunction( + CombinedUnawareBatchSource combinedUnawareBatchSource = + new CombinedUnawareBatchSource( catalogLoader, includingPattern, excludingPattern, databasePattern); - StreamSource - sourceOperator = new StreamSource<>(function); MultiTableCompactionTaskTypeInfo compactionTaskTypeInfo = new MultiTableCompactionTaskTypeInfo(); SingleOutputStreamOperator source = - new DataStreamSource<>( - env, - compactionTaskTypeInfo, - sourceOperator, - false, + env.fromSource( + combinedUnawareBatchSource, + WatermarkStrategy.noWatermarks(), name, - Boundedness.BOUNDED) + compactionTaskTypeInfo) .forceNonParallel(); if (partitionIdleTime != null) { @@ -167,12 +182,4 @@ private static Long getPartitionInfo( } return partitionInfo.get(partition); } - - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } - } } 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/CombinedUnawareStreamingSource.java similarity index 59% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSourceFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/CombinedUnawareStreamingSource.java index e398e09a84518..6ea1ead4db301 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/CombinedUnawareStreamingSource.java @@ -23,13 +23,16 @@ import org.apache.paimon.flink.compact.MultiTableScanBase; import org.apache.paimon.flink.compact.MultiUnawareBucketTableScan; import org.apache.paimon.flink.sink.MultiTableCompactionTaskTypeInfo; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.operators.StreamSource; import java.util.regex.Pattern; @@ -39,13 +42,12 @@ /** * It is responsible for monitoring compactor source in stream mode for the table of unaware bucket. */ -public class CombinedUnawareStreamingSourceFunction - extends CombinedCompactorSourceFunction { +public class CombinedUnawareStreamingSource + extends CombinedCompactorSource { private final long monitorInterval; - private MultiTableScanBase tableScan; - public CombinedUnawareStreamingSourceFunction( + public CombinedUnawareStreamingSource( Catalog.Loader catalogLoader, Pattern includingPattern, Pattern excludingPattern, @@ -56,29 +58,46 @@ public CombinedUnawareStreamingSourceFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - tableScan = - new MultiUnawareBucketTableScan( - catalogLoader, - includingPattern, - excludingPattern, - databasePattern, - isStreaming, - isRunning); + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); } - @SuppressWarnings("BusyWait") - @Override - void scanTable() throws Exception { - while (isRunning.get()) { - MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(ctx); + private class Reader + extends AbstractNonCoordinatedSourceReader { + private MultiTableScanBase tableScan; + + @Override + public void start() { + super.start(); + tableScan = + new MultiUnawareBucketTableScan( + catalogLoader, + includingPattern, + excludingPattern, + databasePattern, + isStreaming); + } + + @Override + public InputStatus pollNext( + ReaderOutput readerOutput) throws Exception { + MultiTableScanBase.ScanResult scanResult = tableScan.scanTable(readerOutput); if (scanResult == FINISHED) { - return; + return InputStatus.END_OF_INPUT; } if (scanResult == IS_EMPTY) { Thread.sleep(monitorInterval); } + return InputStatus.MORE_AVAILABLE; + } + + @Override + public void close() throws Exception { + super.close(); + if (tableScan != null) { + tableScan.close(); + } } } @@ -91,33 +110,18 @@ public static DataStream buildSource( Pattern databasePattern, long monitorInterval) { - CombinedUnawareStreamingSourceFunction function = - new CombinedUnawareStreamingSourceFunction( + CombinedUnawareStreamingSource source = + new CombinedUnawareStreamingSource( catalogLoader, includingPattern, excludingPattern, databasePattern, monitorInterval); - StreamSource - sourceOperator = new StreamSource<>(function); - boolean isParallel = false; MultiTableCompactionTaskTypeInfo compactionTaskTypeInfo = new MultiTableCompactionTaskTypeInfo(); - return new DataStreamSource<>( - env, - compactionTaskTypeInfo, - sourceOperator, - isParallel, - name, - Boundedness.CONTINUOUS_UNBOUNDED) - .forceNonParallel(); - } - @Override - public void close() throws Exception { - super.close(); - if (tableScan != null) { - tableScan.close(); - } + return env.fromSource( + source, WatermarkStrategy.noWatermarks(), name, compactionTaskTypeInfo) + .forceNonParallel(); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java similarity index 53% rename from paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java rename to paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java index 3805f6f8c536e..4ec0a4f99d9fd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorFunction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MonitorSource.java @@ -18,6 +18,10 @@ package org.apache.paimon.flink.source.operator; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.SplitListState; import org.apache.paimon.flink.utils.JavaTypeInfo; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.sink.ChannelComputer; @@ -27,22 +31,18 @@ import org.apache.paimon.table.source.Split; import org.apache.paimon.table.source.StreamTableScan; -import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -71,33 +71,23 @@ *

    Currently, there are two features that rely on this monitor: * *

      - *
    1. Consumer-id: rely on this function to do aligned snapshot consumption, and ensure that all + *
    2. Consumer-id: rely on this source to do aligned snapshot consumption, and ensure that all * data in a snapshot is consumed within each checkpoint. *
    3. Snapshot-watermark: when there is no watermark definition, the default Paimon table will * pass the watermark recorded in the snapshot. *
    */ -public class MonitorFunction extends RichSourceFunction - implements CheckpointedFunction, CheckpointListener { +public class MonitorSource extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 1L; - private static final Logger LOG = LoggerFactory.getLogger(MonitorFunction.class); + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final ReadBuilder readBuilder; private final long monitorInterval; private final boolean emitSnapshotWatermark; - private volatile boolean isRunning = true; - - private transient StreamTableScan scan; - private transient SourceContext ctx; - - private transient ListState checkpointState; - private transient ListState> nextSnapshotState; - private transient TreeMap nextSnapshotPerCheckpoint; - - public MonitorFunction( + public MonitorSource( ReadBuilder readBuilder, long monitorInterval, boolean emitSnapshotWatermark) { this.readBuilder = readBuilder; this.monitorInterval = monitorInterval; @@ -105,40 +95,74 @@ public MonitorFunction( } @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - this.scan = readBuilder.newStreamScan(); - - this.checkpointState = - context.getOperatorStateStore() - .getListState( - new ListStateDescriptor<>( - "next-snapshot", LongSerializer.INSTANCE)); - - @SuppressWarnings("unchecked") - final Class> typedTuple = - (Class>) (Class) Tuple2.class; - this.nextSnapshotState = - context.getOperatorStateStore() - .getListState( - new ListStateDescriptor<>( - "next-snapshot-per-checkpoint", - new TupleSerializer<>( - typedTuple, - new TypeSerializer[] { - LongSerializer.INSTANCE, LongSerializer.INSTANCE - }))); - - this.nextSnapshotPerCheckpoint = new TreeMap<>(); - - if (context.isRestored()) { - LOG.info("Restoring state for the {}.", getClass().getSimpleName()); + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(); + } + + private class Reader extends AbstractNonCoordinatedSourceReader { + private static final String CHECKPOINT_STATE = "CS"; + private static final String NEXT_SNAPSHOT_STATE = "NSS"; + + private final StreamTableScan scan = readBuilder.newStreamScan(); + private final SplitListState checkpointState = + new SplitListState<>(CHECKPOINT_STATE, x -> Long.toString(x), Long::parseLong); + private final SplitListState> nextSnapshotState = + new SplitListState<>( + NEXT_SNAPSHOT_STATE, + x -> x.f0 + ":" + x.f1, + x -> + Tuple2.of( + Long.parseLong(x.split(":")[0]), + Long.parseLong(x.split(":")[1]))); + private final TreeMap nextSnapshotPerCheckpoint = new TreeMap<>(); + + @Override + public void notifyCheckpointComplete(long checkpointId) { + NavigableMap nextSnapshots = + nextSnapshotPerCheckpoint.headMap(checkpointId, true); + OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max(); + max.ifPresent(scan::notifyCheckpointComplete); + nextSnapshots.clear(); + } - List retrievedStates = new ArrayList<>(); - for (Long entry : this.checkpointState.get()) { - retrievedStates.add(entry); + @Override + public List snapshotState(long checkpointId) { + this.checkpointState.clear(); + Long nextSnapshot = this.scan.checkpoint(); + if (nextSnapshot != null) { + this.checkpointState.add(nextSnapshot); + this.nextSnapshotPerCheckpoint.put(checkpointId, nextSnapshot); } - // given that the parallelism of the function is 1, we can only have 1 retrieved items. + List> nextSnapshots = new ArrayList<>(); + this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v))); + this.nextSnapshotState.update(nextSnapshots); + + if (LOG.isDebugEnabled()) { + LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); + } + + List results = new ArrayList<>(); + results.addAll(checkpointState.snapshotState()); + results.addAll(nextSnapshotState.snapshotState()); + return results; + } + + @Override + public void addSplits(List list) { + LOG.info("Restoring state for the {}.", getClass().getSimpleName()); + checkpointState.restoreState(list); + nextSnapshotState.restoreState(list); + + List retrievedStates = checkpointState.get(); + + // given that the parallelism of the source is 1, we can only have 1 retrieved items. Preconditions.checkArgument( retrievedStates.size() <= 1, getClass().getSimpleName() + " retrieved invalid state."); @@ -150,80 +174,31 @@ public void initializeState(FunctionInitializationContext context) throws Except for (Tuple2 tuple2 : nextSnapshotState.get()) { nextSnapshotPerCheckpoint.put(tuple2.f0, tuple2.f1); } - } else { - LOG.info("No state to restore for the {}.", getClass().getSimpleName()); } - } - - @Override - public void snapshotState(FunctionSnapshotContext ctx) throws Exception { - this.checkpointState.clear(); - Long nextSnapshot = this.scan.checkpoint(); - if (nextSnapshot != null) { - this.checkpointState.add(nextSnapshot); - this.nextSnapshotPerCheckpoint.put(ctx.getCheckpointId(), nextSnapshot); - } - - List> nextSnapshots = new ArrayList<>(); - this.nextSnapshotPerCheckpoint.forEach((k, v) -> nextSnapshots.add(new Tuple2<>(k, v))); - this.nextSnapshotState.update(nextSnapshots); - if (LOG.isDebugEnabled()) { - LOG.debug("{} checkpoint {}.", getClass().getSimpleName(), nextSnapshot); - } - } - - @SuppressWarnings("BusyWait") - @Override - public void run(SourceContext ctx) throws Exception { - this.ctx = ctx; - while (isRunning) { + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { boolean isEmpty; - synchronized (ctx.getCheckpointLock()) { - if (!isRunning) { - return; - } - try { - List splits = scan.plan().splits(); - isEmpty = splits.isEmpty(); - splits.forEach(ctx::collect); - - if (emitSnapshotWatermark) { - Long watermark = scan.watermark(); - if (watermark != null) { - ctx.emitWatermark(new Watermark(watermark)); - } + try { + List splits = scan.plan().splits(); + isEmpty = splits.isEmpty(); + splits.forEach(readerOutput::collect); + + if (emitSnapshotWatermark) { + Long watermark = scan.watermark(); + if (watermark != null) { + readerOutput.emitWatermark(new Watermark(watermark)); } - } catch (EndOfScanException esf) { - LOG.info("Catching EndOfStreamException, the stream is finished."); - return; } + } catch (EndOfScanException esf) { + LOG.info("Catching EndOfStreamException, the stream is finished."); + return InputStatus.END_OF_INPUT; } if (isEmpty) { Thread.sleep(monitorInterval); } - } - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - NavigableMap nextSnapshots = - nextSnapshotPerCheckpoint.headMap(checkpointId, true); - OptionalLong max = nextSnapshots.values().stream().mapToLong(Long::longValue).max(); - max.ifPresent(scan::notifyCheckpointComplete); - nextSnapshots.clear(); - } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (ctx != null) { - synchronized (ctx.getCheckpointLock()) { - isRunning = false; - } - } else { - isRunning = false; + return InputStatus.MORE_AVAILABLE; } } @@ -237,9 +212,10 @@ public static DataStream buildSource( boolean shuffleBucketWithPartition, BucketMode bucketMode) { SingleOutputStreamOperator singleOutputStreamOperator = - env.addSource( - new MonitorFunction( + env.fromSource( + new MonitorSource( readBuilder, monitorInterval, emitSnapshotWatermark), + WatermarkStrategy.noWatermarks(), name + "-Monitor", new JavaTypeInfo<>(Split.class)) .forceNonParallel(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java index 73d46ae1e3f19..fbc8bb9d756a3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/operator/MultiTablesReadOperator.java @@ -52,9 +52,8 @@ /** * The operator that reads the Tuple2<{@link Split}, String> received from the preceding {@link - * CombinedAwareBatchSourceFunction} or {@link CombinedAwareStreamingSourceFunction}. Contrary to - * the {@link CombinedCompactorSourceFunction} which has a parallelism of 1, this operator can have - * DOP > 1. + * CombinedAwareBatchSource} or {@link CombinedAwareStreamingSource}. Contrary to the {@link + * CombinedCompactorSource} which has a parallelism of 1, this operator can have DOP > 1. */ public class MultiTablesReadOperator extends AbstractStreamOperator implements OneInputStreamOperator, RowData> { 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 c501c2519b412..0864741a178f1 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 @@ -44,7 +44,7 @@ /** * The operator is used for historical partition compaction. It reads {@link * MultiTableUnawareAppendCompactionTask} received from the preceding {@link - * CombinedUnawareBatchSourceFunction} and filter partitions which is not historical. + * CombinedUnawareBatchSource} and filter partitions which is not historical. */ public class MultiUnawareTablesReadOperator extends AbstractStreamOperator 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 80c85f7cdb35a..ccc66194560e9 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 @@ -38,8 +38,8 @@ /** * The operator that reads the {@link Split splits} received from the preceding {@link - * MonitorFunction}. Contrary to the {@link MonitorFunction} which has a parallelism of 1, this - * operator can have DOP > 1. + * MonitorSource}. Contrary to the {@link MonitorSource} which has a parallelism of 1, this operator + * can have DOP > 1. */ public class ReadOperator extends AbstractStreamOperator implements OneInputStreamOperator { @@ -54,9 +54,11 @@ 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 + // we create our own gauge for currentEmitEventTimeLag and sourceIdleTime, 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 long idleStartTime = FileStoreSourceReaderMetrics.ACTIVE; private transient Counter numRecordsIn; public ReadOperator(ReadBuilder readBuilder) { @@ -69,6 +71,7 @@ public void open() throws Exception { this.sourceReaderMetrics = new FileStoreSourceReaderMetrics(getMetricGroup()); getMetricGroup().gauge(MetricNames.CURRENT_EMIT_EVENT_TIME_LAG, () -> emitEventTimeLag); + getMetricGroup().gauge(MetricNames.SOURCE_IDLE_TIME, this::getIdleTime); this.numRecordsIn = InternalSourceReaderMetricGroup.wrap(getMetricGroup()) .getIOMetricGroup() @@ -83,6 +86,7 @@ public void open() throws Exception { this.read = readBuilder.newRead().withIOManager(ioManager); this.reuseRow = new FlinkRowData(null); this.reuseRecord = new StreamRecord<>(reuseRow); + this.idlingStarted(); } @Override @@ -94,6 +98,8 @@ public void processElement(StreamRecord record) throws Exception { .earliestFileCreationEpochMillis() .orElse(FileStoreSourceReaderMetrics.UNDEFINED); sourceReaderMetrics.recordSnapshotUpdate(eventTime); + // update idleStartTime when reading a new split + idleStartTime = FileStoreSourceReaderMetrics.ACTIVE; boolean firstRecord = true; try (CloseableIterator iterator = @@ -113,6 +119,8 @@ public void processElement(StreamRecord record) throws Exception { output.collect(reuseRecord); } } + // start idle when data sending is completed + this.idlingStarted(); } @Override @@ -122,4 +130,18 @@ public void close() throws Exception { ioManager.close(); } } + + private void idlingStarted() { + if (!isIdling()) { + idleStartTime = System.currentTimeMillis(); + } + } + + private boolean isIdling() { + return idleStartTime != FileStoreSourceReaderMetrics.ACTIVE; + } + + private long getIdleTime() { + return isIdling() ? System.currentTimeMillis() - idleStartTime : 0; + } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java index b0f99a6e89e43..fa84a1ca070d0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkCatalogPropertiesUtil.java @@ -20,8 +20,7 @@ import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableSet; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.WatermarkSpec; +import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.types.DataType; @@ -36,48 +35,23 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.flink.table.descriptors.DescriptorProperties.COMMENT; -import static org.apache.flink.table.descriptors.DescriptorProperties.DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; -import static org.apache.flink.table.descriptors.DescriptorProperties.METADATA; -import static org.apache.flink.table.descriptors.DescriptorProperties.NAME; -import static org.apache.flink.table.descriptors.DescriptorProperties.VIRTUAL; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.COMMENT; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.EXPR; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.METADATA; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.NAME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.VIRTUAL; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_EXPR; /** * Utilities for ser/deserializing non-physical columns and watermark into/from a map of string * properties. */ public class FlinkCatalogPropertiesUtil { - - public static Map serializeNonPhysicalColumns( - Map indexMap, List nonPhysicalColumns) { - Map serialized = new HashMap<>(); - for (TableColumn c : nonPhysicalColumns) { - int index = indexMap.get(c.getName()); - serialized.put(compoundKey(SCHEMA, index, NAME), c.getName()); - serialized.put( - compoundKey(SCHEMA, index, DATA_TYPE), - c.getType().getLogicalType().asSerializableString()); - if (c instanceof TableColumn.ComputedColumn) { - TableColumn.ComputedColumn computedColumn = (TableColumn.ComputedColumn) c; - serialized.put(compoundKey(SCHEMA, index, EXPR), computedColumn.getExpression()); - } else { - TableColumn.MetadataColumn metadataColumn = (TableColumn.MetadataColumn) c; - serialized.put( - compoundKey(SCHEMA, index, METADATA), - metadataColumn.getMetadataAlias().orElse(metadataColumn.getName())); - serialized.put( - compoundKey(SCHEMA, index, VIRTUAL), - Boolean.toString(metadataColumn.isVirtual())); - } - } - return serialized; - } + public static final String SCHEMA = "schema"; /** Serialize non-physical columns of new api. */ public static Map serializeNonPhysicalNewColumns(ResolvedSchema schema) { @@ -119,22 +93,6 @@ public static Map serializeNonPhysicalNewColumns(ResolvedSchema return serialized; } - public static Map serializeWatermarkSpec(WatermarkSpec watermarkSpec) { - Map serializedWatermarkSpec = new HashMap<>(); - String watermarkPrefix = compoundKey(SCHEMA, WATERMARK, 0); - serializedWatermarkSpec.put( - compoundKey(watermarkPrefix, WATERMARK_ROWTIME), - watermarkSpec.getRowtimeAttribute()); - serializedWatermarkSpec.put( - compoundKey(watermarkPrefix, WATERMARK_STRATEGY_EXPR), - watermarkSpec.getWatermarkExpr()); - serializedWatermarkSpec.put( - compoundKey(watermarkPrefix, WATERMARK_STRATEGY_DATA_TYPE), - watermarkSpec.getWatermarkExprOutputType().getLogicalType().asSerializableString()); - - return serializedWatermarkSpec; - } - public static Map serializeNewWatermarkSpec( org.apache.flink.table.catalog.WatermarkSpec watermarkSpec) { Map serializedWatermarkSpec = new HashMap<>(); @@ -219,7 +177,8 @@ private static boolean isColumnNameKey(String key) { && SCHEMA_COLUMN_NAME_SUFFIX.matcher(key.substring(SCHEMA.length() + 1)).matches(); } - public static TableColumn deserializeNonPhysicalColumn(Map options, int index) { + public static void deserializeNonPhysicalColumn( + Map options, int index, Schema.Builder builder) { String nameKey = compoundKey(SCHEMA, index, NAME); String dataTypeKey = compoundKey(SCHEMA, index, DATA_TYPE); String exprKey = compoundKey(SCHEMA, index, EXPR); @@ -227,45 +186,42 @@ public static TableColumn deserializeNonPhysicalColumn(Map optio String virtualKey = compoundKey(SCHEMA, index, VIRTUAL); String name = options.get(nameKey); - DataType dataType = - TypeConversions.fromLogicalToDataType( - LogicalTypeParser.parse(options.get(dataTypeKey))); - TableColumn column; if (options.containsKey(exprKey)) { - column = TableColumn.computed(name, dataType, options.get(exprKey)); + final String expr = options.get(exprKey); + builder.columnByExpression(name, expr); } else if (options.containsKey(metadataKey)) { String metadataAlias = options.get(metadataKey); boolean isVirtual = Boolean.parseBoolean(options.get(virtualKey)); - column = - metadataAlias.equals(name) - ? TableColumn.metadata(name, dataType, isVirtual) - : TableColumn.metadata(name, dataType, metadataAlias, isVirtual); + DataType dataType = + TypeConversions.fromLogicalToDataType( + LogicalTypeParser.parse( + options.get(dataTypeKey), + Thread.currentThread().getContextClassLoader())); + if (metadataAlias.equals(name)) { + builder.columnByMetadata(name, dataType, isVirtual); + } else { + builder.columnByMetadata(name, dataType, metadataAlias, isVirtual); + } } else { throw new RuntimeException( String.format( "Failed to build non-physical column. Current index is %s, options are %s", index, options)); } - - return column; } - public static WatermarkSpec deserializeWatermarkSpec(Map options) { + public static void deserializeWatermarkSpec( + Map options, Schema.Builder builder) { String watermarkPrefixKey = compoundKey(SCHEMA, WATERMARK); String rowtimeKey = compoundKey(watermarkPrefixKey, 0, WATERMARK_ROWTIME); String exprKey = compoundKey(watermarkPrefixKey, 0, WATERMARK_STRATEGY_EXPR); - String dataTypeKey = compoundKey(watermarkPrefixKey, 0, WATERMARK_STRATEGY_DATA_TYPE); String rowtimeAttribute = options.get(rowtimeKey); String watermarkExpressionString = options.get(exprKey); - DataType watermarkExprOutputType = - TypeConversions.fromLogicalToDataType( - LogicalTypeParser.parse(options.get(dataTypeKey))); - return new WatermarkSpec( - rowtimeAttribute, watermarkExpressionString, watermarkExprOutputType); + builder.watermark(rowtimeAttribute, watermarkExpressionString); } public static String compoundKey(Object... components) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java new file mode 100644 index 0000000000000..edc73ca7bf41f --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/FlinkDescriptorProperties.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.flink.table.api.Schema; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Utility class for having a unified string-based representation of Table API related classes such + * as Schema, TypeInformation, etc. + * + *

    Note to implementers: Please try to reuse key names as much as possible. Key-names should be + * hierarchical and lower case. Use "-" instead of dots or camel case. E.g., + * connector.schema.start-from = from-earliest. Try not to use the higher level in a key-name. E.g., + * instead of connector.kafka.kafka-version use connector.kafka.version. + * + *

    Properties with key normalization enabled contain only lower-case keys. + */ +public class FlinkDescriptorProperties { + + public static final String NAME = "name"; + + public static final String DATA_TYPE = "data-type"; + + public static final String EXPR = "expr"; + + public static final String METADATA = "metadata"; + + public static final String VIRTUAL = "virtual"; + + public static final String WATERMARK = "watermark"; + + public static final String WATERMARK_ROWTIME = "rowtime"; + + public static final String WATERMARK_STRATEGY = "strategy"; + + public static final String WATERMARK_STRATEGY_EXPR = WATERMARK_STRATEGY + '.' + EXPR; + + public static final String WATERMARK_STRATEGY_DATA_TYPE = WATERMARK_STRATEGY + '.' + DATA_TYPE; + + public static final String PRIMARY_KEY_NAME = "primary-key.name"; + + public static final String PRIMARY_KEY_COLUMNS = "primary-key.columns"; + + public static final String COMMENT = "comment"; + + public static void removeSchemaKeys(String key, Schema schema, Map options) { + checkNotNull(key); + checkNotNull(schema); + + List subKeys = Arrays.asList(NAME, DATA_TYPE, EXPR, METADATA, VIRTUAL); + for (int idx = 0; idx < schema.getColumns().size(); idx++) { + for (String subKey : subKeys) { + options.remove(key + '.' + idx + '.' + subKey); + } + } + + if (!schema.getWatermarkSpecs().isEmpty()) { + subKeys = + Arrays.asList( + WATERMARK_ROWTIME, + WATERMARK_STRATEGY_EXPR, + WATERMARK_STRATEGY_DATA_TYPE); + for (int idx = 0; idx < schema.getWatermarkSpecs().size(); idx++) { + for (String subKey : subKeys) { + options.remove(key + '.' + WATERMARK + '.' + idx + '.' + subKey); + } + } + } + + schema.getPrimaryKey() + .ifPresent( + pk -> { + options.remove(key + '.' + PRIMARY_KEY_NAME); + options.remove(key + '.' + PRIMARY_KEY_COLUMNS); + }); + } +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java index 4ea5db9f34d48..60898421ddea2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/InternalTypeInfo.java @@ -22,6 +22,7 @@ import org.apache.paimon.types.RowType; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -73,8 +74,17 @@ public boolean isKeyType() { return false; } - @Override - public TypeSerializer createSerializer(ExecutionConfig config) { + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ + public TypeSerializer createSerializer(ExecutionConfig executionConfig) { return serializer.duplicate(); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java index a36243c5bdacb..4aea809b51bc8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/JavaTypeInfo.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -78,7 +79,16 @@ public boolean isKeyType() { return Comparable.class.isAssignableFrom(typeClass); } - @Override + /** + * Do not annotate with @override here to maintain compatibility with Flink 1.18-. + */ + public TypeSerializer createSerializer(SerializerConfig config) { + return this.createSerializer((ExecutionConfig) null); + } + + /** + * Do not annotate with @override here to maintain compatibility with Flink 2.0+. + */ public TypeSerializer createSerializer(ExecutionConfig config) { return new JavaSerializer<>(this.typeClass); } @@ -91,7 +101,9 @@ public TypeComparator createComparator( @SuppressWarnings("rawtypes") GenericTypeComparator comparator = new GenericTypeComparator( - sortOrderAscending, createSerializer(executionConfig), this.typeClass); + sortOrderAscending, + new JavaSerializer<>(this.typeClass), + this.typeClass); return (TypeComparator) comparator; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.java new file mode 100644 index 0000000000000..34e0d041b6a0c --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/utils/RuntimeContextUtils.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.utils; + +import org.apache.flink.api.common.functions.RuntimeContext; + +/** Utility methods about Flink runtime context to resolve compatibility issues. */ +public class RuntimeContextUtils { + public static int getNumberOfParallelSubtasks(RuntimeContext context) { + return context.getTaskInfo().getNumberOfParallelSubtasks(); + } + + public static int getIndexOfThisSubtask(RuntimeContext context) { + return context.getTaskInfo().getIndexOfThisSubtask(); + } +} 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 e5d063979da62..46117a8f01efd 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 @@ -61,6 +61,7 @@ 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.RollbackToWatermarkProcedure org.apache.paimon.flink.procedure.MigrateTableProcedure org.apache.paimon.flink.procedure.MigrateDatabaseProcedure org.apache.paimon.flink.procedure.MigrateFileProcedure 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 c30e6cd5612d3..cdc114b048a11 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 @@ -34,6 +34,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import java.math.BigDecimal; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -573,6 +574,24 @@ public void testCountStarPK() { validateCount1NotPushDown(sql); } + @Test + public void testParquetRowDecimalAndTimestamp() { + sql( + "CREATE TABLE parquet_row_decimal(`row` ROW) WITH ('file.format' = 'parquet')"); + sql("INSERT INTO parquet_row_decimal VALUES ( (ROW(1.2)) )"); + + assertThat(sql("SELECT * FROM parquet_row_decimal")) + .containsExactly(Row.of(Row.of(new BigDecimal("1.2")))); + + sql( + "CREATE TABLE parquet_row_timestamp(`row` ROW) WITH ('file.format' = 'parquet')"); + sql("INSERT INTO parquet_row_timestamp VALUES ( (ROW(TIMESTAMP'2024-11-13 18:00:00')) )"); + + assertThat(sql("SELECT * FROM parquet_row_timestamp")) + .containsExactly( + Row.of(Row.of(DateTimeUtils.toLocalDateTime("2024-11-13 18:00:00", 0)))); + } + private void validateCount1PushDown(String sql) { Transformation transformation = AbstractTestBase.translate(tEnv, sql); while (!transformation.getInputs().isEmpty()) { 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 1d33a9e8a6f25..2566fbe92e4c7 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 @@ -27,6 +27,7 @@ import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.io.IOException; import java.util.ArrayList; @@ -347,6 +348,11 @@ public void testBranchOptionsTable() throws Exception { "+I[bucket, 2]", "+I[snapshot.time-retained, 1 h]", "+I[scan.infer-parallelism, false]"); + assertThat(collectResult("SELECT * FROM t$options /*+ OPTIONS('branch'='test') */")) + .containsExactlyInAnyOrder( + "+I[bucket, 2]", + "+I[snapshot.time-retained, 1 h]", + "+I[scan.infer-parallelism, false]"); } @Test @@ -360,6 +366,10 @@ public void testBranchSchemasTable() throws Exception { sql("ALTER TABLE t$branch_b1 SET ('snapshot.time-retained' = '5 h')"); assertThat(collectResult("SELECT schema_id FROM t$branch_b1$schemas order by schema_id")) .containsExactlyInAnyOrder("+I[0]", "+I[1]"); + assertThat( + collectResult( + "SELECT schema_id FROM t$schemas /*+ OPTIONS('branch'='b1') */ order by schema_id")) + .containsExactlyInAnyOrder("+I[0]", "+I[1]"); } @Test @@ -373,6 +383,8 @@ public void testBranchAuditLogTable() throws Exception { sql("INSERT INTO t$branch_b1 VALUES (3, 4)"); assertThat(collectResult("SELECT * FROM t$branch_b1$audit_log")) .containsExactlyInAnyOrder("+I[+I, 3, 4]"); + assertThat(collectResult("SELECT * FROM t$audit_log /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[+I, 3, 4]"); } @Test @@ -385,6 +397,8 @@ public void testBranchReadOptimizedTable() throws Exception { sql("INSERT INTO t$branch_b1 VALUES (3, 4)"); assertThat(collectResult("SELECT * FROM t$branch_b1$ro")) .containsExactlyInAnyOrder("+I[3, 4]"); + assertThat(collectResult("SELECT * FROM t$ro /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[3, 4]"); } @Test @@ -400,6 +414,10 @@ public void testBranchFilesTable() throws Exception { .containsExactlyInAnyOrder("+I[{a=1, b=2}]"); assertThat(collectResult("SELECT min_value_stats FROM t$branch_b1$files")) .containsExactlyInAnyOrder("+I[{a=3, b=4}]", "+I[{a=5, b=6}]"); + assertThat( + collectResult( + "SELECT min_value_stats FROM t$files /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[{a=3, b=4}]", "+I[{a=5, b=6}]"); } @Test @@ -416,9 +434,14 @@ public void testBranchTagsTable() throws Exception { .containsExactlyInAnyOrder("+I[tag1, 1, 1]"); assertThat(collectResult("SELECT tag_name,snapshot_id,record_count FROM t$branch_b1$tags")) .containsExactlyInAnyOrder("+I[tag1, 1, 1]", "+I[tag2, 2, 2]"); + assertThat( + collectResult( + "SELECT tag_name,snapshot_id,record_count FROM t$tags /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[tag1, 1, 1]", "+I[tag2, 2, 2]"); } @Test + @Timeout(60) public void testBranchConsumersTable() throws Exception { sql("CREATE TABLE t (a INT, b INT)"); sql("INSERT INTO t VALUES (1, 2), (3,4)"); @@ -430,10 +453,19 @@ public void testBranchConsumersTable() throws Exception { "SELECT * FROM t$branch_b1 /*+ OPTIONS('consumer-id'='id1','consumer.expiration-time'='3h') */")); sql("INSERT INTO t$branch_b1 VALUES (5, 6), (7, 8)"); assertThat(iterator.collect(2)).containsExactlyInAnyOrder(Row.of(5, 6), Row.of(7, 8)); + List branchResult; + do { + branchResult = collectResult("SELECT * FROM t$branch_b1$consumers"); + if (!branchResult.isEmpty()) { + break; + } + Thread.sleep(1000); + } while (true); iterator.close(); assertThat(collectResult("SELECT * FROM t$consumers")).isEmpty(); - assertThat(collectResult("SELECT * FROM t$branch_b1$consumers")) + assertThat(branchResult).containsExactlyInAnyOrder("+I[id1, 2]"); + assertThat(collectResult("SELECT * FROM t$consumers /*+ OPTIONS('branch'='b1') */")) .containsExactlyInAnyOrder("+I[id1, 2]"); } @@ -458,6 +490,31 @@ public void testBranchManifestsTable() { .isTrue(); assertThat((long) row.getField(2)).isGreaterThan(0L); }); + List dynamicOptionRes = + sql( + "SELECT schema_id, file_name, file_size FROM t$manifests /*+ OPTIONS('branch'='b1') */"); + assertThat(dynamicOptionRes).containsExactlyInAnyOrderElementsOf(res); + } + + @Test + public void testBranchSnapshotsTable() throws Exception { + sql("CREATE TABLE t (a INT, b INT)"); + sql("INSERT INTO t VALUES (1, 2)"); + + sql("CALL sys.create_branch('default.t', 'b1')"); + sql("INSERT INTO t$branch_b1 VALUES (3, 4)"); + sql("INSERT INTO t$branch_b1 VALUES (5, 6)"); + + assertThat(collectResult("SELECT snapshot_id, schema_id, commit_kind FROM t$snapshots")) + .containsExactlyInAnyOrder("+I[1, 0, APPEND]"); + assertThat( + collectResult( + "SELECT snapshot_id, schema_id, commit_kind FROM t$branch_b1$snapshots")) + .containsExactlyInAnyOrder("+I[1, 0, APPEND]", "+I[2, 0, APPEND]"); + assertThat( + collectResult( + "SELECT snapshot_id, schema_id, commit_kind FROM t$snapshots /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[1, 0, APPEND]", "+I[2, 0, APPEND]"); } @Test @@ -479,6 +536,10 @@ public void testBranchPartitionsTable() throws Exception { collectResult( "SELECT `partition`, record_count, file_count FROM t$branch_b1$partitions")) .containsExactlyInAnyOrder("+I[[1], 2, 2]", "+I[[2], 3, 2]"); + assertThat( + collectResult( + "SELECT `partition`, record_count, file_count FROM t$partitions /*+ OPTIONS('branch'='b1') */")) + .containsExactlyInAnyOrder("+I[[1], 2, 2]", "+I[[2], 3, 2]"); } @Test 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 ba063248ee460..10b03b7139ae3 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 @@ -21,8 +21,6 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.table.system.AllTableOptionsTable; import org.apache.paimon.table.system.CatalogOptionsTable; -import org.apache.paimon.table.system.SinkTableLineageTable; -import org.apache.paimon.table.system.SourceTableLineageTable; import org.apache.paimon.utils.BlockingIterator; import org.apache.commons.lang3.StringUtils; @@ -33,6 +31,7 @@ import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; import org.apache.flink.types.Row; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import javax.annotation.Nonnull; @@ -199,9 +198,7 @@ public void testSystemDatabase() { assertThat(sql("SHOW TABLES")) .containsExactlyInAnyOrder( Row.of(AllTableOptionsTable.ALL_TABLE_OPTIONS), - Row.of(CatalogOptionsTable.CATALOG_OPTIONS), - Row.of(SourceTableLineageTable.SOURCE_TABLE_LINEAGE), - Row.of(SinkTableLineageTable.SINK_TABLE_LINEAGE)); + Row.of(CatalogOptionsTable.CATALOG_OPTIONS)); } @Test @@ -250,17 +247,20 @@ public void testSchemasTable() { sql("ALTER TABLE T SET ('snapshot.num-retained.min' = '18')"); sql("ALTER TABLE T SET ('manifest.format' = 'avro')"); - assertThat(sql("SHOW CREATE TABLE T$schemas").toString()) - .isEqualTo( - "[+I[CREATE TABLE `PAIMON`.`default`.`T$schemas` (\n" - + " `schema_id` BIGINT NOT NULL,\n" - + " `fields` VARCHAR(2147483647) NOT NULL,\n" - + " `partition_keys` VARCHAR(2147483647) NOT NULL,\n" - + " `primary_keys` VARCHAR(2147483647) NOT NULL,\n" - + " `options` VARCHAR(2147483647) NOT NULL,\n" - + " `comment` VARCHAR(2147483647),\n" - + " `update_time` TIMESTAMP(3) NOT NULL\n" - + ") ]]"); + String actualResult = sql("SHOW CREATE TABLE T$schemas").toString(); + String expectedResult = + "[+I[CREATE TABLE `PAIMON`.`default`.`T$schemas` (\n" + + " `schema_id` BIGINT NOT NULL,\n" + + " `fields` VARCHAR(2147483647) NOT NULL,\n" + + " `partition_keys` VARCHAR(2147483647) NOT NULL,\n" + + " `primary_keys` VARCHAR(2147483647) NOT NULL,\n" + + " `options` VARCHAR(2147483647) NOT NULL,\n" + + " `comment` VARCHAR(2147483647),\n" + + " `update_time` TIMESTAMP(3) NOT NULL\n" + + ") ]]"; + actualResult = actualResult.replace(" ", "").replace("\n", ""); + expectedResult = expectedResult.replace(" ", "").replace("\n", ""); + assertThat(actualResult).isEqualTo(expectedResult); List result = sql( @@ -940,6 +940,7 @@ public void testTagsTable() throws Exception { } @Test + @Timeout(60) public void testConsumersTable() throws Exception { batchSql("CREATE TABLE T (a INT, b INT)"); batchSql("INSERT INTO T VALUES (1, 2)"); @@ -952,9 +953,17 @@ public void testConsumersTable() throws Exception { batchSql("INSERT INTO T VALUES (5, 6), (7, 8)"); assertThat(iterator.collect(2)).containsExactlyInAnyOrder(Row.of(1, 2), Row.of(3, 4)); + + List result; + do { + result = sql("SELECT * FROM T$consumers"); + if (!result.isEmpty()) { + break; + } + Thread.sleep(1000); + } while (true); iterator.close(); - List result = sql("SELECT * FROM T$consumers"); assertThat(result).hasSize(1); assertThat(result.get(0).getField(0)).isEqualTo("my1"); assertThat((Long) result.get(0).getField(1)).isGreaterThanOrEqualTo(3); @@ -972,7 +981,8 @@ public void testConsumerIdExpInBatchMode() { "SELECT * FROM T /*+ OPTIONS('consumer-id' = 'test-id') */ WHERE a = 1")) .rootCause() .isInstanceOf(IllegalArgumentException.class) - .hasMessage("consumer.expiration-time should be specified when using consumer-id."); + .hasMessageContaining( + "You need to configure 'consumer.expiration-time' (ALTER TABLE) and restart your write job for it"); } @Test @@ -985,7 +995,8 @@ public void testConsumerIdExpInStreamingMode() { streamSqlIter( "SELECT * FROM T /*+ OPTIONS('consumer-id'='test-id') */")) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("consumer.expiration-time should be specified when using consumer-id."); + .hasMessageContaining( + "You need to configure 'consumer.expiration-time' (ALTER TABLE) and restart your write job for it"); } @Test 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 cf97f7b67d4d1..b448858328041 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 @@ -28,6 +28,7 @@ import org.apache.flink.table.api.StatementSet; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; import org.apache.flink.util.CloseableIterator; @@ -43,6 +44,7 @@ import java.util.Set; import java.util.stream.Collectors; +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; @@ -118,7 +120,14 @@ public void testConsumerId() throws Exception { assertThat(iterator.collect(2)) .containsExactlyInAnyOrder(Row.of("1", "2", "3"), Row.of("4", "5", "6")); - Thread.sleep(1000); + List result; + do { + result = sql("SELECT * FROM %s$consumers", table); + if (!result.isEmpty()) { + break; + } + Thread.sleep(1000); + } while (true); iterator.close(); iterator = @@ -629,4 +638,29 @@ public void testScanFromChangelog(String changelogProducer) throws Exception { assertThat(iterator.collect(1)).containsExactlyInAnyOrder(Row.of("10", "11", "12")); iterator.close(); } + + @Test + public void testAvroRetractNotNullField() { + List input = + Arrays.asList( + Row.ofKind(RowKind.INSERT, 1, "A"), Row.ofKind(RowKind.DELETE, 1, "A")); + String id = TestValuesTableFactory.registerData(input); + sEnv.executeSql( + String.format( + "CREATE TEMPORARY TABLE source (pk INT PRIMARY KEY NOT ENFORCED, a STRING) " + + "WITH ('connector'='values', 'bounded'='true', 'data-id'='%s', " + + "'changelog-mode' = 'I,D,UA,UB')", + id)); + + sql( + "CREATE TABLE avro_sink (pk INT PRIMARY KEY NOT ENFORCED, a STRING NOT NULL) " + + " WITH ('file.format' = 'avro', 'merge-engine' = 'aggregation')"); + + assertThatThrownBy( + () -> sEnv.executeSql("INSERT INTO avro_sink select * from source").await()) + .satisfies( + anyCauseMatches( + RuntimeException.class, + "Caught NullPointerException, the possible reason is you have set following options together")); + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java index 6a2c7b071d2d1..5245114e80eed 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java @@ -36,6 +36,7 @@ import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.FailingFileIO; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.dag.Transformation; @@ -450,7 +451,12 @@ private void sinkAndValidate( throw new UnsupportedOperationException(); } DataStreamSource source = - env.addSource(new FiniteTestSource<>(src, true), InternalTypeInfo.of(TABLE_TYPE)); + env.fromSource( + new FiniteTestSource<>(src, true), + WatermarkStrategy.noWatermarks(), + "FiniteTestSource", + InternalTypeInfo.of(TABLE_TYPE)); + source.forceNonParallel(); new FlinkSinkBuilder(table).forRowData(source).build(); env.execute(); assertThat(iterator.collect(expected.length)).containsExactlyInAnyOrder(expected); @@ -521,9 +527,13 @@ public static DataStreamSource buildTestSource( StreamExecutionEnvironment env, boolean isBatch) { return isBatch ? env.fromCollection(SOURCE_DATA, InternalTypeInfo.of(TABLE_TYPE)) - : env.addSource( - new FiniteTestSource<>(SOURCE_DATA, false), - InternalTypeInfo.of(TABLE_TYPE)); + : (DataStreamSource) + env.fromSource( + new FiniteTestSource<>(SOURCE_DATA, false), + WatermarkStrategy.noWatermarks(), + "FiniteTestSource", + InternalTypeInfo.of(TABLE_TYPE)) + .forceNonParallel(); } public static List executeAndCollect(DataStream source) throws Exception { 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 239043ff79e1d..915c93680a0dc 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 @@ -27,7 +27,6 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.utils.BlockingIterator; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; @@ -60,7 +59,7 @@ public void setup() { tableEnvironmentBuilder() .streamingMode() .parallelism(1) - .setConf(ExecutionCheckpointingOptions.ENABLE_UNALIGNED, false) + .setString("execution.checkpointing.unaligned.enabled", "false") .build(); path = getTempDirPath(); tEnv.executeSql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java index 9c5254d6283b7..6691b9c095140 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FiniteTestSource.java @@ -18,16 +18,18 @@ package org.apache.paimon.flink; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.SplitListState; import org.apache.paimon.utils.Preconditions; -import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.base.IntSerializer; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.source.SourceFunction; import java.util.ArrayList; import java.util.List; @@ -39,8 +41,7 @@ * *

    The reason this class is rewritten is to support {@link CheckpointedFunction}. */ -public class FiniteTestSource - implements SourceFunction, CheckpointedFunction, CheckpointListener { +public class FiniteTestSource extends AbstractNonCoordinatedSource { private static final long serialVersionUID = 1L; @@ -48,27 +49,78 @@ public class FiniteTestSource private final boolean emitOnce; - private volatile boolean running = true; - - private transient int numCheckpointsComplete; - - private transient ListState checkpointedState; - - private volatile int numTimesEmitted; - public FiniteTestSource(List elements, boolean emitOnce) { this.elements = elements; this.emitOnce = emitOnce; } @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - this.checkpointedState = - context.getOperatorStateStore() - .getListState( - new ListStateDescriptor<>("emit-times", IntSerializer.INSTANCE)); + public Boundedness getBoundedness() { + return Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext sourceReaderContext) + throws Exception { + return new Reader<>(elements, emitOnce); + } + + private static class Reader extends AbstractNonCoordinatedSourceReader { + + private final List elements; + + private final boolean emitOnce; + + private final SplitListState checkpointedState = + new SplitListState<>("emit-times", x -> Integer.toString(x), Integer::parseInt); + + private int numTimesEmitted = 0; + + private int numCheckpointsComplete; + + private Integer checkpointToAwait; + + private Reader(List elements, boolean emitOnce) { + this.elements = elements; + this.emitOnce = emitOnce; + this.numCheckpointsComplete = 0; + } + + @Override + public synchronized InputStatus pollNext(ReaderOutput readerOutput) { + if (checkpointToAwait == null) { + checkpointToAwait = numCheckpointsComplete + 2; + } + switch (numTimesEmitted) { + case 0: + emitElements(readerOutput, false); + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + emitElements(readerOutput, true); + if (numCheckpointsComplete < checkpointToAwait + 2) { + return InputStatus.MORE_AVAILABLE; + } + break; + case 1: + emitElements(readerOutput, true); + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + break; + case 2: + // Maybe missed notifyCheckpointComplete, wait next notifyCheckpointComplete + if (numCheckpointsComplete < checkpointToAwait) { + return InputStatus.MORE_AVAILABLE; + } + break; + } + return InputStatus.END_OF_INPUT; + } - if (context.isRestored()) { + @Override + public void addSplits(List list) { + checkpointedState.restoreState(list); List retrievedStates = new ArrayList<>(); for (Integer entry : this.checkpointedState.get()) { retrievedStates.add(entry); @@ -85,76 +137,27 @@ public void initializeState(FunctionInitializationContext context) throws Except getClass().getSimpleName() + " retrieved invalid numTimesEmitted: " + numTimesEmitted); - } else { - this.numTimesEmitted = 0; } - } - @Override - public void run(SourceContext ctx) throws Exception { - switch (numTimesEmitted) { - case 0: - emitElementsAndWaitForCheckpoints(ctx, false); - emitElementsAndWaitForCheckpoints(ctx, true); - break; - case 1: - emitElementsAndWaitForCheckpoints(ctx, true); - break; - case 2: - // Maybe missed notifyCheckpointComplete, wait next notifyCheckpointComplete - final Object lock = ctx.getCheckpointLock(); - synchronized (lock) { - int checkpointToAwait = numCheckpointsComplete + 2; - while (running && numCheckpointsComplete < checkpointToAwait) { - lock.wait(1); - } - } - break; + @Override + public List snapshotState(long l) { + this.checkpointedState.clear(); + this.checkpointedState.add(this.numTimesEmitted); + return this.checkpointedState.snapshotState(); } - } - private void emitElementsAndWaitForCheckpoints(SourceContext ctx, boolean isSecond) - throws InterruptedException { - final Object lock = ctx.getCheckpointLock(); + @Override + public void notifyCheckpointComplete(long checkpointId) { + numCheckpointsComplete++; + } - final int checkpointToAwait; - synchronized (lock) { - checkpointToAwait = numCheckpointsComplete + 2; + private void emitElements(ReaderOutput readerOutput, boolean isSecond) { if (!isSecond || !emitOnce) { for (T t : elements) { - ctx.collect(t); + readerOutput.collect(t); } } numTimesEmitted++; } - - synchronized (lock) { - while (running && numCheckpointsComplete < checkpointToAwait) { - lock.wait(1); - } - } - } - - @Override - public void cancel() { - running = false; - } - - @Override - public void notifyCheckpointComplete(long checkpointId) { - numCheckpointsComplete++; - } - - @Override - public void notifyCheckpointAborted(long checkpointId) {} - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - Preconditions.checkState( - this.checkpointedState != null, - "The " + getClass().getSimpleName() + " has not been properly initialized."); - - this.checkpointedState.clear(); - this.checkpointedState.add(this.numTimesEmitted); } } 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 index ad8a2d45a0368..e186080d9f45d 100644 --- 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 @@ -22,11 +22,13 @@ 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.utils.DateTimeUtils; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import java.util.Collections; import java.util.Map; import java.util.Optional; @@ -51,7 +53,8 @@ public void testAnalyzeTable() throws Catalog.TableNotExistException { sql("INSERT INTO T VALUES ('2', 'aaa', 1, 2)"); sql("ANALYZE TABLE T COMPUTE STATISTICS"); - Optional statisticsOpt = paimonTable("T").statistics(); + FileStoreTable table = paimonTable("T"); + Optional statisticsOpt = table.statistics(); assertThat(statisticsOpt.isPresent()).isTrue(); Statistics stats = statisticsOpt.get(); @@ -60,6 +63,16 @@ public void testAnalyzeTable() throws Catalog.TableNotExistException { Assertions.assertTrue(stats.mergedRecordSize().isPresent()); Assertions.assertTrue(stats.colStats().isEmpty()); + + // by default, caching catalog should cache it + Optional newStats = table.statistics(); + assertThat(newStats.isPresent()).isTrue(); + assertThat(newStats.get()).isSameAs(stats); + + // copy the table + newStats = table.copy(Collections.singletonMap("a", "b")).statistics(); + assertThat(newStats.isPresent()).isTrue(); + assertThat(newStats.get()).isSameAs(stats); } @Test diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java index 9268a236b6cb6..e32150b1fe822 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogPropertiesUtilTest.java @@ -21,27 +21,35 @@ import org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil; import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.WatermarkSpec; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.WatermarkSpec; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionVisitor; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.SqlCallExpression; +import org.apache.flink.table.types.DataType; import org.junit.jupiter.api.Test; 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 static org.apache.flink.table.descriptors.DescriptorProperties.DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.EXPR; -import static org.apache.flink.table.descriptors.DescriptorProperties.METADATA; -import static org.apache.flink.table.descriptors.DescriptorProperties.NAME; -import static org.apache.flink.table.descriptors.DescriptorProperties.VIRTUAL; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_ROWTIME; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; -import static org.apache.flink.table.descriptors.DescriptorProperties.WATERMARK_STRATEGY_EXPR; -import static org.apache.flink.table.descriptors.Schema.SCHEMA; +import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.SCHEMA; import static org.apache.paimon.flink.utils.FlinkCatalogPropertiesUtil.compoundKey; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.EXPR; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.METADATA; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.NAME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.VIRTUAL; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_ROWTIME; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_DATA_TYPE; +import static org.apache.paimon.flink.utils.FlinkDescriptorProperties.WATERMARK_STRATEGY_EXPR; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link FlinkCatalogPropertiesUtil}. */ @@ -49,18 +57,27 @@ public class FlinkCatalogPropertiesUtilTest { @Test public void testSerDeNonPhysicalColumns() { - Map indexMap = new HashMap<>(); - indexMap.put("comp", 2); - indexMap.put("meta1", 3); - indexMap.put("meta2", 5); - List columns = new ArrayList<>(); - columns.add(TableColumn.computed("comp", DataTypes.INT(), "`k` * 2")); - columns.add(TableColumn.metadata("meta1", DataTypes.VARCHAR(10))); - columns.add(TableColumn.metadata("meta2", DataTypes.BIGINT().notNull(), "price", true)); + List columns = new ArrayList<>(); + columns.add(new Schema.UnresolvedComputedColumn("comp", new SqlCallExpression("`k` * 2"))); + columns.add( + new Schema.UnresolvedMetadataColumn("meta1", DataTypes.VARCHAR(10), null, false)); + columns.add( + new Schema.UnresolvedMetadataColumn( + "meta2", DataTypes.BIGINT().notNull(), "price", true, null)); + + List resolvedColumns = new ArrayList<>(); + resolvedColumns.add(Column.physical("phy1", DataTypes.INT())); + resolvedColumns.add(Column.physical("phy2", DataTypes.INT())); + resolvedColumns.add( + Column.computed("comp", new TestResolvedExpression("`k` * 2", DataTypes.INT()))); + resolvedColumns.add(Column.metadata("meta1", DataTypes.VARCHAR(10), null, false)); + resolvedColumns.add(Column.physical("phy3", DataTypes.INT())); + resolvedColumns.add(Column.metadata("meta2", DataTypes.BIGINT().notNull(), "price", true)); // validate serialization Map serialized = - FlinkCatalogPropertiesUtil.serializeNonPhysicalColumns(indexMap, columns); + FlinkCatalogPropertiesUtil.serializeNonPhysicalNewColumns( + new ResolvedSchema(resolvedColumns, Collections.emptyList(), null)); Map expected = new HashMap<>(); expected.put(compoundKey(SCHEMA, 2, NAME), "comp"); @@ -80,27 +97,26 @@ public void testSerDeNonPhysicalColumns() { assertThat(serialized).containsExactlyInAnyOrderEntriesOf(expected); // validate deserialization - List deserialized = new ArrayList<>(); - deserialized.add(FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 2)); - deserialized.add(FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 3)); - deserialized.add(FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 5)); + Schema.Builder builder = Schema.newBuilder(); + FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 2, builder); + FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 3, builder); + FlinkCatalogPropertiesUtil.deserializeNonPhysicalColumn(serialized, 5, builder); - assertThat(deserialized).isEqualTo(columns); - - // validate that + assertThat(builder.build().getColumns()) + .containsExactly(columns.toArray(new Schema.UnresolvedColumn[0])); } @Test public void testSerDeWatermarkSpec() { WatermarkSpec watermarkSpec = - new WatermarkSpec( + WatermarkSpec.of( "test_time", - "`test_time` - INTERVAL '0.001' SECOND", - DataTypes.TIMESTAMP(3)); + new TestResolvedExpression( + "`test_time` - INTERVAL '0.001' SECOND", DataTypes.TIMESTAMP(3))); // validate serialization Map serialized = - FlinkCatalogPropertiesUtil.serializeWatermarkSpec(watermarkSpec); + FlinkCatalogPropertiesUtil.serializeNewWatermarkSpec(watermarkSpec); Map expected = new HashMap<>(); String watermarkPrefix = compoundKey(SCHEMA, WATERMARK, 0); @@ -113,9 +129,13 @@ public void testSerDeWatermarkSpec() { assertThat(serialized).containsExactlyInAnyOrderEntriesOf(expected); // validate serialization - WatermarkSpec deserialized = - FlinkCatalogPropertiesUtil.deserializeWatermarkSpec(serialized); - assertThat(deserialized).isEqualTo(watermarkSpec); + Schema.Builder builder = Schema.newBuilder(); + FlinkCatalogPropertiesUtil.deserializeWatermarkSpec(serialized, builder); + assertThat(builder.build().getWatermarkSpecs()).hasSize(1); + Schema.UnresolvedWatermarkSpec actual = builder.build().getWatermarkSpecs().get(0); + assertThat(actual.getColumnName()).isEqualTo(watermarkSpec.getRowtimeAttribute()); + assertThat(actual.getWatermarkExpression().asSummaryString()) + .isEqualTo(watermarkSpec.getWatermarkExpression().asSummaryString()); } @Test @@ -150,4 +170,44 @@ public void testNonPhysicalColumnsCount() { oldStyleOptions, Arrays.asList("phy1", "phy2"))) .isEqualTo(3); } + + private static class TestResolvedExpression implements ResolvedExpression { + private final String name; + private final DataType outputDataType; + + private TestResolvedExpression(String name, DataType outputDataType) { + this.name = name; + this.outputDataType = outputDataType; + } + + @Override + public DataType getOutputDataType() { + return outputDataType; + } + + @Override + public List getResolvedChildren() { + return Collections.emptyList(); + } + + @Override + public String asSummaryString() { + return new SqlCallExpression(name).asSummaryString(); + } + + @Override + public String asSerializableString() { + return name; + } + + @Override + public List getChildren() { + return Collections.emptyList(); + } + + @Override + public R accept(ExpressionVisitor expressionVisitor) { + return null; + } + } } 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 27a89510975f3..e4286eb18172d 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 @@ -850,7 +850,7 @@ private static void checkEquals(CatalogBaseTable t1, CatalogBaseTable t2) { assertThat(t2.getComment()).isEqualTo(t1.getComment()); assertThat(t2.getOptions()).isEqualTo(t1.getOptions()); if (t1.getTableKind() == CatalogBaseTable.TableKind.TABLE) { - assertThat(t2.getSchema()).isEqualTo(t1.getSchema()); + assertThat(t2.getUnresolvedSchema()).isEqualTo(t1.getUnresolvedSchema()); assertThat(((CatalogTable) (t2)).getPartitionKeys()) .isEqualTo(((CatalogTable) (t1)).getPartitionKeys()); assertThat(((CatalogTable) (t2)).isPartitioned()) @@ -864,7 +864,12 @@ private static void checkEquals(CatalogBaseTable t1, CatalogBaseTable t2) { t2.getUnresolvedSchema() .resolve(new TestSchemaResolver())) .build()) - .isEqualTo(t1.getSchema().toSchema()); + .isEqualTo( + Schema.newBuilder() + .fromResolvedSchema( + t1.getUnresolvedSchema() + .resolve(new TestSchemaResolver())) + .build()); assertThat(mt2.getPartitionKeys()).isEqualTo(mt1.getPartitionKeys()); assertThat(mt2.isPartitioned()).isEqualTo(mt1.isPartitioned()); // validate definition query diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java index c46c4c3589222..8df379a71b786 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkJobRecoveryITCase.java @@ -65,7 +65,7 @@ public void before() throws IOException { .set( CheckpointingOptions.EXTERNALIZED_CHECKPOINT_RETENTION, ExternalizedCheckpointRetention.RETAIN_ON_CANCELLATION) - .removeConfig(CheckpointingOptions.CHECKPOINTING_INTERVAL); + .removeKey("execution.checkpointing.interval"); // insert source data batchSql("INSERT INTO source_table1 VALUES (1, 'test-1', '20241030')"); @@ -219,10 +219,9 @@ private void testRecoverFromSavepoint( batchSql(sql); } - Configuration config = - sEnv.getConfig() - .getConfiguration() - .set(StateRecoveryOptions.SAVEPOINT_PATH, checkpointPath); + Configuration config = sEnv.getConfig().getConfiguration(); + // use config string to stay compatible with flink 1.19- + config.setString("execution.state-recovery.path", checkpointPath); for (Map.Entry entry : recoverOptions.entrySet()) { config.setString(entry.getKey(), entry.getValue()); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkLineageITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkLineageITCase.java deleted file mode 100644 index 5b61d5272f80e..0000000000000 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkLineageITCase.java +++ /dev/null @@ -1,206 +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; - -import org.apache.paimon.lineage.DataLineageEntity; -import org.apache.paimon.lineage.LineageMeta; -import org.apache.paimon.lineage.LineageMetaFactory; -import org.apache.paimon.lineage.TableLineageEntity; -import org.apache.paimon.predicate.Predicate; - -import org.apache.flink.configuration.PipelineOptions; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.junit.jupiter.api.Test; - -import javax.annotation.Nullable; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -import static org.apache.paimon.options.CatalogOptions.LINEAGE_META; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** ITCase for flink table and data lineage. */ -public class FlinkLineageITCase extends CatalogITCaseBase { - private static final String THROWING_META = "throwing-meta"; - private static final Map> jobSourceTableLineages = - new HashMap<>(); - private static final Map> jobSinkTableLineages = - new HashMap<>(); - - @Override - protected List ddl() { - return Collections.singletonList("CREATE TABLE IF NOT EXISTS T (a INT, b INT, c INT)"); - } - - @Override - protected Map catalogOptions() { - return Collections.singletonMap(LINEAGE_META.key(), THROWING_META); - } - - @Test - public void testTableLineage() throws Exception { - // Validate for source and sink lineage when pipeline name is null - assertThatThrownBy( - () -> tEnv.executeSql("INSERT INTO T VALUES (1, 2, 3),(4, 5, 6);").await()) - .hasCauseExactlyInstanceOf(ValidationException.class) - .hasRootCauseMessage("Cannot get pipeline name for lineage meta."); - assertThatThrownBy(() -> tEnv.executeSql("SELECT * FROM T").collect().close()) - .hasCauseExactlyInstanceOf(ValidationException.class) - .hasRootCauseMessage("Cannot get pipeline name for lineage meta."); - - // Call storeSinkTableLineage and storeSourceTableLineage methods - tEnv.getConfig().getConfiguration().set(PipelineOptions.NAME, "insert_t_job"); - tEnv.executeSql("INSERT INTO T VALUES (1, 2, 3),(4, 5, 6);").await(); - assertThat(jobSinkTableLineages).isNotEmpty(); - TableLineageEntity sinkTableLineage = - jobSinkTableLineages.get("insert_t_job").get("default.T.insert_t_job"); - assertThat(sinkTableLineage.getTable()).isEqualTo("T"); - - List sinkTableRows = new ArrayList<>(); - try (CloseableIterator iterator = - tEnv.executeSql("SELECT * FROM sys.sink_table_lineage").collect()) { - while (iterator.hasNext()) { - sinkTableRows.add(iterator.next()); - } - } - assertThat(sinkTableRows.size()).isEqualTo(1); - Row sinkTableRow = sinkTableRows.get(0); - assertThat(sinkTableRow.getField("database_name")).isEqualTo("default"); - assertThat(sinkTableRow.getField("table_name")).isEqualTo("T"); - assertThat(sinkTableRow.getField("job_name")).isEqualTo("insert_t_job"); - - tEnv.getConfig().getConfiguration().set(PipelineOptions.NAME, "select_t_job"); - tEnv.executeSql("SELECT * FROM T").collect().close(); - assertThat(jobSourceTableLineages).isNotEmpty(); - TableLineageEntity sourceTableLineage = - jobSourceTableLineages.get("select_t_job").get("default.T.select_t_job"); - assertThat(sourceTableLineage.getTable()).isEqualTo("T"); - - List sourceTableRows = new ArrayList<>(); - try (CloseableIterator iterator = - tEnv.executeSql("SELECT * FROM sys.source_table_lineage").collect()) { - while (iterator.hasNext()) { - sourceTableRows.add(iterator.next()); - } - } - assertThat(sourceTableRows.size()).isEqualTo(1); - Row sourceTableRow = sourceTableRows.get(0); - assertThat(sourceTableRow.getField("database_name")).isEqualTo("default"); - assertThat(sourceTableRow.getField("table_name")).isEqualTo("T"); - assertThat(sourceTableRow.getField("job_name")).isEqualTo("select_t_job"); - } - - private static String getTableLineageKey(TableLineageEntity entity) { - return String.format("%s.%s.%s", entity.getDatabase(), entity.getTable(), entity.getJob()); - } - - /** Factory to create throwing lineage meta. */ - public static class TestingMemoryLineageMetaFactory implements LineageMetaFactory { - private static final long serialVersionUID = 1L; - - @Override - public String identifier() { - return THROWING_META; - } - - @Override - public LineageMeta create(LineageMetaContext context) { - return new TestingMemoryLineageMeta(); - } - } - - /** Throwing specific exception in each method. */ - private static class TestingMemoryLineageMeta implements LineageMeta { - - @Override - public void saveSourceTableLineage(TableLineageEntity entity) { - jobSourceTableLineages - .computeIfAbsent(entity.getJob(), key -> new HashMap<>()) - .put(getTableLineageKey(entity), entity); - } - - @Override - public void deleteSourceTableLineage(String job) { - jobSourceTableLineages.remove(job); - } - - @Override - public Iterator sourceTableLineages(@Nullable Predicate predicate) { - return jobSourceTableLineages.values().stream() - .flatMap(v -> v.values().stream()) - .iterator(); - } - - @Override - public void saveSinkTableLineage(TableLineageEntity entity) { - assertThat(entity.getJob()).isEqualTo("insert_t_job"); - assertThat(entity.getTable()).isEqualTo("T"); - assertThat(entity.getDatabase()).isEqualTo("default"); - jobSinkTableLineages - .computeIfAbsent(entity.getJob(), key -> new HashMap<>()) - .put(getTableLineageKey(entity), entity); - } - - @Override - public Iterator sinkTableLineages(@Nullable Predicate predicate) { - return jobSinkTableLineages.values().stream() - .flatMap(v -> v.values().stream()) - .iterator(); - } - - @Override - public void deleteSinkTableLineage(String job) { - jobSinkTableLineages.remove(job); - } - - @Override - public void saveSourceDataLineage(DataLineageEntity entity) { - assertThat(entity.getJob()).isEqualTo("select_t_job"); - assertThat(entity.getTable()).isEqualTo("T"); - assertThat(entity.getDatabase()).isEqualTo("default"); - throw new UnsupportedOperationException("Method saveSinkTableLineage is not supported"); - } - - @Override - public Iterator sourceDataLineages(@Nullable Predicate predicate) { - throw new UnsupportedOperationException(); - } - - @Override - public void saveSinkDataLineage(DataLineageEntity entity) { - throw new UnsupportedOperationException(); - } - - @Override - public Iterator sinkDataLineages(@Nullable Predicate predicate) { - throw new UnsupportedOperationException(); - } - - @Override - public void close() throws Exception {} - } -} 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 3e9ba2194aed4..a6abde57b80c8 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 @@ -25,6 +25,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.ValueSource; import java.util.Collections; import java.util.List; @@ -977,4 +978,32 @@ public void testPartialCacheBucketKeyOrder(LookupCacheMode mode) throws Exceptio iterator.close(); } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testOverwriteDimTable(boolean isPkTable) throws Exception { + sql( + "CREATE TABLE DIM (i INT %s, v int, pt STRING) " + + "PARTITIONED BY (pt) WITH ('continuous.discovery-interval'='1 ms')", + isPkTable ? "PRIMARY KEY NOT ENFORCED" : ""); + + BlockingIterator iterator = + streamSqlBlockIter( + "SELECT T.i, D.v, D.pt FROM T LEFT JOIN DIM FOR SYSTEM_TIME AS OF T.proctime AS D ON T.i = D.i"); + + sql("INSERT INTO DIM VALUES (1, 11, 'A'), (2, 22, 'B')"); + sql("INSERT INTO T VALUES (1), (2)"); + + List result = iterator.collect(2); + assertThat(result).containsExactlyInAnyOrder(Row.of(1, 11, "A"), Row.of(2, 22, "B")); + + sql("INSERT OVERWRITE DIM PARTITION (pt='B') VALUES (3, 33)"); + Thread.sleep(2000); // wait refresh + sql("INSERT INTO T VALUES (3)"); + + result = iterator.collect(1); + assertThat(result).containsExactlyInAnyOrder(Row.of(3, 33, "B")); + + iterator.close(); + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ObjectTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ObjectTableITCase.java index b9e30035b093b..d3ad1d4a52f4f 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ObjectTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ObjectTableITCase.java @@ -80,4 +80,31 @@ public void testObjectTableRefresh() throws IOException { .hasMessageContaining("Object table does not support Write."); assertThat(sql("SELECT name, length FROM T")).containsExactlyInAnyOrder(Row.of("f1", 5L)); } + + @Test + public void testObjectTableRefreshInPrivileged() throws IOException { + sql("CALL sys.init_file_based_privilege('root-passwd')"); + + tEnv.executeSql( + String.format( + "CREATE CATALOG rootcat WITH (\n" + + " 'type' = 'paimon',\n" + + " 'warehouse' = '%s',\n" + + " 'user' = 'root',\n" + + " 'password' = 'root-passwd'\n" + + ")", + path)); + tEnv.useCatalog("rootcat"); + + Path objectLocation = new Path(path + "/object-location"); + FileIO fileIO = LocalFileIO.create(); + sql( + "CREATE TABLE T WITH ('type' = 'object-table', 'object-location' = '%s')", + objectLocation); + + // add new file + fileIO.overwriteFileUtf8(new Path(objectLocation, "f0"), "1,2,3"); + sql("CALL sys.refresh_object_table('default.T')"); + assertThat(sql("SELECT name, length FROM T")).containsExactlyInAnyOrder(Row.of("f0", 5L)); + } } 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 3fa95edb8682f..4ee539c4fd270 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 @@ -59,6 +59,8 @@ /** Tests for changelog table with primary keys. */ public class PrimaryKeyFileStoreTableITCase extends AbstractTestBase { + private static final int TIMEOUT = 180; + // ------------------------------------------------------------------------ // Test Utilities // ------------------------------------------------------------------------ @@ -95,12 +97,38 @@ private String createCatalogSql(String catalogName, String warehouse) { catalogName, warehouse, defaultPropertyString); } + private CloseableIterator collect(TableResult result) { + return collect(result, TIMEOUT); + } + + private CloseableIterator collect(TableResult result, int timeout) { + JobClient client = result.getJobClient().get(); + Thread timeoutThread = + new Thread( + () -> { + for (int i = 0; i < timeout; i++) { + try { + Thread.sleep(1000); + if (client.getJobStatus().get().isGloballyTerminalState()) { + return; + } + } catch (Exception e) { + client.cancel(); + throw new RuntimeException(e); + } + } + client.cancel(); + }); + timeoutThread.start(); + return result.collect(); + } + // ------------------------------------------------------------------------ // Constructed Tests // ------------------------------------------------------------------------ @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testFullCompactionTriggerInterval() throws Exception { innerTestChangelogProducing( Arrays.asList( @@ -109,7 +137,7 @@ public void testFullCompactionTriggerInterval() throws Exception { } @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testFullCompactionWithLongCheckpointInterval() throws Exception { // create table TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().parallelism(1).build(); @@ -135,7 +163,7 @@ public void testFullCompactionWithLongCheckpointInterval() throws Exception { .build(); sEnv.executeSql(createCatalogSql("testCatalog", path)); sEnv.executeSql("USE CATALOG testCatalog"); - CloseableIterator it = sEnv.executeSql("SELECT * FROM T").collect(); + CloseableIterator it = collect(sEnv.executeSql("SELECT * FROM T")); // run compact job StreamExecutionEnvironment env = @@ -168,7 +196,7 @@ public void testFullCompactionWithLongCheckpointInterval() throws Exception { } @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testLookupChangelog() throws Exception { innerTestChangelogProducing(Collections.singletonList("'changelog-producer' = 'lookup'")); } @@ -190,7 +218,7 @@ public void testTableReadWriteBranch() throws Exception { + "'bucket' = '2'" + ")"); - CloseableIterator it = sEnv.executeSql("SELECT * FROM T2").collect(); + CloseableIterator it = collect(sEnv.executeSql("SELECT * FROM T2")); // insert data sEnv.executeSql("INSERT INTO T2 VALUES (1, 'A')").await(); @@ -213,7 +241,7 @@ public void testTableReadWriteBranch() throws Exception { sEnv.executeSql("ALTER TABLE T2 SET ('changelog-producer'='full-compaction')"); CloseableIterator branchIt = - sEnv.executeSql("select * from T2 /*+ OPTIONS('branch' = 'branch1') */").collect(); + collect(sEnv.executeSql("select * from T2 /*+ OPTIONS('branch' = 'branch1') */")); // insert data to branch sEnv.executeSql( "INSERT INTO T2/*+ OPTIONS('branch' = 'branch1') */ VALUES (10, 'v10'),(11, 'v11'),(12, 'v12')") @@ -261,7 +289,7 @@ private void innerTestChangelogProducing(List options) throws Exception sEnv.executeSql( "INSERT INTO T SELECT SUM(i) AS k, g AS v FROM `default_catalog`.`default_database`.`S` GROUP BY g"); - CloseableIterator it = sEnv.executeSql("SELECT * FROM T").collect(); + CloseableIterator it = collect(sEnv.executeSql("SELECT * FROM T")); // write initial data sEnv.executeSql( @@ -329,7 +357,7 @@ public void testBatchJobWithConflictAndRestart() throws Exception { result1.await(); result2.await(); - try (CloseableIterator it = tEnv.executeSql("SELECT * FROM t").collect()) { + try (CloseableIterator it = collect(tEnv.executeSql("SELECT * FROM t"))) { for (int i = 0; i < 3; i++) { assertThat(it).hasNext(); Row row = it.next(); @@ -338,7 +366,7 @@ public void testBatchJobWithConflictAndRestart() throws Exception { } } - @Timeout(60) + @Timeout(TIMEOUT) @ParameterizedTest() @ValueSource(booleans = {false, true}) public void testRecreateTableWithException(boolean isReloadData) throws Exception { @@ -361,7 +389,7 @@ public void testRecreateTableWithException(boolean isReloadData) throws Exceptio .build(); sEnv.executeSql(createCatalogSql("testCatalog", path + "/warehouse")); sEnv.executeSql("USE CATALOG testCatalog"); - CloseableIterator it = sEnv.executeSql("SELECT * FROM t").collect(); + CloseableIterator it = collect(sEnv.executeSql("SELECT * FROM t")); // first write List values = new ArrayList<>(); @@ -414,7 +442,7 @@ public void testRecreateTableWithException(boolean isReloadData) throws Exceptio } @Test - @Timeout(120) + @Timeout(TIMEOUT) public void testChangelogCompactInBatchWrite() throws Exception { TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().build(); String catalogDdl = @@ -504,7 +532,7 @@ public void testChangelogCompactInBatchWrite() throws Exception { } @Test - @Timeout(120) + @Timeout(TIMEOUT) public void testChangelogCompactInStreamWrite() throws Exception { TableEnvironment sEnv = tableEnvironmentBuilder() @@ -533,7 +561,7 @@ public void testChangelogCompactInStreamWrite() throws Exception { + "', 'source.monitor-interval' = '500ms' )"); sEnv.executeSql("INSERT INTO t SELECT * FROM `default_catalog`.`default_database`.`s`"); - CloseableIterator it = sEnv.executeSql("SELECT * FROM t").collect(); + CloseableIterator it = collect(sEnv.executeSql("SELECT * FROM t")); // write initial data List values = new ArrayList<>(); @@ -589,7 +617,7 @@ private List listAllFilesWithPrefix(String prefix) throws Exception { private void assertStreamingResult(TableResult result, List expected) throws Exception { List actual = new ArrayList<>(); - try (CloseableIterator it = result.collect()) { + try (CloseableIterator it = collect(result)) { while (actual.size() < expected.size() && it.hasNext()) { actual.add(it.next()); } @@ -611,14 +639,14 @@ private void assertStreamingResult(CloseableIterator it, List expected // ------------------------------------------------------------------------ @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testNoChangelogProducerBatchRandom() throws Exception { TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().build(); testNoChangelogProducerRandom(bEnv, 1, false); } @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testNoChangelogProducerStreamingRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = @@ -631,14 +659,14 @@ public void testNoChangelogProducerStreamingRandom() throws Exception { } @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testFullCompactionChangelogProducerBatchRandom() throws Exception { TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().build(); testFullCompactionChangelogProducerRandom(bEnv, 1, false); } @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testFullCompactionChangelogProducerStreamingRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = @@ -651,7 +679,7 @@ public void testFullCompactionChangelogProducerStreamingRandom() throws Exceptio } @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testStandAloneFullCompactJobRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = @@ -664,14 +692,14 @@ public void testStandAloneFullCompactJobRandom() throws Exception { } @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testLookupChangelogProducerBatchRandom() throws Exception { TableEnvironment bEnv = tableEnvironmentBuilder().batchMode().build(); testLookupChangelogProducerRandom(bEnv, 1, false); } @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testLookupChangelogProducerStreamingRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = @@ -684,7 +712,7 @@ public void testLookupChangelogProducerStreamingRandom() throws Exception { } @Test - @Timeout(180) + @Timeout(TIMEOUT) public void testStandAloneLookupJobRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = @@ -868,7 +896,7 @@ private void checkChangelogTestResult(int numProducers) throws Exception { ResultChecker checker = new ResultChecker(); int endCnt = 0; - try (CloseableIterator it = sEnv.executeSql("SELECT * FROM T").collect()) { + try (CloseableIterator it = collect(sEnv.executeSql("SELECT * FROM T"))) { while (it.hasNext()) { Row row = it.next(); checker.addChangelog(row); @@ -986,7 +1014,7 @@ private void checkBatchResult(int numProducers) throws Exception { bEnv.executeSql("USE CATALOG testCatalog"); ResultChecker checker = new ResultChecker(); - try (CloseableIterator it = bEnv.executeSql("SELECT * FROM T").collect()) { + try (CloseableIterator it = collect(bEnv.executeSql("SELECT * FROM T"))) { while (it.hasNext()) { checker.addChangelog(it.next()); } 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 10de1ae4839f1..732e964542368 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 @@ -801,6 +801,43 @@ public void testStreamingReadOverwriteWithoutPartitionedRecords() throws Excepti streamingItr.close(); } + @Test + public void testStreamingReadOverwriteWithDeleteRecords() throws Exception { + String table = + createTable( + Arrays.asList("currency STRING", "rate BIGINT", "dt STRING"), + Collections.singletonList("currency"), + Collections.emptyList(), + Collections.emptyList(), + streamingReadOverwrite); + + insertInto( + table, + "('US Dollar', 102, '2022-01-01')", + "('Yen', 1, '2022-01-02')", + "('Euro', 119, '2022-01-02')"); + + bEnv.executeSql(String.format("DELETE FROM %s WHERE currency = 'Euro'", table)).await(); + + checkFileStorePath(table, Collections.emptyList()); + + // test projection and filter + BlockingIterator streamingItr = + testStreamingRead( + buildQuery(table, "currency, rate", "WHERE dt = '2022-01-02'"), + Collections.singletonList(changelogRow("+I", "Yen", 1L))); + + insertOverwrite(table, "('US Dollar', 100, '2022-01-02')", "('Yen', 10, '2022-01-01')"); + + validateStreamingReadResult( + streamingItr, + Arrays.asList( + changelogRow("-D", "Yen", 1L), changelogRow("+I", "US Dollar", 100L))); + assertNoMoreRecords(streamingItr); + + streamingItr.close(); + } + @Test public void testUnsupportStreamingReadOverwriteWithoutPk() { assertThatThrownBy( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java index 08969bddfdb3a..d5747d2e28d4b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/RescaleBucketITCase.java @@ -26,7 +26,6 @@ import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.types.Row; import org.junit.jupiter.api.Test; @@ -106,9 +105,10 @@ public void testSuspendAndRecoverAfterRescaleOverwrite() throws Exception { assertThat(batchSql("SELECT * FROM T3")).containsExactlyInAnyOrderElementsOf(committedData); // step5: resume streaming job + // use config string to stay compatible with flink 1.19- sEnv.getConfig() .getConfiguration() - .set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + .setString("execution.state-recovery.path", savepointPath); JobClient resumedJobClient = startJobAndCommitSnapshot(streamSql, snapshotAfterRescale.id()); // stop job 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 08f79efccb763..a8e8332156b37 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 @@ -25,8 +25,11 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.types.Row; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.time.format.DateTimeFormatter; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -35,6 +38,7 @@ 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; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** ITCase for schema changes. */ @@ -248,7 +252,8 @@ public void testModifyColumnTypeFromNumericToDecimal() { public void testModifyColumnTypeBooleanAndNumeric() { // boolean To numeric and numeric To boolean sql("CREATE TABLE T (a BOOLEAN, b BOOLEAN, c TINYINT, d INT, e BIGINT, f DOUBLE)"); - sql("INSERT INTO T VALUES(true, false, cast(0 as TINYINT), 1 , 123, 3.14)"); + sql( + "INSERT INTO T VALUES(true, false, cast(0 as TINYINT), 1 , -9223372036854775808, 3.14)"); sql("ALTER TABLE T MODIFY (a TINYINT, b INT, c BOOLEAN, d BOOLEAN, e BOOLEAN)"); List result = sql("SHOW CREATE TABLE T"); @@ -1015,7 +1020,6 @@ public void testSequenceFieldSortOrder() { 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 @@ -1024,7 +1028,6 @@ public void testSequenceFieldSortOrder() { 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 @@ -1033,7 +1036,6 @@ public void testSequenceFieldSortOrder() { 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]]"); } @@ -1089,4 +1091,112 @@ public void testAlterBucket() { UnsupportedOperationException.class, "Cannot change bucket to -1.")); } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testUpdateNestedColumn(String formatType) { + sql( + "CREATE TABLE T " + + "( k INT, v ROW(f1 INT, f2 ROW(f1 STRING, f2 INT NOT NULL)), PRIMARY KEY (k) NOT ENFORCED ) " + + "WITH ( 'bucket' = '1', 'file.format' = '" + + formatType + + "' )"); + sql( + "INSERT INTO T VALUES (1, ROW(10, ROW('apple', 100))), (2, ROW(20, ROW('banana', 200)))"); + assertThat(sql("SELECT * FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, Row.of(10, Row.of("apple", 100))), + Row.of(2, Row.of(20, Row.of("banana", 200)))); + + sql("ALTER TABLE T MODIFY (v ROW(f1 BIGINT, f2 ROW(f3 DOUBLE, f2 INT), f3 STRING))"); + sql( + "INSERT INTO T VALUES " + + "(1, ROW(1000000000001, ROW(101.0, 101), 'cat')), " + + "(3, ROW(3000000000001, ROW(301.0, CAST(NULL AS INT)), 'dog'))"); + assertThat(sql("SELECT * FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, Row.of(1000000000001L, Row.of(101.0, 101), "cat")), + Row.of(2, Row.of(20L, Row.of(null, 200), null)), + Row.of(3, Row.of(3000000000001L, Row.of(301.0, null), "dog"))); + + sql( + "ALTER TABLE T MODIFY (v ROW(f1 BIGINT, f2 ROW(f3 DOUBLE, f1 STRING, f2 INT), f3 STRING))"); + sql( + "INSERT INTO T VALUES " + + "(1, ROW(1000000000002, ROW(102.0, 'APPLE', 102), 'cat')), " + + "(4, ROW(4000000000002, ROW(402.0, 'LEMON', 402), 'tiger'))"); + assertThat(sql("SELECT k, v.f2.f1, v.f3 FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, "APPLE", "cat"), + Row.of(2, null, null), + Row.of(3, null, "dog"), + Row.of(4, "LEMON", "tiger")); + + assertThatCode(() -> sql("ALTER TABLE T MODIFY (v ROW(f1 BIGINT, f2 INT, f3 STRING))")) + .hasRootCauseMessage( + "Column v.f2 can only be updated to row type, and cannot be updated to INTEGER type"); + } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testUpdateRowInArrayAndMap(String formatType) { + sql( + "CREATE TABLE T " + + "( k INT, v1 ARRAY, v2 MAP, PRIMARY KEY (k) NOT ENFORCED ) " + + "WITH ( 'bucket' = '1', 'file.format' = '" + + formatType + + "' )"); + sql( + "INSERT INTO T VALUES " + + "(1, ARRAY[ROW(100, 'apple'), ROW(101, 'banana')], MAP[100, ROW('cat', 1000), 101, ROW('dog', 1001)]), " + + "(2, ARRAY[ROW(200, 'pear'), ROW(201, 'grape')], MAP[200, ROW('tiger', 2000), 201, ROW('wolf', 2001)])"); + + Map map1 = new HashMap<>(); + map1.put(100, Row.of("cat", 1000)); + map1.put(101, Row.of("dog", 1001)); + Map map2 = new HashMap<>(); + map2.put(200, Row.of("tiger", 2000)); + map2.put(201, Row.of("wolf", 2001)); + assertThat(sql("SELECT * FROM T")) + .containsExactlyInAnyOrder( + Row.of(1, new Row[] {Row.of(100, "apple"), Row.of(101, "banana")}, map1), + Row.of(2, new Row[] {Row.of(200, "pear"), Row.of(201, "grape")}, map2)); + + sql( + "ALTER TABLE T MODIFY (v1 ARRAY, v2 MAP)"); + sql( + "INSERT INTO T VALUES " + + "(1, ARRAY[ROW(1000000000000, 'apple', 'A'), ROW(1000000000001, 'banana', 'B')], MAP[100, ROW(1000.0, 1000), 101, ROW(1001.0, 1001)]), " + + "(3, ARRAY[ROW(3000000000000, 'mango', 'M'), ROW(3000000000001, 'cherry', 'C')], MAP[300, ROW(3000.0, 3000), 301, ROW(3001.0, 3001)])"); + + map1.clear(); + map1.put(100, Row.of(1000.0, 1000)); + map1.put(101, Row.of(1001.0, 1001)); + map2.clear(); + map2.put(200, Row.of(null, 2000)); + map2.put(201, Row.of(null, 2001)); + Map map3 = new HashMap<>(); + map3.put(300, Row.of(3000.0, 3000)); + map3.put(301, Row.of(3001.0, 3001)); + assertThat(sql("SELECT v2, v1, k FROM T")) + .containsExactlyInAnyOrder( + Row.of( + map1, + new Row[] { + Row.of(1000000000000L, "apple", "A"), + Row.of(1000000000001L, "banana", "B") + }, + 1), + Row.of( + map2, + new Row[] {Row.of(200L, "pear", null), Row.of(201L, "grape", null)}, + 2), + Row.of( + map3, + new Row[] { + Row.of(3000000000000L, "mango", "M"), + Row.of(3000000000001L, "cherry", "C") + }, + 3)); + } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java index 594affc124ebb..75b96cbe02eb3 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SerializableRowData.java @@ -47,8 +47,10 @@ public SerializableRowData(RowData row, TypeSerializer serializer) { this.serializer = serializer; } - private void writeObject(ObjectOutputStream out) throws IOException { + private synchronized void writeObject(ObjectOutputStream out) throws IOException { out.defaultWriteObject(); + // This following invocation needs to be synchronized to avoid racing problems when the + // serializer is reused across multiple subtasks. serializer.serialize(row, new DataOutputViewStreamWrapper(out)); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.java new file mode 100644 index 0000000000000..771f4acc5e585 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/SystemTableITCase.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; + +import org.apache.paimon.utils.BlockingIterator; + +import org.apache.flink.types.Row; +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** ITCase for system table. */ +public class SystemTableITCase extends CatalogTableITCase { + + @Test + public void testBinlogTableStreamRead() throws Exception { + sql( + "CREATE TABLE T (a INT, b INT, primary key (a) NOT ENFORCED) with ('changelog-producer' = 'lookup', " + + "'bucket' = '2')"); + BlockingIterator iterator = + streamSqlBlockIter("SELECT * FROM T$binlog /*+ OPTIONS('scan.mode' = 'latest') */"); + sql("INSERT INTO T VALUES (1, 2)"); + sql("INSERT INTO T VALUES (1, 3)"); + sql("INSERT INTO T VALUES (2, 2)"); + List rows = iterator.collect(3); + assertThat(rows) + .containsExactly( + Row.of("+I", new Integer[] {1}, new Integer[] {2}), + Row.of("+U", new Integer[] {1, 1}, new Integer[] {2, 3}), + Row.of("+I", new Integer[] {2}, new Integer[] {2})); + iterator.close(); + } + + @Test + public void testBinlogTableBatchRead() throws Exception { + sql( + "CREATE TABLE T (a INT, b INT, primary key (a) NOT ENFORCED) with ('changelog-producer' = 'lookup', " + + "'bucket' = '2')"); + sql("INSERT INTO T VALUES (1, 2)"); + sql("INSERT INTO T VALUES (1, 3)"); + sql("INSERT INTO T VALUES (2, 2)"); + List rows = sql("SELECT * FROM T$binlog /*+ OPTIONS('scan.mode' = 'latest') */"); + assertThat(rows) + .containsExactly( + Row.of("+I", new Integer[] {1}, new Integer[] {3}), + Row.of("+I", new Integer[] {2}, new Integer[] {2})); + } +} 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 cb323542d4c11..fb8bee5d59620 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 @@ -20,6 +20,9 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSource; +import org.apache.paimon.flink.source.AbstractNonCoordinatedSourceReader; +import org.apache.paimon.flink.source.SimpleSourceSplit; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.reader.RecordReader; @@ -27,10 +30,16 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.utils.FailingFileIO; - +import org.apache.paimon.utils.TimeUtils; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.core.io.InputStatus; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.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; @@ -49,7 +58,6 @@ import java.util.List; import java.util.Random; -import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -202,7 +210,11 @@ public void testCompactionInStreamingMode() throws Exception { batchSql("ALTER TABLE append_table SET ('compaction.early-max.file-num' = '4')"); batchSql("ALTER TABLE append_table SET ('continuous.discovery-interval' = '1 s')"); - sEnv.getConfig().getConfiguration().set(CHECKPOINTING_INTERVAL, Duration.ofMillis(500)); + sEnv.getConfig() + .getConfiguration() + .setString( + "execution.checkpointing.interval", + TimeUtils.formatWithHighestUnit(Duration.ofMillis(500))); sEnv.executeSql( "CREATE TEMPORARY TABLE Orders_in (\n" + " f0 INT,\n" @@ -223,7 +235,11 @@ public void testCompactionInStreamingModeWithMaxWatermark() throws Exception { batchSql("ALTER TABLE append_table SET ('compaction.early-max.file-num' = '4')"); batchSql("ALTER TABLE append_table SET ('continuous.discovery-interval' = '1 s')"); - sEnv.getConfig().getConfiguration().set(CHECKPOINTING_INTERVAL, Duration.ofMillis(500)); + sEnv.getConfig() + .getConfiguration() + .setString( + "execution.checkpointing.interval", + TimeUtils.formatWithHighestUnit(Duration.ofMillis(500))); sEnv.executeSql( "CREATE TEMPORARY TABLE Orders_in (\n" + " f0 INT,\n" @@ -371,7 +387,12 @@ public void testStatelessWriter() throws Exception { .checkpointIntervalMs(500) .build(); DataStream source = - env.addSource(new TestStatelessWriterSource(table)).setParallelism(2).forward(); + env.fromSource( + new TestStatelessWriterSource(table), + WatermarkStrategy.noWatermarks(), + "TestStatelessWriterSource") + .setParallelism(2) + .forward(); StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); tEnv.registerCatalog("mycat", sEnv.getCatalog("PAIMON").get()); @@ -383,46 +404,59 @@ public void testStatelessWriter() throws Exception { .containsExactlyInAnyOrder(Row.of(1, "test"), Row.of(2, "test")); } - private static class TestStatelessWriterSource extends RichParallelSourceFunction { + private static class TestStatelessWriterSource extends AbstractNonCoordinatedSource { 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) { + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SourceReader createReader( + SourceReaderContext sourceReaderContext) throws Exception { + return new Reader(sourceReaderContext.getIndexOfSubtask()); + } + + private class Reader extends AbstractNonCoordinatedSourceReader { + private final int taskId; + private int waitCount; + + private Reader(int taskId) { + this.taskId = taskId; + this.waitCount = (taskId == 0 ? 0 : 10); + } + + @Override + public InputStatus pollNext(ReaderOutput readerOutput) throws Exception { + if (taskId == 0) { + if (waitCount == 0) { + readerOutput.collect(1); + } else if (countNumRecords() >= 1) { + // wait for the record to commit before exiting + Thread.sleep(1000); + return InputStatus.END_OF_INPUT; + } + } else { + int numRecords = countNumRecords(); + if (numRecords >= 1) { 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; - } + readerOutput.collect(2); + } else if (countNumRecords() >= 2) { + // make sure the next checkpoint is successful + Thread.sleep(1000); + return InputStatus.END_OF_INPUT; } } - waitCount--; } + waitCount--; Thread.sleep(1000); + return InputStatus.MORE_AVAILABLE; } } @@ -438,11 +472,6 @@ private int countNumRecords() throws Exception { } return ret; } - - @Override - public void cancel() { - isRunning = false; - } } @Override diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java index 71672551abcba..a55b01cc203be 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java @@ -32,6 +32,7 @@ import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; +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; @@ -44,8 +45,10 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches; import static org.apache.paimon.utils.Preconditions.checkState; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** IT cases for {@link CloneAction}. */ public class CloneActionITCase extends ActionITCaseBase { @@ -640,6 +643,46 @@ public void testCloneTableWithExpiration(String invoker) throws Exception { .isEqualTo(Collections.singletonList("+I[1]")); } + // ------------------------------------------------------------------------ + // Negative Tests + // ------------------------------------------------------------------------ + + @Test + public void testEmptySourceCatalog() { + String sourceWarehouse = getTempDirPath("source-ware"); + + TableEnvironment tEnv = tableEnvironmentBuilder().batchMode().parallelism(1).build(); + tEnv.executeSql( + "CREATE CATALOG sourcecat WITH (\n" + + " 'type' = 'paimon',\n" + + String.format(" 'warehouse' = '%s'\n", sourceWarehouse) + + ")"); + + String targetWarehouse = getTempDirPath("target-ware"); + + String[] args = + new String[] { + "clone", + "--warehouse", + sourceWarehouse, + "--target_warehouse", + targetWarehouse, + "--parallelism", + "1" + }; + CloneAction action = (CloneAction) ActionFactory.createAction(args).get(); + + StreamExecutionEnvironment env = + streamExecutionEnvironmentBuilder().streamingMode().allowRestart().build(); + action.withStreamExecutionEnvironment(env); + + assertThatThrownBy(action::run) + .satisfies( + anyCauseMatches( + IllegalStateException.class, + "Didn't find any table in source catalog.")); + } + // ------------------------------------------------------------------------ // Utils // ------------------------------------------------------------------------ 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 bc849f0a135ff..2c4fb64f331c3 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 @@ -23,13 +23,9 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.table.sink.StreamWriteBuilder; import org.apache.paimon.table.source.DataSplit; import org.apache.paimon.table.source.StreamTableScan; import org.apache.paimon.table.source.TableScan; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.RowType; import org.apache.paimon.utils.CommonTestUtils; import org.apache.paimon.utils.SnapshotManager; @@ -56,12 +52,6 @@ /** IT cases for {@link CompactAction}. */ public class CompactActionITCase extends CompactActionITCaseBase { - private static final DataType[] FIELD_TYPES = - new DataType[] {DataTypes.INT(), DataTypes.INT(), DataTypes.INT(), DataTypes.STRING()}; - - private static final RowType ROW_TYPE = - RowType.of(FIELD_TYPES, new String[] {"k", "v", "hh", "dt"}); - @Test @Timeout(60) public void testBatchCompact() throws Exception { @@ -402,31 +392,6 @@ public void testWrongUsage() throws Exception { .hasMessage("sort compact do not support 'partition_idle_time'."); } - private FileStoreTable prepareTable( - List partitionKeys, - List primaryKeys, - List bucketKey, - Map tableOptions) - throws Exception { - FileStoreTable table = - createFileStoreTable(ROW_TYPE, partitionKeys, primaryKeys, bucketKey, tableOptions); - - StreamWriteBuilder streamWriteBuilder = - table.newStreamWriteBuilder().withCommitUser(commitUser); - write = streamWriteBuilder.newWrite(); - commit = streamWriteBuilder.newCommit(); - - return table; - } - - private void checkLatestSnapshot( - FileStoreTable table, long snapshotId, Snapshot.CommitKind commitKind) { - SnapshotManager snapshotManager = table.snapshotManager(); - Snapshot snapshot = snapshotManager.snapshot(snapshotManager.latestSnapshotId()); - assertThat(snapshot.id()).isEqualTo(snapshotId); - assertThat(snapshot.commitKind()).isEqualTo(commitKind); - } - private void runAction(boolean isStreaming) throws Exception { runAction(isStreaming, false); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCaseBase.java index 4c646444cb72c..41d01bdf7f35e 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCaseBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CompactActionITCaseBase.java @@ -18,17 +18,22 @@ package org.apache.paimon.flink.action; +import org.apache.paimon.Snapshot; import org.apache.paimon.manifest.FileKind; import org.apache.paimon.manifest.ManifestEntry; import org.apache.paimon.operation.FileStoreScan; import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.sink.StreamWriteBuilder; import org.apache.paimon.table.source.StreamTableScan; import org.apache.paimon.table.source.TableScan; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.SnapshotManager; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.concurrent.TimeoutException; @@ -37,6 +42,12 @@ /** Base IT cases for {@link CompactAction} and {@link CompactDatabaseAction} . */ public class CompactActionITCaseBase extends ActionITCaseBase { + protected static final DataType[] FIELD_TYPES = + new DataType[] {DataTypes.INT(), DataTypes.INT(), DataTypes.INT(), DataTypes.STRING()}; + + protected static final RowType ROW_TYPE = + RowType.of(FIELD_TYPES, new String[] {"k", "v", "hh", "dt"}); + protected void validateResult( FileStoreTable table, RowType rowType, @@ -87,4 +98,29 @@ protected void checkFileAndRowSize( assertThat(files.size()).isEqualTo(fileNum); assertThat(count).isEqualTo(rowCount); } + + protected void checkLatestSnapshot( + FileStoreTable table, long snapshotId, Snapshot.CommitKind commitKind) { + SnapshotManager snapshotManager = table.snapshotManager(); + Snapshot snapshot = snapshotManager.snapshot(snapshotManager.latestSnapshotId()); + assertThat(snapshot.id()).isEqualTo(snapshotId); + assertThat(snapshot.commitKind()).isEqualTo(commitKind); + } + + protected FileStoreTable prepareTable( + List partitionKeys, + List primaryKeys, + List bucketKey, + Map tableOptions) + throws Exception { + FileStoreTable table = + createFileStoreTable(ROW_TYPE, partitionKeys, primaryKeys, bucketKey, tableOptions); + + StreamWriteBuilder streamWriteBuilder = + table.newStreamWriteBuilder().withCommitUser(commitUser); + write = streamWriteBuilder.newWrite(); + commit = streamWriteBuilder.newCommit(); + + return table; + } } 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 e2243ddf269a5..6fb8c81eb7440 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 @@ -26,8 +26,11 @@ import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BlockingIterator; import org.apache.flink.table.api.TableException; +import org.apache.flink.types.Row; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -46,6 +49,7 @@ public class ConsumerActionITCase extends ActionITCaseBase { @ParameterizedTest + @Timeout(60) @ValueSource(strings = {"action", "procedure_indexed", "procedure_named"}) public void testResetConsumer(String invoker) throws Exception { init(warehouse); @@ -72,18 +76,22 @@ public void testResetConsumer(String invoker) throws Exception { writeData(rowData(3L, BinaryString.fromString("Paimon"))); // use consumer streaming read table - testStreamingRead( + BlockingIterator iterator = + testStreamingRead( "SELECT * FROM `" + tableName + "` /*+ OPTIONS('consumer-id'='myid','consumer.expiration-time'='3h') */", Arrays.asList( changelogRow("+I", 1L, "Hi"), changelogRow("+I", 2L, "Hello"), - changelogRow("+I", 3L, "Paimon"))) - .close(); + changelogRow("+I", 3L, "Paimon"))); - Thread.sleep(1000); ConsumerManager consumerManager = new ConsumerManager(table.fileIO(), table.location()); + while (!consumerManager.consumer("myid").isPresent()) { + Thread.sleep(1000); + } + iterator.close(); + Optional consumer1 = consumerManager.consumer("myid"); assertThat(consumer1).isPresent(); assertThat(consumer1.get().nextSnapshot()).isEqualTo(4); @@ -191,6 +199,7 @@ public void testResetConsumer(String invoker) throws Exception { } @ParameterizedTest + @Timeout(60) @ValueSource(strings = {"action", "procedure_indexed", "procedure_named"}) public void testResetBranchConsumer(String invoker) throws Exception { init(warehouse); @@ -222,18 +231,23 @@ public void testResetBranchConsumer(String invoker) throws Exception { String branchTableName = tableName + "$branch_b1"; // use consumer streaming read table - testStreamingRead( + BlockingIterator iterator = + testStreamingRead( "SELECT * FROM `" + branchTableName + "` /*+ OPTIONS('consumer-id'='myid','consumer.expiration-time'='3h') */", Arrays.asList( changelogRow("+I", 1L, "Hi"), changelogRow("+I", 2L, "Hello"), - changelogRow("+I", 3L, "Paimon"))) - .close(); + changelogRow("+I", 3L, "Paimon"))); ConsumerManager consumerManager = new ConsumerManager(table.fileIO(), table.location(), branchName); + while (!consumerManager.consumer("myid").isPresent()) { + Thread.sleep(1000); + } + iterator.close(); + Optional consumer1 = consumerManager.consumer("myid"); assertThat(consumer1).isPresent(); assertThat(consumer1.get().nextSnapshot()).isEqualTo(4); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MinorCompactActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MinorCompactActionITCase.java new file mode 100644 index 0000000000000..0373eb01a2d92 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MinorCompactActionITCase.java @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.Snapshot; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.source.DataSplit; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT cases for compact strategy {@link CompactAction}. */ +public class MinorCompactActionITCase extends CompactActionITCaseBase { + + @Test + @Timeout(60) + public void testBatchMinorCompactStrategy() throws Exception { + FileStoreTable table = + prepareTable( + Arrays.asList("dt", "hh"), + Arrays.asList("dt", "hh", "k"), + Collections.emptyList(), + Collections.singletonMap(CoreOptions.WRITE_ONLY.key(), "true")); + + writeData( + rowData(1, 100, 15, BinaryString.fromString("20221208")), + rowData(1, 100, 16, BinaryString.fromString("20221208"))); + + writeData( + rowData(2, 100, 15, BinaryString.fromString("20221208")), + rowData(2, 100, 16, BinaryString.fromString("20221208"))); + + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + CompactAction action = + createAction( + CompactAction.class, + "compact", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--compact_strategy", + "minor", + "--table_conf", + CoreOptions.NUM_SORTED_RUNS_COMPACTION_TRIGGER.key() + "=3"); + StreamExecutionEnvironment env = streamExecutionEnvironmentBuilder().batchMode().build(); + action.withStreamExecutionEnvironment(env).build(); + env.execute(); + + // Due to the limitation of parameter 'num-sorted-run.compaction-trigger', so compact is not + // performed. + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + // Make par-15 has 3 datafile and par-16 has 2 datafile, so par-16 will not be picked out to + // compact. + writeData(rowData(2, 100, 15, BinaryString.fromString("20221208"))); + + env = streamExecutionEnvironmentBuilder().batchMode().build(); + action.withStreamExecutionEnvironment(env).build(); + env.execute(); + + checkLatestSnapshot(table, 4, Snapshot.CommitKind.COMPACT); + + List splits = table.newSnapshotReader().read().dataSplits(); + assertThat(splits.size()).isEqualTo(2); + for (DataSplit split : splits) { + // Par-16 is not compacted. + assertThat(split.dataFiles().size()) + .isEqualTo(split.partition().getInt(1) == 16 ? 2 : 1); + } + } + + @Test + @Timeout(60) + public void testBatchFullCompactStrategy() throws Exception { + FileStoreTable table = + prepareTable( + Arrays.asList("dt", "hh"), + Arrays.asList("dt", "hh", "k"), + Collections.emptyList(), + Collections.singletonMap(CoreOptions.WRITE_ONLY.key(), "true")); + + writeData( + rowData(1, 100, 15, BinaryString.fromString("20221208")), + rowData(1, 100, 16, BinaryString.fromString("20221208"))); + + writeData( + rowData(2, 100, 15, BinaryString.fromString("20221208")), + rowData(2, 100, 16, BinaryString.fromString("20221208"))); + + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + CompactAction action = + createAction( + CompactAction.class, + "compact", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--compact_strategy", + "full", + "--table_conf", + CoreOptions.NUM_SORTED_RUNS_COMPACTION_TRIGGER.key() + "=3"); + StreamExecutionEnvironment env = streamExecutionEnvironmentBuilder().batchMode().build(); + action.withStreamExecutionEnvironment(env).build(); + env.execute(); + + checkLatestSnapshot(table, 3, Snapshot.CommitKind.COMPACT); + + List splits = table.newSnapshotReader().read().dataSplits(); + assertThat(splits.size()).isEqualTo(2); + for (DataSplit split : splits) { + assertThat(split.dataFiles().size()).isEqualTo(1); + } + } + + @Test + @Timeout(60) + public void testStreamingFullCompactStrategy() throws Exception { + prepareTable( + Arrays.asList("dt", "hh"), + Arrays.asList("dt", "hh", "k"), + Collections.emptyList(), + Collections.singletonMap(CoreOptions.WRITE_ONLY.key(), "true")); + CompactAction action = + createAction( + CompactAction.class, + "compact", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--compact_strategy", + "full", + "--table_conf", + CoreOptions.NUM_SORTED_RUNS_COMPACTION_TRIGGER.key() + "=3"); + StreamExecutionEnvironment env = + streamExecutionEnvironmentBuilder().streamingMode().build(); + Assertions.assertThatThrownBy(() -> action.withStreamExecutionEnvironment(env).build()) + .hasMessage( + "The full compact strategy is only supported in batch mode. Please add -Dexecution.runtime-mode=BATCH."); + } + + @Test + @Timeout(60) + public void testCompactStrategyWithWrongUsage() throws Exception { + prepareTable( + Arrays.asList("dt", "hh"), + Arrays.asList("dt", "hh", "k"), + Collections.emptyList(), + Collections.singletonMap(CoreOptions.WRITE_ONLY.key(), "true")); + Assertions.assertThatThrownBy( + () -> + createAction( + CompactAction.class, + "compact", + "--warehouse", + warehouse, + "--database", + database, + "--table", + tableName, + "--compact_strategy", + "wrong_usage", + "--table_conf", + CoreOptions.NUM_SORTED_RUNS_COMPACTION_TRIGGER.key() + + "=3")) + .hasMessage( + "The compact strategy only supports 'full' or 'minor', but 'wrong_usage' is configured."); + } +} 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 938a8ce1be7a2..a92e529aa2cf2 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 @@ -18,324 +18,5 @@ package org.apache.paimon.flink.action; -import org.apache.paimon.CoreOptions; -import org.apache.paimon.data.BinaryString; -import org.apache.paimon.data.GenericRow; -import org.apache.paimon.fs.FileIO; -import org.apache.paimon.fs.Path; -import org.apache.paimon.options.Options; -import org.apache.paimon.schema.SchemaChange; -import org.apache.paimon.schema.SchemaManager; -import org.apache.paimon.schema.TableSchema; -import org.apache.paimon.table.FileStoreTable; -import org.apache.paimon.table.FileStoreTableFactory; -import org.apache.paimon.table.sink.StreamTableCommit; -import org.apache.paimon.table.sink.StreamTableWrite; -import org.apache.paimon.table.sink.StreamWriteBuilder; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.RowType; - -import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; - -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; - -import static org.apache.paimon.CoreOptions.SCAN_FALLBACK_BRANCH; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatCode; - -/** IT cases for {@link RemoveOrphanFilesAction}. */ -public class RemoveOrphanFilesActionITCase extends ActionITCaseBase { - - private static final String ORPHAN_FILE_1 = "bucket-0/orphan_file1"; - private static final String ORPHAN_FILE_2 = "bucket-0/orphan_file2"; - - private FileStoreTable createTableAndWriteData(String tableName) throws Exception { - RowType rowType = - RowType.of( - new DataType[] {DataTypes.BIGINT(), DataTypes.STRING()}, - new String[] {"k", "v"}); - - FileStoreTable table = - createFileStoreTable( - tableName, - rowType, - 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"))); - - Path orphanFile1 = getOrphanFilePath(table, ORPHAN_FILE_1); - Path orphanFile2 = getOrphanFilePath(table, ORPHAN_FILE_2); - - FileIO fileIO = table.fileIO(); - fileIO.writeFile(orphanFile1, "a", true); - Thread.sleep(2000); - fileIO.writeFile(orphanFile2, "b", true); - - return table; - } - - private Path getOrphanFilePath(FileStoreTable table, String orphanFile) { - return new Path(table.location(), orphanFile); - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testRunWithoutException(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 withDryRun = - String.format( - isNamedArgument - ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59', dry_run => true)" - : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59', true)", - database, - tableName); - ImmutableList actualDryRunDeleteFile = ImmutableList.copyOf(executeSQL(withDryRun)); - assertThat(actualDryRunDeleteFile).containsOnly(Row.of("2")); - - String withOlderThan = - String.format( - isNamedArgument - ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59')" - : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59')", - database, - tableName); - ImmutableList actualDeleteFile = ImmutableList.copyOf(executeSQL(withOlderThan)); - - assertThat(actualDeleteFile).containsExactlyInAnyOrder(Row.of("2")); - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testRemoveDatabaseOrphanFilesITCase(boolean isNamedArgument) throws Exception { - createTableAndWriteData("tableName1"); - createTableAndWriteData("tableName2"); - - List args = - new ArrayList<>( - Arrays.asList( - "remove_orphan_files", - "--warehouse", - warehouse, - "--database", - database, - "--table", - "*")); - 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(); - - args.add("--parallelism"); - args.add("5"); - RemoveOrphanFilesAction action3 = createAction(RemoveOrphanFilesAction.class, args); - assertThatCode(action3::run).doesNotThrowAnyException(); - - String withoutOlderThan = - String.format( - isNamedArgument - ? "CALL sys.remove_orphan_files(`table` => '%s.%s')" - : "CALL sys.remove_orphan_files('%s.%s')", - database, - "*"); - CloseableIterator withoutOlderThanCollect = executeSQL(withoutOlderThan); - assertThat(ImmutableList.copyOf(withoutOlderThanCollect)).containsOnly(Row.of("0")); - - String withParallelism = - String.format("CALL sys.remove_orphan_files('%s.%s','',true,5)", database, "*"); - CloseableIterator withParallelismCollect = executeSQL(withParallelism); - assertThat(ImmutableList.copyOf(withParallelismCollect)).containsOnly(Row.of("0")); - - String withDryRun = - String.format( - isNamedArgument - ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59', dry_run => true)" - : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59', true)", - database, - "*"); - ImmutableList actualDryRunDeleteFile = ImmutableList.copyOf(executeSQL(withDryRun)); - assertThat(actualDryRunDeleteFile).containsOnly(Row.of("4")); - - String withOlderThan = - String.format( - isNamedArgument - ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59')" - : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59')", - database, - "*"); - ImmutableList actualDeleteFile = ImmutableList.copyOf(executeSQL(withOlderThan)); - - assertThat(actualDeleteFile).containsOnly(Row.of("4")); - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testCleanWithBranch(boolean isNamedArgument) throws Exception { - // create main branch - FileStoreTable table = createTableAndWriteData(tableName); - - // create first branch and write some data - table.createBranch("br"); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location(), "br"); - TableSchema branchSchema = - schemaManager.commitChanges(SchemaChange.addColumn("v2", DataTypes.INT())); - Options branchOptions = new Options(branchSchema.options()); - branchOptions.set(CoreOptions.BRANCH, "br"); - branchSchema = branchSchema.copy(branchOptions.toMap()); - FileStoreTable branchTable = - FileStoreTableFactory.create(table.fileIO(), table.location(), branchSchema); - - String commitUser = UUID.randomUUID().toString(); - StreamTableWrite write = branchTable.newWrite(commitUser); - StreamTableCommit commit = branchTable.newCommit(commitUser); - write.write(GenericRow.of(2L, BinaryString.fromString("Hello"), 20)); - commit.commit(1, write.prepareCommit(false, 1)); - write.close(); - commit.close(); - - // create orphan file in snapshot directory of first branch - Path orphanFile3 = new Path(table.location(), "branch/branch-br/snapshot/orphan_file3"); - branchTable.fileIO().writeFile(orphanFile3, "x", true); - - // create second branch, which is empty - table.createBranch("br2"); - - // create orphan file in snapshot directory of second branch - Path orphanFile4 = new Path(table.location(), "branch/branch-br2/snapshot/orphan_file4"); - branchTable.fileIO().writeFile(orphanFile4, "y", true); - - if (ThreadLocalRandom.current().nextBoolean()) { - executeSQL( - String.format( - "ALTER TABLE `%s`.`%s` SET ('%s' = 'br')", - database, tableName, SCAN_FALLBACK_BRANCH.key()), - false, - true); - } - String procedure = - String.format( - isNamedArgument - ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59')" - : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59')", - database, - "*"); - 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"); - } -} +/** IT cases base for {@link RemoveOrphanFilesAction} in Flink Common. */ +public class RemoveOrphanFilesActionITCase extends RemoveOrphanFilesActionITCaseBase {} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java new file mode 100644 index 0000000000000..77f3be2f0c765 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java @@ -0,0 +1,341 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.GenericRow; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.options.Options; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.schema.TableSchema; +import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.table.FileStoreTableFactory; +import org.apache.paimon.table.sink.StreamTableCommit; +import org.apache.paimon.table.sink.StreamTableWrite; +import org.apache.paimon.table.sink.StreamWriteBuilder; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; + +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.paimon.CoreOptions.SCAN_FALLBACK_BRANCH; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; + +/** IT cases base for {@link RemoveOrphanFilesAction}. */ +public abstract class RemoveOrphanFilesActionITCaseBase extends ActionITCaseBase { + + private static final String ORPHAN_FILE_1 = "bucket-0/orphan_file1"; + private static final String ORPHAN_FILE_2 = "bucket-0/orphan_file2"; + + private FileStoreTable createTableAndWriteData(String tableName) throws Exception { + RowType rowType = + RowType.of( + new DataType[] {DataTypes.BIGINT(), DataTypes.STRING()}, + new String[] {"k", "v"}); + + FileStoreTable table = + createFileStoreTable( + tableName, + rowType, + 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"))); + + Path orphanFile1 = getOrphanFilePath(table, ORPHAN_FILE_1); + Path orphanFile2 = getOrphanFilePath(table, ORPHAN_FILE_2); + + FileIO fileIO = table.fileIO(); + fileIO.writeFile(orphanFile1, "a", true); + Thread.sleep(2000); + fileIO.writeFile(orphanFile2, "b", true); + + return table; + } + + private Path getOrphanFilePath(FileStoreTable table, String orphanFile) { + return new Path(table.location(), orphanFile); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testRunWithoutException(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 withDryRun = + String.format( + isNamedArgument + ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59', dry_run => true)" + : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59', true)", + database, + tableName); + ImmutableList actualDryRunDeleteFile = ImmutableList.copyOf(executeSQL(withDryRun)); + assertThat(actualDryRunDeleteFile).containsOnly(Row.of("2")); + + String withOlderThan = + String.format( + isNamedArgument + ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59')" + : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59')", + database, + tableName); + ImmutableList actualDeleteFile = ImmutableList.copyOf(executeSQL(withOlderThan)); + + assertThat(actualDeleteFile).containsExactlyInAnyOrder(Row.of("2"), Row.of("2")); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testRemoveDatabaseOrphanFilesITCase(boolean isNamedArgument) throws Exception { + createTableAndWriteData("tableName1"); + createTableAndWriteData("tableName2"); + + List args = + new ArrayList<>( + Arrays.asList( + "remove_orphan_files", + "--warehouse", + warehouse, + "--database", + database, + "--table", + "*")); + 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(); + + args.add("--parallelism"); + args.add("5"); + RemoveOrphanFilesAction action3 = createAction(RemoveOrphanFilesAction.class, args); + assertThatCode(action3::run).doesNotThrowAnyException(); + + String withoutOlderThan = + String.format( + isNamedArgument + ? "CALL sys.remove_orphan_files(`table` => '%s.%s')" + : "CALL sys.remove_orphan_files('%s.%s')", + database, + "*"); + CloseableIterator withoutOlderThanCollect = executeSQL(withoutOlderThan); + assertThat(ImmutableList.copyOf(withoutOlderThanCollect)).containsOnly(Row.of("0")); + + String withParallelism = + String.format("CALL sys.remove_orphan_files('%s.%s','',true,5)", database, "*"); + CloseableIterator withParallelismCollect = executeSQL(withParallelism); + assertThat(ImmutableList.copyOf(withParallelismCollect)).containsOnly(Row.of("0")); + + String withDryRun = + String.format( + isNamedArgument + ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59', dry_run => true)" + : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59', true)", + database, + "*"); + ImmutableList actualDryRunDeleteFile = ImmutableList.copyOf(executeSQL(withDryRun)); + assertThat(actualDryRunDeleteFile).containsOnly(Row.of("4")); + + String withOlderThan = + String.format( + isNamedArgument + ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59')" + : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59')", + database, + "*"); + ImmutableList actualDeleteFile = ImmutableList.copyOf(executeSQL(withOlderThan)); + + assertThat(actualDeleteFile).containsOnly(Row.of("4")); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testCleanWithBranch(boolean isNamedArgument) throws Exception { + // create main branch + FileStoreTable table = createTableAndWriteData(tableName); + + // create first branch and write some data + table.createBranch("br"); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location(), "br"); + TableSchema branchSchema = + schemaManager.commitChanges(SchemaChange.addColumn("v2", DataTypes.INT())); + Options branchOptions = new Options(branchSchema.options()); + branchOptions.set(CoreOptions.BRANCH, "br"); + branchSchema = branchSchema.copy(branchOptions.toMap()); + FileStoreTable branchTable = + FileStoreTableFactory.create(table.fileIO(), table.location(), branchSchema); + + String commitUser = UUID.randomUUID().toString(); + StreamTableWrite write = branchTable.newWrite(commitUser); + StreamTableCommit commit = branchTable.newCommit(commitUser); + write.write(GenericRow.of(2L, BinaryString.fromString("Hello"), 20)); + commit.commit(1, write.prepareCommit(false, 1)); + write.close(); + commit.close(); + + // create orphan file in snapshot directory of first branch + Path orphanFile3 = new Path(table.location(), "branch/branch-br/snapshot/orphan_file3"); + branchTable.fileIO().writeFile(orphanFile3, "x", true); + + // create second branch, which is empty + table.createBranch("br2"); + + // create orphan file in snapshot directory of second branch + Path orphanFile4 = new Path(table.location(), "branch/branch-br2/snapshot/orphan_file4"); + branchTable.fileIO().writeFile(orphanFile4, "y", true); + + if (ThreadLocalRandom.current().nextBoolean()) { + executeSQL( + String.format( + "ALTER TABLE `%s`.`%s` SET ('%s' = 'br')", + database, tableName, SCAN_FALLBACK_BRANCH.key()), + false, + true); + } + String procedure = + String.format( + isNamedArgument + ? "CALL sys.remove_orphan_files(`table` => '%s.%s', older_than => '2999-12-31 23:59:59')" + : "CALL sys.remove_orphan_files('%s.%s', '2999-12-31 23:59:59')", + database, + "*"); + 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/iceberg/FlinkIcebergITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/iceberg/FlinkIcebergITCaseBase.java index 413a404c41ed2..9202cfb8fefb4 100644 --- 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 @@ -404,6 +404,61 @@ public void testDropAndRecreateTable(String format) throws Exception { .containsExactlyInAnyOrder(Row.of("munich"), Row.of("cologne")); } + @ParameterizedTest + @ValueSource(strings = {"orc", "parquet", "avro"}) + public void testNestedTypes(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" + + " k INT,\n" + + " v MAP>,\n" + + " v2 BIGINT\n" + + ") WITH (\n" + + " 'metadata.iceberg.storage' = 'hadoop-catalog',\n" + + " 'file.format' = '" + + format + + "'\n" + + ")"); + tEnv.executeSql( + "INSERT INTO paimon.`default`.T VALUES " + + "(1, MAP[10, ARRAY[ROW('apple', 100), ROW('banana', 101)], 20, ARRAY[ROW('cat', 102), ROW('dog', 103)]], 1000), " + + "(2, MAP[10, ARRAY[ROW('cherry', 200), ROW('pear', 201)], 20, ARRAY[ROW('tiger', 202), ROW('wolf', 203)]], 2000)") + .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 k, v[10], v2 FROM iceberg.`default`.T"))) + .containsExactlyInAnyOrder( + Row.of(1, new Row[] {Row.of("apple", 100), Row.of("banana", 101)}, 1000L), + Row.of(2, new Row[] {Row.of("cherry", 200), Row.of("pear", 201)}, 2000L)); + + tEnv.executeSql( + "INSERT INTO paimon.`default`.T VALUES " + + "(3, MAP[10, ARRAY[ROW('mango', 300), ROW('watermelon', 301)], 20, ARRAY[ROW('rabbit', 302), ROW('lion', 303)]], 3000)") + .await(); + assertThat( + collect( + tEnv.executeSql( + "SELECT k, v[10][2].f1, v2 FROM iceberg.`default`.T WHERE v[20][1].f2 > 200"))) + .containsExactlyInAnyOrder( + Row.of(2, "pear", 2000L), Row.of(3, "watermelon", 3000L)); + } + private List collect(TableResult result) throws Exception { List rows = new ArrayList<>(); try (CloseableIterator it = result.collect()) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java index bec669acd30d3..d79d13f0260cd 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/CompactProcedureITCase.java @@ -31,6 +31,7 @@ import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import java.util.List; @@ -240,6 +241,117 @@ public void testDynamicBucketSortCompact() throws Exception { checkLatestSnapshot(table, 21, Snapshot.CommitKind.OVERWRITE); } + // ----------------------- Minor Compact ----------------------- + + @Test + public void testBatchMinorCompactStrategy() throws Exception { + sql( + "CREATE TABLE T (" + + " k INT," + + " v INT," + + " hh INT," + + " dt STRING," + + " PRIMARY KEY (k, dt, hh) NOT ENFORCED" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'" + + ")"); + FileStoreTable table = paimonTable("T"); + tEnv.getConfig().set(TableConfigOptions.TABLE_DML_SYNC, true); + + sql("INSERT INTO T VALUES (1, 100, 15, '20221208'), (1, 100, 16, '20221208')"); + sql("INSERT INTO T VALUES (2, 100, 15, '20221208'), (2, 100, 16, '20221208')"); + + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + sql( + "CALL sys.compact(`table` => 'default.T', compact_strategy => 'minor', " + + "options => 'num-sorted-run.compaction-trigger=3')"); + + // Due to the limitation of parameter 'num-sorted-run.compaction-trigger' = 3, so compact is + // not + // performed. + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + // Make par-15 has 3 datafile and par-16 has 2 datafile, so par-16 will not be picked out to + // compact. + sql("INSERT INTO T VALUES (1, 100, 15, '20221208')"); + + sql( + "CALL sys.compact(`table` => 'default.T', compact_strategy => 'minor', " + + "options => 'num-sorted-run.compaction-trigger=3')"); + + checkLatestSnapshot(table, 4, Snapshot.CommitKind.COMPACT); + + List splits = table.newSnapshotReader().read().dataSplits(); + assertThat(splits.size()).isEqualTo(2); + for (DataSplit split : splits) { + // Par-16 is not compacted. + assertThat(split.dataFiles().size()) + .isEqualTo(split.partition().getInt(1) == 16 ? 2 : 1); + } + } + + @Test + public void testBatchFullCompactStrategy() throws Exception { + sql( + "CREATE TABLE T (" + + " k INT," + + " v INT," + + " hh INT," + + " dt STRING," + + " PRIMARY KEY (k, dt, hh) NOT ENFORCED" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'" + + ")"); + FileStoreTable table = paimonTable("T"); + tEnv.getConfig().set(TableConfigOptions.TABLE_DML_SYNC, true); + + sql("INSERT INTO T VALUES (1, 100, 15, '20221208'), (1, 100, 16, '20221208')"); + sql("INSERT INTO T VALUES (2, 100, 15, '20221208'), (2, 100, 16, '20221208')"); + + checkLatestSnapshot(table, 2, Snapshot.CommitKind.APPEND); + + sql( + "CALL sys.compact(`table` => 'default.T', compact_strategy => 'full', " + + "options => 'num-sorted-run.compaction-trigger=3')"); + + checkLatestSnapshot(table, 3, Snapshot.CommitKind.COMPACT); + + List splits = table.newSnapshotReader().read().dataSplits(); + assertThat(splits.size()).isEqualTo(2); + for (DataSplit split : splits) { + // Par-16 is not compacted. + assertThat(split.dataFiles().size()).isEqualTo(1); + } + } + + @Test + public void testStreamFullCompactStrategy() throws Exception { + sql( + "CREATE TABLE T (" + + " k INT," + + " v INT," + + " hh INT," + + " dt STRING," + + " PRIMARY KEY (k, dt, hh) NOT ENFORCED" + + ") PARTITIONED BY (dt, hh) WITH (" + + " 'write-only' = 'true'," + + " 'bucket' = '1'" + + ")"); + tEnv.getConfig().set(TableConfigOptions.TABLE_DML_SYNC, true); + + Assertions.assertThatThrownBy( + () -> + streamSqlIter( + "CALL sys.compact(`table` => 'default.T', compact_strategy => 'full', " + + "options => 'num-sorted-run.compaction-trigger=3')") + .close()) + .hasMessageContaining( + "The full compact strategy is only supported in batch mode. Please add -Dexecution.runtime-mode=BATCH."); + } + private void checkLatestSnapshot( FileStoreTable table, long snapshotId, Snapshot.CommitKind commitKind) { SnapshotManager snapshotManager = table.snapshotManager(); 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 bc2e84902f35a..a40968e067bca 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 @@ -402,6 +402,56 @@ public void testSortAndLimitExpirePartition() throws Exception { .containsExactlyInAnyOrder("4:2024-06-03:01:00", "Never-expire:9999-09-09:99:99"); } + @Test + public void testNullPartitionExpire() { + sql("CREATE TABLE T (k INT, ds STRING) PARTITIONED BY (ds);"); + sql("INSERT INTO T VALUES (1, CAST (NULL AS STRING))"); + assertThat( + callExpirePartitions( + "CALL sys.expire_partitions(" + + "`table` => 'default.T'" + + ", expiration_time => '1 d'" + + ", timestamp_formatter => 'yyyyMMdd')")) + .containsExactly("No expired partitions."); + } + + @Test + public void testExpirePartitionsWithDefaultNum() throws Exception { + sql( + "CREATE TABLE T (" + + " k STRING," + + " dt STRING," + + " PRIMARY KEY (k, dt) NOT ENFORCED" + + ") PARTITIONED BY (dt) WITH (" + + " 'bucket' = '1'," + + " 'partition.expiration-max-num'='2'" + + ")"); + FileStoreTable table = paimonTable("T"); + + sql("INSERT INTO T VALUES ('a', '2024-06-01')"); + sql("INSERT INTO T VALUES ('b', '2024-06-02')"); + sql("INSERT INTO T VALUES ('c', '2024-06-03')"); + // This partition never expires. + sql("INSERT INTO T VALUES ('Never-expire', '9999-09-09')"); + Function consumerReadResult = + (InternalRow row) -> row.getString(0) + ":" + row.getString(1); + + assertThat(read(table, consumerReadResult)) + .containsExactlyInAnyOrder( + "a:2024-06-01", "b:2024-06-02", "c:2024-06-03", "Never-expire:9999-09-09"); + + assertThat( + callExpirePartitions( + "CALL sys.expire_partitions(" + + "`table` => 'default.T'" + + ", expiration_time => '1 d'" + + ", timestamp_formatter => 'yyyy-MM-dd')")) + .containsExactlyInAnyOrder("dt=2024-06-01", "dt=2024-06-02"); + + assertThat(read(table, consumerReadResult)) + .containsExactlyInAnyOrder("c:2024-06-03", "Never-expire:9999-09-09"); + } + /** 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/RollbackToWatermarkProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/RollbackToWatermarkProcedureITCase.java new file mode 100644 index 0000000000000..f87ecd24756b2 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/RollbackToWatermarkProcedureITCase.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.paimon.table.FileStoreTable; + +import org.apache.flink.types.Row; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** IT Case for {@link RollbackToWatermarkProcedure}. */ +public class RollbackToWatermarkProcedureITCase extends CatalogITCaseBase { + + @Test + public void testCreateTagsFromSnapshotsWatermark() throws Exception { + sql( + "CREATE TABLE T (" + + " k STRING," + + " dt STRING," + + " PRIMARY KEY (k, dt) NOT ENFORCED" + + ") PARTITIONED BY (dt) WITH (" + + " 'bucket' = '1'" + + ")"); + + // create snapshot 1 with watermark 1000. + sql( + "insert into T/*+ OPTIONS('end-input.watermark'= '1000') */ values('k1', '2024-12-02')"); + // create snapshot 2 with watermark 2000. + sql( + "insert into T/*+ OPTIONS('end-input.watermark'= '2000') */ values('k2', '2024-12-02')"); + // create snapshot 3 with watermark 3000. + sql( + "insert into T/*+ OPTIONS('end-input.watermark'= '3000') */ values('k3', '2024-12-02')"); + + FileStoreTable table = paimonTable("T"); + + long watermark1 = table.snapshotManager().snapshot(1).watermark(); + long watermark2 = table.snapshotManager().snapshot(2).watermark(); + long watermark3 = table.snapshotManager().snapshot(3).watermark(); + + assertThat(watermark1 == 1000).isTrue(); + assertThat(watermark2 == 2000).isTrue(); + assertThat(watermark3 == 3000).isTrue(); + + assertThat(sql("select * from T").stream().map(Row::toString)) + .containsExactlyInAnyOrder( + "+I[k1, 2024-12-02]", "+I[k2, 2024-12-02]", "+I[k3, 2024-12-02]"); + + sql("CALL sys.rollback_to_watermark(`table` => 'default.T',`watermark` => 2001)"); + + // check for snapshot 2 + assertThat(sql("select * from T").stream().map(Row::toString)) + .containsExactlyInAnyOrder("+I[k1, 2024-12-02]", "+I[k2, 2024-12-02]"); + + sql("CALL sys.rollback_to_watermark(`table` => 'default.T',`watermark` => 1001)"); + + // check for snapshot 1 + assertThat(sql("select * from T").stream().map(Row::toString)) + .containsExactlyInAnyOrder("+I[k1, 2024-12-02]"); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java index d589459d9b969..949c2c7a66a36 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlyMultiTableCompactionWorkerOperatorTest.java @@ -25,7 +25,13 @@ import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.CommitMessageImpl; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -45,8 +51,17 @@ public class AppendOnlyMultiTableCompactionWorkerOperatorTest extends TableTestB public void testAsyncCompactionWorks() throws Exception { AppendOnlyMultiTableCompactionWorkerOperator workerOperator = - new AppendOnlyMultiTableCompactionWorkerOperator( - () -> catalog, "user", new Options()); + new AppendOnlyMultiTableCompactionWorkerOperator.Factory( + () -> catalog, "user", new Options()) + .createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask( + new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); List> records = new ArrayList<>(); // create table and write diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java index d04032817cf01..6238a9cbf3ea0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java @@ -32,7 +32,13 @@ import org.apache.paimon.table.sink.CommitMessageImpl; import org.apache.paimon.types.DataTypes; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -49,7 +55,16 @@ public class AppendOnlySingleTableCompactionWorkerOperatorTest extends TableTest public void testAsyncCompactionWorks() throws Exception { createTableDefault(); AppendOnlySingleTableCompactionWorkerOperator workerOperator = - new AppendOnlySingleTableCompactionWorkerOperator(getTableDefault(), "user"); + new AppendOnlySingleTableCompactionWorkerOperator.Factory(getTableDefault(), "user") + .createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask( + new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); // write 200 files List commitMessages = writeDataDefault(200, 20); @@ -102,7 +117,16 @@ public void testAsyncCompactionWorks() throws Exception { public void testAsyncCompactionFileDeletedWhenShutdown() throws Exception { createTableDefault(); AppendOnlySingleTableCompactionWorkerOperator workerOperator = - new AppendOnlySingleTableCompactionWorkerOperator(getTableDefault(), "user"); + new AppendOnlySingleTableCompactionWorkerOperator.Factory(getTableDefault(), "user") + .createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask( + new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); // write 200 files List commitMessages = writeDataDefault(200, 40); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java index 3b58c24d16b18..ee930a06fc3d0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperatorTest.java @@ -32,7 +32,7 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.SavepointType; import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.junit.jupiter.api.Test; @@ -198,13 +198,15 @@ private void processCommittable( } @Override - protected OneInputStreamOperator createCommitterOperator( - FileStoreTable table, - String commitUser, - CommittableStateManager committableStateManager) { - return new AutoTagForSavepointCommitterOperator<>( - (CommitterOperator) - super.createCommitterOperator(table, commitUser, committableStateManager), + protected OneInputStreamOperatorFactory + createCommitterOperatorFactory( + FileStoreTable table, + String commitUser, + CommittableStateManager committableStateManager) { + return new AutoTagForSavepointCommitterOperatorFactory<>( + (CommitterOperatorFactory) + super.createCommitterOperatorFactory( + table, commitUser, committableStateManager), table::snapshotManager, table::tagManager, () -> table.store().newTagDeletion(), @@ -213,14 +215,15 @@ protected OneInputStreamOperator createCommitterOperat } @Override - protected OneInputStreamOperator createCommitterOperator( - FileStoreTable table, - String commitUser, - CommittableStateManager committableStateManager, - ThrowingConsumer initializeFunction) { - return new AutoTagForSavepointCommitterOperator<>( - (CommitterOperator) - super.createCommitterOperator( + protected OneInputStreamOperatorFactory + createCommitterOperatorFactory( + FileStoreTable table, + String commitUser, + CommittableStateManager committableStateManager, + ThrowingConsumer initializeFunction) { + return new AutoTagForSavepointCommitterOperatorFactory<>( + (CommitterOperatorFactory) + super.createCommitterOperatorFactory( table, commitUser, committableStateManager, initializeFunction), table::snapshotManager, table::tagManager, diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java index 147110637aef6..68162832eac9c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperatorTest.java @@ -27,13 +27,21 @@ import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.TagManager; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.junit.jupiter.api.Test; import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; import java.time.format.DateTimeFormatter; +import java.util.ArrayList; import java.util.HashMap; import java.util.Objects; @@ -54,12 +62,23 @@ public void testBatchWriteGeneratorTag() throws Exception { StreamTableWrite write = table.newStreamWriteBuilder().withCommitUser(initialCommitUser).newWrite(); - OneInputStreamOperator committerOperator = - createCommitterOperator( + OneInputStreamOperatorFactory committerOperatorFactory = + createCommitterOperatorFactory( table, initialCommitUser, new RestoreAndFailCommittableStateManager<>( ManifestCommittableSerializer::new)); + + OneInputStreamOperator committerOperator = + committerOperatorFactory.createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask(new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); + committerOperator.open(); TableCommitImpl tableCommit = table.newCommit(initialCommitUser); @@ -106,13 +125,15 @@ public void testBatchWriteGeneratorTag() throws Exception { } @Override - protected OneInputStreamOperator createCommitterOperator( - FileStoreTable table, - String commitUser, - CommittableStateManager committableStateManager) { - return new BatchWriteGeneratorTagOperator<>( - (CommitterOperator) - super.createCommitterOperator(table, commitUser, committableStateManager), + protected OneInputStreamOperatorFactory + createCommitterOperatorFactory( + FileStoreTable table, + String commitUser, + CommittableStateManager committableStateManager) { + return new BatchWriteGeneratorTagOperatorFactory<>( + (CommitterOperatorFactory) + super.createCommitterOperatorFactory( + table, commitUser, committableStateManager), table); } } 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 668d651236fdc..28c93ca79be02 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 @@ -51,10 +51,13 @@ import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +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.watermark.Watermark; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.util.Preconditions; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -259,8 +262,8 @@ public void testRestoreCommitUser() throws Exception { // 3. Check whether success List actual = new ArrayList<>(); - OneInputStreamOperator operator = - createCommitterOperator( + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( table, initialCommitUser, new NoopCommittableStateManager(), @@ -274,7 +277,7 @@ public void testRestoreCommitUser() throws Exception { }); OneInputStreamOperatorTestHarness testHarness1 = - createTestHarness(operator); + createTestHarness(operatorFactory); testHarness1.initializeState(snapshot); testHarness1.close(); @@ -315,10 +318,11 @@ public void testRestoreEmptyMarkDoneState() throws Exception { public void testCommitInputEnd() throws Exception { FileStoreTable table = createFileStoreTable(); String commitUser = UUID.randomUUID().toString(); - OneInputStreamOperator operator = - createCommitterOperator(table, commitUser, new NoopCommittableStateManager()); + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( + table, commitUser, new NoopCommittableStateManager()); OneInputStreamOperatorTestHarness testHarness = - createTestHarness(operator); + createTestHarness(operatorFactory); testHarness.open(); Assertions.assertThatCode( () -> { @@ -378,10 +382,10 @@ public void testCommitInputEnd() throws Exception { }) .doesNotThrowAnyException(); - if (operator instanceof CommitterOperator) { + if (operatorFactory instanceof CommitterOperator) { Assertions.assertThat( ((ManifestCommittable) - ((CommitterOperator) operator) + ((CommitterOperator) operatorFactory) .committablesPerCheckpoint.get(Long.MAX_VALUE)) .fileCommittables() .size()) @@ -604,14 +608,14 @@ public void testCalcDataBytesSend() throws Exception { public void testCommitMetrics() throws Exception { FileStoreTable table = createFileStoreTable(); - OneInputStreamOperator operator = - createCommitterOperator( + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( table, null, new RestoreAndFailCommittableStateManager<>( ManifestCommittableSerializer::new)); OneInputStreamOperatorTestHarness testHarness = - createTestHarness(operator); + createTestHarness(operatorFactory); testHarness.open(); long timestamp = 0; StreamTableWrite write = @@ -627,7 +631,9 @@ public void testCommitMetrics() throws Exception { testHarness.notifyOfCompletedCheckpoint(cpId); MetricGroup commitMetricGroup = - operator.getMetricGroup() + testHarness + .getOneInputOperator() + .getMetricGroup() .addGroup("paimon") .addGroup("table", table.name()) .addGroup("commit"); @@ -685,10 +691,11 @@ public void testCommitMetrics() throws Exception { public void testParallelism() throws Exception { FileStoreTable table = createFileStoreTable(); String commitUser = UUID.randomUUID().toString(); - OneInputStreamOperator operator = - createCommitterOperator(table, commitUser, new NoopCommittableStateManager()); + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( + table, commitUser, new NoopCommittableStateManager()); try (OneInputStreamOperatorTestHarness testHarness = - createTestHarness(operator, 10, 10, 3)) { + createTestHarness(operatorFactory, 10, 10, 3)) { Assertions.assertThatCode(testHarness::open) .hasMessage("Committer Operator parallelism in paimon MUST be one."); } @@ -700,13 +707,13 @@ public void testParallelism() throws Exception { protected OneInputStreamOperatorTestHarness createRecoverableTestHarness(FileStoreTable table) throws Exception { - OneInputStreamOperator operator = - createCommitterOperator( + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( table, null, new RestoreAndFailCommittableStateManager<>( ManifestCommittableSerializer::new)); - return createTestHarness(operator); + return createTestHarness(operatorFactory); } private OneInputStreamOperatorTestHarness createLossyTestHarness( @@ -716,18 +723,20 @@ private OneInputStreamOperatorTestHarness createLossyT private OneInputStreamOperatorTestHarness createLossyTestHarness( FileStoreTable table, String commitUser) throws Exception { - OneInputStreamOperator operator = - createCommitterOperator(table, commitUser, new NoopCommittableStateManager()); - return createTestHarness(operator); + OneInputStreamOperatorFactory operatorFactory = + createCommitterOperatorFactory( + table, commitUser, new NoopCommittableStateManager()); + return createTestHarness(operatorFactory); } private OneInputStreamOperatorTestHarness createTestHarness( - OneInputStreamOperator operator) throws Exception { - return createTestHarness(operator, 1, 1, 0); + OneInputStreamOperatorFactory operatorFactory) + throws Exception { + return createTestHarness(operatorFactory, 1, 1, 0); } private OneInputStreamOperatorTestHarness createTestHarness( - OneInputStreamOperator operator, + OneInputStreamOperatorFactory operatorFactory, int maxParallelism, int parallelism, int subTaskIndex) @@ -736,22 +745,23 @@ private OneInputStreamOperatorTestHarness createTestHa new CommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>( - operator, + operatorFactory, maxParallelism, parallelism, subTaskIndex, - serializer, new OperatorID()); + harness.getStreamConfig().setupNetworkInputs(Preconditions.checkNotNull(serializer)); + harness.getStreamConfig().serializeAllConfigs(); harness.setup(serializer); return harness; } - protected OneInputStreamOperator createCommitterOperator( - FileStoreTable table, - String commitUser, - CommittableStateManager committableStateManager) { - return new CommitterOperator<>( - true, + protected OneInputStreamOperatorFactory + createCommitterOperatorFactory( + FileStoreTable table, + String commitUser, + CommittableStateManager committableStateManager) { + return new CommitterOperatorFactory<>( true, true, commitUser == null ? initialCommitUser : commitUser, @@ -765,13 +775,13 @@ protected OneInputStreamOperator createCommitterOperat committableStateManager); } - protected OneInputStreamOperator createCommitterOperator( - FileStoreTable table, - String commitUser, - CommittableStateManager committableStateManager, - ThrowingConsumer initializeFunction) { - return new CommitterOperator( - true, + protected OneInputStreamOperatorFactory + createCommitterOperatorFactory( + FileStoreTable table, + String commitUser, + CommittableStateManager committableStateManager, + ThrowingConsumer initializeFunction) { + return new CommitterOperatorFactory( true, true, commitUser == null ? initialCommitUser : commitUser, @@ -784,8 +794,24 @@ protected OneInputStreamOperator createCommitterOperat context), committableStateManager) { @Override - public void initializeState(StateInitializationContext context) throws Exception { - initializeFunction.accept(context); + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters parameters) { + return (T) + new CommitterOperator( + parameters, + streamingCheckpointEnabled, + forceSingleParallelism, + initialCommitUser, + committerFactory, + committableStateManager, + endInputWatermark) { + @Override + public void initializeState(StateInitializationContext context) + throws Exception { + initializeFunction.accept(context); + } + }; } }; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java index a5f260fb25a59..d487d75925eb9 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java @@ -132,7 +132,7 @@ public void testCompact() throws Exception { .withContinuousMode(false) .withPartitionPredicate(predicate) .build(); - new CompactorSinkBuilder(table).withInput(source).build(); + new CompactorSinkBuilder(table, true).withInput(source).build(); env.execute(); snapshot = snapshotManager.snapshot(snapshotManager.latestSnapshotId()); @@ -181,7 +181,8 @@ public void testCompactParallelism() throws Exception { FlinkConnectorOptions.SINK_PARALLELISM.key(), String.valueOf(sinkParalellism)); } - })) + }), + false) .withInput(source) .build(); @@ -253,8 +254,8 @@ private OneInputStreamOperatorTestHarness createTestHarnes return harness; } - protected StoreCompactOperator createCompactOperator(FileStoreTable table) { - return new StoreCompactOperator( + protected StoreCompactOperator.Factory createCompactOperator(FileStoreTable table) { + return new StoreCompactOperator.Factory( table, (t, commitUser, state, ioManager, memoryPool, metricGroup) -> new StoreSinkWriteImpl( @@ -267,13 +268,20 @@ protected StoreCompactOperator createCompactOperator(FileStoreTable table) { false, memoryPool, metricGroup), - "test"); + "test", + true); } - protected MultiTablesStoreCompactOperator createMultiTablesCompactOperator( + protected MultiTablesStoreCompactOperator.Factory createMultiTablesCompactOperator( Catalog.Loader catalogLoader) throws Exception { - return new MultiTablesStoreCompactOperator( - catalogLoader, commitUser, new CheckpointConfig(), false, false, new Options()); + return new MultiTablesStoreCompactOperator.Factory( + catalogLoader, + commitUser, + new CheckpointConfig(), + false, + false, + true, + new Options()); } private static byte[] partition(String dt, int hh) { 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 c335568344b32..5f21858e61a58 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 @@ -42,7 +42,7 @@ 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.OneInputStreamOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.junit.jupiter.api.Test; @@ -82,20 +82,22 @@ private boolean testSpillable( Collections.singletonList(GenericRow.of(1, 1))); FlinkSink flinkSink = new FixedBucketSink(fileStoreTable, null, null); DataStream written = flinkSink.doWrite(source, "123", 1); - RowDataStoreWriteOperator operator = - ((RowDataStoreWriteOperator) - ((SimpleOperatorFactory) - ((OneInputTransformation) written.getTransformation()) - .getOperatorFactory()) - .getOperator()); + OneInputStreamOperatorFactory operatorFactory = + (OneInputStreamOperatorFactory) + ((OneInputTransformation) + written.getTransformation()) + .getOperatorFactory(); TypeSerializer serializer = new CommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator); + new OneInputStreamOperatorTestHarness<>(operatorFactory); harness.setup(serializer); harness.initializeEmptyState(); + RowDataStoreWriteOperator operator = + (RowDataStoreWriteOperator) harness.getOneInputOperator(); + return ((KeyValueFileStoreWrite) ((StoreSinkWriteImpl) operator.write).write.getWrite()) .bufferSpillable(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LocalMergeOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LocalMergeOperatorTest.java index 1162e20b155b2..fc45eceb3fd5b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LocalMergeOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/LocalMergeOperatorTest.java @@ -26,12 +26,18 @@ import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.apache.flink.util.OutputTag; import org.junit.jupiter.api.Test; @@ -151,7 +157,17 @@ private void prepareHashOperator(Map options) throws Exception { Collections.singletonList("f0"), options, null); - operator = new LocalMergeOperator(schema); + operator = + new LocalMergeOperator.Factory(schema) + .createStreamOperator( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask( + new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null)); operator.open(); assertThat(operator.merger()).isInstanceOf(HashMapLocalMerger.class); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java index 6b912d2e57fe9..b1486deacb0c6 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/SinkSavepointITCase.java @@ -137,7 +137,7 @@ private JobClient runRecoverFromSavepointJob(String failingPath, String savepoin .parallelism(1) .allowRestart() .setConf(conf) - .setConf(StateBackendOptions.STATE_BACKEND, "filesystem") + .setConf(StateBackendOptions.STATE_BACKEND, "hashmap") .setConf( CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + path + "/checkpoint") 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 3f2daedffd480..3740033e025e9 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 @@ -48,17 +48,18 @@ public void testCompactExactlyOnce(boolean streamingMode) throws Exception { CompactRememberStoreWrite compactRememberStoreWrite = new CompactRememberStoreWrite(streamingMode); - StoreCompactOperator operator = - new StoreCompactOperator( + StoreCompactOperator.Factory operatorFactory = + new StoreCompactOperator.Factory( getTableDefault(), (table, commitUser, state, ioManager, memoryPool, metricGroup) -> compactRememberStoreWrite, - "10086"); + "10086", + !streamingMode); TypeSerializer serializer = new CommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator); + new OneInputStreamOperatorTestHarness<>(operatorFactory); harness.setup(serializer); harness.initializeEmptyState(); harness.open(); @@ -69,7 +70,7 @@ public void testCompactExactlyOnce(boolean streamingMode) throws Exception { harness.processElement(new StreamRecord<>(data(1))); harness.processElement(new StreamRecord<>(data(2))); - operator.prepareCommit(true, 1); + ((StoreCompactOperator) harness.getOneInputOperator()).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/StoreMultiCommitterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java index 10e432f3c8c21..752679fb59037 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java @@ -645,11 +645,10 @@ public void testCommitMetrics() throws Exception { private OneInputStreamOperatorTestHarness createRecoverableTestHarness() throws Exception { - CommitterOperator operator = - new CommitterOperator<>( + CommitterOperatorFactory operator = + new CommitterOperatorFactory<>( true, false, - true, initialCommitUser, context -> new StoreMultiCommitter(catalogLoader, context), new RestoreAndFailCommittableStateManager<>( @@ -659,11 +658,10 @@ public void testCommitMetrics() throws Exception { private OneInputStreamOperatorTestHarness createLossyTestHarness() throws Exception { - CommitterOperator operator = - new CommitterOperator<>( + CommitterOperatorFactory operator = + new CommitterOperatorFactory<>( true, false, - true, initialCommitUser, context -> new StoreMultiCommitter(catalogLoader, context), new CommittableStateManager() { @@ -682,12 +680,13 @@ public void snapshotState( private OneInputStreamOperatorTestHarness createTestHarness( - CommitterOperator operator) + CommitterOperatorFactory + operatorFactory) throws Exception { TypeSerializer serializer = new MultiTableCommittableTypeInfo().createSerializer(new ExecutionConfig()); OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(operator, serializer); + new OneInputStreamOperatorTestHarness<>(operatorFactory, serializer); harness.setup(serializer); return harness; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java index 298f3155ba34e..b0aa76f157ac8 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WrappedManifestCommittableSerializerTest.java @@ -98,7 +98,7 @@ public static void addFileCommittables( if (!committable.logOffsets().containsKey(bucket)) { int offset = ID.incrementAndGet(); - committable.addLogOffset(bucket, offset); + committable.addLogOffset(bucket, offset, false); assertThat(committable.logOffsets().get(bucket)).isEqualTo(offset); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java new file mode 100644 index 0000000000000..24fb529b59ea2 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterChainingStrategyTest.java @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.flink.api.dag.Transformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.CompiledPlan; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.CompiledPlanUtils; +import org.apache.flink.util.TimeUtils; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Tests for {@link org.apache.flink.streaming.api.operators.ChainingStrategy} of writer operators. + */ +public class WriterChainingStrategyTest { + private static final String TABLE_NAME = "paimon_table"; + + @TempDir java.nio.file.Path tempDir; + + private StreamTableEnvironment tEnv; + + @BeforeEach + public void beforeEach() { + Configuration config = new Configuration(); + config.setString( + "execution.checkpointing.interval", + TimeUtils.formatWithHighestUnit(Duration.ofMillis(500))); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + tEnv = StreamTableEnvironment.create(env); + + String catalog = "PAIMON"; + Map options = new HashMap<>(); + options.put("type", "paimon"); + options.put("warehouse", tempDir.toString()); + tEnv.executeSql( + String.format( + "CREATE CATALOG %s WITH ( %s )", + catalog, + options.entrySet().stream() + .map(e -> String.format("'%s'='%s'", e.getKey(), e.getValue())) + .collect(Collectors.joining(",")))); + tEnv.useCatalog(catalog); + } + + @Test + public void testAppendTable() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING) " + + "WITH ('bucket' = '1', 'bucket-key'='id', 'write-only' = 'true')", + TABLE_NAME)) + .await(); + + verifyChaining(false, true); + } + + @Test + public void testAppendTableWithUnawareBucket() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING) " + + "WITH ('bucket' = '-1', 'write-only' = 'true')", + TABLE_NAME)) + .await(); + + verifyChaining(true, true); + } + + @Test + public void testPrimaryKeyTable() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING, PRIMARY KEY (id) NOT ENFORCED) " + + "WITH ('bucket' = '1', 'bucket-key'='id', 'write-only' = 'true')", + TABLE_NAME)) + .await(); + + verifyChaining(false, true); + } + + @Test + public void testPrimaryKeyTableWithDynamicBucket() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING, PRIMARY KEY (id) NOT ENFORCED) " + + "WITH ('bucket' = '-1', 'write-only' = 'true')", + TABLE_NAME)) + .await(); + + verifyChaining(false, true); + } + + @Test + public void testPrimaryKeyTableWithMultipleWriter() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING, PRIMARY KEY (id) NOT ENFORCED) " + + "WITH ('bucket' = '1', 'bucket-key'='id', 'write-only' = 'true', 'sink.parallelism' = '2')", + TABLE_NAME)) + .await(); + + verifyChaining(false, false); + } + + @Test + public void testPrimaryKeyTableWithCrossPartitionUpdate() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING, PRIMARY KEY (id) NOT ENFORCED) " + + "PARTITIONED BY ( dt ) WITH ('bucket' = '-1', 'write-only' = 'true')", + TABLE_NAME)) + .await(); + + List vertices = verifyChaining(false, true); + JobVertex vertex = findVertex(vertices, "INDEX_BOOTSTRAP"); + assertThat(vertex.toString()).contains("Source"); + } + + @Test + public void testPrimaryKeyTableWithLocalMerge() throws Exception { + tEnv.executeSql( + String.format( + "CREATE TABLE %s (id INT, data STRING, dt STRING, PRIMARY KEY (id) NOT ENFORCED) " + + "WITH ('bucket' = '-1', 'write-only' = 'true', 'local-merge-buffer-size' = '1MB')", + TABLE_NAME)) + .await(); + + List vertices = verifyChaining(false, true); + JobVertex vertex = findVertex(vertices, "local merge"); + assertThat(vertex.toString()).contains("Source"); + } + + private List verifyChaining( + boolean isWriterChainedWithUpstream, boolean isWriterChainedWithDownStream) { + CompiledPlan plan = + tEnv.compilePlanSql( + String.format( + "INSERT INTO %s VALUES (1, 'AAA', ''), (2, 'BBB', '')", + TABLE_NAME)); + List> transformations = CompiledPlanUtils.toTransformations(tEnv, plan); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + transformations.forEach(env::addOperator); + + List vertices = new ArrayList<>(); + env.getStreamGraph().getJobGraph().getVertices().forEach(vertices::add); + JobVertex vertex = findVertex(vertices, "Writer(write-only)"); + + if (isWriterChainedWithUpstream) { + assertThat(vertex.toString()).contains("Source"); + } else { + assertThat(vertex.toString()).doesNotContain("Source"); + } + + if (isWriterChainedWithDownStream) { + assertThat(vertex.toString()).contains("Committer"); + } else { + assertThat(vertex.toString()).doesNotContain("Committer"); + } + + return vertices; + } + + private JobVertex findVertex(List vertices, String key) { + for (JobVertex vertex : vertices) { + if (vertex.toString().contains(key)) { + return vertex; + } + } + throw new IllegalStateException( + String.format( + "Cannot find vertex with keyword %s among job vertices %s", key, vertices)); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java index 3a8c1557122f7..83af157450780 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java @@ -115,9 +115,10 @@ public void testAppendOnlyTableMetrics() throws Exception { private void testMetricsImpl(FileStoreTable fileStoreTable) throws Exception { String tableName = tablePath.getName(); - RowDataStoreWriteOperator operator = getStoreSinkWriteOperator(fileStoreTable); + RowDataStoreWriteOperator.Factory operatorFactory = + getStoreSinkWriteOperatorFactory(fileStoreTable); OneInputStreamOperatorTestHarness harness = - createHarness(operator); + createHarness(operatorFactory); TypeSerializer serializer = new CommittableTypeInfo().createSerializer(new ExecutionConfig()); @@ -133,7 +134,7 @@ private void testMetricsImpl(FileStoreTable fileStoreTable) throws Exception { harness.snapshot(1, 2); harness.notifyOfCompletedCheckpoint(1); - OperatorMetricGroup metricGroup = operator.getMetricGroup(); + OperatorMetricGroup metricGroup = harness.getOneInputOperator().getMetricGroup(); MetricGroup writerBufferMetricGroup = metricGroup .addGroup("paimon") @@ -173,9 +174,10 @@ public void testAsyncLookupWithFailure() throws Exception { rowType, Arrays.asList("pt", "k"), Collections.singletonList("k"), options); // we don't wait for compaction because this is async lookup test - RowDataStoreWriteOperator operator = getAsyncLookupWriteOperator(fileStoreTable, false); + RowDataStoreWriteOperator.Factory operatorFactory = + getAsyncLookupWriteOperatorFactory(fileStoreTable, false); OneInputStreamOperatorTestHarness harness = - createHarness(operator); + createHarness(operatorFactory); TableCommitImpl commit = fileStoreTable.newCommit(commitUser); @@ -205,8 +207,8 @@ public void testAsyncLookupWithFailure() throws Exception { harness.close(); // re-create operator from state, this time wait for compaction to check result - operator = getAsyncLookupWriteOperator(fileStoreTable, true); - harness = createHarness(operator); + operatorFactory = getAsyncLookupWriteOperatorFactory(fileStoreTable, true); + harness = createHarness(operatorFactory); harness.setup(serializer); harness.initializeState(state); harness.open(); @@ -263,9 +265,10 @@ private void testChangelog(boolean insertOnly) throws Exception { FileStoreTable fileStoreTable = createFileStoreTable( rowType, Arrays.asList("pt", "k"), Collections.singletonList("k"), options); - RowDataStoreWriteOperator operator = getStoreSinkWriteOperator(fileStoreTable); + RowDataStoreWriteOperator.Factory operatorFactory = + getStoreSinkWriteOperatorFactory(fileStoreTable); OneInputStreamOperatorTestHarness harness = - createHarness(operator); + createHarness(operatorFactory); TableCommitImpl commit = fileStoreTable.newCommit(commitUser); @@ -277,7 +280,7 @@ private void testChangelog(boolean insertOnly) throws Exception { if (insertOnly) { Field field = TableWriteOperator.class.getDeclaredField("write"); field.setAccessible(true); - StoreSinkWrite write = (StoreSinkWrite) field.get(operator); + StoreSinkWrite write = (StoreSinkWrite) field.get(harness.getOneInputOperator()); write.withInsertOnly(true); } @@ -339,17 +342,17 @@ public void testNumWritersMetric() throws Exception { options); TableCommitImpl commit = fileStoreTable.newCommit(commitUser); - RowDataStoreWriteOperator rowDataStoreWriteOperator = - getStoreSinkWriteOperator(fileStoreTable); + RowDataStoreWriteOperator.Factory operatorFactory = + getStoreSinkWriteOperatorFactory(fileStoreTable); OneInputStreamOperatorTestHarness harness = - createHarness(rowDataStoreWriteOperator); + createHarness(operatorFactory); TypeSerializer serializer = new CommittableTypeInfo().createSerializer(new ExecutionConfig()); harness.setup(serializer); harness.open(); - OperatorMetricGroup metricGroup = rowDataStoreWriteOperator.getMetricGroup(); + OperatorMetricGroup metricGroup = harness.getOneInputOperator().getMetricGroup(); MetricGroup writerBufferMetricGroup = metricGroup .addGroup("paimon") @@ -408,8 +411,9 @@ public void testNumWritersMetric() throws Exception { // Test utils // ------------------------------------------------------------------------ - private RowDataStoreWriteOperator getStoreSinkWriteOperator(FileStoreTable fileStoreTable) { - return new RowDataStoreWriteOperator( + private RowDataStoreWriteOperator.Factory getStoreSinkWriteOperatorFactory( + FileStoreTable fileStoreTable) { + return new RowDataStoreWriteOperator.Factory( fileStoreTable, null, (table, commitUser, state, ioManager, memoryPool, metricGroup) -> @@ -426,9 +430,9 @@ private RowDataStoreWriteOperator getStoreSinkWriteOperator(FileStoreTable fileS commitUser); } - private RowDataStoreWriteOperator getAsyncLookupWriteOperator( + private RowDataStoreWriteOperator.Factory getAsyncLookupWriteOperatorFactory( FileStoreTable fileStoreTable, boolean waitCompaction) { - return new RowDataStoreWriteOperator( + return new RowDataStoreWriteOperator.Factory( fileStoreTable, null, (table, commitUser, state, ioManager, memoryPool, metricGroup) -> @@ -471,10 +475,11 @@ private FileStoreTable createFileStoreTable( } private OneInputStreamOperatorTestHarness createHarness( - RowDataStoreWriteOperator operator) throws Exception { + RowDataStoreWriteOperator.Factory operatorFactory) throws Exception { InternalTypeInfo internalRowInternalTypeInfo = new InternalTypeInfo<>(new InternalRowTypeSerializer(RowType.builder().build())); return new OneInputStreamOperatorTestHarness<>( - operator, internalRowInternalTypeInfo.createSerializer(new ExecutionConfig())); + operatorFactory, + internalRowInternalTypeInfo.createSerializer(new ExecutionConfig())); } } 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 19c22d137c7f2..fca5dcf0ed69d 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 @@ -66,7 +66,8 @@ public void markDone(LinkedHashMap partitionSpec) public void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { throw new UnsupportedOperationException(); } 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 index 0050f3083a8ec..f245940da57d8 100644 --- 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 @@ -117,7 +117,8 @@ public void markDone(LinkedHashMap partitionSpec) public void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) throws Exception { partitionParams.put( PartitionPathUtils.generatePartitionPath(partitionSpec), 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 index 8404d994fa9f2..0c5d485af7bc9 100644 --- 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 @@ -18,10 +18,10 @@ package org.apache.paimon.flink.source; +import org.apache.commons.collections.IteratorUtils; 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; @@ -67,7 +67,7 @@ public void testParallelSourceExecution() throws Exception { "iterator source"); final List result = - DataStreamUtils.collectBoundedStream(stream, "Iterator Source Test"); + IteratorUtils.toList(stream.executeAndCollect("Iterator Source Test")); verifySequence(result, 1L, 1_000L); } 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 85679e5fd30af..d2bb9eb982741 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 @@ -69,7 +69,7 @@ public class TestChangelogDataReadWrite { private static final RowType KEY_TYPE = new RowType(singletonList(new DataField(0, "k", new BigIntType()))); private static final RowType VALUE_TYPE = - new RowType(singletonList(new DataField(0, "v", new BigIntType()))); + new RowType(singletonList(new DataField(1, "v", new BigIntType()))); private static final RowType PARTITION_TYPE = new RowType(singletonList(new DataField(0, "p", new IntType()))); private static final Comparator COMPARATOR = @@ -87,7 +87,7 @@ public List keyFields(TableSchema schema) { @Override public List valueFields(TableSchema schema) { return Collections.singletonList( - new DataField(0, "v", new org.apache.paimon.types.BigIntType(false))); + new DataField(1, "v", new org.apache.paimon.types.BigIntType(false))); } }; @@ -110,7 +110,8 @@ public TestChangelogDataReadWrite(String root) { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); 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 61a03a29a21bd..0cd969707cfa6 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 @@ -33,12 +33,17 @@ import org.apache.paimon.table.source.TableRead; import org.apache.paimon.types.DataTypes; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.RecordAttributes; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.data.GenericRowData; @@ -46,6 +51,7 @@ import org.apache.flink.table.runtime.typeutils.InternalSerializers; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.function.SupplierWithException; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -58,11 +64,13 @@ import java.util.List; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import static org.apache.paimon.CoreOptions.CONSUMER_ID; import static org.assertj.core.api.Assertions.assertThat; -/** Test for {@link MonitorFunction} and {@link ReadOperator}. */ +/** Test for {@link MonitorSource} and {@link ReadOperator}. */ public class OperatorSourceTest { @TempDir Path tempDir; @@ -114,28 +122,39 @@ private List> readSplit(Split split) throws IOException { } @Test - public void testMonitorFunction() throws Exception { + public void testMonitorSource() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // 1. run first OperatorSubtaskState snapshot; { - MonitorFunction function = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource src = new StreamSource<>(function); + MonitorSource source = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operator = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + source.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarness = - new AbstractStreamOperatorTestHarness<>(src, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operator, 1, 1, 0); testHarness.open(); - snapshot = testReadSplit(function, () -> testHarness.snapshot(0, 0), 1, 1, 1); + snapshot = testReadSplit(operator, () -> testHarness.snapshot(0, 0), 1, 1, 1); } // 2. restore from state { - MonitorFunction functionCopy1 = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource srcCopy1 = new StreamSource<>(functionCopy1); + MonitorSource sourceCopy1 = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operatorCopy1 = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + sourceCopy1.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarnessCopy1 = - new AbstractStreamOperatorTestHarness<>(srcCopy1, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operatorCopy1, 1, 1, 0); testHarnessCopy1.initializeState(snapshot); testHarnessCopy1.open(); testReadSplit( - functionCopy1, + operatorCopy1, () -> { testHarnessCopy1.snapshot(1, 1); testHarnessCopy1.notifyOfCompletedCheckpoint(1); @@ -148,12 +167,17 @@ public void testMonitorFunction() throws Exception { // 3. restore from consumer id { - MonitorFunction functionCopy2 = new MonitorFunction(table.newReadBuilder(), 10, false); - StreamSource srcCopy2 = new StreamSource<>(functionCopy2); + MonitorSource sourceCopy2 = new MonitorSource(table.newReadBuilder(), 10, false); + TestingSourceOperator operatorCopy2 = + (TestingSourceOperator) + TestingSourceOperator.createTestOperator( + sourceCopy2.createReader(null), + WatermarkStrategy.noWatermarks(), + false); AbstractStreamOperatorTestHarness testHarnessCopy2 = - new AbstractStreamOperatorTestHarness<>(srcCopy2, 1, 1, 0); + new AbstractStreamOperatorTestHarness<>(operatorCopy2, 1, 1, 0); testHarnessCopy2.open(); - testReadSplit(functionCopy2, () -> null, 3, 3, 3); + testReadSplit(operatorCopy2, () -> null, 3, 3, 3); } } @@ -204,6 +228,14 @@ public void testReadOperatorMetricsRegisterAndUpdate() throws Exception { .getValue()) .isEqualTo(-1L); + Thread.sleep(300L); + assertThat( + (Long) + TestingMetricUtils.getGauge( + readerOperatorMetricGroup, "sourceIdleTime") + .getValue()) + .isGreaterThan(299L); + harness.processElement(new StreamRecord<>(splits.get(0))); assertThat( (Long) @@ -228,10 +260,18 @@ public void testReadOperatorMetricsRegisterAndUpdate() throws Exception { "currentEmitEventTimeLag") .getValue()) .isEqualTo(emitEventTimeLag); + + assertThat( + (Long) + TestingMetricUtils.getGauge( + readerOperatorMetricGroup, "sourceIdleTime") + .getValue()) + .isGreaterThan(99L) + .isLessThan(300L); } private T testReadSplit( - MonitorFunction function, + SourceOperator operator, SupplierWithException beforeClose, int a, int b, @@ -239,20 +279,36 @@ private T testReadSplit( throws Exception { Throwable[] error = new Throwable[1]; ArrayBlockingQueue queue = new ArrayBlockingQueue<>(10); + AtomicReference> iteratorRef = new AtomicReference<>(); - DummySourceContext sourceContext = - new DummySourceContext() { + PushingAsyncDataInput.DataOutput output = + new PushingAsyncDataInput.DataOutput() { @Override - public void collect(Split element) { - queue.add(element); + public void emitRecord(StreamRecord streamRecord) { + queue.add(streamRecord.getValue()); } + + @Override + public void emitWatermark(Watermark watermark) {} + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {} + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) {} + + @Override + public void emitRecordAttributes(RecordAttributes recordAttributes) {} }; + AtomicBoolean isRunning = new AtomicBoolean(true); Thread runner = new Thread( () -> { try { - function.run(sourceContext); + while (isRunning.get()) { + operator.emitNext(output); + } } catch (Throwable t) { t.printStackTrace(); error[0] = t; @@ -266,34 +322,15 @@ public void collect(Split element) { assertThat(readSplit(split)).containsExactlyInAnyOrder(Arrays.asList(a, b, c)); T t = beforeClose.get(); - function.cancel(); + CloseableIterator iterator = iteratorRef.get(); + if (iterator != null) { + iterator.close(); + } + isRunning.set(false); runner.join(); assertThat(error[0]).isNull(); return t; } - - private abstract static class DummySourceContext - implements SourceFunction.SourceContext { - - private final Object lock = new Object(); - - @Override - public void collectWithTimestamp(Split element, long timestamp) {} - - @Override - public void emitWatermark(Watermark mark) {} - - @Override - public void markAsTemporarilyIdle() {} - - @Override - public Object getCheckpointLock() { - return lock; - } - - @Override - public void close() {} - } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java new file mode 100644 index 0000000000000..77b44d5b0e5c7 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/operator/TestingSourceOperator.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.operator; + +import org.apache.paimon.flink.source.SimpleSourceSplit; +import org.apache.paimon.flink.source.SimpleSourceSplitSerializer; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.DummyEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackendParametersImpl; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.flink.streaming.api.operators.SourceOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; +import org.apache.flink.streaming.util.MockStreamingRuntimeContext; + +import java.util.ArrayList; +import java.util.Collections; + +/** + * A SourceOperator extension to simplify test setup. + * + *

    This class is implemented in reference to {@link + * org.apache.flink.streaming.api.operators.source.TestingSourceOperator}. + * + *

    See Flink + * PR that introduced this class + */ +public class TestingSourceOperator extends SourceOperator { + + private static final long serialVersionUID = 1L; + + private final int subtaskIndex; + private final int parallelism; + + public TestingSourceOperator( + StreamOperatorParameters parameters, + SourceReader reader, + WatermarkStrategy watermarkStrategy, + ProcessingTimeService timeService, + boolean emitProgressiveWatermarks) { + + this( + parameters, + reader, + watermarkStrategy, + timeService, + new TestingOperatorEventGateway(), + 1, + 5, + emitProgressiveWatermarks); + } + + public TestingSourceOperator( + StreamOperatorParameters parameters, + SourceReader reader, + WatermarkStrategy watermarkStrategy, + ProcessingTimeService timeService, + OperatorEventGateway eventGateway, + int subtaskIndex, + int parallelism, + boolean emitProgressiveWatermarks) { + + super( + (context) -> reader, + eventGateway, + new SimpleSourceSplitSerializer(), + watermarkStrategy, + timeService, + new Configuration(), + "localhost", + emitProgressiveWatermarks, + () -> false); + + this.subtaskIndex = subtaskIndex; + this.parallelism = parallelism; + this.metrics = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup(); + initSourceMetricGroup(); + + // unchecked wrapping is okay to keep tests simpler + try { + initReader(); + } catch (Exception e) { + throw new RuntimeException(e); + } + + setup(parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + } + + @Override + public StreamingRuntimeContext getRuntimeContext() { + return new MockStreamingRuntimeContext(false, parallelism, subtaskIndex); + } + + // this is overridden to avoid complex mock injection through the "containingTask" + @Override + public ExecutionConfig getExecutionConfig() { + ExecutionConfig cfg = new ExecutionConfig(); + cfg.setAutoWatermarkInterval(100); + return cfg; + } + + public static SourceOperator createTestOperator( + SourceReader reader, + WatermarkStrategy watermarkStrategy, + boolean emitProgressiveWatermarks) + throws Exception { + + AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); + Environment env = new MockEnvironmentBuilder().build(); + CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); + final OperatorStateStore operatorStateStore = + abstractStateBackend.createOperatorStateBackend( + new OperatorStateBackendParametersImpl( + env, + "test-operator", + Collections.emptyList(), + cancelStreamRegistry)); + + final StateInitializationContext stateContext = + new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + + TestProcessingTimeService timeService = new TestProcessingTimeService(); + timeService.setCurrentTime(Integer.MAX_VALUE); // start somewhere that is not zero + + final SourceOperator sourceOperator = + new TestingSourceOperator<>( + new StreamOperatorParameters<>( + new SourceOperatorStreamTask(new DummyEnvironment()), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>()), + null, + null, + null), + reader, + watermarkStrategy, + timeService, + emitProgressiveWatermarks); + sourceOperator.initializeState(stateContext); + sourceOperator.open(); + + return sourceOperator; + } + + private static class TestingOperatorEventGateway implements OperatorEventGateway { + @Override + public void sendEventToCoordinator(OperatorEvent event) {} + } +} 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 ce0017eb18747..ee838ed68255e 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 @@ -19,6 +19,7 @@ package org.apache.paimon.flink.util; import org.apache.paimon.utils.FileIOUtils; +import org.apache.paimon.utils.TimeUtils; import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.dag.Transformation; @@ -29,7 +30,6 @@ import org.apache.flink.runtime.client.JobStatusMessage; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; @@ -164,6 +164,11 @@ public TableEnvironmentBuilder setConf(ConfigOption option, T value) { return this; } + public TableEnvironmentBuilder setString(String key, String value) { + conf.setString(key, value); + return this; + } + public TableEnvironmentBuilder setConf(Configuration conf) { this.conf.addAll(conf); return this; @@ -182,9 +187,10 @@ public TableEnvironment build() { if (checkpointIntervalMs != null) { tEnv.getConfig() .getConfiguration() - .set( - ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL, - Duration.ofMillis(checkpointIntervalMs)); + .setString( + "execution.checkpointing.interval", + TimeUtils.formatWithHighestUnit( + Duration.ofMillis(checkpointIntervalMs))); } } else { tEnv = diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java index cfc23a0a44d8d..39939f78670b9 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/MiniClusterWithClientExtension.java @@ -29,7 +29,6 @@ import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.test.junit5.InjectClusterClient; -import org.apache.flink.test.util.TestEnvironment; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.AfterEachCallback; import org.junit.jupiter.api.extension.BeforeAllCallback; @@ -167,17 +166,12 @@ private void registerEnv(InternalMiniClusterExtension internalMiniClusterExtensi .getOptional(CoreOptions.DEFAULT_PARALLELISM) .orElse(internalMiniClusterExtension.getNumberSlots()); - TestEnvironment executionEnvironment = - new TestEnvironment( - internalMiniClusterExtension.getMiniCluster(), defaultParallelism, false); - executionEnvironment.setAsContext(); TestStreamEnvironment.setAsContext( internalMiniClusterExtension.getMiniCluster(), defaultParallelism); } private void unregisterEnv(InternalMiniClusterExtension internalMiniClusterExtension) { TestStreamEnvironment.unsetAsContext(); - TestEnvironment.unsetAsContext(); } private MiniClusterClient createMiniClusterClient( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java index 86b0014eb39cf..0eac2ed2936e0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java @@ -23,8 +23,8 @@ import org.apache.paimon.utils.BlockingIterator; import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Time; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.RestartStrategyOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; @@ -36,6 +36,7 @@ import javax.annotation.Nullable; import java.nio.file.Paths; +import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -52,7 +53,7 @@ /** Test util for {@link ReadWriteTableITCase}. */ public class ReadWriteTableTestUtil { - private static final Time TIME_OUT = Time.seconds(10); + private static final Duration TIME_OUT = Duration.ofSeconds(10); public static final int DEFAULT_PARALLELISM = 2; @@ -75,12 +76,11 @@ public static void init(String warehouse) { } public static void init(String warehouse, int parallelism) { - StreamExecutionEnvironment sExeEnv = buildStreamEnv(parallelism); - sExeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + // Using `none` to avoid compatibility issues with Flink 1.18-. + StreamExecutionEnvironment sExeEnv = buildStreamEnv(parallelism, "none"); sEnv = StreamTableEnvironment.create(sExeEnv); - bExeEnv = buildBatchEnv(parallelism); - bExeEnv.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + bExeEnv = buildBatchEnv(parallelism, "none"); bEnv = StreamTableEnvironment.create(bExeEnv, EnvironmentSettings.inBatchMode()); ReadWriteTableTestUtil.warehouse = warehouse; @@ -95,16 +95,24 @@ public static void init(String warehouse, int parallelism) { bEnv.useCatalog(catalog); } - public static StreamExecutionEnvironment buildStreamEnv(int parallelism) { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + public static StreamExecutionEnvironment buildStreamEnv( + int parallelism, String restartStrategy) { + Configuration configuration = new Configuration(); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY, restartStrategy); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); env.setRuntimeMode(RuntimeExecutionMode.STREAMING); env.enableCheckpointing(100); env.setParallelism(parallelism); return env; } - public static StreamExecutionEnvironment buildBatchEnv(int parallelism) { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + public static StreamExecutionEnvironment buildBatchEnv( + int parallelism, String restartStrategy) { + Configuration configuration = new Configuration(); + configuration.set(RestartStrategyOptions.RESTART_STRATEGY, restartStrategy); + final StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configuration); env.setRuntimeMode(RuntimeExecutionMode.BATCH); env.setParallelism(parallelism); return env; @@ -270,7 +278,7 @@ public static void testBatchRead(String query, List expected) throws Except try (BlockingIterator iterator = BlockingIterator.of(resultItr)) { if (!expected.isEmpty()) { List result = - iterator.collect(expected.size(), TIME_OUT.getSize(), TIME_OUT.getUnit()); + iterator.collect(expected.size(), TIME_OUT.getSeconds(), TimeUnit.SECONDS); assertThat(toInsertOnlyRows(result)) .containsExactlyInAnyOrderElementsOf(toInsertOnlyRows(expected)); } diff --git a/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory b/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory index fcb6fe982943f..3c05b5fba3ec9 100644 --- a/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory +++ b/paimon-flink/paimon-flink-common/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory @@ -15,8 +15,5 @@ org.apache.paimon.flink.FlinkCatalogTest$TestingLogSoreRegisterFactory -# Lineage meta factory -org.apache.paimon.flink.FlinkLineageITCase$TestingMemoryLineageMetaFactory - # Catalog lock factory org.apache.paimon.flink.FileSystemCatalogITCase$FileSystemCatalogDummyLockFactory \ No newline at end of file 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 index 6c3af4e50043f..93aa0719caea5 100644 --- a/paimon-format/src/main/java/org/apache/orc/impl/RecordReaderImpl.java +++ b/paimon-format/src/main/java/org/apache/orc/impl/RecordReaderImpl.java @@ -62,6 +62,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; import java.math.BigDecimal; import java.sql.Timestamp; @@ -127,7 +129,7 @@ public class RecordReaderImpl implements RecordReader { 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; + @Nullable 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" diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java index 7f3e275183cf3..a06ca9948c444 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroBulkFormat.java @@ -22,7 +22,7 @@ import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; -import org.apache.paimon.reader.RecordReader; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.types.RowType; import org.apache.paimon.utils.IOUtils; import org.apache.paimon.utils.IteratorResultIterator; @@ -49,12 +49,12 @@ public AvroBulkFormat(RowType projectedRowType) { } @Override - public RecordReader createReader(FormatReaderFactory.Context context) + public FileRecordReader createReader(FormatReaderFactory.Context context) throws IOException { return new AvroReader(context.fileIO(), context.filePath(), context.fileSize()); } - private class AvroReader implements RecordReader { + private class AvroReader implements FileRecordReader { private final FileIO fileIO; private final DataFileReader reader; @@ -90,7 +90,7 @@ private DataFileReader createReaderFromPath(Path path, long fileSiz @Nullable @Override - public RecordIterator readBatch() throws IOException { + public IteratorResultIterator readBatch() throws IOException { Object ticket; try { ticket = pool.pollEntry(); 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 63a51c0a13a94..fcce9ae505305 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 @@ -105,7 +105,7 @@ private CodecFactory createCodecFactory(String compression) { if (compression.equalsIgnoreCase("zstd")) { return CodecFactory.zstandardCodec(zstdLevel); } - return CodecFactory.fromString(options.get(AVRO_OUTPUT_CODEC)); + return CodecFactory.fromString(compression); } /** A {@link FormatWriterFactory} to write {@link InternalRow}. */ diff --git a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroRowDatumWriter.java b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroRowDatumWriter.java index c2bd81d0038fd..d302451625720 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroRowDatumWriter.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/avro/AvroRowDatumWriter.java @@ -56,6 +56,15 @@ public void write(InternalRow datum, Encoder out) throws IOException { // top Row is a UNION type out.writeIndex(1); } - this.writer.writeRow(datum, out); + try { + this.writer.writeRow(datum, out); + } catch (NullPointerException npe) { + throw new RuntimeException( + "Caught NullPointerException, the possible reason is you have set following options together:\n" + + " 1. file.format = avro;\n" + + " 2. merge-function = aggregation/partial-update;\n" + + " 3. some fields are not null.", + npe); + } } } 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 792fefabf03c1..5ed10bdfdd1e4 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 @@ -28,7 +28,6 @@ import org.apache.paimon.format.orc.filter.OrcFilters; import org.apache.paimon.format.orc.filter.OrcPredicateFunctionVisitor; import org.apache.paimon.format.orc.filter.OrcSimpleStatsExtractor; -import org.apache.paimon.format.orc.reader.OrcSplitReaderUtil; import org.apache.paimon.format.orc.writer.RowDataVectorizer; import org.apache.paimon.format.orc.writer.Vectorizer; import org.apache.paimon.fs.ObjectCacheManager; @@ -142,7 +141,7 @@ public FormatReaderFactory createReaderFactory( @Override public void validateDataFields(RowType rowType) { DataType refinedType = refineDataType(rowType); - OrcSplitReaderUtil.toOrcType(refinedType); + OrcTypeUtil.convertToOrcSchema((RowType) refinedType); } /** @@ -160,9 +159,8 @@ public FormatWriterFactory createWriterFactory(RowType type) { DataType refinedType = refineDataType(type); DataType[] orcTypes = getFieldTypes(refinedType).toArray(new DataType[0]); - TypeDescription typeDescription = OrcSplitReaderUtil.toOrcType(refinedType); - Vectorizer vectorizer = - new RowDataVectorizer(typeDescription.toString(), orcTypes); + TypeDescription typeDescription = OrcTypeUtil.convertToOrcSchema((RowType) refinedType); + Vectorizer vectorizer = new RowDataVectorizer(typeDescription, orcTypes); return new OrcWriterFactory(vectorizer, orcProperties, writerConf, writeBatchSize); } 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 dbc5de265c492..db17357bfd705 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 @@ -24,12 +24,14 @@ import org.apache.paimon.data.columnar.ColumnarRowIterator; import org.apache.paimon.data.columnar.VectorizedColumnBatch; import org.apache.paimon.fileindex.FileIndexResult; +import org.apache.paimon.fileindex.bitmap.BitmapIndexResult; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.OrcFormatReaderContext; import org.apache.paimon.format.fs.HadoopReadOnlyFileSystem; import org.apache.paimon.format.orc.filter.OrcFilters; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.reader.RecordReader.RecordIterator; import org.apache.paimon.types.DataType; import org.apache.paimon.types.RowType; @@ -54,8 +56,8 @@ import java.io.IOException; import java.util.List; +import static org.apache.paimon.format.orc.OrcTypeUtil.convertToOrcSchema; import static org.apache.paimon.format.orc.reader.AbstractOrcColumnVector.createPaimonVector; -import static org.apache.paimon.format.orc.reader.OrcSplitReaderUtil.toOrcType; import static org.apache.paimon.utils.Preconditions.checkNotNull; /** An ORC reader that produces a stream of {@link ColumnarRow} records. */ @@ -80,7 +82,7 @@ public OrcReaderFactory( final int batchSize, final boolean deletionVectorsEnabled) { this.hadoopConfig = checkNotNull(hadoopConfig); - this.schema = toOrcType(readType); + this.schema = convertToOrcSchema(readType); this.tableType = readType; this.conjunctPredicates = checkNotNull(conjunctPredicates); this.batchSize = batchSize; @@ -184,7 +186,7 @@ public VectorizedRowBatch orcVectorizedRowBatch() { return orcVectorizedRowBatch; } - private RecordIterator convertAndGetIterator( + private ColumnarRowIterator convertAndGetIterator( VectorizedRowBatch orcBatch, long rowNumber) { // no copying from the ORC column vectors to the Paimon columns vectors necessary, // because they point to the same data arrays internally design @@ -209,8 +211,7 @@ private RecordIterator convertAndGetIterator( * batch is addressed by the starting row number of the batch, plus the number of records to be * skipped before. */ - private static final class OrcVectorizedReader - implements org.apache.paimon.reader.RecordReader { + private static final class OrcVectorizedReader implements FileRecordReader { private final RecordReader orcReader; private final Pool pool; @@ -222,7 +223,7 @@ private OrcVectorizedReader(final RecordReader orcReader, final Pool readBatch() throws IOException { + public ColumnarRowIterator readBatch() throws IOException { final OrcReaderBatch batch = getCachedEntry(); final VectorizedRowBatch orcVectorBatch = batch.orcVectorizedRowBatch(); @@ -258,7 +259,7 @@ private static RecordReader createRecordReader( org.apache.paimon.fs.Path path, long splitStart, long splitLength, - FileIndexResult fileIndexResult, + @Nullable FileIndexResult fileIndexResult, boolean deletionVectorsEnabled) throws IOException { org.apache.orc.Reader orcReader = createReader(conf, fileIO, path, fileIndexResult); @@ -276,9 +277,11 @@ private static RecordReader createRecordReader( .skipCorruptRecords(OrcConf.SKIP_CORRUPT_DATA.getBoolean(conf)) .tolerateMissingSchema( OrcConf.TOLERATE_MISSING_SCHEMA.getBoolean(conf)); - if (!conjunctPredicates.isEmpty() && !deletionVectorsEnabled) { - // deletion vectors can not enable this feature, cased by getRowNumber would be - // changed. + if (!conjunctPredicates.isEmpty() + && !deletionVectorsEnabled + && !(fileIndexResult instanceof BitmapIndexResult)) { + // row group filter push down will make row number change incorrect + // so deletion vectors mode and bitmap index cannot work with row group push down options.useSelected(OrcConf.READER_USE_SELECTED.getBoolean(conf)); options.allowSARGToFilter(OrcConf.ALLOW_SARG_TO_FILTER.getBoolean(conf)); } @@ -342,7 +345,7 @@ public static org.apache.orc.Reader createReader( org.apache.hadoop.conf.Configuration conf, FileIO fileIO, org.apache.paimon.fs.Path path, - FileIndexResult fileIndexResult) + @Nullable FileIndexResult fileIndexResult) throws IOException { // open ORC file and create reader org.apache.hadoop.fs.Path hPath = new org.apache.hadoop.fs.Path(path.toUri()); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcTypeUtil.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcTypeUtil.java new file mode 100644 index 0000000000000..f7d3d626d44f7 --- /dev/null +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/OrcTypeUtil.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.orc; + +import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.table.SpecialFields; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.CharType; +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.MapType; +import org.apache.paimon.types.RowType; +import org.apache.paimon.types.VarCharType; + +import org.apache.orc.TypeDescription; + +/** Util for orc types. */ +public class OrcTypeUtil { + + public static final String PAIMON_ORC_FIELD_ID_KEY = "paimon.id"; + + public static TypeDescription convertToOrcSchema(RowType rowType) { + TypeDescription struct = TypeDescription.createStruct(); + for (DataField dataField : rowType.getFields()) { + TypeDescription child = convertToOrcType(dataField.type(), dataField.id(), 0); + struct.addField(dataField.name(), child); + } + return struct; + } + + @VisibleForTesting + static TypeDescription convertToOrcType(DataType type, int fieldId, int depth) { + type = type.copy(true); + switch (type.getTypeRoot()) { + case CHAR: + return TypeDescription.createChar() + .withMaxLength(((CharType) type).getLength()) + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case VARCHAR: + int len = ((VarCharType) type).getLength(); + if (len == VarCharType.MAX_LENGTH) { + return TypeDescription.createString() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + } else { + return TypeDescription.createVarchar() + .withMaxLength(len) + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + } + case BOOLEAN: + return TypeDescription.createBoolean() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case VARBINARY: + if (type.equals(DataTypes.BYTES())) { + return TypeDescription.createBinary() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + } else { + throw new UnsupportedOperationException( + "Not support other binary type: " + type); + } + case DECIMAL: + DecimalType decimalType = (DecimalType) type; + return TypeDescription.createDecimal() + .withScale(decimalType.getScale()) + .withPrecision(decimalType.getPrecision()) + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case TINYINT: + return TypeDescription.createByte() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case SMALLINT: + return TypeDescription.createShort() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case INTEGER: + case TIME_WITHOUT_TIME_ZONE: + return TypeDescription.createInt() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case BIGINT: + return TypeDescription.createLong() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case FLOAT: + return TypeDescription.createFloat() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case DOUBLE: + return TypeDescription.createDouble() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case DATE: + return TypeDescription.createDate() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case TIMESTAMP_WITHOUT_TIME_ZONE: + return TypeDescription.createTimestamp() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return TypeDescription.createTimestampInstant() + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case ARRAY: + ArrayType arrayType = (ArrayType) type; + + String elementFieldId = + String.valueOf(SpecialFields.getArrayElementFieldId(fieldId, depth + 1)); + TypeDescription elementOrcType = + convertToOrcType(arrayType.getElementType(), fieldId, depth + 1) + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, elementFieldId); + + return TypeDescription.createList(elementOrcType) + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case MAP: + MapType mapType = (MapType) type; + + String mapKeyFieldId = + String.valueOf(SpecialFields.getMapKeyFieldId(fieldId, depth + 1)); + TypeDescription mapKeyOrcType = + convertToOrcType(mapType.getKeyType(), fieldId, depth + 1) + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, mapKeyFieldId); + + String mapValueFieldId = + String.valueOf(SpecialFields.getMapValueFieldId(fieldId, depth + 1)); + TypeDescription mapValueOrcType = + convertToOrcType(mapType.getValueType(), fieldId, depth + 1) + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, mapValueFieldId); + + return TypeDescription.createMap(mapKeyOrcType, mapValueOrcType) + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + case ROW: + return convertToOrcSchema((RowType) type) + .setAttribute(PAIMON_ORC_FIELD_ID_KEY, String.valueOf(fieldId)); + default: + throw new UnsupportedOperationException("Unsupported type: " + type); + } + } +} diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/reader/OrcSplitReaderUtil.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/reader/OrcSplitReaderUtil.java deleted file mode 100644 index 882f1c753991f..0000000000000 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/reader/OrcSplitReaderUtil.java +++ /dev/null @@ -1,99 +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.format.orc.reader; - -import org.apache.paimon.types.ArrayType; -import org.apache.paimon.types.CharType; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; -import org.apache.paimon.types.DecimalType; -import org.apache.paimon.types.MapType; -import org.apache.paimon.types.RowType; -import org.apache.paimon.types.VarCharType; - -import org.apache.orc.TypeDescription; - -/** Util for orc types. */ -public class OrcSplitReaderUtil { - - public static TypeDescription toOrcType(DataType type) { - type = type.copy(true); - switch (type.getTypeRoot()) { - case CHAR: - return TypeDescription.createChar().withMaxLength(((CharType) type).getLength()); - case VARCHAR: - int len = ((VarCharType) type).getLength(); - if (len == VarCharType.MAX_LENGTH) { - return TypeDescription.createString(); - } else { - return TypeDescription.createVarchar().withMaxLength(len); - } - case BOOLEAN: - return TypeDescription.createBoolean(); - case VARBINARY: - if (type.equals(DataTypes.BYTES())) { - return TypeDescription.createBinary(); - } else { - throw new UnsupportedOperationException( - "Not support other binary type: " + type); - } - case DECIMAL: - DecimalType decimalType = (DecimalType) type; - return TypeDescription.createDecimal() - .withScale(decimalType.getScale()) - .withPrecision(decimalType.getPrecision()); - case TINYINT: - return TypeDescription.createByte(); - case SMALLINT: - return TypeDescription.createShort(); - case INTEGER: - case TIME_WITHOUT_TIME_ZONE: - return TypeDescription.createInt(); - case BIGINT: - return TypeDescription.createLong(); - case FLOAT: - return TypeDescription.createFloat(); - case DOUBLE: - return TypeDescription.createDouble(); - case DATE: - return TypeDescription.createDate(); - case TIMESTAMP_WITHOUT_TIME_ZONE: - return TypeDescription.createTimestamp(); - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return TypeDescription.createTimestampInstant(); - case ARRAY: - ArrayType arrayType = (ArrayType) type; - return TypeDescription.createList(toOrcType(arrayType.getElementType())); - case MAP: - MapType mapType = (MapType) type; - return TypeDescription.createMap( - toOrcType(mapType.getKeyType()), toOrcType(mapType.getValueType())); - case ROW: - RowType rowType = (RowType) type; - TypeDescription struct = TypeDescription.createStruct(); - for (int i = 0; i < rowType.getFieldCount(); i++) { - struct.addField( - rowType.getFieldNames().get(i), toOrcType(rowType.getTypeAt(i))); - } - return struct; - default: - throw new UnsupportedOperationException("Unsupported type: " + type); - } - } -} diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/RowDataVectorizer.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/RowDataVectorizer.java index 21443cdf94634..46c936a0263ee 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/RowDataVectorizer.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/RowDataVectorizer.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hive.ql.exec.vector.ColumnVector; import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; +import org.apache.orc.TypeDescription; import java.util.Arrays; import java.util.List; @@ -35,7 +36,7 @@ public class RowDataVectorizer extends Vectorizer { private final List fieldWriters; - public RowDataVectorizer(String schema, DataType[] fieldTypes) { + public RowDataVectorizer(TypeDescription schema, DataType[] fieldTypes) { super(schema); this.fieldWriters = Arrays.stream(fieldTypes) diff --git a/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/Vectorizer.java b/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/Vectorizer.java index 0f0e6bba74a84..2add46531a618 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/Vectorizer.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/orc/writer/Vectorizer.java @@ -39,9 +39,9 @@ public abstract class Vectorizer implements Serializable { private final TypeDescription schema; - public Vectorizer(final String schema) { + public Vectorizer(final TypeDescription schema) { checkNotNull(schema); - this.schema = TypeDescription.fromString(schema); + this.schema = schema; } /** 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 2a62c0bc89471..6f8cab2202d62 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 @@ -23,16 +23,17 @@ 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.data.columnar.heap.ElementCountable; import org.apache.paimon.data.columnar.writable.WritableColumnVector; import org.apache.paimon.format.FormatReaderFactory; import org.apache.paimon.format.parquet.reader.ColumnReader; import org.apache.paimon.format.parquet.reader.ParquetDecimalVector; +import org.apache.paimon.format.parquet.reader.ParquetReadState; import org.apache.paimon.format.parquet.reader.ParquetTimestampVector; import org.apache.paimon.format.parquet.type.ParquetField; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; -import org.apache.paimon.reader.RecordReader; -import org.apache.paimon.reader.RecordReader.RecordIterator; +import org.apache.paimon.reader.FileRecordReader; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataType; @@ -88,8 +89,8 @@ public class ParquetReaderFactory implements FormatReaderFactory { private final Options conf; private final RowType projectedType; - private final String[] projectedFields; - private final DataType[] projectedTypes; + private final String[] projectedColumnNames; + private final DataField[] projectedFields; private final int batchSize; private final FilterCompat.Filter filter; private final Set unknownFieldsIndices = new HashSet<>(); @@ -98,14 +99,15 @@ public ParquetReaderFactory( Options conf, RowType projectedType, int batchSize, FilterCompat.Filter filter) { this.conf = conf; this.projectedType = projectedType; - this.projectedFields = projectedType.getFieldNames().toArray(new String[0]); - this.projectedTypes = projectedType.getFieldTypes().toArray(new DataType[0]); + this.projectedColumnNames = projectedType.getFieldNames().toArray(new String[0]); + this.projectedFields = projectedType.getFields().toArray(new DataField[0]); this.batchSize = batchSize; this.filter = filter; } @Override - public ParquetReader createReader(FormatReaderFactory.Context context) throws IOException { + public FileRecordReader createReader(FormatReaderFactory.Context context) + throws IOException { ParquetReadOptions.Builder builder = ParquetReadOptions.builder().withRange(0, context.fileSize()); setReadOptions(builder); @@ -129,7 +131,7 @@ public ParquetReader createReader(FormatReaderFactory.Context context) throws IO buildFieldsList(projectedType.getFields(), projectedType.getFieldNames(), columnIO); return new ParquetReader( - reader, requestedSchema, reader.getRecordCount(), poolOfBatches, fields); + reader, requestedSchema, reader.getFilteredRecordCount(), poolOfBatches, fields); } private void setReadOptions(ParquetReadOptions.Builder builder) { @@ -154,20 +156,20 @@ private void setReadOptions(ParquetReadOptions.Builder builder) { /** Clips `parquetSchema` according to `fieldNames`. */ private MessageType clipParquetSchema(GroupType parquetSchema) { - Type[] types = new Type[projectedFields.length]; - for (int i = 0; i < projectedFields.length; ++i) { - String fieldName = projectedFields[i]; + Type[] types = new Type[projectedColumnNames.length]; + for (int i = 0; i < projectedColumnNames.length; ++i) { + String fieldName = projectedColumnNames[i]; if (!parquetSchema.containsField(fieldName)) { LOG.warn( "{} does not exist in {}, will fill the field with null.", fieldName, parquetSchema); types[i] = - ParquetSchemaConverter.convertToParquetType(fieldName, projectedTypes[i]); + ParquetSchemaConverter.convertToParquetType(fieldName, projectedFields[i]); unknownFieldsIndices.add(i); } else { Type parquetType = parquetSchema.getType(fieldName); - types[i] = clipParquetType(projectedTypes[i], parquetType); + types[i] = clipParquetType(projectedFields[i].type(), parquetType); } } @@ -221,7 +223,7 @@ private Type clipParquetType(DataType readType, Type parquetType) { private void checkSchema(MessageType fileSchema, MessageType requestedSchema) throws IOException, UnsupportedOperationException { - if (projectedFields.length != requestedSchema.getFieldCount()) { + if (projectedColumnNames.length != requestedSchema.getFieldCount()) { throw new RuntimeException( "The quality of field type is incompatible with the request schema!"); } @@ -269,13 +271,13 @@ private ParquetReaderBatch createReaderBatch( } private WritableColumnVector[] createWritableVectors(MessageType requestedSchema) { - WritableColumnVector[] columns = new WritableColumnVector[projectedTypes.length]; + WritableColumnVector[] columns = new WritableColumnVector[projectedFields.length]; List types = requestedSchema.getFields(); - for (int i = 0; i < projectedTypes.length; i++) { + for (int i = 0; i < projectedFields.length; i++) { columns[i] = createWritableColumnVector( batchSize, - projectedTypes[i], + projectedFields[i].type(), types.get(i), requestedSchema.getColumns(), 0); @@ -291,9 +293,12 @@ private VectorizedColumnBatch createVectorizedColumnBatch( WritableColumnVector[] writableVectors) { ColumnVector[] vectors = new ColumnVector[writableVectors.length]; for (int i = 0; i < writableVectors.length; i++) { - switch (projectedTypes[i].getTypeRoot()) { + switch (projectedFields[i].type().getTypeRoot()) { case DECIMAL: - vectors[i] = new ParquetDecimalVector(writableVectors[i]); + vectors[i] = + new ParquetDecimalVector( + writableVectors[i], + ((ElementCountable) writableVectors[i]).getLen()); break; case TIMESTAMP_WITHOUT_TIME_ZONE: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: @@ -307,7 +312,7 @@ private VectorizedColumnBatch createVectorizedColumnBatch( return new VectorizedColumnBatch(vectors); } - private class ParquetReader implements RecordReader { + private class ParquetReader implements FileRecordReader { private ParquetFileReader reader; @@ -332,6 +337,10 @@ private class ParquetReader implements RecordReader { private long nextRowPosition; + private ParquetReadState currentRowGroupReadState; + + private long currentRowGroupFirstRowIndex; + /** * For each request column, the reader to read this column. This is NULL if this column is * missing from the file, in which case we populate the attribute with NULL. @@ -355,12 +364,13 @@ private ParquetReader( this.totalCountLoadedSoFar = 0; this.currentRowPosition = 0; this.nextRowPosition = 0; + this.currentRowGroupFirstRowIndex = 0; this.fields = fields; } @Nullable @Override - public RecordIterator readBatch() throws IOException { + public ColumnarRowIterator readBatch() throws IOException { final ParquetReaderBatch batch = getCachedEntry(); if (!nextBatch(batch)) { @@ -386,7 +396,8 @@ private boolean nextBatch(ParquetReaderBatch batch) throws IOException { currentRowPosition = nextRowPosition; } - int num = (int) Math.min(batchSize, totalCountLoadedSoFar - rowsReturned); + int num = getBachSize(); + for (int i = 0; i < columnReaders.length; ++i) { if (columnReaders[i] == null) { batch.writableVectors[i].fillWithNulls(); @@ -396,13 +407,13 @@ private boolean nextBatch(ParquetReaderBatch batch) throws IOException { } } rowsReturned += num; - nextRowPosition = currentRowPosition + num; + nextRowPosition = getNextRowPosition(num); batch.columnarBatch.setNumRows(num); return true; } private void readNextRowGroup() throws IOException { - PageReadStore rowGroup = reader.readNextRowGroup(); + PageReadStore rowGroup = reader.readNextFilteredRowGroup(); if (rowGroup == null) { throw new IOException( "expecting more rows but reached last block. Read " @@ -411,13 +422,16 @@ private void readNextRowGroup() throws IOException { + totalRowCount); } + this.currentRowGroupReadState = + new ParquetReadState(rowGroup.getRowIndexes().orElse(null)); + List types = requestedSchema.getFields(); columnReaders = new ColumnReader[types.size()]; for (int i = 0; i < types.size(); ++i) { if (!unknownFieldsIndices.contains(i)) { columnReaders[i] = createColumnReader( - projectedTypes[i], + projectedFields[i].type(), types.get(i), requestedSchema.getColumns(), rowGroup, @@ -425,18 +439,62 @@ private void readNextRowGroup() throws IOException { 0); } } + totalCountLoadedSoFar += rowGroup.getRowCount(); - if (rowGroup.getRowIndexOffset().isPresent()) { - currentRowPosition = rowGroup.getRowIndexOffset().get(); + + if (rowGroup.getRowIndexOffset().isPresent()) { // filter + currentRowGroupFirstRowIndex = rowGroup.getRowIndexOffset().get(); + long pageIndex = 0; + if (!this.currentRowGroupReadState.isMaxRange()) { + pageIndex = this.currentRowGroupReadState.currentRangeStart(); + } + currentRowPosition = currentRowGroupFirstRowIndex + pageIndex; } else { if (reader.rowGroupsFiltered()) { throw new RuntimeException( "There is a bug, rowIndexOffset must be present when row groups are filtered."); } + currentRowGroupFirstRowIndex = nextRowPosition; currentRowPosition = nextRowPosition; } } + private int getBachSize() throws IOException { + + long rangeBatchSize = Long.MAX_VALUE; + if (this.currentRowGroupReadState.isFinished()) { + throw new IOException( + "expecting more rows but reached last page block. Read " + + rowsReturned + + " out of " + + totalRowCount); + } else if (!this.currentRowGroupReadState.isMaxRange()) { + long pageIndex = this.currentRowPosition - this.currentRowGroupFirstRowIndex; + rangeBatchSize = this.currentRowGroupReadState.currentRangeEnd() - pageIndex + 1; + } + + return (int) + Math.min( + batchSize, + Math.min(rangeBatchSize, totalCountLoadedSoFar - rowsReturned)); + } + + private long getNextRowPosition(int num) { + if (this.currentRowGroupReadState.isMaxRange()) { + return this.currentRowPosition + num; + } else { + long pageIndex = this.currentRowPosition - this.currentRowGroupFirstRowIndex; + long nextIndex = pageIndex + num; + + if (this.currentRowGroupReadState.currentRangeEnd() < nextIndex) { + this.currentRowGroupReadState.nextRange(); + nextIndex = this.currentRowGroupReadState.currentRangeStart(); + } + + return this.currentRowGroupFirstRowIndex + nextIndex; + } + } + private ParquetReaderBatch getCachedEntry() throws IOException { try { return pool.pollEntry(); @@ -488,7 +546,7 @@ public void recycle() { recycler.recycle(this); } - public RecordIterator convertAndGetIterator(long rowNumber) { + public ColumnarRowIterator convertAndGetIterator(long rowNumber) { result.reset(rowNumber); return result; } 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 5cf81c375a37d..708e5eb7ea3d4 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 @@ -22,7 +22,6 @@ 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; @@ -39,9 +38,6 @@ import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types; -import java.util.ArrayList; -import java.util.List; - import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; @@ -59,86 +55,110 @@ public static MessageType convertToParquetMessageType(String name, RowType rowTy return new MessageType(name, convertToParquetTypes(rowType)); } - public static Type convertToParquetType(String name, DataType type) { - Type.Repetition repetition = - type.isNullable() ? Type.Repetition.OPTIONAL : Type.Repetition.REQUIRED; - return convertToParquetType(name, type, repetition); + public static Type convertToParquetType(String name, DataField field) { + return convertToParquetType(name, field.type(), field.id(), 0); + } + + private static Type[] convertToParquetTypes(RowType rowType) { + return rowType.getFields().stream() + .map(f -> convertToParquetType(f.name(), f.type(), f.id(), 0)) + .toArray(Type[]::new); } - private static Type convertToParquetType( - String name, DataType type, Type.Repetition repetition) { + private static Type convertToParquetType(String name, DataType type, int fieldId, int depth) { + Type.Repetition repetition = + type.isNullable() ? Type.Repetition.OPTIONAL : Type.Repetition.REQUIRED; switch (type.getTypeRoot()) { case CHAR: case VARCHAR: return Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition) .as(LogicalTypeAnnotation.stringType()) - .named(name); + .named(name) + .withId(fieldId); case BOOLEAN: return Types.primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, repetition) - .named(name); + .named(name) + .withId(fieldId); case BINARY: case VARBINARY: return Types.primitive(PrimitiveType.PrimitiveTypeName.BINARY, repetition) - .named(name); + .named(name) + .withId(fieldId); case DECIMAL: int precision = ((DecimalType) type).getPrecision(); int scale = ((DecimalType) type).getScale(); if (is32BitDecimal(precision)) { return Types.primitive(INT32, repetition) .as(LogicalTypeAnnotation.decimalType(scale, precision)) - .named(name); + .named(name) + .withId(fieldId); } else if (is64BitDecimal(precision)) { return Types.primitive(INT64, repetition) .as(LogicalTypeAnnotation.decimalType(scale, precision)) - .named(name); + .named(name) + .withId(fieldId); } else { return Types.primitive(FIXED_LEN_BYTE_ARRAY, repetition) .as(LogicalTypeAnnotation.decimalType(scale, precision)) .length(computeMinBytesForDecimalPrecision(precision)) - .named(name); + .named(name) + .withId(fieldId); } case TINYINT: return Types.primitive(INT32, repetition) .as(LogicalTypeAnnotation.intType(8, true)) - .named(name); + .named(name) + .withId(fieldId); case SMALLINT: return Types.primitive(INT32, repetition) .as(LogicalTypeAnnotation.intType(16, true)) - .named(name); + .named(name) + .withId(fieldId); case INTEGER: - return Types.primitive(INT32, repetition).named(name); + return Types.primitive(INT32, repetition).named(name).withId(fieldId); case BIGINT: - return Types.primitive(INT64, repetition).named(name); + return Types.primitive(INT64, repetition).named(name).withId(fieldId); case FLOAT: return Types.primitive(PrimitiveType.PrimitiveTypeName.FLOAT, repetition) - .named(name); + .named(name) + .withId(fieldId); case DOUBLE: return Types.primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, repetition) - .named(name); + .named(name) + .withId(fieldId); case DATE: return Types.primitive(INT32, repetition) .as(LogicalTypeAnnotation.dateType()) - .named(name); + .named(name) + .withId(fieldId); case TIME_WITHOUT_TIME_ZONE: return Types.primitive(INT32, repetition) .as( LogicalTypeAnnotation.timeType( true, LogicalTypeAnnotation.TimeUnit.MILLIS)) - .named(name); + .named(name) + .withId(fieldId); case TIMESTAMP_WITHOUT_TIME_ZONE: TimestampType timestampType = (TimestampType) type; return createTimestampWithLogicalType( - name, timestampType.getPrecision(), repetition, false); + name, timestampType.getPrecision(), repetition, false) + .withId(fieldId); case TIMESTAMP_WITH_LOCAL_TIME_ZONE: LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) type; return createTimestampWithLogicalType( - name, localZonedTimestampType.getPrecision(), repetition, true); + name, localZonedTimestampType.getPrecision(), repetition, true) + .withId(fieldId); case ARRAY: ArrayType arrayType = (ArrayType) type; - return ConversionPatterns.listOfElements( - repetition, - name, - convertToParquetType(LIST_ELEMENT_NAME, arrayType.getElementType())); + Type elementParquetType = + convertToParquetType( + LIST_ELEMENT_NAME, + arrayType.getElementType(), + fieldId, + depth + 1) + .withId(SpecialFields.getArrayElementFieldId(fieldId, depth + 1)); + return ConversionPatterns.listOfElements(repetition, name, elementParquetType) + .withId(fieldId); case MAP: MapType mapType = (MapType) type; DataType keyType = mapType.getKeyType(); @@ -147,12 +167,20 @@ private static Type convertToParquetType( // it as not nullable keyType = keyType.copy(false); } + Type mapKeyParquetType = + convertToParquetType(MAP_KEY_NAME, keyType, fieldId, depth + 1) + .withId(SpecialFields.getMapKeyFieldId(fieldId, depth + 1)); + Type mapValueParquetType = + convertToParquetType( + MAP_VALUE_NAME, mapType.getValueType(), fieldId, depth + 1) + .withId(SpecialFields.getMapValueFieldId(fieldId, depth + 1)); return ConversionPatterns.mapType( - repetition, - name, - MAP_REPEATED_NAME, - convertToParquetType(MAP_KEY_NAME, keyType), - convertToParquetType(MAP_VALUE_NAME, mapType.getValueType())); + repetition, + name, + MAP_REPEATED_NAME, + mapKeyParquetType, + mapValueParquetType) + .withId(fieldId); case MULTISET: MultisetType multisetType = (MultisetType) type; DataType elementType = multisetType.getElementType(); @@ -161,15 +189,23 @@ private static Type convertToParquetType( // so we configure it as not nullable elementType = elementType.copy(false); } + Type multisetKeyParquetType = + convertToParquetType(MAP_KEY_NAME, elementType, fieldId, depth + 1) + .withId(SpecialFields.getMapKeyFieldId(fieldId, depth + 1)); + Type multisetValueParquetType = + convertToParquetType(MAP_VALUE_NAME, new IntType(false), fieldId, depth + 1) + .withId(SpecialFields.getMapValueFieldId(fieldId, depth + 1)); return ConversionPatterns.mapType( - repetition, - name, - MAP_REPEATED_NAME, - convertToParquetType(MAP_KEY_NAME, elementType), - convertToParquetType(MAP_VALUE_NAME, new IntType(false))); + repetition, + name, + MAP_REPEATED_NAME, + multisetKeyParquetType, + multisetValueParquetType) + .withId(fieldId); case ROW: RowType rowType = (RowType) type; - return new GroupType(repetition, name, convertToParquetTypes(rowType)); + return new GroupType(repetition, name, convertToParquetTypes(rowType)) + .withId(fieldId); default: throw new UnsupportedOperationException("Unsupported type: " + type); } @@ -194,53 +230,6 @@ private static Type createTimestampWithLogicalType( } } - private static List convertToParquetTypes(RowType rowType) { - List types = new ArrayList<>(rowType.getFieldCount()); - 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; - } - public static int computeMinBytesForDecimalPrecision(int precision) { int numBytes = 1; while (Math.pow(2.0, 8 * numBytes - 1) < Math.pow(10.0, precision)) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/CollectionPosition.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/CollectionPosition.java index e72a4280f4aa8..beb5de7a92e54 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/CollectionPosition.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/CollectionPosition.java @@ -22,14 +22,14 @@ /** To represent collection's position in repeated type. */ public class CollectionPosition { + @Nullable private final boolean[] isNull; private final long[] offsets; - private final long[] length; - private final int valueCount; - public CollectionPosition(boolean[] isNull, long[] offsets, long[] length, int valueCount) { + public CollectionPosition( + @Nullable boolean[] isNull, long[] offsets, long[] length, int valueCount) { this.isNull = isNull; this.offsets = offsets; this.length = length; diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/LevelDelegation.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/LevelDelegation.java index 25bbedc861d12..8e30d90ba2c7a 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/LevelDelegation.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/position/LevelDelegation.java @@ -20,6 +20,7 @@ /** To delegate repetition level and definition level. */ public class LevelDelegation { + private final int[] repetitionLevel; private final int[] definitionLevel; diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/AbstractColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/AbstractColumnReader.java index 7e2ab6d5e7f05..d4a0ab039b536 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/AbstractColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/AbstractColumnReader.java @@ -32,6 +32,7 @@ import org.apache.parquet.column.page.DataPageV1; import org.apache.parquet.column.page.DataPageV2; import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.column.page.PageReader; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.io.ParquetDecodingException; @@ -65,20 +66,13 @@ public abstract class AbstractColumnReader protected final ColumnDescriptor descriptor; - /** Total number of values read. */ - private long valuesRead; - - /** - * value that indicates the end of the current page. That is, if valuesRead == - * endOfPageValueCount, we are at the end of the page. - */ - private long endOfPageValueCount; - /** If true, the current page is dictionary encoded. */ private boolean isCurrentPageDictionaryEncoded; - /** Total values in the current page. */ - private int pageValueCount; + /** + * Helper struct to track intermediate states while reading Parquet pages in the column chunk. + */ + private final ParquetReadState readState; /* * Input streams: @@ -93,7 +87,7 @@ public abstract class AbstractColumnReader */ /** Run length decoder for data and dictionary. */ - protected RunLengthDecoder runLenDecoder; + RunLengthDecoder runLenDecoder; /** Data input stream. */ ByteBufferInputStream dataInputStream; @@ -101,12 +95,14 @@ public abstract class AbstractColumnReader /** Dictionary decoder to wrap dictionary ids input stream. */ private RunLengthDecoder dictionaryIdsDecoder; - public AbstractColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public AbstractColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) throws IOException { this.descriptor = descriptor; - this.pageReader = pageReader; + this.pageReader = pageReadStore.getPageReader(descriptor); this.maxDefLevel = descriptor.getMaxDefinitionLevel(); + this.readState = new ParquetReadState(pageReadStore.getRowIndexes().orElse(null)); + DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); if (dictionaryPage != null) { try { @@ -147,56 +143,136 @@ public final void readToVector(int readNumber, VECTOR vector) throws IOException if (dictionary != null) { dictionaryIds = vector.reserveDictionaryIds(readNumber); } - while (readNumber > 0) { + + readState.resetForNewBatch(readNumber); + + while (readState.rowsToReadInBatch > 0) { // Compute the number of values we want to read in this page. - int leftInPage = (int) (endOfPageValueCount - valuesRead); - if (leftInPage == 0) { - DataPage page = pageReader.readPage(); - if (page instanceof DataPageV1) { - readPageV1((DataPageV1) page); - } else if (page instanceof DataPageV2) { - readPageV2((DataPageV2) page); - } else { - throw new RuntimeException("Unsupported page type: " + page.getClass()); + if (readState.valuesToReadInPage == 0) { + int pageValueCount = readPage(); + if (pageValueCount < 0) { + // we've read all the pages; this could happen when we're reading a repeated + // list and we + // don't know where the list will end until we've seen all the pages. + break; } - leftInPage = (int) (endOfPageValueCount - valuesRead); } - int num = Math.min(readNumber, leftInPage); - if (isCurrentPageDictionaryEncoded) { - // Read and decode dictionary ids. - runLenDecoder.readDictionaryIds( - num, dictionaryIds, vector, rowId, maxDefLevel, this.dictionaryIdsDecoder); - - if (vector.hasDictionary() || (rowId == 0 && supportLazyDecode())) { - // Column vector supports lazy decoding of dictionary values so just set the - // dictionary. - // We can't do this if rowId != 0 AND the column doesn't have a dictionary (i.e. - // some - // non-dictionary encoded values have already been added). - vector.setDictionary(new ParquetDictionary(dictionary)); + + if (readState.isFinished()) { + break; + } + + long pageRowId = readState.rowId; + int leftInBatch = readState.rowsToReadInBatch; + int leftInPage = readState.valuesToReadInPage; + + int readBatch = Math.min(leftInBatch, leftInPage); + + long rangeStart = readState.currentRangeStart(); + long rangeEnd = readState.currentRangeEnd(); + + if (pageRowId < rangeStart) { + int toSkip = (int) (rangeStart - pageRowId); + if (toSkip >= leftInPage) { // drop page + pageRowId += leftInPage; + leftInPage = 0; } else { - readBatchFromDictionaryIds(rowId, num, vector, dictionaryIds); + if (isCurrentPageDictionaryEncoded) { + runLenDecoder.skipDictionaryIds( + toSkip, maxDefLevel, this.dictionaryIdsDecoder); + pageRowId += toSkip; + leftInPage -= toSkip; + } else { + skipBatch(toSkip); + pageRowId += toSkip; + leftInPage -= toSkip; + } } + } else if (pageRowId > rangeEnd) { + readState.nextRange(); } else { - if (vector.hasDictionary() && rowId != 0) { - // This batch already has dictionary encoded values but this new page is not. - // The batch - // does not support a mix of dictionary and not so we will decode the - // dictionary. - readBatchFromDictionaryIds(0, rowId, vector, vector.getDictionaryIds()); + long start = pageRowId; + long end = Math.min(rangeEnd, pageRowId + readBatch - 1); + int num = (int) (end - start + 1); + + if (isCurrentPageDictionaryEncoded) { + // Read and decode dictionary ids. + runLenDecoder.readDictionaryIds( + num, + dictionaryIds, + vector, + rowId, + maxDefLevel, + this.dictionaryIdsDecoder); + + if (vector.hasDictionary() || (rowId == 0 && supportLazyDecode())) { + // Column vector supports lazy decoding of dictionary values so just set the + // dictionary. + // We can't do this if rowId != 0 AND the column doesn't have a dictionary + // (i.e. + // some + // non-dictionary encoded values have already been added). + vector.setDictionary(new ParquetDictionary(dictionary)); + } else { + readBatchFromDictionaryIds(rowId, num, vector, dictionaryIds); + } + } else { + if (vector.hasDictionary() && rowId != 0) { + // This batch already has dictionary encoded values but this new page is + // not. + // The batch + // does not support a mix of dictionary and not so we will decode the + // dictionary. + readBatchFromDictionaryIds(0, rowId, vector, vector.getDictionaryIds()); + } + vector.setDictionary(null); + readBatch(rowId, num, vector); } - vector.setDictionary(null); - readBatch(rowId, num, vector); + leftInBatch -= num; + pageRowId += num; + leftInPage -= num; + rowId += num; } + readState.rowsToReadInBatch = leftInBatch; + readState.valuesToReadInPage = leftInPage; + readState.rowId = pageRowId; + } + } - valuesRead += num; - rowId += num; - readNumber -= num; + private int readPage() { + DataPage page = pageReader.readPage(); + if (page == null) { + return -1; } + long pageFirstRowIndex = page.getFirstRowIndex().orElse(0L); + + int pageValueCount = + page.accept( + new DataPage.Visitor() { + @Override + public Integer visit(DataPageV1 dataPageV1) { + try { + return readPageV1(dataPageV1); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Integer visit(DataPageV2 dataPageV2) { + try { + return readPageV2(dataPageV2); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + readState.resetForNewPage(pageValueCount, pageFirstRowIndex); + return pageValueCount; } - private void readPageV1(DataPageV1 page) throws IOException { - this.pageValueCount = page.getValueCount(); + private int readPageV1(DataPageV1 page) throws IOException { + int pageValueCount = page.getValueCount(); ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); // Initialize the decoders. @@ -211,30 +287,31 @@ private void readPageV1(DataPageV1 page) throws IOException { ByteBufferInputStream in = bytes.toInputStream(); rlReader.initFromPage(pageValueCount, in); this.runLenDecoder.initFromStream(pageValueCount, in); - prepareNewPage(page.getValueEncoding(), in); + prepareNewPage(page.getValueEncoding(), in, pageValueCount); + return pageValueCount; } catch (IOException e) { throw new IOException("could not read page " + page + " in col " + descriptor, e); } } - private void readPageV2(DataPageV2 page) throws IOException { - this.pageValueCount = page.getValueCount(); + private int readPageV2(DataPageV2 page) throws IOException { + int pageValueCount = page.getValueCount(); int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); // do not read the length from the stream. v2 pages handle dividing the page bytes. this.runLenDecoder = new RunLengthDecoder(bitWidth, false); this.runLenDecoder.initFromStream( - this.pageValueCount, page.getDefinitionLevels().toInputStream()); + pageValueCount, page.getDefinitionLevels().toInputStream()); try { - prepareNewPage(page.getDataEncoding(), page.getData().toInputStream()); + prepareNewPage(page.getDataEncoding(), page.getData().toInputStream(), pageValueCount); + return pageValueCount; } catch (IOException e) { throw new IOException("could not read page " + page + " in col " + descriptor, e); } } - private void prepareNewPage(Encoding dataEncoding, ByteBufferInputStream in) + private void prepareNewPage(Encoding dataEncoding, ByteBufferInputStream in, int pageValueCount) throws IOException { - this.endOfPageValueCount = valuesRead + pageValueCount; if (dataEncoding.usesDictionary()) { if (dictionary == null) { throw new IOException( @@ -269,6 +346,14 @@ private void prepareNewPage(Encoding dataEncoding, ByteBufferInputStream in) afterReadPage(); } + final void skipDataBuffer(int length) { + try { + dataInputStream.skipFully(length); + } catch (IOException e) { + throw new ParquetDecodingException("Failed to skip " + length + " bytes", e); + } + } + final ByteBuffer readDataBuffer(int length) { try { return dataInputStream.slice(length).order(ByteOrder.LITTLE_ENDIAN); @@ -291,6 +376,8 @@ protected boolean supportLazyDecode() { /** Read batch from {@link #runLenDecoder} and {@link #dataInputStream}. */ protected abstract void readBatch(int rowId, int num, VECTOR column); + protected abstract void skipBatch(int num); + /** * Decode dictionary ids to data. From {@link #runLenDecoder} and {@link #dictionaryIdsDecoder}. */ diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BooleanColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BooleanColumnReader.java index d5dc231d8436d..4355392bf5520 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BooleanColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BooleanColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.io.ParquetDecodingException; import org.apache.parquet.schema.PrimitiveType; @@ -36,17 +36,12 @@ public class BooleanColumnReader extends AbstractColumnReader 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + for (int i = 0; i < n; i++) { + readBoolean(); + } + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + readBoolean(); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + private boolean readBoolean() { if (bitOffset == 0) { try { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ByteColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ByteColumnReader.java index bed9923d9be35..804b8bc0275ea 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ByteColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ByteColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -31,8 +31,9 @@ /** Byte {@link ColumnReader}. Using INT32 to store byte, so just cast int to byte. */ public class ByteColumnReader extends AbstractColumnReader { - public ByteColumnReader(ColumnDescriptor descriptor, PageReader pageReader) throws IOException { - super(descriptor, pageReader); + public ByteColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) + throws IOException { + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.INT32); } @@ -69,6 +70,38 @@ protected void readBatch(int rowId, int num, WritableByteVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipByte(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipByte(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipByte(int num) { + skipDataBuffer(4 * num); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableByteVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BytesColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BytesColumnReader.java index e83115c8a69f4..6ee395e585684 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BytesColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/BytesColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -31,9 +31,9 @@ /** Bytes {@link ColumnReader}. A int length and bytes data. */ public class BytesColumnReader extends AbstractColumnReader { - public BytesColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public BytesColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) throws IOException { - super(descriptor, pageReader); + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.BINARY); } @@ -70,6 +70,41 @@ protected void readBatch(int rowId, int num, WritableBytesVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipBinary(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipBinary(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipBinary(int num) { + for (int i = 0; i < num; i++) { + int len = readDataBuffer(4).getInt(); + skipDataBuffer(len); + } + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableBytesVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/DoubleColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/DoubleColumnReader.java index d6d8aa2bbb22b..2cffd406248e5 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/DoubleColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/DoubleColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -31,9 +31,9 @@ /** Double {@link ColumnReader}. */ public class DoubleColumnReader extends AbstractColumnReader { - public DoubleColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public DoubleColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) throws IOException { - super(descriptor, pageReader); + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.DOUBLE); } @@ -70,6 +70,38 @@ protected void readBatch(int rowId, int num, WritableDoubleVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipDouble(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipDouble(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipDouble(int num) { + skipDataBuffer(8 * num); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableDoubleVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FixedLenBytesColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FixedLenBytesColumnReader.java index afce717a67197..25e1b466e4658 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FixedLenBytesColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FixedLenBytesColumnReader.java @@ -25,7 +25,7 @@ import org.apache.paimon.format.parquet.ParquetSchemaConverter; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.PrimitiveType; @@ -39,8 +39,9 @@ public class FixedLenBytesColumnReader private final int precision; public FixedLenBytesColumnReader( - ColumnDescriptor descriptor, PageReader pageReader, int precision) throws IOException { - super(descriptor, pageReader); + ColumnDescriptor descriptor, PageReadStore pageReadStore, int precision) + throws IOException { + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY); this.precision = precision; } @@ -79,6 +80,35 @@ protected void readBatch(int rowId, int num, VECTOR column) { } } + @Override + protected void skipBatch(int num) { + int bytesLen = descriptor.getPrimitiveType().getTypeLength(); + if (ParquetSchemaConverter.is32BitDecimal(precision)) { + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + skipDataBinary(bytesLen); + } + } + } else if (ParquetSchemaConverter.is64BitDecimal(precision)) { + + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + skipDataBinary(bytesLen); + } + } + } else { + for (int i = 0; i < num; i++) { + if (runLenDecoder.readInteger() == maxDefLevel) { + skipDataBinary(bytesLen); + } + } + } + } + + private void skipDataBinary(int len) { + skipDataBuffer(len); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, VECTOR column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FloatColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FloatColumnReader.java index 1f4adfa4b9c8c..e9eec13df5fc5 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FloatColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/FloatColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -31,9 +31,9 @@ /** Float {@link ColumnReader}. */ public class FloatColumnReader extends AbstractColumnReader { - public FloatColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public FloatColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) throws IOException { - super(descriptor, pageReader); + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.FLOAT); } @@ -70,6 +70,38 @@ protected void readBatch(int rowId, int num, WritableFloatVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipFloat(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipFloat(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipFloat(int num) { + skipDataBuffer(4 * num); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableFloatVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/IntColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/IntColumnReader.java index e38e916d187e6..521ad998f6f1f 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/IntColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/IntColumnReader.java @@ -21,7 +21,7 @@ import org.apache.paimon.data.columnar.writable.WritableIntVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -30,8 +30,9 @@ /** Int {@link ColumnReader}. */ public class IntColumnReader extends AbstractColumnReader { - public IntColumnReader(ColumnDescriptor descriptor, PageReader pageReader) throws IOException { - super(descriptor, pageReader); + public IntColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) + throws IOException { + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.INT32); } @@ -68,6 +69,38 @@ protected void readBatch(int rowId, int num, WritableIntVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipInteger(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipInteger(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipInteger(int num) { + skipDataBuffer(4 * num); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableIntVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/LongColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/LongColumnReader.java index a8e04eae673af..c4af086a7026a 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/LongColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/LongColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableLongVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -31,8 +31,9 @@ /** Long {@link ColumnReader}. */ public class LongColumnReader extends AbstractColumnReader { - public LongColumnReader(ColumnDescriptor descriptor, PageReader pageReader) throws IOException { - super(descriptor, pageReader); + public LongColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) + throws IOException { + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.INT64); } @@ -69,6 +70,38 @@ protected void readBatch(int rowId, int num, WritableLongVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipValue(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipValue(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipValue(int num) { + skipDataBuffer(num * 8); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableLongVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java index c89c77603dac2..8f20be2754471 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedColumnReader.java @@ -20,6 +20,7 @@ import org.apache.paimon.data.columnar.ColumnVector; import org.apache.paimon.data.columnar.heap.AbstractHeapVector; +import org.apache.paimon.data.columnar.heap.ElementCountable; import org.apache.paimon.data.columnar.heap.HeapArrayVector; import org.apache.paimon.data.columnar.heap.HeapMapVector; import org.apache.paimon.data.columnar.heap.HeapRowVector; @@ -134,7 +135,7 @@ private Pair readRow( String.format("Row field does not have any children: %s.", field)); } - int len = ((AbstractHeapVector) finalChildrenVectors[0]).getLen(); + int len = ((ElementCountable) finalChildrenVectors[0]).getLen(); boolean[] isNull = new boolean[len]; Arrays.fill(isNull, true); boolean hasNull = false; @@ -278,7 +279,7 @@ private Pair readPrimitive( reader = new NestedPrimitiveColumnReader( descriptor, - pages.getPageReader(descriptor), + pages, isUtcTimestamp, descriptor.getPrimitiveType(), field.getType(), diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java index 7ee33a0bb5cc8..f0a82a6d711e8 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/NestedPrimitiveColumnReader.java @@ -44,6 +44,7 @@ import org.apache.parquet.column.page.DataPageV1; import org.apache.parquet.column.page.DataPageV2; import org.apache.parquet.column.page.DictionaryPage; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.column.page.PageReader; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder; @@ -64,6 +65,7 @@ /** Reader to read nested primitive column. */ public class NestedPrimitiveColumnReader implements ColumnReader { + private static final Logger LOG = LoggerFactory.getLogger(NestedPrimitiveColumnReader.class); private final IntArrayList repetitionLevelList = new IntArrayList(0); @@ -82,15 +84,6 @@ public class NestedPrimitiveColumnReader implements ColumnReader valueList = new ArrayList<>(); + int valueIndex = collectDataFromParquetPage(readNumber, valueList); + + return fillColumnVector(valueIndex, valueList); + } + + private int collectDataFromParquetPage(int total, List valueList) throws IOException { + int valueIndex = 0; // repeated type need two loops to read data. - while (!eof && index < readNumber) { + + readState.resetForNewBatch(total); + + while (!eof && readState.rowsToReadInBatch > 0) { + + if (readState.isFinished()) { // finished to read + eof = true; + break; + } + + long pageRowId = readState.rowId; + long rangeStart = readState.currentRangeStart(); + long rangeEnd = readState.currentRangeEnd(); + + if (pageRowId > rangeEnd) { + readState.nextRange(); + continue; + } + + boolean needFilterSkip = pageRowId < rangeStart; + do { - if (!lastValue.shouldSkip) { + + if (!lastValue.shouldSkip && !needFilterSkip) { valueList.add(lastValue.value); valueIndex++; } } while (readValue() && (repetitionLevel != 0)); - index++; + + if (pageRowId == readState.rowId) { + readState.rowId = readState.rowId + 1; + } + + if (!needFilterSkip) { + readState.rowsToReadInBatch = readState.rowsToReadInBatch - 1; + } } - return fillColumnVector(valueIndex, valueList); + return valueIndex; } public LevelDelegation getLevelDelegation() { @@ -255,20 +285,24 @@ private void readAndSaveRepetitionAndDefinitionLevels() { // get the values of repetition and definitionLevel repetitionLevel = repetitionLevelColumn.nextInt(); definitionLevel = definitionLevelColumn.nextInt(); - valuesRead++; + readState.valuesToReadInPage = readState.valuesToReadInPage - 1; repetitionLevelList.add(repetitionLevel); definitionLevelList.add(definitionLevel); } private int readPageIfNeed() throws IOException { // Compute the number of values we want to read in this page. - int leftInPage = (int) (endOfPageValueCount - valuesRead); - if (leftInPage == 0) { - // no data left in current page, load data from new page - readPage(); - leftInPage = (int) (endOfPageValueCount - valuesRead); + if (readState.valuesToReadInPage == 0) { + int pageValueCount = readPage(); + // 返回当前 page 的数据量 + if (pageValueCount < 0) { + // we've read all the pages; this could happen when we're reading a repeated list + // and we + // don't know where the list will end until we've seen all the pages. + return -1; + } } - return leftInPage; + return readState.valuesToReadInPage; } private Object readPrimitiveTypedRow(DataType category) { @@ -495,7 +529,7 @@ private WritableColumnVector fillColumnVector(int total, List valueList) { phiv.vector[i] = ((List) valueList).get(i); } } - return new ParquetDecimalVector(phiv); + return new ParquetDecimalVector(phiv, total); case INT64: HeapLongVector phlv = new HeapLongVector(total); for (int i = 0; i < valueList.size(); i++) { @@ -505,10 +539,10 @@ private WritableColumnVector fillColumnVector(int total, List valueList) { phlv.vector[i] = ((List) valueList).get(i); } } - return new ParquetDecimalVector(phlv); + return new ParquetDecimalVector(phlv, total); default: HeapBytesVector phbv = getHeapBytesVector(total, valueList); - return new ParquetDecimalVector(phbv); + return new ParquetDecimalVector(phbv, total); } default: throw new RuntimeException("Unsupported type in the list: " + type); @@ -528,33 +562,36 @@ private static HeapBytesVector getHeapBytesVector(int total, List valueList) { return phbv; } - protected void readPage() { + protected int readPage() { DataPage page = pageReader.readPage(); if (page == null) { - return; + return -1; } - page.accept( - new DataPage.Visitor() { - @Override - public Void visit(DataPageV1 dataPageV1) { - readPageV1(dataPageV1); - return null; - } + long pageFirstRowIndex = page.getFirstRowIndex().orElse(0L); - @Override - public Void visit(DataPageV2 dataPageV2) { - readPageV2(dataPageV2); - return null; - } - }); + int pageValueCount = + page.accept( + new DataPage.Visitor() { + @Override + public Integer visit(DataPageV1 dataPageV1) { + return readPageV1(dataPageV1); + } + + @Override + public Integer visit(DataPageV2 dataPageV2) { + return readPageV2(dataPageV2); + } + }); + readState.resetForNewPage(pageValueCount, pageFirstRowIndex); + return pageValueCount; } private void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) throws IOException { - this.pageValueCount = valueCount; - this.endOfPageValueCount = valuesRead + pageValueCount; + // this.pageValueCount = valueCount; + // this.endOfPageValueCount = valuesRead + pageValueCount; if (dataEncoding.usesDictionary()) { this.dataColumn = null; if (dictionary == null) { @@ -577,13 +614,14 @@ private void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int } try { - dataColumn.initFromPage(pageValueCount, in); + dataColumn.initFromPage(valueCount, in); } catch (IOException e) { throw new IOException(String.format("Could not read page in col %s.", descriptor), e); } } - private void readPageV1(DataPageV1 page) { + private int readPageV1(DataPageV1 page) { + int pageValueCount = page.getValueCount(); ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); ValuesReader dlReader = page.getDlEncoding().getValuesReader(descriptor, DEFINITION_LEVEL); this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader); @@ -597,15 +635,16 @@ private void readPageV1(DataPageV1 page) { LOG.debug("Reading definition levels at {}.", in.position()); dlReader.initFromPage(pageValueCount, in); LOG.debug("Reading data at {}.", in.position()); - initDataReader(page.getValueEncoding(), in, page.getValueCount()); + initDataReader(page.getValueEncoding(), in, pageValueCount); + return pageValueCount; } catch (IOException e) { throw new ParquetDecodingException( String.format("Could not read page %s in col %s.", page, descriptor), e); } } - private void readPageV2(DataPageV2 page) { - this.pageValueCount = page.getValueCount(); + private int readPageV2(DataPageV2 page) { + int pageValueCount = page.getValueCount(); this.repetitionLevelColumn = newRLEIterator(descriptor.getMaxRepetitionLevel(), page.getRepetitionLevels()); this.definitionLevelColumn = @@ -615,8 +654,8 @@ private void readPageV2(DataPageV2 page) { "Page data size {} bytes and {} records.", page.getData().size(), pageValueCount); - initDataReader( - page.getDataEncoding(), page.getData().toInputStream(), page.getValueCount()); + initDataReader(page.getDataEncoding(), page.getData().toInputStream(), pageValueCount); + return pageValueCount; } catch (IOException e) { throw new ParquetDecodingException( String.format("Could not read page %s in col %s.", page, descriptor), e); diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetDecimalVector.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetDecimalVector.java index 28d308bac61f5..42714ab066dae 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetDecimalVector.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetDecimalVector.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.columnar.Dictionary; import org.apache.paimon.data.columnar.IntColumnVector; import org.apache.paimon.data.columnar.LongColumnVector; +import org.apache.paimon.data.columnar.heap.ElementCountable; import org.apache.paimon.data.columnar.writable.WritableBytesVector; import org.apache.paimon.data.columnar.writable.WritableColumnVector; import org.apache.paimon.data.columnar.writable.WritableIntVector; @@ -38,12 +39,18 @@ * {@link DecimalColumnVector} interface. */ public class ParquetDecimalVector - implements DecimalColumnVector, WritableLongVector, WritableIntVector, WritableBytesVector { + implements DecimalColumnVector, + WritableLongVector, + WritableIntVector, + WritableBytesVector, + ElementCountable { private final ColumnVector vector; + private final int len; - public ParquetDecimalVector(ColumnVector vector) { + public ParquetDecimalVector(ColumnVector vector, int len) { this.vector = vector; + this.len = len; } @Override @@ -225,4 +232,9 @@ public void fill(long value) { ((WritableLongVector) vector).fill(value); } } + + @Override + public int getLen() { + return len; + } } diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReadState.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReadState.java new file mode 100644 index 0000000000000..aa89ea9821442 --- /dev/null +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ParquetReadState.java @@ -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.format.parquet.reader; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.PrimitiveIterator; + +/** Parquet reader state for column index. */ +public class ParquetReadState { + + /** A special row range used when there is no row indexes (hence all rows must be included). */ + private static final RowRange MAX_ROW_RANGE = new RowRange(Long.MIN_VALUE, Long.MAX_VALUE); + + /** + * A special row range used when the row indexes are present AND all the row ranges have been + * processed. This serves as a sentinel at the end indicating that all rows come after the last + * row range should be skipped. + */ + private static final RowRange END_ROW_RANGE = new RowRange(Long.MAX_VALUE, Long.MIN_VALUE); + + private final Iterator rowRanges; + + private RowRange currentRange; + + /** row index for the next read. */ + long rowId; + + int valuesToReadInPage; + int rowsToReadInBatch; + + public ParquetReadState(PrimitiveIterator.OfLong rowIndexes) { + this.rowRanges = constructRanges(rowIndexes); + nextRange(); + } + + /** + * Construct a list of row ranges from the given `rowIndexes`. For example, suppose the + * `rowIndexes` are `[0, 1, 2, 4, 5, 7, 8, 9]`, it will be converted into 3 row ranges: `[0-2], + * [4-5], [7-9]`. + */ + private Iterator constructRanges(PrimitiveIterator.OfLong rowIndexes) { + if (rowIndexes == null) { + return null; + } + + List rowRanges = new ArrayList<>(); + long currentStart = Long.MIN_VALUE; + long previous = Long.MIN_VALUE; + + while (rowIndexes.hasNext()) { + long idx = rowIndexes.nextLong(); + if (currentStart == Long.MIN_VALUE) { + currentStart = idx; + } else if (previous + 1 != idx) { + RowRange range = new RowRange(currentStart, previous); + rowRanges.add(range); + currentStart = idx; + } + previous = idx; + } + + if (previous != Long.MIN_VALUE) { + rowRanges.add(new RowRange(currentStart, previous)); + } + + return rowRanges.iterator(); + } + + /** Must be called at the beginning of reading a new batch. */ + void resetForNewBatch(int batchSize) { + this.rowsToReadInBatch = batchSize; + } + + /** Must be called at the beginning of reading a new page. */ + void resetForNewPage(int totalValuesInPage, long pageFirstRowIndex) { + this.valuesToReadInPage = totalValuesInPage; + this.rowId = pageFirstRowIndex; + } + + /** Returns the start index of the current row range. */ + public long currentRangeStart() { + return currentRange.start; + } + + /** Returns the end index of the current row range. */ + public long currentRangeEnd() { + return currentRange.end; + } + + public boolean isFinished() { + return this.currentRange.equals(END_ROW_RANGE); + } + + public boolean isMaxRange() { + return this.currentRange.equals(MAX_ROW_RANGE); + } + + /** Advance to the next range. */ + public void nextRange() { + if (rowRanges == null) { + currentRange = MAX_ROW_RANGE; + } else if (!rowRanges.hasNext()) { + currentRange = END_ROW_RANGE; + } else { + currentRange = rowRanges.next(); + } + } + + /** Helper struct to represent a range of row indexes `[start, end]`. */ + public static class RowRange { + final long start; + final long end; + + RowRange(long start, long end) { + this.start = start; + this.end = end; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof RowRange)) { + return false; + } + return ((RowRange) obj).start == this.start && ((RowRange) obj).end == this.end; + } + } +} 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 860ec54fa88b0..a2be77414d5ac 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 @@ -87,58 +87,45 @@ public static ColumnReader createColumnReader( getAllColumnDescriptorByType(depth, type, columnDescriptors); switch (fieldType.getTypeRoot()) { case BOOLEAN: - return new BooleanColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new BooleanColumnReader(descriptors.get(0), pages); case TINYINT: - return new ByteColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new ByteColumnReader(descriptors.get(0), pages); case DOUBLE: - return new DoubleColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new DoubleColumnReader(descriptors.get(0), pages); case FLOAT: - return new FloatColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new FloatColumnReader(descriptors.get(0), pages); case INTEGER: case DATE: case TIME_WITHOUT_TIME_ZONE: - return new IntColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new IntColumnReader(descriptors.get(0), pages); case BIGINT: - return new LongColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new LongColumnReader(descriptors.get(0), pages); case SMALLINT: - return new ShortColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new ShortColumnReader(descriptors.get(0), pages); case CHAR: case VARCHAR: case BINARY: case VARBINARY: - return new BytesColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new BytesColumnReader(descriptors.get(0), pages); case TIMESTAMP_WITHOUT_TIME_ZONE: case TIMESTAMP_WITH_LOCAL_TIME_ZONE: if (descriptors.get(0).getPrimitiveType().getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT64) { - return new LongColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new LongColumnReader(descriptors.get(0), pages); } - return new TimestampColumnReader( - true, descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new TimestampColumnReader(true, descriptors.get(0), pages); case DECIMAL: switch (descriptors.get(0).getPrimitiveType().getPrimitiveTypeName()) { case INT32: - return new IntColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new IntColumnReader(descriptors.get(0), pages); case INT64: - return new LongColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new LongColumnReader(descriptors.get(0), pages); case BINARY: - return new BytesColumnReader( - descriptors.get(0), pages.getPageReader(descriptors.get(0))); + return new BytesColumnReader(descriptors.get(0), pages); case FIXED_LEN_BYTE_ARRAY: return new FixedLenBytesColumnReader( descriptors.get(0), - pages.getPageReader(descriptors.get(0)), + pages, ((DecimalType) fieldType).getPrecision()); } case ARRAY: diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RowColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RowColumnReader.java deleted file mode 100644 index fa2da03ef312f..0000000000000 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RowColumnReader.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.format.parquet.reader; - -import org.apache.paimon.data.columnar.heap.HeapRowVector; -import org.apache.paimon.data.columnar.writable.WritableColumnVector; - -import java.io.IOException; -import java.util.List; - -/** Row {@link ColumnReader}. */ -public class RowColumnReader implements ColumnReader { - - private final List fieldReaders; - - public RowColumnReader(List fieldReaders) { - this.fieldReaders = fieldReaders; - } - - @Override - public void readToVector(int readNumber, WritableColumnVector vector) throws IOException { - HeapRowVector rowVector = (HeapRowVector) vector; - WritableColumnVector[] vectors = rowVector.getFields(); - // row vector null array - boolean[] isNulls = new boolean[readNumber]; - for (int i = 0; i < vectors.length; i++) { - fieldReaders.get(i).readToVector(readNumber, vectors[i]); - - for (int j = 0; j < readNumber; j++) { - if (i == 0) { - isNulls[j] = vectors[i].isNullAt(j); - } else { - isNulls[j] = isNulls[j] && vectors[i].isNullAt(j); - } - if (i == vectors.length - 1 && isNulls[j]) { - // rowColumnVector[j] is null only when all fields[j] of rowColumnVector[j] is - // null - rowVector.setNullAt(j); - } - } - } - } -} diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RunLengthDecoder.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RunLengthDecoder.java index 2dd1655d571f0..ebb8f28fa1eee 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RunLengthDecoder.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/RunLengthDecoder.java @@ -194,6 +194,51 @@ private void readDictionaryIdData(int total, WritableIntVector c, int rowId) { } } + void skipDictionaryIds(int total, int level, RunLengthDecoder data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) { + this.readNextGroup(); + } + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.skipDictionaryIdData(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + data.readInteger(); + } + } + break; + } + left -= n; + currentCount -= n; + } + } + + private void skipDictionaryIdData(int total) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) { + this.readNextGroup(); + } + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + break; + case PACKED: + currentBufferIdx += n; + break; + } + left -= n; + currentCount -= n; + } + } + /** Reads the next varint encoded int. */ private int readUnsignedVarInt() throws IOException { int value = 0; diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ShortColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ShortColumnReader.java index 7b32232261a7b..bdb2f401fa3f3 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ShortColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/ShortColumnReader.java @@ -22,7 +22,7 @@ import org.apache.paimon.data.columnar.writable.WritableShortVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.schema.PrimitiveType; import java.io.IOException; @@ -30,9 +30,9 @@ /** Short {@link ColumnReader}. Using INT32 to store short, so just cast int to short. */ public class ShortColumnReader extends AbstractColumnReader { - public ShortColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + public ShortColumnReader(ColumnDescriptor descriptor, PageReadStore pageReadStore) throws IOException { - super(descriptor, pageReader); + super(descriptor, pageReadStore); checkTypeName(PrimitiveType.PrimitiveTypeName.INT32); } @@ -71,6 +71,38 @@ protected void readBatch(int rowId, int num, WritableShortVector column) { } } + @Override + protected void skipBatch(int num) { + int left = num; + while (left > 0) { + if (runLenDecoder.currentCount == 0) { + runLenDecoder.readNextGroup(); + } + int n = Math.min(left, runLenDecoder.currentCount); + switch (runLenDecoder.mode) { + case RLE: + if (runLenDecoder.currentValue == maxDefLevel) { + skipShot(n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (runLenDecoder.currentBuffer[runLenDecoder.currentBufferIdx++] + == maxDefLevel) { + skipShot(1); + } + } + break; + } + left -= n; + runLenDecoder.currentCount -= n; + } + } + + private void skipShot(int num) { + skipDataBuffer(4 * num); + } + @Override protected void readBatchFromDictionaryIds( int rowId, int num, WritableShortVector column, WritableIntVector dictionaryIds) { diff --git a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/TimestampColumnReader.java b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/TimestampColumnReader.java index 4a279ff90e15f..d6ac96ea44586 100644 --- a/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/TimestampColumnReader.java +++ b/paimon-format/src/main/java/org/apache/paimon/format/parquet/reader/TimestampColumnReader.java @@ -23,7 +23,7 @@ import org.apache.paimon.data.columnar.writable.WritableTimestampVector; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.PrimitiveType; @@ -36,8 +36,9 @@ /** * Timestamp {@link ColumnReader}. We only support INT96 bytes now, julianDay(4) + nanosOfDay(8). - * See https://github.com/apache/parquet-format/blob/master/DataTypes.md#timestamp TIMESTAMP_MILLIS - * and TIMESTAMP_MICROS are the deprecated ConvertedType. + * See Parquet + * Timestamp TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType. */ public class TimestampColumnReader extends AbstractColumnReader { @@ -49,9 +50,9 @@ public class TimestampColumnReader extends AbstractColumnReader createWriter(OutputFile out, String compressio conf.getInt( ParquetOutputFormat.PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE)) + .withPageRowCountLimit( + conf.getInt( + ParquetOutputFormat.PAGE_ROW_COUNT_LIMIT, + ParquetProperties.DEFAULT_PAGE_ROW_COUNT_LIMIT)) .withDictionaryPageSize( conf.getInt( ParquetOutputFormat.DICTIONARY_PAGE_SIZE, diff --git a/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java b/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java index 3f6486baaef20..9c0dbb43fe625 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/avro/AvroFileFormatTest.java @@ -198,4 +198,16 @@ private void checkException() throws IOException { .isInstanceOf(IOException.class) .hasMessageContaining("Artificial exception"); } + + @Test + void testCompression() throws IOException { + RowType rowType = DataTypes.ROW(DataTypes.INT().notNull()); + AvroFileFormat format = new AvroFileFormat(new FormatContext(new Options(), 1024, 1024)); + LocalFileIO localFileIO = LocalFileIO.create(); + Path file = new Path(new Path(tempPath.toUri()), UUID.randomUUID().toString()); + try (PositionOutputStream out = localFileIO.newOutputStream(file, false)) { + assertThatThrownBy(() -> format.createWriterFactory(rowType).create(out, "unsupported")) + .hasMessageContaining("Unrecognized codec: unsupported"); + } + } } diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcSplitReaderUtilTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcSplitReaderUtilTest.java deleted file mode 100644 index c07838dfa34ca..0000000000000 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcSplitReaderUtilTest.java +++ /dev/null @@ -1,68 +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.format.orc; - -import org.apache.paimon.format.orc.reader.OrcSplitReaderUtil; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypes; - -import org.junit.jupiter.api.Test; - -import static org.apache.paimon.format.orc.reader.OrcSplitReaderUtil.toOrcType; -import static org.assertj.core.api.Assertions.assertThat; - -/** Test for {@link OrcSplitReaderUtil}. */ -class OrcSplitReaderUtilTest { - - @Test - void testDataTypeToOrcType() { - test("boolean", DataTypes.BOOLEAN()); - test("char(123)", DataTypes.CHAR(123)); - test("varchar(123)", DataTypes.VARCHAR(123)); - test("string", DataTypes.STRING()); - test("binary", DataTypes.BYTES()); - test("tinyint", DataTypes.TINYINT()); - test("smallint", DataTypes.SMALLINT()); - test("int", DataTypes.INT()); - test("bigint", DataTypes.BIGINT()); - test("float", DataTypes.FLOAT()); - test("double", DataTypes.DOUBLE()); - test("date", DataTypes.DATE()); - test("timestamp", DataTypes.TIMESTAMP()); - test("array", DataTypes.ARRAY(DataTypes.FLOAT())); - test("map", DataTypes.MAP(DataTypes.FLOAT(), DataTypes.BIGINT())); - test( - "struct>", - DataTypes.ROW( - DataTypes.FIELD(0, "int0", DataTypes.INT()), - DataTypes.FIELD(1, "str1", DataTypes.STRING()), - DataTypes.FIELD(2, "double2", DataTypes.DOUBLE()), - DataTypes.FIELD( - 3, - "row3", - DataTypes.ROW( - DataTypes.FIELD(4, "int0", DataTypes.INT()), - DataTypes.FIELD(5, "int1", DataTypes.INT()))))); - test("decimal(4,2)", DataTypes.DECIMAL(4, 2)); - } - - private void test(String expected, DataType type) { - assertThat(toOrcType(type)).hasToString(expected); - } -} diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcTypeUtilTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcTypeUtilTest.java new file mode 100644 index 0000000000000..5669ac33d443e --- /dev/null +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcTypeUtilTest.java @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.orc; + +import org.apache.paimon.format.FileFormatFactory; +import org.apache.paimon.format.FormatWriter; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.PositionOutputStream; +import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.options.Options; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.DataTypes; +import org.apache.paimon.types.RowType; + +import org.apache.paimon.shade.guava30.com.google.common.base.Objects; + +import org.apache.hadoop.conf.Configuration; +import org.apache.orc.Reader; +import org.apache.orc.TypeDescription; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.paimon.format.orc.OrcFileFormat.refineDataType; +import static org.apache.paimon.format.orc.OrcTypeUtil.PAIMON_ORC_FIELD_ID_KEY; +import static org.apache.paimon.format.orc.OrcTypeUtil.convertToOrcSchema; +import static org.apache.paimon.format.orc.OrcTypeUtil.convertToOrcType; +import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** Test for {@link OrcTypeUtil}. */ +class OrcTypeUtilTest { + + @Test + void testDataTypeToOrcType() { + test("boolean", DataTypes.BOOLEAN()); + test("char(123)", DataTypes.CHAR(123)); + test("varchar(123)", DataTypes.VARCHAR(123)); + test("string", DataTypes.STRING()); + test("binary", DataTypes.BYTES()); + test("tinyint", DataTypes.TINYINT()); + test("smallint", DataTypes.SMALLINT()); + test("int", DataTypes.INT()); + test("bigint", DataTypes.BIGINT()); + test("float", DataTypes.FLOAT()); + test("double", DataTypes.DOUBLE()); + test("date", DataTypes.DATE()); + test("timestamp", DataTypes.TIMESTAMP()); + test("array", DataTypes.ARRAY(DataTypes.FLOAT())); + test("map", DataTypes.MAP(DataTypes.FLOAT(), DataTypes.BIGINT())); + test( + "struct>", + DataTypes.ROW( + DataTypes.FIELD(0, "int0", DataTypes.INT()), + DataTypes.FIELD(1, "str1", DataTypes.STRING()), + DataTypes.FIELD(2, "double2", DataTypes.DOUBLE()), + DataTypes.FIELD( + 3, + "row3", + DataTypes.ROW( + DataTypes.FIELD(4, "int0", DataTypes.INT()), + DataTypes.FIELD(5, "int1", DataTypes.INT()))))); + test("decimal(4,2)", DataTypes.DECIMAL(4, 2)); + } + + private void test(String expected, DataType type) { + assertThat(convertToOrcType(type, -1, -1)).hasToString(expected); + } + + @Test + void testFieldIdAttribute(@TempDir java.nio.file.Path tempPath) throws IOException { + RowType rowType = + RowType.builder() + .field("a", DataTypes.INT()) + .field( + "b", + RowType.builder(true, new AtomicInteger(10)) + .field("f0", DataTypes.STRING()) + .field("f1", DataTypes.INT()) + .build()) + .field("c", DataTypes.ARRAY(DataTypes.INT())) + .field("d", DataTypes.MAP(DataTypes.INT(), DataTypes.STRING())) + .field( + "e", + DataTypes.ARRAY( + RowType.builder(true, new AtomicInteger(20)) + .field("f0", DataTypes.STRING()) + .field("f1", DataTypes.INT()) + .build())) + .field( + "f", + RowType.builder(true, new AtomicInteger(30)) + .field("f0", DataTypes.ARRAY(DataTypes.INT())) + .build()) + .build(); + + // write schema to orc file then get + FileIO fileIO = LocalFileIO.create(); + Path tempFile = new Path(new Path(tempPath.toUri()), UUID.randomUUID().toString()); + + OrcFileFormat format = + new OrcFileFormat(new FileFormatFactory.FormatContext(new Options(), 1024, 1024)); + PositionOutputStream out = fileIO.newOutputStream(tempFile, false); + FormatWriter writer = format.createWriterFactory(rowType).create(out, "zstd"); + writer.close(); + out.close(); + + Reader orcReader = + OrcReaderFactory.createReader(new Configuration(), fileIO, tempFile, null); + TypeDescription orcSchema = orcReader.getSchema(); + + RowType refined = (RowType) refineDataType(rowType); + + assertThatNoException() + .isThrownBy(() -> checkStruct(convertToOrcSchema(refined), orcSchema)); + + assertThatNoException() + .isThrownBy( + () -> + checkStruct( + convertToOrcSchema(refined.project("c", "b", "d")), + orcSchema)); + + assertThatNoException() + .isThrownBy( + () -> + checkStruct( + convertToOrcSchema(refined.project("a", "e", "f")), + orcSchema)); + } + + private void checkStruct(TypeDescription requiredStruct, TypeDescription orcStruct) { + List requiredFields = requiredStruct.getFieldNames(); + List requiredTypes = requiredStruct.getChildren(); + List orcFields = orcStruct.getFieldNames(); + List orcTypes = orcStruct.getChildren(); + + for (int i = 0; i < requiredFields.size(); i++) { + String field = requiredFields.get(i); + int orcIndex = orcFields.indexOf(field); + checkArgument(orcIndex != -1, "Cannot find field %s in orc file meta.", field); + TypeDescription requiredType = requiredTypes.get(i); + TypeDescription orcType = orcTypes.get(orcIndex); + checkField(field, requiredType, orcType); + } + } + + private void checkField( + String fieldName, TypeDescription requiredType, TypeDescription orcType) { + checkFieldIdAttribute(fieldName, requiredType, orcType); + if (requiredType.getCategory().isPrimitive()) { + return; + } + + switch (requiredType.getCategory()) { + case LIST: + checkField( + "_elem", requiredType.getChildren().get(0), orcType.getChildren().get(0)); + return; + case MAP: + checkField("_key", requiredType.getChildren().get(0), orcType.getChildren().get(0)); + checkField( + "_value", requiredType.getChildren().get(1), orcType.getChildren().get(1)); + return; + case STRUCT: + checkStruct(requiredType, orcType); + return; + default: + throw new UnsupportedOperationException("Unsupported orc type: " + requiredType); + } + } + + private void checkFieldIdAttribute( + String fieldName, TypeDescription requiredType, TypeDescription orcType) { + String requiredId = requiredType.getAttributeValue(PAIMON_ORC_FIELD_ID_KEY); + String orcId = orcType.getAttributeValue(PAIMON_ORC_FIELD_ID_KEY); + checkArgument( + Objects.equal(requiredId, orcId), + "Field %s has different id: read type id is %s but orc type id is %s. This is unexpected.", + fieldName, + requiredId, + orcId); + } +} diff --git a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcWriterFactoryTest.java b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcWriterFactoryTest.java index 2511d7ed7a9e5..52df5afb4efd6 100644 --- a/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcWriterFactoryTest.java +++ b/paimon-format/src/test/java/org/apache/paimon/format/orc/OrcWriterFactoryTest.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path; import org.apache.orc.MemoryManager; import org.apache.orc.OrcFile; +import org.apache.orc.TypeDescription; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -47,7 +48,7 @@ void testNotOverrideInMemoryManager(@TempDir java.nio.file.Path tmpDir) throws I OrcWriterFactory factory = new TestOrcWriterFactory( new RowDataVectorizer( - "struct<_col0:string,_col1:int>", + TypeDescription.fromString("struct<_col0:string,_col1:int>"), new DataType[] {DataTypes.STRING(), DataTypes.INT()}), memoryManager); factory.create(new LocalPositionOutputStream(tmpDir.resolve("file1").toFile()), "LZ4"); 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 099811d98b528..ffe4d6008296a 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 @@ -97,6 +97,7 @@ import java.util.stream.Collectors; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -489,11 +490,17 @@ public void testConvertToParquetTypeWithId() { 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)))); + 2, + "c", + DataTypes.MAP( + DataTypes.INT(), + DataTypes.MAP( + DataTypes.BIGINT(), new RowType(nestedFields))))); RowType rowType = new RowType(fields); int baseId = 536870911; - Type mapValueType = + int depthLimit = 1 << 10; + Type innerMapValueType = new GroupType( Type.Repetition.OPTIONAL, "value", @@ -506,7 +513,17 @@ public void testConvertToParquetTypeWithId() { .as(LogicalTypeAnnotation.stringType()) .named("v2") .withId(4)) - .withId(baseId - 2); + .withId(baseId + depthLimit * 2 + 2); + Type outerMapValueType = + ConversionPatterns.mapType( + Type.Repetition.OPTIONAL, + "value", + "key_value", + Types.primitive(INT64, Type.Repetition.REQUIRED) + .named("key") + .withId(baseId - depthLimit * 2 - 2), + innerMapValueType) + .withId(baseId + depthLimit * 2 + 1); Type expected = new MessageType( "table", @@ -519,7 +536,7 @@ public void testConvertToParquetTypeWithId() { Type.Repetition.OPTIONAL) .as(LogicalTypeAnnotation.stringType()) .named("element") - .withId(baseId + 1)) + .withId(baseId + depthLimit + 1)) .withId(1), ConversionPatterns.mapType( Type.Repetition.OPTIONAL, @@ -527,8 +544,8 @@ public void testConvertToParquetTypeWithId() { "key_value", Types.primitive(INT32, Type.Repetition.REQUIRED) .named("key") - .withId(baseId + 2), - mapValueType) + .withId(baseId - depthLimit * 2 - 1), + outerMapValueType) .withId(2)); Type actual = ParquetSchemaConverter.convertToParquetMessageType("table", rowType); assertThat(actual).isEqualTo(expected); diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveAlterTableUtils.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveAlterTableUtils.java new file mode 100644 index 0000000000000..8f77499486fd4 --- /dev/null +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveAlterTableUtils.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.paimon.catalog.Identifier; + +import org.apache.hadoop.hive.common.StatsSetupConst; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.EnvironmentContext; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.thrift.TException; + +/** Utils for hive alter table. */ +public class HiveAlterTableUtils { + + public static void alterTable(IMetaStoreClient client, Identifier identifier, Table table) + throws TException { + try { + alterTableWithEnv(client, identifier, table); + } catch (NoClassDefFoundError | NoSuchMethodError e) { + alterTableWithoutEnv(client, identifier, table); + } + } + + private static void alterTableWithEnv( + IMetaStoreClient client, Identifier identifier, Table table) throws TException { + EnvironmentContext environmentContext = new EnvironmentContext(); + environmentContext.putToProperties(StatsSetupConst.CASCADE, "true"); + environmentContext.putToProperties(StatsSetupConst.DO_NOT_UPDATE_STATS, "false"); + client.alter_table_with_environmentContext( + identifier.getDatabaseName(), identifier.getTableName(), table, environmentContext); + } + + private static void alterTableWithoutEnv( + IMetaStoreClient client, Identifier identifier, Table table) throws TException { + client.alter_table(identifier.getDatabaseName(), identifier.getTableName(), table, true); + } +} 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 2bf16c0f44b1d..c74ede9815465 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 @@ -48,6 +48,7 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.Pair; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.view.View; import org.apache.paimon.view.ViewImpl; @@ -83,7 +84,6 @@ import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -112,7 +112,6 @@ 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. */ @@ -122,7 +121,7 @@ public class HiveCatalog extends AbstractCatalog { // Reserved properties public static final String TABLE_TYPE_PROP = "table_type"; - public static final String PAIMON_TABLE_TYPE_VALUE = "paimon"; + public static final String PAIMON_TABLE_IDENTIFIER = "PAIMON"; // we don't include paimon-hive-connector as dependencies because it depends on // hive-exec @@ -135,6 +134,7 @@ public class HiveCatalog extends AbstractCatalog { "org.apache.paimon.hive.PaimonStorageHandler"; private static final String HIVE_PREFIX = "hive."; public static final String HIVE_SITE_FILE = "hive-site.xml"; + private static final String HIVE_EXTERNAL_TABLE_PROP = "EXTERNAL"; private final HiveConf hiveConf; private final String clientClassName; @@ -210,6 +210,20 @@ public Path getTableLocation(Identifier identifier) { return getTableLocation(identifier, table); } + private Pair initialTableLocation( + Map tableOptions, Identifier identifier) { + boolean externalTable; + Path location; + if (tableOptions.containsKey(CoreOptions.PATH.key())) { + externalTable = true; + location = new Path(tableOptions.get(CoreOptions.PATH.key())); + } else { + externalTable = usingExternalTable(tableOptions); + location = getTableLocation(identifier, null); + } + return Pair.of(location, externalTable); + } + private Path getTableLocation(Identifier identifier, @Nullable Table table) { try { String databaseName = identifier.getDatabaseName(); @@ -283,6 +297,8 @@ private Database convertToHiveDatabase(String name, Map properti (key, value) -> { if (key.equals(COMMENT_PROP)) { database.setDescription(value); + } else if (key.equals(OWNER_PROP)) { + database.setOwnerName(value); } else if (key.equals(DB_LOCATION_PROP)) { database.setLocationUri(value); } else if (value != null) { @@ -299,12 +315,15 @@ public org.apache.paimon.catalog.Database getDatabaseImpl(String name) try { 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()); } + if (database.getOwnerName() != null) { + options.put(OWNER_PROP, database.getOwnerName()); + } + if (database.getLocationUri() != null) { + options.put(DB_LOCATION_PROP, database.getLocationUri()); + } return org.apache.paimon.catalog.Database.of(name, options, database.getDescription()); } catch (NoSuchObjectException e) { throw new DatabaseNotExistException(name); @@ -411,6 +430,18 @@ protected List listTablesImpl(String databaseName) { } } + @Override + protected TableMeta getDataTableMeta(Identifier identifier) throws TableNotExistException { + return getDataTableMeta(identifier, getHmsTable(identifier)); + } + + private TableMeta getDataTableMeta(Identifier identifier, Table table) + throws TableNotExistException { + return new TableMeta( + getDataTableSchema(identifier, table), + identifier.getFullName() + "." + table.getCreateTime()); + } + @Override public TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException { Table table = getHmsTable(identifier); @@ -567,19 +598,19 @@ public org.apache.paimon.table.Table getDataOrFormatTable(Identifier identifier) Preconditions.checkArgument(identifier.getSystemTableName() == null); Table table = getHmsTable(identifier); try { - TableSchema tableSchema = getDataTableSchema(identifier, table); + TableMeta tableMeta = getDataTableMeta(identifier, table); return FileStoreTableFactory.create( fileIO, getTableLocation(identifier, table), - tableSchema, + tableMeta.schema(), new CatalogEnvironment( identifier, + tableMeta.uuid(), Lock.factory( lockFactory().orElse(null), lockContext().orElse(null), identifier), - metastoreClientFactory(identifier, tableSchema).orElse(null), - lineageMetaFactory)); + metastoreClientFactory(identifier, tableMeta.schema()).orElse(null))); } catch (TableNotExistException ignore) { } @@ -616,8 +647,10 @@ public void createFormatTable(Identifier identifier, Schema schema) { options, schema.comment()); try { - Path location = getTableLocation(identifier, null); - Table hiveTable = createHiveFormatTable(identifier, newSchema, location); + Pair pair = initialTableLocation(schema.options(), identifier); + Path location = pair.getLeft(); + boolean externalTable = pair.getRight(); + Table hiveTable = createHiveFormatTable(identifier, newSchema, location, externalTable); 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. @@ -625,29 +658,36 @@ public void createFormatTable(Identifier identifier, Schema schema) { } } - private boolean usingExternalTable() { + private boolean usingExternalTable(Map tableOptions) { CatalogTableType tableType = OptionsUtils.convertToEnum( hiveConf.get(TABLE_TYPE.key(), CatalogTableType.MANAGED.toString()), CatalogTableType.class); - return CatalogTableType.EXTERNAL.equals(tableType); + + String externalPropValue = + tableOptions.getOrDefault( + HIVE_EXTERNAL_TABLE_PROP.toLowerCase(), + tableOptions.get(HIVE_EXTERNAL_TABLE_PROP.toUpperCase())); + return CatalogTableType.EXTERNAL.equals(tableType) + || "TRUE".equalsIgnoreCase(externalPropValue); } @Override protected void dropTableImpl(Identifier identifier) { try { + boolean externalTable = isExternalTable(getHmsTable(identifier)); clients.execute( client -> client.dropTable( identifier.getDatabaseName(), identifier.getTableName(), - true, + !externalTable, false, true)); // When drop a Hive external table, only the hive metadata is deleted and the data files // are not deleted. - if (usingExternalTable()) { + if (externalTable) { return; } @@ -662,7 +702,7 @@ protected void dropTableImpl(Identifier identifier) { } catch (Exception ee) { LOG.error("Delete directory[{}] fail for table {}", path, identifier, ee); } - } catch (TException e) { + } catch (TException | TableNotExistException e) { throw new RuntimeException("Failed to drop table " + identifier.getFullName(), e); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -673,28 +713,27 @@ protected void dropTableImpl(Identifier identifier) { @Override 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); + Pair pair = initialTableLocation(schema.options(), identifier); + Path location = pair.getLeft(); + boolean externalTable = pair.getRight(); TableSchema tableSchema; try { - tableSchema = - schemaManager(identifier, location).createTable(schema, usingExternalTable()); + tableSchema = schemaManager(identifier, location).createTable(schema, externalTable); } catch (Exception e) { - throw new RuntimeException( - "Failed to commit changes of table " - + identifier.getFullName() - + " to underlying files.", - e); + throw new RuntimeException("Failed to create table " + identifier.getFullName(), e); } try { clients.execute( client -> - client.createTable(createHiveTable(identifier, tableSchema, location))); + client.createTable( + createHiveTable( + identifier, tableSchema, location, externalTable))); } catch (Exception e) { try { - fileIO.deleteDirectoryQuietly(location); + if (!externalTable) { + fileIO.deleteDirectoryQuietly(location); + } } catch (Exception ee) { LOG.error("Delete directory[{}] fail for table {}", location, identifier, ee); } @@ -702,7 +741,8 @@ protected void createTableImpl(Identifier identifier, Schema schema) { } } - private Table createHiveTable(Identifier identifier, TableSchema tableSchema, Path location) { + private Table createHiveTable( + Identifier identifier, TableSchema tableSchema, Path location, boolean externalTable) { checkArgument(Options.fromMap(tableSchema.options()).get(TYPE) != FORMAT_TABLE); Map tblProperties; @@ -722,45 +762,39 @@ private Table createHiveTable(Identifier identifier, TableSchema tableSchema, Pa } } - Table table = newHmsTable(identifier, tblProperties, PAIMON_TABLE_TYPE_VALUE); - updateHmsTable(table, identifier, tableSchema, PAIMON_TABLE_TYPE_VALUE, location); + Table table = newHmsTable(identifier, tblProperties, null, externalTable); + updateHmsTable(table, identifier, tableSchema, null, location); return table; } private Table createHiveFormatTable( - Identifier identifier, TableSchema tableSchema, Path location) { - Options options = Options.fromMap(tableSchema.options()); - checkArgument(options.get(TYPE) == FORMAT_TABLE); + Identifier identifier, TableSchema tableSchema, Path location, boolean externalTable) { + CoreOptions coreOptions = new CoreOptions(tableSchema.options()); + checkArgument(coreOptions.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()); + // file.format option has a default value and cannot be empty. + FormatTable.Format provider = FormatTable.parseFormat(coreOptions.formatType()); Map tblProperties = new HashMap<>(); - Table table = newHmsTable(identifier, tblProperties, provider); + Table table = newHmsTable(identifier, tblProperties, provider, externalTable); 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; } @Override protected void renameTableImpl(Identifier fromTable, Identifier toTable) { try { - Table table = renameHiveTable(fromTable, toTable); - + // Get fromTable's location before rename Path fromPath = getTableLocation(fromTable); - if (!new SchemaManager(fileIO, fromPath).listAllIds().isEmpty()) { + Table table = renameHiveTable(fromTable, toTable); + Path toPath = getTableLocation(toTable); + if (!isExternalTable(table) + && !fromPath.equals(toPath) + && !new SchemaManager(fileIO, fromPath).listAllIds().isEmpty()) { // Rename the file system's table directory. Maintain consistency between tables in // the file system and tables in the Hive Metastore. - Path toPath = getTableLocation(toTable); try { fileIO.rename(fromPath, toPath); } catch (IOException e) { @@ -832,14 +866,9 @@ private void alterTableToHms(Table table, Identifier identifier, TableSchema new throws TException, InterruptedException { updateHmsTablePars(table, newSchema); Path location = getTableLocation(identifier, table); - updateHmsTable(table, identifier, newSchema, newSchema.options().get("provider"), location); - clients.execute( - client -> - client.alter_table( - identifier.getDatabaseName(), - identifier.getTableName(), - table, - true)); + // file format is null, because only data table support alter table. + updateHmsTable(table, identifier, newSchema, null, location); + clients.execute(client -> HiveAlterTableUtils.alterTable(client, identifier, table)); } @Override @@ -847,6 +876,11 @@ public boolean allowUpperCase() { return catalogOptions.getOptional(ALLOW_UPPER_CASE).orElse(false); } + @Override + protected boolean allowCustomTablePath() { + return true; + } + public boolean syncAllProperties() { return catalogOptions.get(SYNC_ALL_PROPERTIES); } @@ -903,10 +937,13 @@ public void repairTable(Identifier identifier) throws TableNotExistException { TableSchema tableSchema = tableSchemaInFileSystem(location, identifier.getBranchNameOrDefault()) .orElseThrow(() -> new TableNotExistException(identifier)); - Table newTable = createHiveTable(identifier, tableSchema, location); + try { + Table newTable = null; try { Table table = getHmsTable(identifier); + newTable = + createHiveTable(identifier, tableSchema, location, isExternalTable(table)); checkArgument( isPaimonTable(table), "Table %s is not a paimon table in hive metastore.", @@ -917,7 +954,16 @@ public void repairTable(Identifier identifier) throws TableNotExistException { } } catch (TableNotExistException e) { // hive table does not exist. - clients.execute(client -> client.createTable(newTable)); + if (newTable == null) { + newTable = + createHiveTable( + identifier, + tableSchema, + location, + usingExternalTable(tableSchema.options())); + } + Table finalNewTable = newTable; + clients.execute(client -> client.createTable(finalNewTable)); } // repair partitions @@ -947,7 +993,7 @@ public String warehouse() { return warehouse; } - private Table getHmsTable(Identifier identifier) throws TableNotExistException { + public Table getHmsTable(Identifier identifier) throws TableNotExistException { try { return clients.run( client -> @@ -968,10 +1014,8 @@ private Table getHmsTable(Identifier identifier) throws TableNotExistException { private boolean isPaimonTable(Identifier identifier, Table table) { return isPaimonTable(table) - && tableSchemaInFileSystem( - getTableLocation(identifier, table), - identifier.getBranchNameOrDefault()) - .isPresent(); + && tableExistsInFileSystem( + getTableLocation(identifier, table), identifier.getBranchNameOrDefault()); } private static boolean isPaimonTable(Table table) { @@ -994,16 +1038,16 @@ public static boolean isView(Table table) { return table != null && TableType.VIRTUAL_VIEW.name().equals(table.getTableType()); } + private boolean isExternalTable(Table table) { + return table != null && TableType.EXTERNAL_TABLE.name().equals(table.getTableType()); + } + private Table newHmsTable( - Identifier identifier, Map tableParameters, String provider) { + Identifier identifier, + Map tableParameters, + @Nullable FormatTable.Format provider, + boolean externalTable) { long currentTimeMillis = System.currentTimeMillis(); - CatalogTableType tableType = - OptionsUtils.convertToEnum( - hiveConf.get(TABLE_TYPE.key(), CatalogTableType.MANAGED.toString()), - CatalogTableType.class); - if (provider == null) { - provider = PAIMON_TABLE_TYPE_VALUE; - } Table table = new Table( identifier.getTableName(), @@ -1018,68 +1062,86 @@ private Table newHmsTable( tableParameters, null, null, - tableType.toString().toUpperCase(Locale.ROOT) + "_TABLE"); - table.getParameters().put(TABLE_TYPE_PROP, provider.toUpperCase()); - if (PAIMON_TABLE_TYPE_VALUE.equalsIgnoreCase(provider)) { + externalTable + ? TableType.EXTERNAL_TABLE.name() + : TableType.MANAGED_TABLE.name()); + + if (provider == null) { + // normal paimon table + table.getParameters().put(TABLE_TYPE_PROP, PAIMON_TABLE_IDENTIFIER); table.getParameters() .put(hive_metastoreConstants.META_TABLE_STORAGE, STORAGE_HANDLER_CLASS_NAME); } else { - table.getParameters().put(FILE_FORMAT.key(), provider.toLowerCase()); + // format table + table.getParameters().put(TABLE_TYPE_PROP, provider.name()); + table.getParameters().put(FILE_FORMAT.key(), provider.name().toLowerCase()); table.getParameters().put(TYPE.key(), FORMAT_TABLE.toString()); } - if (CatalogTableType.EXTERNAL.equals(tableType)) { - table.getParameters().put("EXTERNAL", "TRUE"); + + if (externalTable) { + table.getParameters().put(HIVE_EXTERNAL_TABLE_PROP, "TRUE"); } return table; } - 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 { + private String getSerdeClassName(@Nullable FormatTable.Format provider) { + if (provider == null) { return SERDE_CLASS_NAME; } + switch (provider) { + case CSV: + return "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"; + case PARQUET: + return "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"; + case ORC: + return "org.apache.hadoop.hive.ql.io.orc.OrcSerde"; + } + 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 { + private String getInputFormatName(@Nullable FormatTable.Format provider) { + if (provider == null) { return INPUT_FORMAT_CLASS_NAME; } + switch (provider) { + case CSV: + return "org.apache.hadoop.mapred.TextInputFormat"; + case PARQUET: + return "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"; + case ORC: + return "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat"; + } + 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 { + private String getOutputFormatClassName(@Nullable FormatTable.Format provider) { + if (provider == null) { return OUTPUT_FORMAT_CLASS_NAME; } + switch (provider) { + case CSV: + return "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"; + case PARQUET: + return "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"; + case ORC: + return "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"; + } + return OUTPUT_FORMAT_CLASS_NAME; + } + + private Map setSerDeInfoParam(@Nullable FormatTable.Format provider) { + Map param = new HashMap<>(); + if (provider == FormatTable.Format.CSV) { + param.put(FIELD_DELIM, options.get(FIELD_DELIMITER)); + } + return param; } private void updateHmsTable( Table table, Identifier identifier, TableSchema schema, - String provider, + @Nullable FormatTable.Format provider, Path location) { StorageDescriptor sd = table.getSd() != null ? table.getSd() : new StorageDescriptor(); @@ -1143,14 +1205,6 @@ private void updateHmsTable( 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) { if (syncAllProperties()) { table.getParameters().putAll(schema.options()); @@ -1293,7 +1347,7 @@ public static Catalog createHiveCatalog(CatalogContext context) { return new HiveCatalog( fileIO, hiveConf, - options.get(HiveCatalogFactory.METASTORE_CLIENT_CLASS), + options.get(HiveCatalogOptions.METASTORE_CLIENT_CLASS), options, warehouse.toUri().toString()); } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogFactory.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogFactory.java index 95da0037168c3..eff06831dd4fe 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogFactory.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogFactory.java @@ -21,23 +21,12 @@ import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; -import org.apache.paimon.options.ConfigOption; -import org.apache.paimon.options.ConfigOptions; import static org.apache.paimon.hive.HiveCatalogOptions.IDENTIFIER; /** Factory to create {@link HiveCatalog}. */ public class HiveCatalogFactory implements CatalogFactory { - public static final ConfigOption METASTORE_CLIENT_CLASS = - ConfigOptions.key("metastore.client.class") - .stringType() - .defaultValue("org.apache.hadoop.hive.metastore.HiveMetaStoreClient") - .withDescription( - "Class name of Hive metastore client.\n" - + "NOTE: This class must directly implements " - + "org.apache.hadoop.hive.metastore.IMetaStoreClient."); - @Override public String identifier() { return IDENTIFIER; 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 38f73bc6bd65f..ceab498368205 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 @@ -48,6 +48,15 @@ public final class HiveCatalogOptions { + "If not configured, try to load from 'HADOOP_CONF_DIR' or 'HADOOP_HOME' system environment.\n" + "Configure Priority: 1.from 'hadoop-conf-dir' 2.from HADOOP_CONF_DIR 3.from HADOOP_HOME/conf 4.HADOOP_HOME/etc/hadoop.\n"); + public static final ConfigOption METASTORE_CLIENT_CLASS = + ConfigOptions.key("metastore.client.class") + .stringType() + .defaultValue("org.apache.hadoop.hive.metastore.HiveMetaStoreClient") + .withDescription( + "Class name of Hive metastore client.\n" + + "NOTE: This class must directly implements " + + "org.apache.hadoop.hive.metastore.IMetaStoreClient."); + public static final ConfigOption LOCATION_IN_PROPERTIES = ConfigOptions.key("location-in-properties") .booleanType() 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 cb70e01911456..885fa463e5a7d 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 @@ -31,6 +31,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PartitionEventType; @@ -92,21 +93,15 @@ public void addPartitions(List partitions) throws Exception { @Override public void addPartition(LinkedHashMap partitionSpec) throws Exception { - List partitionValues = new ArrayList<>(partitionSpec.values()); - try { - clients.execute( - client -> - client.getPartition( - identifier.getDatabaseName(), - identifier.getTableName(), - partitionValues)); - // do nothing if the partition already exists - } catch (NoSuchObjectException e) { - // partition not found, create new partition - Partition hivePartition = - toHivePartition(partitionSpec, (int) (System.currentTimeMillis() / 1000)); - clients.execute(client -> client.add_partition(hivePartition)); - } + Partition hivePartition = + toHivePartition(partitionSpec, (int) (System.currentTimeMillis() / 1000)); + clients.execute( + client -> { + try { + client.add_partition(hivePartition); + } catch (AlreadyExistsException ignore) { + } + }); } @Override @@ -124,17 +119,28 @@ public void addPartitionsSpec(List> partitionSpecs public void alterPartition( LinkedHashMap partitionSpec, Map parameters, - long modifyTime) + long modifyTime, + boolean ignoreIfNotExist) 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)); + Partition hivePartition; + try { + hivePartition = + clients.run( + client -> + client.getPartition( + identifier.getDatabaseName(), + identifier.getObjectName(), + partitionValues)); + } catch (NoSuchObjectException e) { + if (ignoreIfNotExist) { + return; + } else { + throw e; + } + } + hivePartition.setValues(partitionValues); hivePartition.setLastAccessTime(currentTime); hivePartition.getParameters().putAll(parameters); 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 index d913f729e3518..ddd21384cbc83 100644 --- 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 @@ -22,7 +22,6 @@ 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; @@ -49,6 +48,8 @@ import java.util.HashMap; import java.util.stream.Collectors; +import static org.apache.paimon.iceberg.AbstractIcebergCommitCallback.catalogDatabasePath; + /** * {@link IcebergMetadataCommitter} to commit Iceberg metadata to Hive metastore, so the table can * be visited by Iceberg's Hive catalog. @@ -98,9 +99,7 @@ public IcebergHiveMetadataCommitter(FileStoreTable table) { this.clients = new CachedClientPool( - hiveConf, - options, - HiveCatalogFactory.METASTORE_CLIENT_CLASS.defaultValue()); + hiveConf, options, options.getString(IcebergOptions.HIVE_CLIENT_CLASS)); } @Override @@ -158,6 +157,7 @@ private boolean databaseExists(String databaseName) throws Exception { private void createDatabase(String databaseName) throws Exception { Database database = new Database(); database.setName(databaseName); + database.setLocationUri(catalogDatabasePath(table).toString()); clients.execute(client -> client.createDatabase(database)); } 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 dcd770c57f516..267bdf0c71002 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,10 +18,12 @@ package org.apache.paimon.hive; +import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.Catalog; import org.apache.paimon.catalog.CatalogTestBase; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.client.ClientPool; +import org.apache.paimon.options.CatalogOptions; import org.apache.paimon.options.Options; import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaChange; @@ -30,6 +32,7 @@ import org.apache.paimon.utils.CommonTestUtils; import org.apache.paimon.utils.HadoopUtils; +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.hadoop.hive.conf.HiveConf; @@ -38,6 +41,7 @@ import org.apache.thrift.TException; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import java.lang.reflect.Field; import java.util.ArrayList; @@ -51,7 +55,7 @@ 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; +import static org.apache.paimon.hive.HiveCatalog.PAIMON_TABLE_IDENTIFIER; import static org.apache.paimon.hive.HiveCatalog.TABLE_TYPE_PROP; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -214,7 +218,7 @@ public void testAddHiveTableParameters() { assertThat(tableProperties).containsEntry("comment", "this is a hive table"); assertThat(tableProperties) .containsEntry( - TABLE_TYPE_PROP, PAIMON_TABLE_TYPE_VALUE.toUpperCase(Locale.ROOT)); + TABLE_TYPE_PROP, PAIMON_TABLE_IDENTIFIER.toUpperCase(Locale.ROOT)); } catch (Exception e) { fail("Test failed due to exception: " + e.getMessage()); } @@ -359,4 +363,36 @@ protected boolean supportsView() { protected boolean supportsFormatTable() { return true; } + + @Test + public void testCreateExternalTableWithLocation(@TempDir java.nio.file.Path tempDir) + throws Exception { + HiveConf hiveConf = new HiveConf(); + String jdoConnectionURL = "jdbc:derby:memory:" + UUID.randomUUID(); + hiveConf.setVar(METASTORECONNECTURLKEY, jdoConnectionURL + ";create=true"); + hiveConf.set(CatalogOptions.TABLE_TYPE.key(), "external"); + String metastoreClientClass = "org.apache.hadoop.hive.metastore.HiveMetaStoreClient"; + HiveCatalog externalWarehouseCatalog = + new HiveCatalog(fileIO, hiveConf, metastoreClientClass, warehouse); + + String externalTablePath = tempDir.toString(); + + Schema schema = + new Schema( + Lists.newArrayList(new DataField(0, "foo", DataTypes.INT())), + Collections.emptyList(), + Collections.emptyList(), + ImmutableMap.of("path", externalTablePath), + ""); + + Identifier identifier = Identifier.create("default", "my_table"); + externalWarehouseCatalog.createTable(identifier, schema, true); + + org.apache.paimon.table.Table table = externalWarehouseCatalog.getTable(identifier); + assertThat(table.options()) + .extracting(CoreOptions.PATH.key()) + .isEqualTo("file:" + externalTablePath); + + externalWarehouseCatalog.close(); + } } diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveTableStatsTest.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveTableStatsTest.java new file mode 100644 index 0000000000000..33016fd083613 --- /dev/null +++ b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveTableStatsTest.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.hive; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; +import org.apache.paimon.options.CatalogOptions; +import org.apache.paimon.options.Options; +import org.apache.paimon.schema.Schema; +import org.apache.paimon.schema.SchemaChange; +import org.apache.paimon.types.DataField; +import org.apache.paimon.types.DataTypes; + +import org.apache.paimon.shade.guava30.com.google.common.collect.Lists; +import org.apache.paimon.shade.guava30.com.google.common.collect.Maps; + +import org.apache.hadoop.hive.common.StatsSetupConst; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Table; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.util.Collections; +import java.util.UUID; + +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTORECONNECTURLKEY; +import static org.assertj.core.api.Assertions.assertThat; + +/** Verify that table stats has been updated. */ +public class HiveTableStatsTest { + @TempDir java.nio.file.Path tempFile; + protected Catalog catalog; + + @BeforeEach + public void setUp() throws Exception { + String warehouse = tempFile.toUri().toString(); + HiveConf hiveConf = new HiveConf(); + String jdoConnectionURL = "jdbc:derby:memory:" + UUID.randomUUID(); + hiveConf.setVar(METASTORECONNECTURLKEY, jdoConnectionURL + ";create=true"); + String metastoreClientClass = "org.apache.hadoop.hive.metastore.HiveMetaStoreClient"; + Options catalogOptions = new Options(); + catalogOptions.set(StatsSetupConst.DO_NOT_UPDATE_STATS, "true"); + catalogOptions.set(CatalogOptions.WAREHOUSE, warehouse); + CatalogContext catalogContext = CatalogContext.create(catalogOptions); + FileIO fileIO = FileIO.get(new Path(warehouse), catalogContext); + catalog = + new HiveCatalog(fileIO, hiveConf, metastoreClientClass, catalogOptions, warehouse); + } + + @Test + public void testAlterTable() throws Exception { + catalog.createDatabase("test_db", false); + // Alter table adds a new column to an existing table,but do not update stats + Identifier identifier = Identifier.create("test_db", "test_table"); + catalog.createTable( + identifier, + new Schema( + Lists.newArrayList(new DataField(0, "col1", DataTypes.STRING())), + Collections.emptyList(), + Collections.emptyList(), + Maps.newHashMap(), + ""), + false); + catalog.alterTable( + identifier, + Lists.newArrayList( + SchemaChange.addColumn("col2", DataTypes.DATE()), + SchemaChange.addColumn("col3", DataTypes.STRING(), "col3 field")), + false); + HiveCatalog hiveCatalog = (HiveCatalog) catalog; + Table table = hiveCatalog.getHmsTable(identifier); + assertThat(table.getParameters().get("COLUMN_STATS_ACCURATE")).isEqualTo(null); + } +} diff --git a/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/HiveTypeUtils.java b/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/HiveTypeUtils.java index f00d675f3750a..33cd45a351a43 100644 --- a/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/HiveTypeUtils.java +++ b/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/HiveTypeUtils.java @@ -185,7 +185,7 @@ public TypeInfo visit(TimestampType timestampType) { @Override public TypeInfo visit(LocalZonedTimestampType localZonedTimestampType) { - return LocalZonedTimestampTypeUtils.toHiveType(localZonedTimestampType); + return LocalZonedTimestampTypeUtils.hiveLocalZonedTimestampType(); } @Override @@ -254,7 +254,7 @@ static DataType visit(TypeInfo type, HiveToPaimonTypeVisitor visitor) { } public DataType atomic(TypeInfo atomic) { - if (LocalZonedTimestampTypeUtils.isLocalZonedTimestampType(atomic)) { + if (LocalZonedTimestampTypeUtils.isHiveLocalZonedTimestampType(atomic)) { return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(); } diff --git a/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/LocalZonedTimestampTypeUtils.java b/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/LocalZonedTimestampTypeUtils.java index fe76debfc333f..b143fcd8caec6 100644 --- a/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/LocalZonedTimestampTypeUtils.java +++ b/paimon-hive/paimon-hive-common/src/main/java/org/apache/paimon/hive/LocalZonedTimestampTypeUtils.java @@ -23,14 +23,27 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -/** To maintain compatibility with Hive 3. */ +import java.lang.reflect.Field; + +/** + * Utils to convert between Hive TimestampLocalTZTypeInfo and Paimon {@link + * LocalZonedTimestampType}, using reflection to solve compatibility between Hive 2 and Hive 3. + */ public class LocalZonedTimestampTypeUtils { - public static boolean isLocalZonedTimestampType(TypeInfo hiveTypeInfo) { - return false; + public static boolean isHiveLocalZonedTimestampType(TypeInfo hiveTypeInfo) { + return "org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo" + .equals(hiveTypeInfo.getClass().getName()); } - public static TypeInfo toHiveType(LocalZonedTimestampType paimonType) { - return TypeInfoFactory.timestampTypeInfo; + public static TypeInfo hiveLocalZonedTimestampType() { + try { + Class typeInfoFactoryClass = + Class.forName("org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory"); + Field field = typeInfoFactoryClass.getField("timestampLocalTZTypeInfo"); + return (TypeInfo) field.get(null); + } catch (Exception e) { + return TypeInfoFactory.timestampTypeInfo; + } } } diff --git a/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java index ebd4684edf1bc..55e6d74084d81 100644 --- a/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java +++ b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java @@ -22,6 +22,7 @@ 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.EnvironmentContext; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Table; @@ -51,4 +52,11 @@ public void alter_table( throws InvalidOperationException, MetaException, TException { throw new TException(); } + + @Override + public void alter_table_with_environmentContext( + String defaultDatabaseName, String tblName, Table table, EnvironmentContext env) + throws InvalidOperationException, MetaException, TException { + throw new TException(); + } } 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 index a9e4ba945440e..7d726e75a17d2 100644 --- 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 @@ -18,5 +18,12 @@ package org.apache.paimon.iceberg; +import org.apache.paimon.hive.CreateFailHiveMetaStoreClient; + /** IT cases for {@link IcebergHiveMetadataCommitter} in Hive 2.3. */ -public class IcebergHive23MetadataCommitterITCase extends IcebergHiveMetadataCommitterITCaseBase {} +public class IcebergHive23MetadataCommitterITCase extends IcebergHiveMetadataCommitterITCaseBase { + @Override + protected String createFailHiveMetaStoreClient() { + return CreateFailHiveMetaStoreClient.class.getName(); + } +} diff --git a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java index ae6a1bb85ac45..eab18feadacaf 100644 --- a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java +++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java @@ -22,6 +22,7 @@ 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.EnvironmentContext; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.MetaException; import org.apache.hadoop.hive.metastore.api.Table; @@ -51,4 +52,11 @@ public void alter_table( throws InvalidOperationException, MetaException, TException { throw new TException(); } + + @Override + public void alter_table_with_environmentContext( + String defaultDatabaseName, String tblName, Table table, EnvironmentContext env) + throws InvalidOperationException, MetaException, TException { + throw new TException(); + } } 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 index 6f4b0afd1ae12..0634adfad3576 100644 --- 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 @@ -18,5 +18,12 @@ package org.apache.paimon.iceberg; +import org.apache.paimon.hive.CreateFailHiveMetaStoreClient; + /** IT cases for {@link IcebergHiveMetadataCommitter} in Hive 3.1. */ -public class IcebergHive31MetadataCommitterITCase extends IcebergHiveMetadataCommitterITCaseBase {} +public class IcebergHive31MetadataCommitterITCase extends IcebergHiveMetadataCommitterITCaseBase { + @Override + protected String createFailHiveMetaStoreClient() { + return CreateFailHiveMetaStoreClient.class.getName(); + } +} diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java index f637651413ed6..108315a96103b 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java @@ -233,9 +233,10 @@ private static void checkFieldsMatched( } } - if (schemaFieldNames.size() != hiveFieldNames.size()) { + // It is OK that hive is a subset of paimon + if (schemaFieldNames.size() < hiveFieldNames.size()) { throw new IllegalArgumentException( - "Hive DDL and paimon schema mismatched! " + "Hive DDL is a superset of paimon schema! " + "It is recommended not to write any column definition " + "as Paimon external table can read schema from the specified location.\n" + "There are " diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/utils/HiveSplitGenerator.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/utils/HiveSplitGenerator.java index 33cbc19e03262..144afab8e1fa8 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/utils/HiveSplitGenerator.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/utils/HiveSplitGenerator.java @@ -96,7 +96,8 @@ public static InputSplit[] generateSplits(FileStoreTable table, JobConf jobConf) scan.withFilter(PredicateBuilder.and(predicatePerPartition)); } } - scan.plan() + scan.dropStats() + .plan() .splits() .forEach( split -> 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 74d2d7e1c3437..2266a8484d9d2 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 @@ -31,12 +31,12 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.utils.IOUtils; +import org.apache.paimon.utils.TimeUtils; import com.klarna.hiverunner.HiveShell; import com.klarna.hiverunner.annotations.HiveSQL; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.Path; -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.TableResult; @@ -139,7 +139,9 @@ private void registerHiveCatalog(String catalogName, Map catalog EnvironmentSettings.newInstance().inStreamingMode().build()); sEnv.getConfig() .getConfiguration() - .set(ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL, Duration.ofSeconds(1)); + .setString( + "execution.checkpointing.interval", + TimeUtils.formatWithHighestUnit(Duration.ofSeconds(1))); sEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1); tEnv.executeSql( diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveTableSchemaTest.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveTableSchemaTest.java index 07cd00c8e67ea..fe7aeac0833ae 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveTableSchemaTest.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveTableSchemaTest.java @@ -153,6 +153,54 @@ public void testMismatchedColumnNameAndType() throws Exception { .hasMessageContaining(expected); } + @Test + public void testSubsetColumnNameAndType() throws Exception { + createSchema(); + Properties properties = new Properties(); + List columns = Arrays.asList("a", "b"); + properties.setProperty("columns", String.join(",", columns)); + properties.setProperty( + "columns.types", + String.join( + ":", + Arrays.asList( + TypeInfoFactory.intTypeInfo.getTypeName(), + TypeInfoFactory.stringTypeInfo.getTypeName(), + TypeInfoFactory.getDecimalTypeInfo(6, 3).getTypeName()))); + properties.setProperty("columns.comments", "\0\0"); + properties.setProperty("location", tempDir.toString()); + List fields = HiveSchema.extract(null, properties).fieldNames(); + assertThat(fields).isEqualTo(columns); + } + + @Test + public void testSupersetColumnNameAndType() throws Exception { + createSchema(); + Properties properties = new Properties(); + properties.setProperty("columns", "a,b,c,d"); + properties.setProperty( + "columns.types", + String.join( + ":", + Arrays.asList( + TypeInfoFactory.intTypeInfo.getTypeName(), + TypeInfoFactory.stringTypeInfo.getTypeName(), + TypeInfoFactory.decimalTypeInfo.getTypeName(), + TypeInfoFactory.stringTypeInfo.getTypeName(), + TypeInfoFactory.getDecimalTypeInfo(6, 3).getTypeName()))); + properties.setProperty("columns.comments", "\0\0"); + properties.setProperty("location", tempDir.toString()); + String expected = + "Hive DDL is a superset of paimon schema! " + + "It is recommended not to write any column definition " + + "as Paimon external table can read schema from the specified location.\n" + + "There are 4 fields in Hive DDL: a, b, c, d\n" + + "There are 3 fields in Paimon schema: a, b, c\n"; + assertThatThrownBy(() -> HiveSchema.extract(null, properties)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining(expected); + } + @Test public void testTooFewColumns() throws Exception { createSchema(); @@ -162,16 +210,7 @@ public void testTooFewColumns() throws Exception { properties.setProperty("columns.types", TypeInfoFactory.intTypeInfo.getTypeName()); properties.setProperty("location", tempDir.toString()); properties.setProperty("columns.comments", ""); - - String expected = - "Hive DDL and paimon schema mismatched! " - + "It is recommended not to write any column definition " - + "as Paimon external table can read schema from the specified location.\n" - + "There are 1 fields in Hive DDL: a\n" - + "There are 3 fields in Paimon schema: a, b, c"; - assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> HiveSchema.extract(null, properties)) - .withMessageContaining(expected); + assertThat(HiveSchema.extract(null, properties)).isInstanceOf(HiveSchema.class); } @Test @@ -194,7 +233,7 @@ public void testTooManyColumns() throws Exception { properties.setProperty("location", tempDir.toString()); String expected = - "Hive DDL and paimon schema mismatched! " + "Hive DDL is a superset of paimon schema! " + "It is recommended not to write any column definition " + "as Paimon external table can read schema from the specified location.\n" + "There are 5 fields in Hive DDL: a, b, c, d, e\n" diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveWriteITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveWriteITCase.java index 57486ec30be9f..c99eb9cd1f461 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveWriteITCase.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveWriteITCase.java @@ -157,7 +157,7 @@ private String writeData(Table table, String path, List data) throw write.close(); commit.close(); - String tableName = "test_table_" + (UUID.randomUUID().toString().substring(0, 4)); + String tableName = "test_table_" + UUID.randomUUID().toString().replace('-', '_'); hiveShell.execute( String.join( "\n", 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 index fab22775751b0..d0c64c5d3b7fc 100644 --- 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 @@ -104,6 +104,12 @@ public void testPrimaryKeyTable() throws Exception { Row.of(2, 1, "cat"), Row.of(2, 2, "elephant")), collect(tEnv.executeSql("SELECT * FROM my_iceberg.test_db.t ORDER BY pt, id"))); + + Assert.assertTrue( + hiveShell + .executeQuery("DESC DATABASE EXTENDED test_db") + .toString() + .contains("iceberg/test_db")); } @Test @@ -150,6 +156,36 @@ public void testAppendOnlyTable() throws Exception { "SELECT data, id, pt FROM my_iceberg.test_db.t WHERE id > 1 ORDER BY pt, id"))); } + @Test + public void testCustomMetastoreClass() { + 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( + String.format( + "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', " + + "'metadata.iceberg.hive-client-class' = '%s')", + createFailHiveMetaStoreClient())); + Assert.assertThrows( + Exception.class, + () -> + tEnv.executeSql( + "INSERT INTO my_paimon.test_db.t VALUES " + + "(1, 1, 'apple'), (1, 2, 'pear'), (2, 1, 'cat'), (2, 2, 'dog')") + .await()); + } + + protected abstract String createFailHiveMetaStoreClient(); + private List collect(TableResult result) throws Exception { List rows = new ArrayList<>(); try (CloseableIterator it = result.collect()) { diff --git a/paimon-open-api/Makefile b/paimon-open-api/Makefile new file mode 100644 index 0000000000000..c3264c83dbd02 --- /dev/null +++ b/paimon-open-api/Makefile @@ -0,0 +1,25 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# See: https://cwiki.apache.org/confluence/display/INFRA/git+-+.asf.yaml+features + + +install: + brew install yq + +generate: + @sh generate.sh diff --git a/paimon-open-api/README.md b/paimon-open-api/README.md new file mode 100644 index 0000000000000..9d14a7cdd3643 --- /dev/null +++ b/paimon-open-api/README.md @@ -0,0 +1,10 @@ +# Open API spec + +The `rest-catalog-open-api.yaml` defines the REST catalog interface. + +## Generate Open API Spec +```sh +make install +cd paimon-open-api +make generate +``` \ No newline at end of file diff --git a/paimon-open-api/generate.sh b/paimon-open-api/generate.sh new file mode 100755 index 0000000000000..619b642ab760b --- /dev/null +++ b/paimon-open-api/generate.sh @@ -0,0 +1,49 @@ +#!/bin/bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# Start the application +cd .. +mvn spotless:apply +mvn clean install -DskipTests +cd ./paimon-open-api +mvn spring-boot:run & +SPRING_PID=$! +# Wait for the application to be ready +RETRY_COUNT=0 +MAX_RETRIES=10 +SLEEP_DURATION=5 + +until $(curl -s -o /dev/null -w "%{http_code}" http://localhost:8080/swagger-api-docs | grep -q "200"); do + ((RETRY_COUNT++)) + if [ $RETRY_COUNT -gt $MAX_RETRIES ]; then + echo "Failed to start the application after $MAX_RETRIES retries." + exit 1 + fi + echo "Application not ready yet. Retrying in $SLEEP_DURATION seconds..." + sleep $SLEEP_DURATION +done + +echo "Application is ready". + +# Generate the OpenAPI specification file +curl -s "http://localhost:8080/swagger-api-docs" | jq -M > ./rest-catalog-open-api.json +yq --prettyPrint -o=yaml ./rest-catalog-open-api.json > ./rest-catalog-open-api.yaml +rm -rf ./rest-catalog-open-api.json +mvn spotless:apply +# Stop the application +echo "Stopping application..." +kill $SPRING_PID \ No newline at end of file diff --git a/paimon-open-api/pom.xml b/paimon-open-api/pom.xml new file mode 100644 index 0000000000000..9422852432705 --- /dev/null +++ b/paimon-open-api/pom.xml @@ -0,0 +1,86 @@ + + + + 4.0.0 + + org.apache.paimon + paimon-parent + 1.0-SNAPSHOT + + + paimon-open-api + Paimon : Open API + + + 8 + 8 + UTF-8 + + + + org.springframework.boot + spring-boot-starter-web + 2.7.18 + + + ch.qos.logback + logback-classic + + + + + + + org.springdoc + springdoc-openapi-ui + 1.7.0 + + + org.apache.paimon + paimon-core + ${project.version} + + + io.swagger.core.v3 + swagger-annotations + 2.2.20 + + + + + + org.springframework.boot + spring-boot-maven-plugin + 2.7.6 + + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + + + + + \ No newline at end of file diff --git a/paimon-open-api/rest-catalog-open-api.yaml b/paimon-open-api/rest-catalog-open-api.yaml new file mode 100644 index 0000000000000..9b69b3de27765 --- /dev/null +++ b/paimon-open-api/rest-catalog-open-api.yaml @@ -0,0 +1,220 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +--- +openapi: 3.0.1 +info: + title: RESTCatalog API + description: This API exposes endpoints to RESTCatalog. + license: + name: Apache 2.0 + url: https://www.apache.org/licenses/LICENSE-2.0.html + version: "1.0" +servers: + - url: http://localhost:8080 + description: Server URL in Development environment +paths: + /v1/{prefix}/databases: + get: + tags: + - database + summary: List Databases + operationId: listDatabases + parameters: + - name: prefix + in: path + required: true + schema: + type: string + responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/ListDatabasesResponse' + "500": + description: Internal Server Error + post: + tags: + - database + summary: Create Databases + operationId: createDatabases + parameters: + - name: prefix + in: path + required: true + schema: + type: string + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/CreateDatabaseRequest' + responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/CreateDatabaseResponse' + "409": + description: Resource has exist + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error + /v1/{prefix}/databases/{database}: + get: + tags: + - database + summary: Get Database + operationId: getDatabases + parameters: + - name: prefix + in: path + required: true + schema: + type: string + - name: database + in: path + required: true + schema: + type: string + responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/GetDatabaseResponse' + "404": + description: Resource not found + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error + delete: + tags: + - database + summary: Drop Database + operationId: dropDatabases + parameters: + - name: prefix + in: path + required: true + schema: + type: string + - name: database + in: path + required: true + schema: + type: string + responses: + "404": + description: Resource not found + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorResponse' + "500": + description: Internal Server Error + /v1/config: + get: + tags: + - config + summary: Get Config + operationId: getConfig + responses: + "200": + description: OK + content: + application/json: + schema: + $ref: '#/components/schemas/ConfigResponse' + "500": + description: Internal Server Error +components: + schemas: + CreateDatabaseRequest: + type: object + properties: + name: + type: string + ignoreIfExists: + type: boolean + options: + type: object + additionalProperties: + type: string + CreateDatabaseResponse: + type: object + properties: + name: + type: string + options: + type: object + additionalProperties: + type: string + ErrorResponse: + type: object + properties: + message: + type: string + code: + type: integer + format: int32 + stack: + type: array + items: + type: string + DatabaseName: + type: object + properties: + name: + type: string + ListDatabasesResponse: + type: object + properties: + databases: + type: array + items: + $ref: '#/components/schemas/DatabaseName' + GetDatabaseResponse: + type: object + properties: + name: + type: string + options: + type: object + additionalProperties: + type: string + ConfigResponse: + type: object + properties: + defaults: + type: object + additionalProperties: + type: string + overrides: + type: object + additionalProperties: + type: string diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.java new file mode 100644 index 0000000000000..76ce4cbf83c61 --- /dev/null +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/OpenApiApplication.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.open.api; + +import org.springframework.boot.SpringApplication; +import org.springframework.boot.autoconfigure.SpringBootApplication; + +/** OpenAPI application. */ +@SpringBootApplication +public class OpenApiApplication { + + public static void main(String[] args) { + SpringApplication.run(OpenApiApplication.class, args); + } +} diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java new file mode 100644 index 0000000000000..19f6f8cdf6734 --- /dev/null +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/RESTCatalogController.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.open.api; + +import org.apache.paimon.rest.ResourcePaths; +import org.apache.paimon.rest.requests.CreateDatabaseRequest; +import org.apache.paimon.rest.responses.ConfigResponse; +import org.apache.paimon.rest.responses.CreateDatabaseResponse; +import org.apache.paimon.rest.responses.DatabaseName; +import org.apache.paimon.rest.responses.ErrorResponse; +import org.apache.paimon.rest.responses.GetDatabaseResponse; +import org.apache.paimon.rest.responses.ListDatabasesResponse; + +import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableList; + +import io.swagger.v3.oas.annotations.Operation; +import io.swagger.v3.oas.annotations.media.Content; +import io.swagger.v3.oas.annotations.media.Schema; +import io.swagger.v3.oas.annotations.parameters.RequestBody; +import io.swagger.v3.oas.annotations.responses.ApiResponse; +import io.swagger.v3.oas.annotations.responses.ApiResponses; +import org.springframework.web.bind.annotation.CrossOrigin; +import org.springframework.web.bind.annotation.DeleteMapping; +import org.springframework.web.bind.annotation.GetMapping; +import org.springframework.web.bind.annotation.PathVariable; +import org.springframework.web.bind.annotation.PostMapping; +import org.springframework.web.bind.annotation.RestController; + +import java.util.HashMap; +import java.util.Map; + +/** * RESTCatalog management APIs. */ +@CrossOrigin(origins = "http://localhost:8081") +@RestController +public class RESTCatalogController { + + @Operation( + summary = "Get Config", + tags = {"config"}) + @ApiResponses({ + @ApiResponse( + responseCode = "200", + content = {@Content(schema = @Schema(implementation = ConfigResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @GetMapping(ResourcePaths.V1_CONFIG) + public ConfigResponse getConfig() { + Map defaults = new HashMap<>(); + Map overrides = new HashMap<>(); + return new ConfigResponse(defaults, overrides); + } + + @Operation( + summary = "List Databases", + tags = {"database"}) + @ApiResponses({ + @ApiResponse( + responseCode = "200", + content = { + @Content(schema = @Schema(implementation = ListDatabasesResponse.class)) + }), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @GetMapping("/v1/{prefix}/databases") + public ListDatabasesResponse listDatabases(@PathVariable String prefix) { + return new ListDatabasesResponse(ImmutableList.of(new DatabaseName("account"))); + } + + @Operation( + summary = "Create Databases", + tags = {"database"}) + @ApiResponses({ + @ApiResponse( + responseCode = "200", + content = { + @Content(schema = @Schema(implementation = CreateDatabaseResponse.class)) + }), + @ApiResponse( + responseCode = "409", + description = "Resource has exist", + content = { + @Content( + schema = @Schema(implementation = ErrorResponse.class), + mediaType = "application/json") + }), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @PostMapping("/v1/{prefix}/databases") + public CreateDatabaseResponse createDatabases( + @PathVariable String prefix, @RequestBody CreateDatabaseRequest request) { + Map properties = new HashMap<>(); + return new CreateDatabaseResponse("name", properties); + } + + @Operation( + summary = "Get Database", + tags = {"database"}) + @ApiResponses({ + @ApiResponse( + responseCode = "200", + content = {@Content(schema = @Schema(implementation = GetDatabaseResponse.class))}), + @ApiResponse( + responseCode = "404", + description = "Resource not found", + content = {@Content(schema = @Schema(implementation = ErrorResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @GetMapping("/v1/{prefix}/databases/{database}") + public GetDatabaseResponse getDatabases( + @PathVariable String prefix, @PathVariable String database) { + Map options = new HashMap<>(); + return new GetDatabaseResponse("name", options); + } + + @Operation( + summary = "Drop Database", + tags = {"database"}) + @ApiResponses({ + @ApiResponse( + responseCode = "404", + description = "Resource not found", + content = {@Content(schema = @Schema(implementation = ErrorResponse.class))}), + @ApiResponse( + responseCode = "500", + content = {@Content(schema = @Schema())}) + }) + @DeleteMapping("/v1/{prefix}/databases/{database}") + public void dropDatabases(@PathVariable String prefix, @PathVariable String database) {} +} diff --git a/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.java new file mode 100644 index 0000000000000..71ac066d4a70c --- /dev/null +++ b/paimon-open-api/src/main/java/org/apache/paimon/open/api/config/OpenAPIConfig.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.open.api.config; + +import io.swagger.v3.oas.models.OpenAPI; +import io.swagger.v3.oas.models.info.Info; +import io.swagger.v3.oas.models.info.License; +import io.swagger.v3.oas.models.responses.ApiResponses; +import io.swagger.v3.oas.models.servers.Server; +import org.springdoc.core.customizers.OpenApiCustomiser; +import org.springframework.beans.factory.annotation.Value; +import org.springframework.context.annotation.Bean; +import org.springframework.context.annotation.Configuration; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; + +/** Config for OpenAPI. */ +@Configuration +public class OpenAPIConfig { + + @Value("${openapi.url}") + private String devUrl; + + @Bean + public OpenAPI restCatalogOpenAPI() { + Server server = new Server(); + server.setUrl(devUrl); + server.setDescription("Server URL in Development environment"); + + License mitLicense = + new License() + .name("Apache 2.0") + .url("https://www.apache.org/licenses/LICENSE-2.0.html"); + + Info info = + new Info() + .title("RESTCatalog API") + .version("1.0") + .description("This API exposes endpoints to RESTCatalog.") + .license(mitLicense); + List servers = new ArrayList<>(); + servers.add(server); + return new OpenAPI().info(info).servers(servers); + } + + /** Sort response alphabetically. So the api generate will in same order everytime. */ + @Bean + public OpenApiCustomiser sortResponseAlphabetically() { + return openApi -> { + openApi.getPaths() + .values() + .forEach( + path -> + path.readOperations() + .forEach( + operation -> { + ApiResponses responses = + operation.getResponses(); + if (responses != null) { + ApiResponses sortedResponses = + new ApiResponses(); + List keys = + new ArrayList<>( + responses.keySet()); + keys.sort(Comparator.naturalOrder()); + + for (String key : keys) { + sortedResponses.addApiResponse( + key, responses.get(key)); + } + + operation.setResponses(sortedResponses); + } + })); + }; + } +} diff --git a/paimon-open-api/src/main/resources/application.properties b/paimon-open-api/src/main/resources/application.properties new file mode 100644 index 0000000000000..1e7a987c9d403 --- /dev/null +++ b/paimon-open-api/src/main/resources/application.properties @@ -0,0 +1,26 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +springdoc.swagger-ui.path=/swagger-api +springdoc.api-docs.path=/swagger-api-docs +springdoc.swagger-ui.deepLinking=true +springdoc.swagger-ui.tryItOutEnabled=true +springdoc.swagger-ui.filter=true +springdoc.swagger-ui.tagsSorter=alpha +springdoc.swagger-ui.operations-sorter=alpha +# define response default media type +springdoc.default-produces-media-type=application/json +openapi.url=http://localhost:8080 diff --git a/paimon-spark/paimon-spark-3.2/pom.xml b/paimon-spark/paimon-spark-3.2/pom.xml index 626bb5bae8330..957319b47dab8 100644 --- a/paimon-spark/paimon-spark-3.2/pom.xml +++ b/paimon-spark/paimon-spark-3.2/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.paimon + paimon-spark3-common + ${project.version} + + org.apache.paimon paimon-spark-common_${scala.binary.version} @@ -63,7 +69,7 @@ under the License. org.apache.paimon - paimon-spark-common_${scala.binary.version} + paimon-spark-ut ${project.version} tests test @@ -126,7 +132,7 @@ under the License. - org.apache.paimon:paimon-spark-common_${scala.binary.version} + org.apache.paimon:paimon-spark3-common diff --git a/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala new file mode 100644 index 0000000000000..e759edd0c2c69 --- /dev/null +++ b/paimon-spark/paimon-spark-3.2/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.parser.extensions + +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.spark.catalyst.plans.logical.{CreatePaimonView, DropPaimonView, ResolvedIdentifier, ShowPaimonViews} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{CTESubstitution, ResolvedNamespace, UnresolvedView} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{CatalogManager, LookupCatalog} + +case class RewritePaimonViewCommands(spark: SparkSession) + extends Rule[LogicalPlan] + with LookupCatalog { + + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + + case CreateViewStatement( + ResolvedIdent(resolved), + userSpecifiedColumns, + comment, + properties, + Some(originalText), + child, + allowExisting, + replace, + _) => + CreatePaimonView( + child = resolved, + queryText = originalText, + query = CTESubstitution.apply(child), + columnAliases = userSpecifiedColumns.map(_._1), + columnComments = userSpecifiedColumns.map(_._2.orElse(Option.empty)), + comment = comment, + properties = properties, + allowExisting = allowExisting, + replace = replace + ) + + case DropView(ResolvedIdent(resolved), ifExists: Boolean) => + DropPaimonView(resolved, ifExists) + + case ShowViews(_, pattern, output) if catalogManager.currentCatalog.isInstanceOf[SupportView] => + ShowPaimonViews( + ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + pattern, + output) + } + + private object ResolvedIdent { + def unapply(unresolved: Any): Option[ResolvedIdentifier] = unresolved match { + case CatalogAndIdentifier(viewCatalog: SupportView, ident) => + Some(ResolvedIdentifier(viewCatalog, ident)) + case UnresolvedView(CatalogAndIdentifier(viewCatalog: SupportView, ident), _, _, _) => + Some(ResolvedIdentifier(viewCatalog, ident)) + case _ => + None + } + } +} diff --git a/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 0000000000000..6ab8a2671b518 --- /dev/null +++ b/paimon-spark/paimon-spark-3.2/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.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 PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-3.3/pom.xml b/paimon-spark/paimon-spark-3.3/pom.xml index 689e4131ccd9e..0a390d9267894 100644 --- a/paimon-spark/paimon-spark-3.3/pom.xml +++ b/paimon-spark/paimon-spark-3.3/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.paimon + paimon-spark3-common + ${project.version} + + org.apache.paimon paimon-spark-common_${scala.binary.version} @@ -63,7 +69,7 @@ under the License. org.apache.paimon - paimon-spark-common_${scala.binary.version} + paimon-spark-ut ${project.version} tests test @@ -126,7 +132,7 @@ under the License. - org.apache.paimon:paimon-spark-common_${scala.binary.version} + org.apache.paimon:paimon-spark3-common diff --git a/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala b/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala new file mode 100644 index 0000000000000..5d57cda2f34b2 --- /dev/null +++ b/paimon-spark/paimon-spark-3.3/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.catalyst.parser.extensions + +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.spark.catalyst.plans.logical.{CreatePaimonView, DropPaimonView, ResolvedIdentifier, ShowPaimonViews} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{CTESubstitution, ResolvedNamespace, UnresolvedDBObjectName, UnresolvedView} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{CatalogManager, LookupCatalog} + +case class RewritePaimonViewCommands(spark: SparkSession) + extends Rule[LogicalPlan] + with LookupCatalog { + + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + + case CreateView( + ResolvedIdent(resolved), + userSpecifiedColumns, + comment, + properties, + Some(queryText), + query, + allowExisting, + replace) => + CreatePaimonView( + child = resolved, + queryText = queryText, + query = CTESubstitution.apply(query), + columnAliases = userSpecifiedColumns.map(_._1), + columnComments = userSpecifiedColumns.map(_._2.orElse(Option.empty)), + comment = comment, + properties = properties, + allowExisting = allowExisting, + replace = replace + ) + + case DropView(ResolvedIdent(resolved), ifExists: Boolean) => + DropPaimonView(resolved, ifExists) + + case ShowViews(_, pattern, output) if catalogManager.currentCatalog.isInstanceOf[SupportView] => + ShowPaimonViews( + ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + pattern, + output) + } + + private object ResolvedIdent { + def unapply(unresolved: Any): Option[ResolvedIdentifier] = unresolved match { + case UnresolvedDBObjectName(CatalogAndIdentifier(viewCatalog: SupportView, ident), _) => + Some(ResolvedIdentifier(viewCatalog, ident)) + case UnresolvedView(CatalogAndIdentifier(viewCatalog: SupportView, ident), _, _, _) => + Some(ResolvedIdentifier(viewCatalog, ident)) + case _ => + None + } + } +} diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/DataFrameWriteTest.scala b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/DataFrameWriteTest.scala index a3cecfc72e1d2..cb449edb4ccbe 100644 --- a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/DataFrameWriteTest.scala +++ b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/DataFrameWriteTest.scala @@ -18,10 +18,15 @@ package org.apache.paimon.spark +import org.apache.spark.SparkConf import org.junit.jupiter.api.Assertions class DataFrameWriteTest extends PaimonSparkTestBase { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.sql.catalog.paimon.cache-enabled", "false") + } + test("Paimon: DataFrameWrite.saveAsTable") { import testImplicits._ diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTest.scala b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTest.scala index 304b814b33d38..219d57c865c8c 100644 --- a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTest.scala +++ b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTest.scala @@ -248,7 +248,7 @@ class InsertOverwriteTest extends PaimonSparkTestBase { spark.sql("SELECT * FROM T ORDER BY a, b"), Row(1, 3, "3") :: Row(2, 4, "4") :: Nil) - withSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { + withSparkSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { // dynamic overwrite the a=1 partition spark.sql("INSERT OVERWRITE T VALUES (1, 5, '5'), (1, 7, '7')") checkAnswer( @@ -289,7 +289,7 @@ class InsertOverwriteTest extends PaimonSparkTestBase { "ptv2", 22) :: Nil) - withSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { + withSparkSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { // dynamic overwrite the pt2=22 partition spark.sql( "INSERT OVERWRITE T PARTITION (pt2 = 22) VALUES (3, 'c2', 'ptv1'), (4, 'd2', 'ptv3')") diff --git a/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 0000000000000..6ab8a2671b518 --- /dev/null +++ b/paimon-spark/paimon-spark-3.3/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.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 PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-3.4/pom.xml b/paimon-spark/paimon-spark-3.4/pom.xml index d1ded508a927c..0f4cb30e4f7f6 100644 --- a/paimon-spark/paimon-spark-3.4/pom.xml +++ b/paimon-spark/paimon-spark-3.4/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.paimon + paimon-spark3-common + ${project.version} + + org.apache.paimon paimon-spark-common_${scala.binary.version} @@ -63,7 +69,7 @@ under the License. org.apache.paimon - paimon-spark-common_${scala.binary.version} + paimon-spark-ut ${project.version} tests test @@ -126,7 +132,7 @@ under the License. - org.apache.paimon:paimon-spark-common_${scala.binary.version} + org.apache.paimon:paimon-spark3-common diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala index 18fb9e116ba43..ab4a9bcd9dbff 100644 --- a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala @@ -18,6 +18,7 @@ package org.apache.paimon.spark +import org.apache.spark.SparkConf import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions.{col, mean, window} @@ -27,6 +28,10 @@ import java.sql.Date class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.sql.catalog.paimon.cache-enabled", "false") + } + import testImplicits._ test("Paimon Sink: forEachBatch") { diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 0000000000000..6ab8a2671b518 --- /dev/null +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.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 PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-3.5/pom.xml b/paimon-spark/paimon-spark-3.5/pom.xml index 92803cda540e4..1b9c968889083 100644 --- a/paimon-spark/paimon-spark-3.5/pom.xml +++ b/paimon-spark/paimon-spark-3.5/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.paimon + paimon-spark3-common + ${project.version} + + org.apache.paimon paimon-spark-common_${scala.binary.version} @@ -63,7 +69,7 @@ under the License. org.apache.paimon - paimon-spark-common_${scala.binary.version} + paimon-spark-ut ${project.version} tests test @@ -126,7 +132,7 @@ under the License. - org.apache.paimon:paimon-spark-common_${scala.binary.version} + org.apache.paimon:paimon-spark3-common diff --git a/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 0000000000000..6ab8a2671b518 --- /dev/null +++ b/paimon-spark/paimon-spark-3.5/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.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 PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-4.0/pom.xml b/paimon-spark/paimon-spark-4.0/pom.xml index 9f819f820ce2c..8e7d166dc55b5 100644 --- a/paimon-spark/paimon-spark-4.0/pom.xml +++ b/paimon-spark/paimon-spark-4.0/pom.xml @@ -36,6 +36,12 @@ under the License. + + org.apache.paimon + paimon-spark4-common + ${project.version} + + org.apache.paimon paimon-spark-common_${scala.binary.version} @@ -63,7 +69,7 @@ under the License. org.apache.paimon - paimon-spark-common_${scala.binary.version} + paimon-spark-ut ${project.version} tests test @@ -126,7 +132,7 @@ under the License. - org.apache.paimon:paimon-spark-common_${scala.binary.version} + org.apache.paimon:paimon-spark4-common diff --git a/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 0000000000000..6ab8a2671b518 --- /dev/null +++ b/paimon-spark/paimon-spark-4.0/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.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 PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-common/pom.xml b/paimon-spark/paimon-spark-common/pom.xml index 1cfc53f42d489..052c4c4265fc4 100644 --- a/paimon-spark/paimon-spark-common/pom.xml +++ b/paimon-spark/paimon-spark-common/pom.xml @@ -38,18 +38,6 @@ under the License. - - org.apache.paimon - ${paimon-sparkx-common} - ${project.version} - - - * - * - - - - org.apache.spark spark-sql_${scala.binary.version} @@ -72,46 +60,6 @@ under the License. org.apache.paimon paimon-bundle - - - - - 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.spark - spark-avro_${scala.binary.version} - ${spark.version} - test - @@ -130,7 +78,6 @@ under the License. org.apache.paimon:paimon-bundle - org.apache.paimon:${paimon-sparkx-common} @@ -155,21 +102,6 @@ under the License. src/main/antlr4 - - - - org.apache.maven.plugins - maven-jar-plugin - - - prepare-test-jar - test-compile - - test-jar - - - - 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/AbstractSparkInternalRow.java similarity index 67% rename from paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkInternalRow.java rename to paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/AbstractSparkInternalRow.java index 147c6c2d77c84..28604a6d62933 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/AbstractSparkInternalRow.java @@ -18,24 +18,15 @@ package org.apache.paimon.spark; -import org.apache.paimon.data.BinaryString; -import org.apache.paimon.data.InternalArray; -import org.apache.paimon.data.InternalMap; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.data.Timestamp; -import org.apache.paimon.spark.util.shim.TypeUtils; +import org.apache.paimon.spark.data.SparkInternalRow; import org.apache.paimon.types.ArrayType; import org.apache.paimon.types.BigIntType; import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypeChecks; -import org.apache.paimon.types.IntType; -import org.apache.paimon.types.MapType; -import org.apache.paimon.types.MultisetType; import org.apache.paimon.types.RowType; -import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; import org.apache.spark.sql.catalyst.util.ArrayData; -import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.BooleanType; @@ -61,19 +52,23 @@ import java.util.Objects; +import static org.apache.paimon.spark.DataConverter.fromPaimon; 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.paimon.shims.InternalRow { +/** + * An abstract {@link SparkInternalRow} that overwrite all the common methods in spark3 and spark4. + */ +public abstract class AbstractSparkInternalRow extends SparkInternalRow { - private final RowType rowType; + protected RowType rowType; - private InternalRow row; + protected InternalRow row; - public SparkInternalRow(RowType rowType) { + public AbstractSparkInternalRow(RowType rowType) { this.rowType = rowType; } + @Override public SparkInternalRow replace(InternalRow row) { this.row = row; return this; @@ -96,7 +91,7 @@ public void update(int i, Object value) { @Override public org.apache.spark.sql.catalyst.InternalRow copy() { - return new SparkInternalRow(rowType).replace(copyInternalRow(row, rowType)); + return SparkInternalRow.create(rowType).replace(copyInternalRow(row, rowType)); } @Override @@ -255,7 +250,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - SparkInternalRow that = (SparkInternalRow) o; + AbstractSparkInternalRow that = (AbstractSparkInternalRow) o; return Objects.equals(rowType, that.rowType) && Objects.equals(row, that.row); } @@ -263,78 +258,4 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(rowType, row); } - - // ================== static methods ========================================= - - public static Object fromPaimon(Object o, DataType type) { - if (o == null) { - return null; - } - switch (type.getTypeRoot()) { - case TIMESTAMP_WITHOUT_TIME_ZONE: - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - return fromPaimon((Timestamp) o); - case CHAR: - case VARCHAR: - return fromPaimon((BinaryString) o); - case DECIMAL: - return fromPaimon((org.apache.paimon.data.Decimal) o); - case ARRAY: - return fromPaimon((InternalArray) o, (ArrayType) type); - case MAP: - case MULTISET: - return fromPaimon((InternalMap) o, type); - case ROW: - return fromPaimon((InternalRow) o, (RowType) type); - default: - return o; - } - } - - public static UTF8String fromPaimon(BinaryString string) { - return UTF8String.fromBytes(string.toBytes()); - } - - public static Decimal fromPaimon(org.apache.paimon.data.Decimal decimal) { - return Decimal.apply(decimal.toBigDecimal()); - } - - public static org.apache.spark.sql.catalyst.InternalRow fromPaimon( - InternalRow row, RowType rowType) { - return new SparkInternalRow(rowType).replace(row); - } - - public static long fromPaimon(Timestamp timestamp) { - if (TypeUtils.treatPaimonTimestampTypeAsSparkTimestampType()) { - return DateTimeUtils.fromJavaTimestamp(timestamp.toSQLTimestamp()); - } else { - return timestamp.toMicros(); - } - } - - public static ArrayData fromPaimon(InternalArray array, ArrayType arrayType) { - return fromPaimonArrayElementType(array, arrayType.getElementType()); - } - - private static ArrayData fromPaimonArrayElementType(InternalArray array, DataType elementType) { - return new SparkArrayData(elementType).replace(array); - } - - public static MapData fromPaimon(InternalMap map, DataType mapType) { - DataType keyType; - DataType valueType; - if (mapType instanceof MapType) { - keyType = ((MapType) mapType).getKeyType(); - valueType = ((MapType) mapType).getValueType(); - } else if (mapType instanceof MultisetType) { - keyType = ((MultisetType) mapType).getElementType(); - valueType = new IntType(); - } else { - throw new UnsupportedOperationException("Unsupported type: " + mapType); - } - - return new ArrayBasedMapData( - fromPaimonArrayElementType(map.keyArray(), keyType), - fromPaimonArrayElementType(map.valueArray(), valueType)); - } } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/DataConverter.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/DataConverter.java new file mode 100644 index 0000000000000..0b5ea899476eb --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/DataConverter.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark; + +import org.apache.paimon.data.BinaryString; +import org.apache.paimon.data.InternalArray; +import org.apache.paimon.data.InternalMap; +import org.apache.paimon.data.InternalRow; +import org.apache.paimon.data.Timestamp; +import org.apache.paimon.spark.data.SparkArrayData; +import org.apache.paimon.spark.data.SparkInternalRow; +import org.apache.paimon.spark.util.shim.TypeUtils; +import org.apache.paimon.types.ArrayType; +import org.apache.paimon.types.DataType; +import org.apache.paimon.types.IntType; +import org.apache.paimon.types.MapType; +import org.apache.paimon.types.MultisetType; +import org.apache.paimon.types.RowType; + +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; + +/** A data converter that convert Paimon data to Spark Data. */ +public class DataConverter { + + public static Object fromPaimon(Object o, DataType type) { + if (o == null) { + return null; + } + switch (type.getTypeRoot()) { + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return fromPaimon((Timestamp) o); + case CHAR: + case VARCHAR: + return fromPaimon((BinaryString) o); + case DECIMAL: + return fromPaimon((org.apache.paimon.data.Decimal) o); + case ARRAY: + return fromPaimon((InternalArray) o, (ArrayType) type); + case MAP: + case MULTISET: + return fromPaimon((InternalMap) o, type); + case ROW: + return fromPaimon((InternalRow) o, (RowType) type); + default: + return o; + } + } + + public static UTF8String fromPaimon(BinaryString string) { + return UTF8String.fromBytes(string.toBytes()); + } + + public static Decimal fromPaimon(org.apache.paimon.data.Decimal decimal) { + return Decimal.apply(decimal.toBigDecimal()); + } + + public static org.apache.spark.sql.catalyst.InternalRow fromPaimon( + InternalRow row, RowType rowType) { + return SparkInternalRow.create(rowType).replace(row); + } + + public static long fromPaimon(Timestamp timestamp) { + if (TypeUtils.treatPaimonTimestampTypeAsSparkTimestampType()) { + return DateTimeUtils.fromJavaTimestamp(timestamp.toSQLTimestamp()); + } else { + return timestamp.toMicros(); + } + } + + public static ArrayData fromPaimon(InternalArray array, ArrayType arrayType) { + return fromPaimonArrayElementType(array, arrayType.getElementType()); + } + + private static ArrayData fromPaimonArrayElementType(InternalArray array, DataType elementType) { + return SparkArrayData.create(elementType).replace(array); + } + + public static MapData fromPaimon(InternalMap map, DataType mapType) { + DataType keyType; + DataType valueType; + if (mapType instanceof MapType) { + keyType = ((MapType) mapType).getKeyType(); + valueType = ((MapType) mapType).getValueType(); + } else if (mapType instanceof MultisetType) { + keyType = ((MultisetType) mapType).getElementType(); + valueType = new IntType(); + } else { + throw new UnsupportedOperationException("Unsupported type: " + mapType); + } + + return new ArrayBasedMapData( + fromPaimonArrayElementType(map.keyArray(), keyType), + fromPaimonArrayElementType(map.valueArray(), valueType)); + } +} 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 deleted file mode 100644 index 9934047a18251..0000000000000 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkArrayData.java +++ /dev/null @@ -1,172 +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.InternalArray; -import org.apache.paimon.types.ArrayType; -import org.apache.paimon.types.BigIntType; -import org.apache.paimon.types.DataType; -import org.apache.paimon.types.DataTypeChecks; -import org.apache.paimon.types.RowType; -import org.apache.paimon.utils.InternalRowUtils; - -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.SpecializedGettersReader; -import org.apache.spark.sql.catalyst.util.ArrayData; -import org.apache.spark.sql.catalyst.util.MapData; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.unsafe.types.CalendarInterval; -import org.apache.spark.unsafe.types.UTF8String; - -import static org.apache.paimon.spark.SparkInternalRow.fromPaimon; -import static org.apache.paimon.utils.InternalRowUtils.copyArray; - -/** Spark {@link ArrayData} to wrap Paimon {@link InternalArray}. */ -public class SparkArrayData extends org.apache.spark.sql.paimon.shims.ArrayData { - - private final DataType elementType; - - private InternalArray array; - - public SparkArrayData(DataType elementType) { - this.elementType = elementType; - } - - public SparkArrayData replace(InternalArray array) { - this.array = array; - return this; - } - - @Override - public int numElements() { - return array.size(); - } - - @Override - public ArrayData copy() { - return new SparkArrayData(elementType).replace(copyArray(array, elementType)); - } - - @Override - public Object[] array() { - Object[] objects = new Object[numElements()]; - for (int i = 0; i < objects.length; i++) { - objects[i] = fromPaimon(InternalRowUtils.get(array, i, elementType), elementType); - } - return objects; - } - - @Override - public void setNullAt(int i) { - throw new UnsupportedOperationException(); - } - - @Override - public void update(int i, Object value) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean isNullAt(int ordinal) { - return array.isNullAt(ordinal); - } - - @Override - public boolean getBoolean(int ordinal) { - return array.getBoolean(ordinal); - } - - @Override - public byte getByte(int ordinal) { - return array.getByte(ordinal); - } - - @Override - public short getShort(int ordinal) { - return array.getShort(ordinal); - } - - @Override - public int getInt(int ordinal) { - return array.getInt(ordinal); - } - - @Override - public long getLong(int ordinal) { - if (elementType instanceof BigIntType) { - return array.getLong(ordinal); - } - - return getTimestampMicros(ordinal); - } - - private long getTimestampMicros(int ordinal) { - return fromPaimon(array.getTimestamp(ordinal, DataTypeChecks.getPrecision(elementType))); - } - - @Override - public float getFloat(int ordinal) { - return array.getFloat(ordinal); - } - - @Override - public double getDouble(int ordinal) { - return array.getDouble(ordinal); - } - - @Override - public Decimal getDecimal(int ordinal, int precision, int scale) { - return fromPaimon(array.getDecimal(ordinal, precision, scale)); - } - - @Override - public UTF8String getUTF8String(int ordinal) { - return fromPaimon(array.getString(ordinal)); - } - - @Override - public byte[] getBinary(int ordinal) { - return array.getBinary(ordinal); - } - - @Override - public CalendarInterval getInterval(int ordinal) { - throw new UnsupportedOperationException(); - } - - @Override - public InternalRow getStruct(int ordinal, int numFields) { - return fromPaimon(array.getRow(ordinal, numFields), (RowType) elementType); - } - - @Override - public ArrayData getArray(int ordinal) { - return fromPaimon(array.getArray(ordinal), (ArrayType) elementType); - } - - @Override - public MapData getMap(int ordinal) { - return fromPaimon(array.getMap(ordinal), elementType); - } - - @Override - public Object get(int ordinal, org.apache.spark.sql.types.DataType dataType) { - return SpecializedGettersReader.read(this, ordinal, dataType, true, true); - } -} 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 5fde2c56596f6..d6318c723fe0e 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,7 @@ import org.apache.paimon.schema.SchemaChange; import org.apache.paimon.spark.catalog.SparkBaseCatalog; import org.apache.paimon.spark.catalog.SupportFunction; +import org.apache.paimon.spark.catalog.SupportView; import org.apache.paimon.table.FormatTable; import org.apache.paimon.table.FormatTableOptions; @@ -72,10 +73,11 @@ import static org.apache.paimon.spark.SparkCatalogOptions.DEFAULT_DATABASE; import static org.apache.paimon.spark.SparkTypeUtils.toPaimonType; import static org.apache.paimon.spark.util.OptionUtils.copyWithSQLConf; -import static org.apache.paimon.utils.Preconditions.checkArgument; +import static org.apache.paimon.spark.utils.CatalogUtils.checkNamespace; +import static org.apache.paimon.spark.utils.CatalogUtils.toIdentifier; /** Spark {@link TableCatalog} for paimon. */ -public class SparkCatalog extends SparkBaseCatalog implements SupportFunction { +public class SparkCatalog extends SparkBaseCatalog implements SupportFunction, SupportView { private static final Logger LOG = LoggerFactory.getLogger(SparkCatalog.class); @@ -126,10 +128,7 @@ public String[] defaultNamespace() { @Override public void createNamespace(String[] namespace, Map metadata) throws NamespaceAlreadyExistsException { - checkArgument( - isValidateNamespace(namespace), - "Namespace %s is not valid", - Arrays.toString(namespace)); + checkNamespace(namespace); try { catalog.createDatabase(namespace[0], false, metadata); } catch (Catalog.DatabaseAlreadyExistException e) { @@ -152,9 +151,7 @@ public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceExcep if (namespace.length == 0) { return listNamespaces(); } - if (!isValidateNamespace(namespace)) { - throw new NoSuchNamespaceException(namespace); - } + checkNamespace(namespace); try { catalog.getDatabase(namespace[0]); return new String[0][]; @@ -166,10 +163,7 @@ public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceExcep @Override public Map loadNamespaceMetadata(String[] namespace) throws NoSuchNamespaceException { - checkArgument( - isValidateNamespace(namespace), - "Namespace %s is not valid", - Arrays.toString(namespace)); + checkNamespace(namespace); String dataBaseName = namespace[0]; try { return catalog.getDatabase(dataBaseName).options(); @@ -207,10 +201,7 @@ public boolean dropNamespace(String[] namespace) throws NoSuchNamespaceException */ public boolean dropNamespace(String[] namespace, boolean cascade) throws NoSuchNamespaceException { - checkArgument( - isValidateNamespace(namespace), - "Namespace %s is not valid", - Arrays.toString(namespace)); + checkNamespace(namespace); try { catalog.dropDatabase(namespace[0], false, cascade); return true; @@ -224,10 +215,7 @@ public boolean dropNamespace(String[] namespace, boolean cascade) @Override public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException { - checkArgument( - isValidateNamespace(namespace), - "Missing database in namespace: %s", - Arrays.toString(namespace)); + checkNamespace(namespace); try { return catalog.listTables(namespace[0]).stream() .map(table -> Identifier.of(namespace, table)) @@ -239,10 +227,7 @@ public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceExcepti @Override public void invalidateTable(Identifier ident) { - try { - catalog.invalidateTable(toIdentifier(ident)); - } catch (NoSuchTableException ignored) { - } + catalog.invalidateTable(toIdentifier(ident)); } @Override @@ -312,26 +297,8 @@ public org.apache.spark.sql.connector.catalog.Table createTable( Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException { try { - String provider = properties.get(TableCatalog.PROP_PROVIDER); - 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); - } + catalog.createTable( + toIdentifier(ident), toInitialSchema(schema, partitions, properties), false); return loadTable(ident); } catch (Catalog.TableAlreadyExistException e) { throw new TableAlreadyExistsException(ident); @@ -347,7 +314,7 @@ public boolean dropTable(Identifier ident) { try { catalog.dropTable(toIdentifier(ident), false); return true; - } catch (Catalog.TableNotExistException | NoSuchTableException e) { + } catch (Catalog.TableNotExistException e) { return false; } } @@ -373,20 +340,20 @@ private SchemaChange toSchemaChange(TableChange change) { TableChange.AddColumn add = (TableChange.AddColumn) change; SchemaChange.Move move = getMove(add.position(), add.fieldNames()); return SchemaChange.addColumn( - Arrays.asList(add.fieldNames()), + add.fieldNames(), toPaimonType(add.dataType()).copy(add.isNullable()), add.comment(), move); } else if (change instanceof TableChange.RenameColumn) { TableChange.RenameColumn rename = (TableChange.RenameColumn) change; - return SchemaChange.renameColumn(Arrays.asList(rename.fieldNames()), rename.newName()); + return SchemaChange.renameColumn(rename.fieldNames(), rename.newName()); } else if (change instanceof TableChange.DeleteColumn) { TableChange.DeleteColumn delete = (TableChange.DeleteColumn) change; - return SchemaChange.dropColumn(Arrays.asList(delete.fieldNames())); + return SchemaChange.dropColumn(delete.fieldNames()); } else if (change instanceof TableChange.UpdateColumnType) { TableChange.UpdateColumnType update = (TableChange.UpdateColumnType) change; return SchemaChange.updateColumnType( - Arrays.asList(update.fieldNames()), toPaimonType(update.newDataType()), true); + update.fieldNames(), toPaimonType(update.newDataType()), true); } else if (change instanceof TableChange.UpdateColumnNullability) { TableChange.UpdateColumnNullability update = (TableChange.UpdateColumnNullability) change; @@ -420,11 +387,18 @@ private static SchemaChange.Move getMove( private Schema toInitialSchema( StructType schema, Transform[] partitions, Map properties) { Map normalizedProperties = new HashMap<>(properties); - if (!normalizedProperties.containsKey(TableCatalog.PROP_PROVIDER)) { - normalizedProperties.put(TableCatalog.PROP_PROVIDER, SparkSource.NAME()); + String provider = properties.get(TableCatalog.PROP_PROVIDER); + if (!usePaimon(provider) && SparkSource.FORMAT_NAMES().contains(provider.toLowerCase())) { + normalizedProperties.put(TYPE.key(), FORMAT_TABLE.toString()); + normalizedProperties.put(FILE_FORMAT.key(), provider.toLowerCase()); } + normalizedProperties.remove(TableCatalog.PROP_PROVIDER); normalizedProperties.remove(PRIMARY_KEY_IDENTIFIER); normalizedProperties.remove(TableCatalog.PROP_COMMENT); + if (normalizedProperties.containsKey(TableCatalog.PROP_LOCATION)) { + String path = normalizedProperties.remove(TableCatalog.PROP_LOCATION); + normalizedProperties.put(CoreOptions.PATH.key(), path); + } String pkAsString = properties.get(PRIMARY_KEY_IDENTIFIER); List primaryKeys = pkAsString == null @@ -454,10 +428,6 @@ private void validateAlterProperty(String alterKey) { } } - private boolean isValidateNamespace(String[] namespace) { - return namespace.length == 1; - } - @Override public void renameTable(Identifier oldIdent, Identifier newIdent) throws NoSuchTableException, TableAlreadyExistsException { @@ -472,15 +442,6 @@ public void renameTable(Identifier oldIdent, Identifier newIdent) // --------------------- tools ------------------------------------------ - protected org.apache.paimon.catalog.Identifier toIdentifier(Identifier ident) - throws NoSuchTableException { - if (!isValidateNamespace(ident.namespace())) { - throw new NoSuchTableException(ident); - } - - return new org.apache.paimon.catalog.Identifier(ident.namespace()[0], ident.name()); - } - protected org.apache.spark.sql.connector.catalog.Table loadSparkTable( Identifier ident, Map extraOptions) throws NoSuchTableException { try { 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 d4b712fcb8ee0..6b7b17b1b1a5e 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,7 +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.paimon.shims.SparkShimLoader; import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.slf4j.Logger; @@ -186,7 +186,7 @@ public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableExcep @Override public void invalidateTable(Identifier ident) { // We do not need to check whether the table exists and whether - // it is an Paimon table to reduce remote service requests. + // it is a Paimon table to reduce remote service requests. sparkCatalog.invalidateTable(ident); asTableCatalog().invalidateTable(ident); } @@ -203,7 +203,8 @@ public Table createTable( return sparkCatalog.createTable(ident, schema, partitions, properties); } else { // delegate to the session catalog - return shims.createTable(asTableCatalog(), ident, schema, partitions, properties); + return SparkShimLoader.getSparkShim() + .createTable(asTableCatalog(), ident, schema, partitions, properties); } } @@ -287,12 +288,6 @@ private CaseInsensitiveStringMap autoFillConfigurations( Map newOptions = new HashMap<>(options.asCaseSensitiveMap()); fillAliyunConfigurations(newOptions, hadoopConf); fillCommonConfigurations(newOptions, sqlConf); - - // if spark is case-insensitive, set allow upper case to catalog - if (!sqlConf.caseSensitiveAnalysis()) { - newOptions.put(ALLOW_UPPER_CASE.key(), "true"); - } - return new CaseInsensitiveStringMap(newOptions); } @@ -312,13 +307,16 @@ private void fillCommonConfigurations(Map options, SQLConf sqlCo String warehouse = sqlConf.warehousePath(); options.put(WAREHOUSE.key(), warehouse); } + if (!options.containsKey(METASTORE.key())) { String metastore = sqlConf.getConf(StaticSQLConf.CATALOG_IMPLEMENTATION()); if (HiveCatalogOptions.IDENTIFIER.equals(metastore)) { 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()); @@ -332,6 +330,11 @@ private void fillCommonConfigurations(Map options, SQLConf sqlCo } else { options.put(DEFAULT_DATABASE.key(), sessionCatalogDefaultDatabase); } + + // if spark is case-insensitive, set allow upper case to catalog + if (!sqlConf.caseSensitiveAnalysis()) { + options.put(ALLOW_UPPER_CASE.key(), "true"); + } } @Override 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 35b65a7b530b8..b2fa66a150906 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 @@ -43,11 +43,13 @@ 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.spark.procedure.RollbackToWatermarkProcedure; import org.apache.paimon.shade.guava30.com.google.common.collect.ImmutableMap; import java.util.Locale; import java.util.Map; +import java.util.Set; import java.util.function.Supplier; /** The {@link Procedure}s including all the stored procedures. */ @@ -62,11 +64,16 @@ public static ProcedureBuilder newBuilder(String name) { return builderSupplier != null ? builderSupplier.get() : null; } + public static Set names() { + return BUILDERS.keySet(); + } + private static Map> initProcedureBuilders() { ImmutableMap.Builder> procedureBuilders = ImmutableMap.builder(); procedureBuilders.put("rollback", RollbackProcedure::builder); procedureBuilders.put("rollback_to_timestamp", RollbackToTimestampProcedure::builder); + procedureBuilders.put("rollback_to_watermark", RollbackToWatermarkProcedure::builder); procedureBuilders.put("create_tag", CreateTagProcedure::builder); procedureBuilders.put("replace_tag", ReplaceTagProcedure::builder); procedureBuilders.put("rename_tag", RenameTagProcedure::builder); 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 8bba676200ce7..f6643f7584065 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 @@ -81,6 +81,10 @@ public static DataType fromPaimonType(org.apache.paimon.types.DataType type) { return type.accept(PaimonToSparkTypeVisitor.INSTANCE); } + public static org.apache.paimon.types.RowType toPaimonRowType(StructType type) { + return (RowType) toPaimonType(type); + } + public static org.apache.paimon.types.DataType toPaimonType(DataType dataType) { return SparkToPaimonTypeVisitor.visit(dataType); } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportView.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportView.java new file mode 100644 index 0000000000000..b8ce86e892866 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/catalog/SupportView.java @@ -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.paimon.spark.catalog; + +import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.spark.SparkTypeUtils; +import org.apache.paimon.view.View; +import org.apache.paimon.view.ViewImpl; + +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.types.StructType; + +import java.util.List; +import java.util.Map; + +import static org.apache.paimon.spark.utils.CatalogUtils.checkNamespace; +import static org.apache.paimon.spark.utils.CatalogUtils.toIdentifier; + +/** Catalog methods for working with Views. */ +public interface SupportView extends WithPaimonCatalog { + + default List listViews(String[] namespace) throws NoSuchNamespaceException { + try { + checkNamespace(namespace); + return paimonCatalog().listViews(namespace[0]); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchNamespaceException(namespace); + } + } + + default View loadView(Identifier ident) throws Catalog.ViewNotExistException { + return paimonCatalog().getView(toIdentifier(ident)); + } + + default void createView( + Identifier ident, + StructType schema, + String queryText, + String comment, + Map properties, + Boolean ignoreIfExists) + throws NoSuchNamespaceException { + org.apache.paimon.catalog.Identifier paimonIdent = toIdentifier(ident); + try { + paimonCatalog() + .createView( + paimonIdent, + new ViewImpl( + paimonIdent, + SparkTypeUtils.toPaimonRowType(schema), + queryText, + comment, + properties), + ignoreIfExists); + } catch (Catalog.ViewAlreadyExistException e) { + throw new RuntimeException("view already exists: " + ident, e); + } catch (Catalog.DatabaseNotExistException e) { + throw new NoSuchNamespaceException(ident.namespace()); + } + } + + default void dropView(Identifier ident, Boolean ignoreIfExists) { + try { + paimonCatalog().dropView(toIdentifier(ident), ignoreIfExists); + } catch (Catalog.ViewNotExistException e) { + throw new RuntimeException("view not exists: " + ident, e); + } + } +} 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 71cf04cf5ef55..4a43e39c31ba1 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 @@ -107,6 +107,7 @@ public class CompactProcedure extends BaseProcedure { new ProcedureParameter[] { ProcedureParameter.required("table", StringType), ProcedureParameter.optional("partitions", StringType), + ProcedureParameter.optional("compact_strategy", StringType), ProcedureParameter.optional("order_strategy", StringType), ProcedureParameter.optional("order_by", StringType), ProcedureParameter.optional("where", StringType), @@ -120,6 +121,9 @@ public class CompactProcedure extends BaseProcedure { new StructField("result", DataTypes.BooleanType, true, Metadata.empty()) }); + private static final String MINOR = "minor"; + private static final String FULL = "full"; + protected CompactProcedure(TableCatalog tableCatalog) { super(tableCatalog); } @@ -138,15 +142,17 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); String partitions = blank(args, 1) ? null : args.getString(1); - String sortType = blank(args, 2) ? TableSorter.OrderType.NONE.name() : args.getString(2); + // make full compact strategy as default. + String compactStrategy = blank(args, 2) ? FULL : args.getString(2); + String sortType = blank(args, 3) ? TableSorter.OrderType.NONE.name() : args.getString(3); List sortColumns = - blank(args, 3) + blank(args, 4) ? Collections.emptyList() - : Arrays.asList(args.getString(3).split(",")); - String where = blank(args, 4) ? null : args.getString(4); - String options = args.isNullAt(5) ? null : args.getString(5); + : Arrays.asList(args.getString(4).split(",")); + String where = blank(args, 5) ? null : args.getString(5); + String options = args.isNullAt(6) ? null : args.getString(6); Duration partitionIdleTime = - blank(args, 6) ? null : TimeUtils.parseDuration(args.getString(6)); + blank(args, 7) ? null : TimeUtils.parseDuration(args.getString(7)); if (TableSorter.OrderType.NONE.name().equals(sortType) && !sortColumns.isEmpty()) { throw new IllegalArgumentException( "order_strategy \"none\" cannot work with order_by columns."); @@ -155,6 +161,14 @@ public InternalRow[] call(InternalRow args) { throw new IllegalArgumentException( "sort compact do not support 'partition_idle_time'."); } + + if (!(compactStrategy.equalsIgnoreCase(FULL) || compactStrategy.equalsIgnoreCase(MINOR))) { + throw new IllegalArgumentException( + String.format( + "The compact strategy only supports 'full' or 'minor', but '%s' is configured.", + compactStrategy)); + } + checkArgument( partitions == null || where == null, "partitions and where cannot be used together."); @@ -192,6 +206,7 @@ public InternalRow[] call(InternalRow args) { newInternalRow( execute( (FileStoreTable) table, + compactStrategy, sortType, sortColumns, relation, @@ -212,6 +227,7 @@ private boolean blank(InternalRow args, int index) { private boolean execute( FileStoreTable table, + String compactStrategy, String sortType, List sortColumns, DataSourceV2Relation relation, @@ -219,6 +235,7 @@ private boolean execute( @Nullable Duration partitionIdleTime) { BucketMode bucketMode = table.bucketMode(); TableSorter.OrderType orderType = TableSorter.OrderType.of(sortType); + boolean fullCompact = compactStrategy.equalsIgnoreCase(FULL); Predicate filter = condition == null ? null @@ -233,7 +250,8 @@ private boolean execute( switch (bucketMode) { case HASH_FIXED: case HASH_DYNAMIC: - compactAwareBucketTable(table, filter, partitionIdleTime, javaSparkContext); + compactAwareBucketTable( + table, fullCompact, filter, partitionIdleTime, javaSparkContext); break; case BUCKET_UNAWARE: compactUnAwareBucketTable(table, filter, partitionIdleTime, javaSparkContext); @@ -259,6 +277,7 @@ private boolean execute( private void compactAwareBucketTable( FileStoreTable table, + boolean fullCompact, @Nullable Predicate filter, @Nullable Duration partitionIdleTime, JavaSparkContext javaSparkContext) { @@ -304,7 +323,7 @@ private void compactAwareBucketTable( SerializationUtils.deserializeBinaryRow( pair.getLeft()), pair.getRight(), - true); + fullCompact); } CommitMessageSerializer serializer = new CommitMessageSerializer(); diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/DeleteBranchProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/DeleteBranchProcedure.java index e398eee0261f8..4a01c33d6af1d 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/DeleteBranchProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/DeleteBranchProcedure.java @@ -18,6 +18,8 @@ package org.apache.paimon.spark.procedure; +import org.apache.paimon.spark.catalog.WithPaimonCatalog; + import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -61,13 +63,20 @@ public StructType outputType() { public InternalRow[] call(InternalRow args) { Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name()); String branchStr = args.getString(1); - return modifyPaimonTable( - tableIdent, - table -> { - table.deleteBranches(branchStr); - InternalRow outputRow = newInternalRow(true); - return new InternalRow[] {outputRow}; - }); + InternalRow[] result = + modifyPaimonTable( + tableIdent, + table -> { + table.deleteBranches(branchStr); + InternalRow outputRow = newInternalRow(true); + return new InternalRow[] {outputRow}; + }); + ((WithPaimonCatalog) tableCatalog()) + .paimonCatalog() + .invalidateTable( + new org.apache.paimon.catalog.Identifier( + tableIdent.namespace()[0], tableIdent.name(), branchStr)); + return result; } 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 7b388227e5a4e..e3a53d2bd2efb 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 @@ -107,9 +107,10 @@ public InternalRow[] call(InternalRow args) { .catalogEnvironment() .metastoreClientFactory()) .map(MetastoreClient.Factory::create) - .orElse(null)); + .orElse(null), + fileStore.options().partitionExpireMaxNum()); if (maxExpires != null) { - partitionExpire.withMaxExpires(maxExpires); + partitionExpire.withMaxExpireNum(maxExpires); } List> expired = partitionExpire.expire(Long.MAX_VALUE); return expired == null || expired.isEmpty() 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 293e84ca14bd6..a929641106c69 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.CleanOrphanFilesResult; import org.apache.paimon.operation.LocalOrphanFilesClean; import org.apache.paimon.operation.OrphanFilesClean; import org.apache.paimon.spark.catalog.WithPaimonCatalog; @@ -66,7 +67,9 @@ public class RemoveOrphanFilesProcedure extends BaseProcedure { private static final StructType OUTPUT_TYPE = new StructType( new StructField[] { - new StructField("result", LongType, true, Metadata.empty()) + new StructField("deletedFileCount", LongType, true, Metadata.empty()), + new StructField( + "deletedFileTotalLenInBytes", LongType, true, Metadata.empty()) }); private RemoveOrphanFilesProcedure(TableCatalog tableCatalog) { @@ -104,11 +107,11 @@ public InternalRow[] call(InternalRow args) { Catalog catalog = ((WithPaimonCatalog) tableCatalog()).paimonCatalog(); String mode = args.isNullAt(4) ? "DISTRIBUTED" : args.getString(4); - long deletedFiles; + CleanOrphanFilesResult cleanOrphanFilesResult; try { switch (mode.toUpperCase(Locale.ROOT)) { case "LOCAL": - deletedFiles = + cleanOrphanFilesResult = LocalOrphanFilesClean.executeDatabaseOrphanFiles( catalog, identifier.getDatabaseName(), @@ -120,7 +123,7 @@ public InternalRow[] call(InternalRow args) { args.isNullAt(3) ? null : args.getInt(3)); break; case "DISTRIBUTED": - deletedFiles = + cleanOrphanFilesResult = SparkOrphanFilesClean.executeDatabaseOrphanFiles( catalog, identifier.getDatabaseName(), @@ -137,7 +140,12 @@ public InternalRow[] call(InternalRow args) { + mode + ". Only 'DISTRIBUTED' and 'LOCAL' are supported."); } - return new InternalRow[] {newInternalRow(deletedFiles)}; + + return new InternalRow[] { + newInternalRow( + cleanOrphanFilesResult.getDeletedFileCount(), + cleanOrphanFilesResult.getDeletedFileTotalLenInBytes()) + }; } catch (Exception e) { throw new RuntimeException(e); } diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RollbackToWatermarkProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RollbackToWatermarkProcedure.java new file mode 100644 index 0000000000000..09185f02c9192 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/RollbackToWatermarkProcedure.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 watermark. */ +public class RollbackToWatermarkProcedure extends BaseProcedure { + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + ProcedureParameter.required("table", StringType), + // watermark value + ProcedureParameter.required("watermark", LongType) + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("result", StringType, true, Metadata.empty()) + }); + + private RollbackToWatermarkProcedure(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 watermark = args.getLong(1); + + return modifyPaimonTable( + tableIdent, + table -> { + FileStoreTable fileStoreTable = (FileStoreTable) table; + Snapshot snapshot = + fileStoreTable.snapshotManager().earlierOrEqualWatermark(watermark); + Preconditions.checkNotNull( + snapshot, + String.format("count not find snapshot earlier than %s", watermark)); + 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 RollbackToWatermarkProcedure doBuild() { + return new RollbackToWatermarkProcedure(tableCatalog()); + } + }; + } + + @Override + public String description() { + return "RollbackToWatermarkProcedure"; + } +} diff --git a/paimon-hive/paimon-hive-connector-3.1/src/main/java/org/apache/paimon/hive/LocalZonedTimestampTypeUtils.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/CatalogUtils.java similarity index 53% rename from paimon-hive/paimon-hive-connector-3.1/src/main/java/org/apache/paimon/hive/LocalZonedTimestampTypeUtils.java rename to paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/CatalogUtils.java index b7c5d26ae6572..fca9df210e708 100644 --- a/paimon-hive/paimon-hive-connector-3.1/src/main/java/org/apache/paimon/hive/LocalZonedTimestampTypeUtils.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/utils/CatalogUtils.java @@ -16,25 +16,26 @@ * limitations under the License. */ -package org.apache.paimon.hive; +package org.apache.paimon.spark.utils; -import org.apache.paimon.types.LocalZonedTimestampType; +import org.apache.spark.sql.connector.catalog.Identifier; -import org.apache.hadoop.hive.serde2.typeinfo.TimestampLocalTZTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import java.util.Arrays; -/** - * Utils to convert between Hive {@link TimestampLocalTZTypeInfo} and Paimon {@link - * LocalZonedTimestampType}. - */ -public class LocalZonedTimestampTypeUtils { +import static org.apache.paimon.utils.Preconditions.checkArgument; + +/** Utils of catalog. */ +public class CatalogUtils { - public static boolean isLocalZonedTimestampType(TypeInfo hiveTypeInfo) { - return hiveTypeInfo instanceof TimestampLocalTZTypeInfo; + public static void checkNamespace(String[] namespace) { + checkArgument( + namespace.length == 1, + "Paimon only support single namespace, but got %s", + Arrays.toString(namespace)); } - public static TypeInfo toHiveType(LocalZonedTimestampType paimonType) { - return TypeInfoFactory.timestampLocalTZTypeInfo; + public static org.apache.paimon.catalog.Identifier toIdentifier(Identifier ident) { + checkNamespace(ident.namespace()); + return new org.apache.paimon.catalog.Identifier(ident.namespace()[0], ident.name()); } } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ColumnPruningAndPushDown.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ColumnPruningAndPushDown.scala index 95c8f4b3a9a8f..f29c146b775a6 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ColumnPruningAndPushDown.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/ColumnPruningAndPushDown.scala @@ -62,7 +62,7 @@ trait ColumnPruningAndPushDown extends Scan with Logging { _readBuilder.withFilter(pushedPredicate) } pushDownLimit.foreach(_readBuilder.withLimit) - _readBuilder + _readBuilder.dropStats() } final def metadataColumns: Seq[PaimonMetadataColumn] = { 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 54970bfe3cb21..840f1341a69d7 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 @@ -21,6 +21,7 @@ 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.spark.data.SparkInternalRow import org.apache.paimon.table.FileStoreTable import org.apache.paimon.table.sink.BatchWriteBuilder import org.apache.paimon.types.RowType @@ -99,7 +100,7 @@ trait PaimonPartitionManagement extends SupportsAtomicPartitionManagement { } override def loadPartitionMetadata(ident: InternalRow): JMap[String, String] = { - throw new UnsupportedOperationException("Load partition is not supported") + Map.empty[String, String].asJava } override def listPartitionIdentifiers( @@ -116,7 +117,7 @@ trait PaimonPartitionManagement extends SupportsAtomicPartitionManagement { s"the partition schema '${partitionSchema.sql}'." ) table.newReadBuilder.newScan.listPartitions.asScala - .map(binaryRow => SparkInternalRow.fromPaimon(binaryRow, partitionRowType)) + .map(binaryRow => DataConverter.fromPaimon(binaryRow, partitionRowType)) .filter( sparkInternalRow => { partitionCols.zipWithIndex diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala index fa9072df31499..526178e28ec33 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReader.scala @@ -20,6 +20,7 @@ package org.apache.paimon.spark import org.apache.paimon.data.{InternalRow => PaimonInternalRow} import org.apache.paimon.reader.RecordReader +import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.spark.schema.PaimonMetadataColumn import org.apache.paimon.table.source.{DataSplit, Split} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReaderFactory.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReaderFactory.scala index 94de0bec3b506..59b07a7944811 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReaderFactory.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/PaimonPartitionReaderFactory.scala @@ -18,10 +18,11 @@ package org.apache.paimon.spark -import org.apache.paimon.data +import org.apache.paimon.data.{InternalRow => PaimonInternalRow} import org.apache.paimon.disk.IOManager import org.apache.paimon.reader.RecordReader import org.apache.paimon.spark.SparkUtils.createIOManager +import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.spark.schema.PaimonMetadataColumn import org.apache.paimon.table.source.{ReadBuilder, Split} import org.apache.paimon.types.RowType @@ -45,13 +46,13 @@ case class PaimonPartitionReaderFactory( val dataFields = new JList(readBuilder.readType().getFields) dataFields.addAll(metadataColumns.map(_.toPaimonDataField).asJava) val rowType = new RowType(dataFields) - new SparkInternalRow(rowType) + SparkInternalRow.create(rowType) } override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { partition match { case paimonInputPartition: PaimonInputPartition => - val readFunc: Split => RecordReader[data.InternalRow] = + val readFunc: Split => RecordReader[PaimonInternalRow] = (split: Split) => readBuilder.newRead().withIOManager(ioManager).createReader(split) PaimonPartitionReader(readFunc, paimonInputPartition, row, metadataColumns) case _ => 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 d8b66e1cd1e03..0393a1cd15786 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 @@ -21,6 +21,7 @@ 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.paimon.table.source.DataSplit import org.apache.spark.sql.connector.expressions.aggregate.Aggregation import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownAggregates, SupportsPushDownLimit} @@ -36,12 +37,12 @@ class PaimonScanBuilder(table: Table) override def pushLimit(limit: Int): Boolean = { // 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 + // just make the 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` + // for now, we only support complete push down, so there is no difference with `pushAggregation` pushAggregation(aggregation) } @@ -66,8 +67,11 @@ class PaimonScanBuilder(table: Table) val pushedPartitionPredicate = PredicateBuilder.and(pushedPredicates.map(_._2): _*) readBuilder.withFilter(pushedPartitionPredicate) } - val scan = readBuilder.newScan() - scan.listPartitionEntries.asScala.foreach(aggregator.update) + val dataSplits = readBuilder.newScan().plan().splits().asScala.map(_.asInstanceOf[DataSplit]) + if (!dataSplits.forall(_.mergedRowCountAvailable())) { + return false + } + dataSplits.foreach(aggregator.update) localScan = Some( PaimonLocalScan( aggregator.result(), 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 28af4ac0a4fdb..8dd4649330327 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,6 +18,7 @@ package org.apache.paimon.spark +import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.stats.ColStats import org.apache.paimon.types.{DataField, DataType, RowType} @@ -118,8 +119,10 @@ object PaimonColumnStats { def apply(dateType: DataType, paimonColStats: ColStats[_]): PaimonColumnStats = { PaimonColumnStats( paimonColStats.nullCount, - Optional.ofNullable(SparkInternalRow.fromPaimon(paimonColStats.min().orElse(null), dateType)), - Optional.ofNullable(SparkInternalRow.fromPaimon(paimonColStats.max().orElse(null), dateType)), + Optional.ofNullable( + DataConverter + .fromPaimon(paimonColStats.min().orElse(null), dateType)), + Optional.ofNullable(DataConverter.fromPaimon(paimonColStats.max().orElse(null), dateType)), paimonColStats.distinctCount, paimonColStats.avgLen, paimonColStats.maxLen 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 0170a29f68d36..d80d7350a6554 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 @@ -118,7 +118,7 @@ object SparkSource { val NAME = "paimon" - val FORMAT_NAMES = Seq("csv", "orc", "parquet") + val FORMAT_NAMES: Seq[String] = Seq("csv", "orc", "parquet") def toBaseRelation(table: FileStoreTable, _sqlContext: SQLContext): BaseRelation = { new BaseRelation { 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 39b1947e4f37b..b9a90d8b5bef2 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 @@ -64,6 +64,9 @@ case class SparkTable(table: Table) if (table.comment.isPresent) { properties.put(TableCatalog.PROP_COMMENT, table.comment.get) } + if (properties.containsKey(CoreOptions.PATH.key())) { + properties.put(TableCatalog.PROP_LOCATION, properties.get(CoreOptions.PATH.key())) + } properties case _ => Collections.emptyMap() } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala index cd9718cf44eb2..8988e7218d1f8 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/aggregate/LocalAggregator.scala @@ -19,9 +19,10 @@ 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.spark.SparkTypeUtils +import org.apache.paimon.spark.data.SparkInternalRow import org.apache.paimon.table.{DataTable, Table} +import org.apache.paimon.table.source.DataSplit import org.apache.paimon.utils.{InternalRowUtils, ProjectedRow} import org.apache.spark.sql.catalyst.InternalRow @@ -77,13 +78,7 @@ class LocalAggregator(table: Table) { } def pushAggregation(aggregation: Aggregation): Boolean = { - if ( - !table.isInstanceOf[DataTable] || - !table.primaryKeys.isEmpty - ) { - return false - } - if (table.asInstanceOf[DataTable].coreOptions.deletionVectorsEnabled) { + if (!table.isInstanceOf[DataTable]) { return false } @@ -104,15 +99,15 @@ class LocalAggregator(table: Table) { 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) + SparkInternalRow.create(partitionType).replace(genericRow) } - def update(partitionEntry: PartitionEntry): Unit = { + def update(dataSplit: DataSplit): Unit = { assert(isInitialized) - val groupByRow = requiredGroupByRow(partitionEntry.partition()) + val groupByRow = requiredGroupByRow(dataSplit.partition()) val aggFuncEvaluator = groupByEvaluatorMap.getOrElseUpdate(groupByRow, aggFuncEvaluatorGetter()) - aggFuncEvaluator.foreach(_.update(partitionEntry)) + aggFuncEvaluator.foreach(_.update(dataSplit)) } def result(): Array[InternalRow] = { @@ -146,7 +141,7 @@ class LocalAggregator(table: Table) { } trait AggFuncEvaluator[T] { - def update(partitionEntry: PartitionEntry): Unit + def update(dataSplit: DataSplit): Unit def result(): T def resultType: DataType def prettyName: String @@ -155,8 +150,8 @@ trait AggFuncEvaluator[T] { class CountStarEvaluator extends AggFuncEvaluator[Long] { private var _result: Long = 0L - override def update(partitionEntry: PartitionEntry): Unit = { - _result += partitionEntry.recordCount() + override def update(dataSplit: DataSplit): Unit = { + _result += dataSplit.mergedRowCount() } override def result(): Long = _result diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonResolvePartitionSpec.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonResolvePartitionSpec.scala new file mode 100644 index 0000000000000..5d6a5a063c06d --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonResolvePartitionSpec.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.PaimonUtils.{normalizePartitionSpec, requireExactMatchedPartitionSpec} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.{PartitionSpec, ResolvedPartitionSpec, UnresolvedPartitionSpec} +import org.apache.spark.sql.catalyst.analysis.ResolvePartitionSpec.conf +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} +import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +object PaimonResolvePartitionSpec { + + def resolve( + catalog: TableCatalog, + tableIndent: Identifier, + partitionSpec: PartitionSpec): ResolvedPartitionSpec = { + val table = catalog.loadTable(tableIndent).asPartitionable + partitionSpec match { + case u: UnresolvedPartitionSpec => + val partitionSchema = table.partitionSchema() + resolvePartitionSpec(table.name(), u, partitionSchema, allowPartitionSpec = false) + case o => o.asInstanceOf[ResolvedPartitionSpec] + } + } + + private def resolvePartitionSpec( + tableName: String, + partSpec: UnresolvedPartitionSpec, + partSchema: StructType, + allowPartitionSpec: Boolean): ResolvedPartitionSpec = { + val normalizedSpec = normalizePartitionSpec(partSpec.spec, partSchema, tableName, conf.resolver) + if (!allowPartitionSpec) { + requireExactMatchedPartitionSpec(tableName, normalizedSpec, partSchema.fieldNames) + } + val partitionNames = normalizedSpec.keySet + val requestedFields = partSchema.filter(field => partitionNames.contains(field.name)) + ResolvedPartitionSpec( + requestedFields.map(_.name), + convertToPartIdent(normalizedSpec, requestedFields), + partSpec.location) + } + + def convertToPartIdent( + partitionSpec: TablePartitionSpec, + schema: Seq[StructField]): InternalRow = { + val partValues = schema.map { + part => + val raw = partitionSpec.get(part.name).orNull + val dt = CharVarcharUtils.replaceCharVarcharWithString(part.dataType) + Cast(Literal.create(raw, StringType), dt, Some(conf.sessionLocalTimeZone)).eval() + } + InternalRow.fromSeq(partValues) + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonViewResolver.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonViewResolver.scala new file mode 100644 index 0000000000000..3da0ddab64172 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonViewResolver.scala @@ -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.spark.catalyst.analysis + +import org.apache.paimon.catalog.Catalog.ViewNotExistException +import org.apache.paimon.spark.SparkTypeUtils +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.view.View + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedRelation, UnresolvedTableOrView} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, UpCast} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.parser.extensions.{CurrentOrigin, Origin} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Project, SubqueryAlias} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{Identifier, PaimonLookupCatalog} + +case class PaimonViewResolver(spark: SparkSession) + extends Rule[LogicalPlan] + with PaimonLookupCatalog { + + protected lazy val catalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { + case u @ UnresolvedRelation(parts @ CatalogAndIdentifier(catalog: SupportView, ident), _, _) => + try { + val view = catalog.loadView(ident) + createViewRelation(parts, view) + } catch { + case _: ViewNotExistException => + u + } + + case u @ UnresolvedTableOrView(CatalogAndIdentifier(catalog: SupportView, ident), _, _) => + try { + catalog.loadView(ident) + ResolvedPaimonView(catalog, ident) + } catch { + case _: ViewNotExistException => + u + } + } + + private def createViewRelation(nameParts: Seq[String], view: View): LogicalPlan = { + val parsedPlan = parseViewText(nameParts.toArray.mkString("."), view.query) + + val aliases = SparkTypeUtils.fromPaimonRowType(view.rowType()).fields.zipWithIndex.map { + case (expected, pos) => + val attr = GetColumnByOrdinal(pos, expected.dataType) + Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = + Some(expected.metadata)) + } + + SubqueryAlias(nameParts, Project(aliases, parsedPlan)) + } + + private def parseViewText(name: String, viewText: String): LogicalPlan = { + val origin = Origin( + objectType = Some("VIEW"), + objectName = Some(name) + ) + try { + CurrentOrigin.withOrigin(origin) { + try { + spark.sessionState.sqlParser.parseQuery(viewText) + } catch { + // For compatibility with Spark 3.2 and below + case _: NoSuchMethodError => + spark.sessionState.sqlParser.parsePlan(viewText) + } + } + } catch { + case _: ParseException => + throw new RuntimeException("Failed to parse view text: " + viewText) + } + } +} + +case class ResolvedPaimonView(catalog: SupportView, identifier: Identifier) extends LeafNode { + override def output: Seq[Attribute] = Nil +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala index c008819fb0cc8..d4010ea338116 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/expressions/ExpressionHelper.scala @@ -23,12 +23,13 @@ import org.apache.paimon.spark.SparkFilterConverter import org.apache.paimon.spark.catalyst.Compatibility import org.apache.paimon.types.RowType +import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.PaimonUtils.{normalizeExprs, translateFilter} -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, Cast, Expression, GetStructField, Literal, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.paimon.shims.SparkShimLoader import org.apache.spark.sql.types.{DataType, NullType} /** An expression helper. */ @@ -36,6 +37,14 @@ trait ExpressionHelper extends PredicateHelper { import ExpressionHelper._ + def toColumn(expr: Expression): Column = { + SparkShimLoader.getSparkShim.column(expr) + } + + def toExpression(spark: SparkSession, col: Column): Expression = { + SparkShimLoader.getSparkShim.convertToExpression(spark, col) + } + protected def resolveExpression( spark: SparkSession)(expr: Expression, plan: LogicalPlan): Expression = { if (expr.resolved) { 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 b0b1a76e7a1f1..3428ed89f0047 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,7 +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.paimon.shims.SparkShimLoader import org.apache.spark.sql.types.{DataType, StructType} import scala.collection.mutable.ArrayBuffer @@ -344,7 +344,7 @@ trait MergePaimonScalarSubqueriesBase extends Rule[LogicalPlan] with PredicateHe val Seq(newPlanSupportsHashAggregate, cachedPlanSupportsHashAggregate) = aggregateExpressionsSeq.zip(groupByExpressionSeq).map { case (aggregateExpressions, groupByExpressions) => - shims.Aggregate.supportsHashAggregate( + SparkShimLoader.getSparkShim.supportsHashAggregate( aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes), groupByExpressions) } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonTableValuedFunctions.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonTableValuedFunctions.scala index 4d63c2a8d2be7..6edbf533cbbc3 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonTableValuedFunctions.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonTableValuedFunctions.scala @@ -19,8 +19,6 @@ package org.apache.paimon.spark.catalyst.plans.logical import org.apache.paimon.CoreOptions -import org.apache.paimon.spark.SparkCatalog -import org.apache.paimon.spark.catalog.Catalogs import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.FunctionIdentifier @@ -28,7 +26,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistryBase import org.apache.spark.sql.catalyst.analysis.TableFunctionRegistry.TableFunctionBuilder import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, ExpressionInfo} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} -import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -40,7 +38,7 @@ object PaimonTableValuedFunctions { val supportedFnNames: Seq[String] = Seq(INCREMENTAL_QUERY) - type TableFunctionDescription = (FunctionIdentifier, ExpressionInfo, TableFunctionBuilder) + private type TableFunctionDescription = (FunctionIdentifier, ExpressionInfo, TableFunctionBuilder) def getTableValueFunctionInjection(fnName: String): TableFunctionDescription = { val (info, builder) = fnName match { @@ -60,13 +58,7 @@ object PaimonTableValuedFunctions { val sessionState = spark.sessionState val catalogManager = sessionState.catalogManager - - val sparkCatalog = new SparkCatalog() - val currentCatalog = catalogManager.currentCatalog.name() - sparkCatalog.initialize( - currentCatalog, - Catalogs.catalogOptions(currentCatalog, spark.sessionState.conf)) - + val sparkCatalog = catalogManager.currentCatalog.asInstanceOf[TableCatalog] val tableId = sessionState.sqlParser.parseTableIdentifier(args.head.eval().toString) val namespace = tableId.database.map(Array(_)).getOrElse(catalogManager.currentNamespace) val ident = Identifier.of(namespace, tableId.table) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonViewCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonViewCommand.scala new file mode 100644 index 0000000000000..24b27bb0e6cca --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonViewCommand.scala @@ -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.spark.catalyst.plans.logical + +import org.apache.paimon.spark.leafnode.{PaimonBinaryCommand, PaimonUnaryCommand} + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, ShowViews, Statistics} +import org.apache.spark.sql.connector.catalog.{CatalogPlugin, Identifier} + +case class CreatePaimonView( + child: LogicalPlan, + queryText: String, + query: LogicalPlan, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String] = Seq.empty, + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean) + extends PaimonBinaryCommand { + + override def left: LogicalPlan = child + + override def right: LogicalPlan = query + + override protected def withNewChildrenInternal( + newLeft: LogicalPlan, + newRight: LogicalPlan): LogicalPlan = + copy(child = newLeft, query = newRight) +} + +case class DropPaimonView(child: LogicalPlan, ifExists: Boolean) extends PaimonUnaryCommand { + + override protected def withNewChildInternal(newChild: LogicalPlan): DropPaimonView = + copy(child = newChild) +} + +case class ShowPaimonViews( + namespace: LogicalPlan, + pattern: Option[String], + override val output: Seq[Attribute] = ShowViews.getOutputAttrs) + extends PaimonUnaryCommand { + + override def child: LogicalPlan = namespace + + override protected def withNewChildInternal(newChild: LogicalPlan): ShowPaimonViews = + copy(namespace = newChild) +} + +/** Copy from spark 3.4+ */ +case class ResolvedIdentifier(catalog: CatalogPlugin, identifier: Identifier) extends LeafNode { + + override def output: Seq[Attribute] = Nil + + override def stats: Statistics = Statistics.DUMMY +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala index f252b3bb130b4..57a8a8e4abfd0 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/BucketProcessor.scala @@ -22,7 +22,7 @@ import org.apache.paimon.crosspartition.{GlobalIndexAssigner, KeyPartOrRow} import org.apache.paimon.data.{BinaryRow, GenericRow, InternalRow => PaimonInternalRow, JoinedRow} import org.apache.paimon.disk.IOManager import org.apache.paimon.index.HashBucketAssigner -import org.apache.paimon.spark.{SparkInternalRow, SparkRow} +import org.apache.paimon.spark.{DataConverter, SparkRow} import org.apache.paimon.spark.SparkUtils.createIOManager import org.apache.paimon.spark.util.EncoderUtils import org.apache.paimon.table.FileStoreTable @@ -179,7 +179,7 @@ class GlobalIndexAssignerIterator( extraRow.setField(1, bucket) queue.enqueue( encoderGroup.internalToRow( - SparkInternalRow.fromPaimon(new JoinedRow(row, extraRow), rowType))) + DataConverter.fromPaimon(new JoinedRow(row, extraRow), rowType))) } ) rowIterator.foreach { 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 f557a0cf38ee0..52e704172fc82 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 @@ -38,7 +38,6 @@ 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 @@ -153,12 +152,12 @@ case class MergeIntoPaimonTable( } if (hasUpdate(matchedActions)) { touchedFilePathsSet ++= findTouchedFiles( - targetDS.join(sourceDS, column(mergeCondition), "inner"), + targetDS.join(sourceDS, toColumn(mergeCondition), "inner"), sparkSession) } if (hasUpdate(notMatchedBySourceActions)) { touchedFilePathsSet ++= findTouchedFiles( - targetDS.join(sourceDS, column(mergeCondition), "left_anti"), + targetDS.join(sourceDS, toColumn(mergeCondition), "left_anti"), sparkSession) } @@ -200,7 +199,7 @@ case class MergeIntoPaimonTable( val sourceDS = createDataset(sparkSession, sourceTable) .withColumn(SOURCE_ROW_COL, lit(true)) - val joinedDS = sourceDS.join(targetDS, column(mergeCondition), "fullOuter") + val joinedDS = sourceDS.join(targetDS, toColumn(mergeCondition), "fullOuter") val joinedPlan = joinedDS.queryExecution.analyzed def resolveOnJoinedPlan(exprs: Seq[Expression]): Seq[Expression] = { @@ -209,9 +208,9 @@ case class MergeIntoPaimonTable( val targetOutput = filteredTargetPlan.output val targetRowNotMatched = resolveOnJoinedPlan( - Seq(convertToExpression(sparkSession, col(SOURCE_ROW_COL).isNull))).head + Seq(toExpression(sparkSession, col(SOURCE_ROW_COL).isNull))).head val sourceRowNotMatched = resolveOnJoinedPlan( - Seq(convertToExpression(sparkSession, col(TARGET_ROW_COL).isNull))).head + Seq(toExpression(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)) @@ -275,7 +274,7 @@ case class MergeIntoPaimonTable( .withColumn(ROW_ID_COL, monotonically_increasing_id()) val sourceDS = createDataset(sparkSession, sourceTable) val count = sourceDS - .join(targetDS, column(mergeCondition), "inner") + .join(targetDS, toColumn(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 19f73cb6cc683..9a88ca2e4c3ab 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 @@ -18,6 +18,7 @@ package org.apache.paimon.spark.commands +import org.apache.paimon.manifest.PartitionEntry import org.apache.paimon.schema.TableSchema import org.apache.paimon.spark.SparkTable import org.apache.paimon.spark.leafnode.PaimonLeafRunnableCommand @@ -64,11 +65,9 @@ case class PaimonAnalyzeTableColumnCommand( // compute stats val totalSize = table .newScan() - .plan() - .splits() + .listPartitionEntries() .asScala - .flatMap { case split: DataSplit => split.dataFiles().asScala } - .map(_.fileSize()) + .map(_.fileSizeInBytes()) .sum val (mergedRecordCount, colStats) = PaimonStatsUtils.computeColumnStats(sparkSession, relation, attributes) 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 191d7a766b716..87583593e3fe8 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 @@ -95,6 +95,10 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper with SQLCon output: Seq[Attribute]): Seq[DataSplit] = { // low level snapshot reader, it can not be affected by 'scan.mode' val snapshotReader = table.newSnapshotReader() + // dropStats after filter push down + if (table.coreOptions().manifestDeleteFileDropStats()) { + snapshotReader.dropStats() + } if (condition != TrueLiteral) { val filter = convertConditionToPaimonPredicate(condition, output, rowType, ignoreFailure = true) 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 f2ea965d14071..47e3f77d0e2cd 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 @@ -33,7 +33,7 @@ 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 import org.apache.spark.sql.functions.lit -import org.apache.spark.sql.paimon.shims.ExpressionUtils.column +import org.apache.spark.sql.paimon.shims.SparkShimLoader case class UpdatePaimonTableCommand( relation: DataSourceV2Relation, @@ -133,7 +133,8 @@ case class UpdatePaimonTableCommand( sparkSession: SparkSession, touchedDataSplits: Array[DataSplit]): Seq[CommitMessage] = { val updateColumns = updateExpressions.zip(relation.output).map { - case (update, origin) => column(update).as(origin.name, origin.metadata) + case (update, origin) => + SparkShimLoader.getSparkShim.column(update).as(origin.name, origin.metadata) } val toUpdateScanRelation = createNewRelation(touchedDataSplits, relation) @@ -156,7 +157,7 @@ case class UpdatePaimonTableCommand( } else { If(condition, update, origin) } - column(updated).as(origin.name, origin.metadata) + SparkShimLoader.getSparkShim.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/data/SparkArrayData.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkArrayData.scala new file mode 100644 index 0000000000000..c6539a493cee9 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkArrayData.scala @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.data + +import org.apache.paimon.data.InternalArray +import org.apache.paimon.spark.DataConverter +import org.apache.paimon.types.{ArrayType => PaimonArrayType, BigIntType, DataType => PaimonDataType, DataTypeChecks, RowType} +import org.apache.paimon.utils.InternalRowUtils + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.SpecializedGettersReader +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} +import org.apache.spark.sql.paimon.shims.SparkShimLoader +import org.apache.spark.sql.types.{DataType, Decimal} +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} + +abstract class SparkArrayData extends org.apache.spark.sql.catalyst.util.ArrayData { + + def replace(array: InternalArray): SparkArrayData +} + +abstract class AbstractSparkArrayData extends SparkArrayData { + + val elementType: PaimonDataType + + var paimonArray: InternalArray = _ + + override def replace(array: InternalArray): SparkArrayData = { + this.paimonArray = array + this + } + + override def numElements(): Int = paimonArray.size() + + override def copy(): ArrayData = { + SparkArrayData.create(elementType).replace(InternalRowUtils.copyArray(paimonArray, elementType)) + } + + override def array: Array[Any] = { + Array.range(0, numElements()).map { + i => + DataConverter + .fromPaimon(InternalRowUtils.get(paimonArray, i, elementType), elementType) + } + } + + override def setNullAt(i: Int): Unit = throw new UnsupportedOperationException() + + override def update(i: Int, value: Any): Unit = throw new UnsupportedOperationException() + + override def isNullAt(ordinal: Int): Boolean = paimonArray.isNullAt(ordinal) + + override def getBoolean(ordinal: Int): Boolean = paimonArray.getBoolean(ordinal) + + override def getByte(ordinal: Int): Byte = paimonArray.getByte(ordinal) + + override def getShort(ordinal: Int): Short = paimonArray.getShort(ordinal) + + override def getInt(ordinal: Int): Int = paimonArray.getInt(ordinal) + + override def getLong(ordinal: Int): Long = elementType match { + case _: BigIntType => paimonArray.getLong(ordinal) + case _ => + DataConverter.fromPaimon( + paimonArray.getTimestamp(ordinal, DataTypeChecks.getPrecision(elementType))) + } + + override def getFloat(ordinal: Int): Float = paimonArray.getFloat(ordinal) + + override def getDouble(ordinal: Int): Double = paimonArray.getDouble(ordinal) + + override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = + DataConverter.fromPaimon(paimonArray.getDecimal(ordinal, precision, scale)) + + override def getUTF8String(ordinal: Int): UTF8String = + DataConverter.fromPaimon(paimonArray.getString(ordinal)) + + override def getBinary(ordinal: Int): Array[Byte] = paimonArray.getBinary(ordinal) + + override def getInterval(ordinal: Int): CalendarInterval = + throw new UnsupportedOperationException() + + override def getStruct(ordinal: Int, numFields: Int): InternalRow = DataConverter + .fromPaimon(paimonArray.getRow(ordinal, numFields), elementType.asInstanceOf[RowType]) + + override def getArray(ordinal: Int): ArrayData = DataConverter.fromPaimon( + paimonArray.getArray(ordinal), + elementType.asInstanceOf[PaimonArrayType]) + + override def getMap(ordinal: Int): MapData = + DataConverter.fromPaimon(paimonArray.getMap(ordinal), elementType) + + override def get(ordinal: Int, dataType: DataType): AnyRef = + SpecializedGettersReader.read(this, ordinal, dataType, true, true) + +} + +object SparkArrayData { + def create(elementType: PaimonDataType): SparkArrayData = { + SparkShimLoader.getSparkShim.createSparkArrayData(elementType) + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala new file mode 100644 index 0000000000000..f3e607e9d7d20 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/data/SparkInternalRow.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.data + +import org.apache.paimon.types.RowType + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.paimon.shims.SparkShimLoader + +abstract class SparkInternalRow extends InternalRow { + def replace(row: org.apache.paimon.data.InternalRow): SparkInternalRow +} + +object SparkInternalRow { + + def create(rowType: RowType): SparkInternalRow = { + SparkShimLoader.getSparkShim.createSparkInternalRow(rowType) + } + +} 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 0c3d3e6b6dc64..fb7bc6b22cd34 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 @@ -19,12 +19,15 @@ package org.apache.paimon.spark.execution import org.apache.paimon.spark.{SparkCatalog, SparkUtils} -import org.apache.paimon.spark.catalyst.plans.logical.{CreateOrReplaceTagCommand, DeleteTagCommand, PaimonCallCommand, RenameTagCommand, ShowTagsCommand} +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.spark.catalyst.analysis.ResolvedPaimonView +import org.apache.paimon.spark.catalyst.plans.logical.{CreateOrReplaceTagCommand, CreatePaimonView, DeleteTagCommand, DropPaimonView, PaimonCallCommand, RenameTagCommand, ResolvedIdentifier, ShowPaimonViews, ShowTagsCommand} import org.apache.spark.sql.{SparkSession, Strategy} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace 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.catalyst.plans.logical.{CreateTableAsSelect, DescribeRelation, LogicalPlan, ShowCreateTable} 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 @@ -65,6 +68,45 @@ case class PaimonStrategy(spark: SparkSession) case RenameTagCommand(PaimonCatalogAndIdentifier(catalog, ident), sourceTag, targetTag) => RenameTagExec(catalog, ident, sourceTag, targetTag) :: Nil + case CreatePaimonView( + ResolvedIdentifier(viewCatalog: SupportView, ident), + queryText, + query, + columnAliases, + columnComments, + queryColumnNames, + comment, + properties, + allowExisting, + replace) => + CreatePaimonViewExec( + viewCatalog, + ident, + queryText, + query.schema, + columnAliases, + columnComments, + queryColumnNames, + comment, + properties, + allowExisting, + replace) :: Nil + + case DropPaimonView(ResolvedIdentifier(viewCatalog: SupportView, ident), ifExists) => + DropPaimonViewExec(viewCatalog, ident, ifExists) :: Nil + + // A new member was added to ResolvedNamespace since spark4.0, + // unapply pattern matching is not used here to ensure compatibility across multiple spark versions. + case ShowPaimonViews(r: ResolvedNamespace, pattern, output) + if r.catalog.isInstanceOf[SupportView] => + ShowPaimonViewsExec(output, r.catalog.asInstanceOf[SupportView], r.namespace, pattern) :: Nil + + case ShowCreateTable(ResolvedPaimonView(viewCatalog, ident), _, output) => + ShowCreatePaimonViewExec(output, viewCatalog, ident) :: Nil + + case DescribeRelation(ResolvedPaimonView(viewCatalog, ident), _, isExtended, output) => + DescribePaimonViewExec(output, viewCatalog, ident, isExtended) :: Nil + case _ => Nil } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonViewExec.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonViewExec.scala new file mode 100644 index 0000000000000..2282f7c34411a --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/execution/PaimonViewExec.scala @@ -0,0 +1,232 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.catalog.SupportView +import org.apache.paimon.spark.leafnode.PaimonLeafV2CommandExec +import org.apache.paimon.view.View + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericInternalRow} +import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIfNeeded, StringUtils} +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.types.UTF8String + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +case class CreatePaimonViewExec( + catalog: SupportView, + ident: Identifier, + queryText: String, + viewSchema: StructType, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String], + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean +) extends PaimonLeafV2CommandExec { + + override def output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + if (columnAliases.nonEmpty || columnComments.nonEmpty || queryColumnNames.nonEmpty) { + throw new UnsupportedOperationException( + "columnAliases, columnComments and queryColumnNames are not supported now") + } + + // Note: for replace just drop then create ,this operation is non-atomic. + if (replace) { + catalog.dropView(ident, true) + } + + catalog.createView( + ident, + viewSchema, + queryText, + comment.orNull, + properties.asJava, + allowExisting) + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"CreatePaimonViewExec: $ident" + } +} + +case class DropPaimonViewExec(catalog: SupportView, ident: Identifier, ifExists: Boolean) + extends PaimonLeafV2CommandExec { + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.dropView(ident, ifExists) + Nil + } + + override def simpleString(maxFields: Int): String = { + s"DropPaimonViewExec: $ident" + } +} + +case class ShowPaimonViewsExec( + output: Seq[Attribute], + catalog: SupportView, + namespace: Seq[String], + pattern: Option[String]) + extends PaimonLeafV2CommandExec { + + override protected def run(): Seq[InternalRow] = { + val rows = new ArrayBuffer[InternalRow]() + catalog.listViews(namespace.toArray).asScala.map { + viewName => + if (pattern.forall(StringUtils.filterPattern(Seq(viewName), _).nonEmpty)) { + rows += new GenericInternalRow( + Array( + UTF8String.fromString(namespace.mkString(".")), + UTF8String.fromString(viewName), + false)) + } + } + rows.toSeq + } + + override def simpleString(maxFields: Int): String = { + s"ShowPaimonViewsExec: $namespace" + } +} + +case class ShowCreatePaimonViewExec(output: Seq[Attribute], catalog: SupportView, ident: Identifier) + extends PaimonLeafV2CommandExec { + + override protected def run(): Seq[InternalRow] = { + val view = catalog.loadView(ident) + + val builder = new StringBuilder + builder ++= s"CREATE VIEW ${view.fullName()} " + showDataColumns(view, builder) + showComment(view, builder) + showProperties(view, builder) + builder ++= s"AS\n${view.query}\n" + + Seq(new GenericInternalRow(values = Array(UTF8String.fromString(builder.toString)))) + } + + private def showDataColumns(view: View, builder: StringBuilder): Unit = { + if (view.rowType().getFields.size() > 0) { + val viewColumns = view.rowType().getFields.asScala.map { + f => + val comment = if (f.description() != null) s" COMMENT '${f.description()}'" else "" + // view columns shouldn't have data type info + s"${quoteIfNeeded(f.name)}$comment" + } + builder ++= concatByMultiLines(viewColumns) + } + } + + private def showComment(view: View, builder: StringBuilder): Unit = { + if (view.comment().isPresent) { + builder ++= s"COMMENT '${view.comment().get()}'\n" + } + } + + private def showProperties(view: View, builder: StringBuilder): Unit = { + if (!view.options().isEmpty) { + val props = view.options().asScala.toSeq.sortBy(_._1).map { + case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + } + builder ++= s"TBLPROPERTIES ${concatByMultiLines(props)}" + } + } + + private def concatByMultiLines(iter: Iterable[String]): String = { + iter.mkString("(\n ", ",\n ", ")\n") + } + + override def simpleString(maxFields: Int): String = { + s"ShowCreatePaimonViewExec: $ident" + } +} + +case class DescribePaimonViewExec( + output: Seq[Attribute], + catalog: SupportView, + ident: Identifier, + isExtended: Boolean) + extends PaimonLeafV2CommandExec { + + override protected def run(): Seq[InternalRow] = { + val rows = new ArrayBuffer[InternalRow]() + val view = catalog.loadView(ident) + + describeColumns(view, rows) + if (isExtended) { + describeExtended(view, rows) + } + + rows.toSeq + } + + private def describeColumns(view: View, rows: ArrayBuffer[InternalRow]) = { + view + .rowType() + .getFields + .asScala + .map(f => rows += row(f.name(), f.`type`().toString, f.description())) + } + + private def describeExtended(view: View, rows: ArrayBuffer[InternalRow]) = { + rows += row("", "", "") + rows += row("# Detailed View Information", "", "") + rows += row("Name", view.fullName(), "") + rows += row("Comment", view.comment().orElse(""), "") + rows += row("View Text", view.query, "") + rows += row( + "View Query Output Columns", + view.rowType().getFieldNames.asScala.mkString("[", ", ", "]"), + "") + rows += row( + "View Properties", + view + .options() + .asScala + .toSeq + .sortBy(_._1) + .map { case (k, v) => s"$k=$v" } + .mkString("[", ", ", "]"), + "") + } + + private def row(s1: String, s2: String, s3: String): InternalRow = { + new GenericInternalRow( + values = + Array(UTF8String.fromString(s1), UTF8String.fromString(s2), UTF8String.fromString(s3))) + } + + override def simpleString(maxFields: Int): String = { + s"DescribePaimonViewExec: $ident" + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala index 4fe217ee09bd8..f73df64fb8ab6 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/extensions/PaimonSparkSessionExtensions.scala @@ -18,25 +18,30 @@ package org.apache.paimon.spark.extensions -import org.apache.paimon.spark.catalyst.analysis.{PaimonAnalysis, PaimonDeleteTable, PaimonIncompatiblePHRRules, PaimonIncompatibleResolutionRules, PaimonMergeInto, PaimonPostHocResolutionRules, PaimonProcedureResolver, PaimonUpdateTable} +import org.apache.paimon.spark.catalyst.analysis.{PaimonAnalysis, PaimonDeleteTable, PaimonIncompatiblePHRRules, PaimonIncompatibleResolutionRules, PaimonMergeInto, PaimonPostHocResolutionRules, PaimonProcedureResolver, PaimonUpdateTable, PaimonViewResolver} import org.apache.paimon.spark.catalyst.optimizer.{EvalSubqueriesForDeleteTable, MergePaimonScalarSubqueries} import org.apache.paimon.spark.catalyst.plans.logical.PaimonTableValuedFunctions import org.apache.paimon.spark.execution.PaimonStrategy import org.apache.paimon.spark.execution.adaptive.DisableUnnecessaryPaimonBucketedScan import org.apache.spark.sql.SparkSessionExtensions -import org.apache.spark.sql.catalyst.parser.extensions.PaimonSparkSqlExtensionsParser +import org.apache.spark.sql.paimon.shims.SparkShimLoader /** Spark session extension to extends the syntax and adds the rules. */ class PaimonSparkSessionExtensions extends (SparkSessionExtensions => Unit) { override def apply(extensions: SparkSessionExtensions): Unit = { // parser extensions - extensions.injectParser { case (_, parser) => new PaimonSparkSqlExtensionsParser(parser) } + extensions.injectParser { + case (_, parser) => SparkShimLoader.getSparkShim.createSparkParser(parser) + } // analyzer extensions extensions.injectResolutionRule(spark => new PaimonAnalysis(spark)) extensions.injectResolutionRule(spark => PaimonProcedureResolver(spark)) + extensions.injectResolutionRule(spark => PaimonViewResolver(spark)) + extensions.injectResolutionRule( + spark => SparkShimLoader.getSparkShim.createCustomResolution(spark)) extensions.injectResolutionRule(spark => PaimonIncompatibleResolutionRules(spark)) extensions.injectPostHocResolutionRule(spark => PaimonPostHocResolutionRules(spark)) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/leafnode/package.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/leafnode/package.scala index 5befb88dae43f..6ebab038480a4 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/leafnode/package.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/leafnode/package.scala @@ -18,7 +18,7 @@ package org.apache.paimon.spark -import org.apache.spark.sql.catalyst.plans.logical.{LeafCommand, LeafParsedStatement} +import org.apache.spark.sql.catalyst.plans.logical.{BinaryCommand, LeafCommand, LeafParsedStatement, UnaryCommand} import org.apache.spark.sql.execution.command.LeafRunnableCommand import org.apache.spark.sql.execution.datasources.v2.LeafV2CommandExec @@ -30,6 +30,9 @@ package object leafnode { trait PaimonLeafCommand extends LeafCommand - trait PaimonLeafV2CommandExec extends LeafV2CommandExec + trait PaimonUnaryCommand extends UnaryCommand + + trait PaimonBinaryCommand extends BinaryCommand + trait PaimonLeafV2CommandExec extends LeafV2CommandExec } 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 488d70e349356..fca0493ede287 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 @@ -22,15 +22,14 @@ import org.apache.paimon.{utils, Snapshot} import org.apache.paimon.catalog.{Catalog, Identifier} import org.apache.paimon.fs.Path import org.apache.paimon.manifest.{ManifestEntry, ManifestFile} -import org.apache.paimon.operation.OrphanFilesClean +import org.apache.paimon.operation.{CleanOrphanFilesResult, OrphanFilesClean} import org.apache.paimon.operation.OrphanFilesClean.retryReadingFiles import org.apache.paimon.table.FileStoreTable import org.apache.paimon.utils.SerializableConsumer import org.apache.spark.internal.Logging -import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.{functions, Dataset, SparkSession} import org.apache.spark.sql.catalyst.SQLConfHelper -import org.apache.spark.sql.functions.sum import java.util import java.util.Collections @@ -50,14 +49,18 @@ case class SparkOrphanFilesClean( with SQLConfHelper with Logging { - def doOrphanClean(): (Dataset[Long], Dataset[BranchAndManifestFile]) = { + def doOrphanClean(): (Dataset[(Long, Long)], Dataset[BranchAndManifestFile]) = { import spark.implicits._ val branches = validBranches() - val deletedInLocal = new AtomicLong(0) + val deletedFilesCountInLocal = new AtomicLong(0) + val deletedFilesLenInBytesInLocal = new AtomicLong(0) // snapshot and changelog files are the root of everything, so they are handled specially // here, and subsequently, we will not count their orphan files. - cleanSnapshotDir(branches, (_: Path) => deletedInLocal.incrementAndGet) + cleanSnapshotDir( + branches, + (_: Path) => deletedFilesCountInLocal.incrementAndGet, + size => deletedFilesLenInBytesInLocal.addAndGet(size)) val maxBranchParallelism = Math.min(branches.size(), parallelism) // find snapshots using branch and find manifests(manifest, index, statistics) using snapshot @@ -121,10 +124,10 @@ case class SparkOrphanFilesClean( .flatMap { dir => tryBestListingDirs(new Path(dir)).asScala.filter(oldEnough).map { - file => (file.getPath.getName, file.getPath.toUri.toString) + file => (file.getPath.getName, file.getPath.toUri.toString, file.getLen) } } - .toDF("name", "path") + .toDF("name", "path", "len") .repartition(parallelism) // use left anti to filter files which is not used @@ -132,21 +135,30 @@ case class SparkOrphanFilesClean( .join(usedFiles, $"name" === $"used_name", "left_anti") .mapPartitions { it => - var deleted = 0L + var deletedFilesCount = 0L + var deletedFilesLenInBytes = 0L + while (it.hasNext) { - val pathToClean = it.next().getString(1) - specifiedFileCleaner.accept(new Path(pathToClean)) + val fileInfo = it.next(); + val pathToClean = fileInfo.getString(1) + val deletedPath = new Path(pathToClean) + deletedFilesLenInBytes += fileInfo.getLong(2) + specifiedFileCleaner.accept(deletedPath) logInfo(s"Cleaned file: $pathToClean") - deleted += 1 + deletedFilesCount += 1 } - logInfo(s"Total cleaned files: $deleted"); - Iterator.single(deleted) + logInfo( + s"Total cleaned files: $deletedFilesCount, Total cleaned files len : $deletedFilesLenInBytes") + Iterator.single((deletedFilesCount, deletedFilesLenInBytes)) + } + val finalDeletedDataset = + if (deletedFilesCountInLocal.get() != 0 || deletedFilesLenInBytesInLocal.get() != 0) { + deleted.union( + spark.createDataset( + Seq((deletedFilesCountInLocal.get(), deletedFilesLenInBytesInLocal.get())))) + } else { + deleted } - val finalDeletedDataset = if (deletedInLocal.get() != 0) { - deleted.union(spark.createDataset(Seq(deletedInLocal.get()))) - } else { - deleted - } (finalDeletedDataset, usedManifestFiles) } @@ -169,7 +181,7 @@ object SparkOrphanFilesClean extends SQLConfHelper { tableName: String, olderThanMillis: Long, fileCleaner: SerializableConsumer[Path], - parallelismOpt: Integer): Long = { + parallelismOpt: Integer): CleanOrphanFilesResult = { val spark = SparkSession.active val parallelism = if (parallelismOpt == null) { Math.max(spark.sparkContext.defaultParallelism, conf.numShufflePartitions) @@ -192,7 +204,7 @@ object SparkOrphanFilesClean extends SQLConfHelper { table.asInstanceOf[FileStoreTable] } if (tables.isEmpty) { - return 0 + return new CleanOrphanFilesResult(0, 0) } val (deleted, waitToRelease) = tables.map { table => @@ -207,15 +219,15 @@ object SparkOrphanFilesClean extends SQLConfHelper { try { val result = deleted .reduce((l, r) => l.union(r)) - .toDF("deleted") - .agg(sum("deleted")) + .toDF("deletedFilesCount", "deletedFilesLenInBytes") + .agg(functions.sum("deletedFilesCount"), functions.sum("deletedFilesLenInBytes")) .head() - assert(result.schema.size == 1, result.schema) + assert(result.schema.size == 2, result.schema) if (result.isNullAt(0)) { // no files can be deleted - 0 + new CleanOrphanFilesResult(0, 0) } else { - result.getLong(0) + new CleanOrphanFilesResult(result.getLong(0), result.getLong(1)) } } finally { waitToRelease.foreach(_.unpersist()) 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 b44a66fce3ff1..7e61d71ac1835 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 @@ -44,7 +44,8 @@ private[spark] trait StreamHelper { var lastTriggerMillis: Long - private lazy val streamScan: StreamDataTableScan = table.newStreamScan() + private lazy val streamScan: StreamDataTableScan = + table.newStreamScan().dropStats().asInstanceOf[StreamDataTableScan] private lazy val partitionSchema: StructType = SparkTypeUtils.fromPaimonRowType(TypeUtils.project(table.rowType(), table.partitionKeys())) diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala index 4492d856ad509..cc49e787dc81c 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/PaimonUtils.scala @@ -20,11 +20,15 @@ package org.apache.spark.sql import org.apache.spark.executor.OutputMetrics import org.apache.spark.rdd.InputFileBlockHolder +import org.apache.spark.sql.catalyst.analysis.Resolver +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.expressions.{FieldReference, NamedReference} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.PartitioningUtils import org.apache.spark.util.{Utils => SparkUtils} /** @@ -87,4 +91,19 @@ object PaimonUtils { outputMetrics.setBytesWritten(bytesWritten) outputMetrics.setRecordsWritten(recordsWritten) } + + def normalizePartitionSpec[T]( + partitionSpec: Map[String, T], + partCols: StructType, + tblName: String, + resolver: Resolver): Map[String, T] = { + PartitioningUtils.normalizePartitionSpec(partitionSpec, partCols, tblName, resolver) + } + + def requireExactMatchedPartitionSpec( + tableName: String, + spec: TablePartitionSpec, + partitionColumnNames: Seq[String]): Unit = { + PartitioningUtils.requireExactMatchedPartitionSpec(tableName, spec, partitionColumnNames) + } } 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/AbstractPaimonSparkSqlExtensionsParser.scala similarity index 93% rename from paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/PaimonSparkSqlExtensionsParser.scala rename to paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/AbstractPaimonSparkSqlExtensionsParser.scala index dd0a48159b2ae..557b0735c74d7 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/AbstractPaimonSparkSqlExtensionsParser.scala @@ -18,12 +18,14 @@ package org.apache.spark.sql.catalyst.parser.extensions +import org.apache.paimon.spark.SparkProcedures + import org.antlr.v4.runtime._ import org.antlr.v4.runtime.atn.PredictionMode import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException} import org.antlr.v4.runtime.tree.TerminalNodeImpl import org.apache.spark.internal.Logging -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface} @@ -34,6 +36,8 @@ import org.apache.spark.sql.types.{DataType, StructType} import java.util.Locale +import scala.collection.JavaConverters._ + /* This file is based on source code from the Iceberg Project (http://iceberg.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. */ @@ -47,8 +51,8 @@ import java.util.Locale * @param delegate * The extension parser. */ -class PaimonSparkSqlExtensionsParser(val delegate: ParserInterface) - extends org.apache.spark.sql.paimon.shims.ParserInterface +abstract class AbstractPaimonSparkSqlExtensionsParser(val delegate: ParserInterface) + extends org.apache.spark.sql.catalyst.parser.ParserInterface with Logging { private lazy val substitutor = new VariableSubstitution() @@ -61,7 +65,7 @@ class PaimonSparkSqlExtensionsParser(val delegate: ParserInterface) parse(sqlTextAfterSubstitution)(parser => astBuilder.visit(parser.singleStatement())) .asInstanceOf[LogicalPlan] } else { - delegate.parsePlan(sqlText) + RewritePaimonViewCommands(SparkSession.active).apply(delegate.parsePlan(sqlText)) } } @@ -100,8 +104,15 @@ class PaimonSparkSqlExtensionsParser(val delegate: ParserInterface) .replaceAll("--.*?\\n", " ") .replaceAll("\\s+", " ") .replaceAll("/\\*.*?\\*/", " ") + .replaceAll("`", "") .trim() - normalized.startsWith("call") || isTagRefDdl(normalized) + isPaimonProcedure(normalized) || isTagRefDdl(normalized) + } + + // All builtin paimon procedures are under the 'sys' namespace + private def isPaimonProcedure(normalized: String): Boolean = { + normalized.startsWith("call") && + SparkProcedures.names().asScala.map("sys." + _).exists(normalized.contains) } private def isTagRefDdl(normalized: String): Boolean = { 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 b864894e7498c..a1289a5f0b50f 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,7 @@ package org.apache.spark.sql.catalyst.parser.extensions import org.apache.paimon.spark.catalyst.plans.logical -import org.apache.paimon.spark.catalyst.plans.logical.{CreateOrReplaceTagCommand, DeleteTagCommand, PaimonCallArgument, PaimonCallStatement, PaimonNamedArgument, PaimonPositionalArgument, RenameTagCommand, ShowTagsCommand, TagOptions} +import org.apache.paimon.spark.catalyst.plans.logical._ import org.apache.paimon.utils.TimeUtils import org.antlr.v4.runtime._ @@ -212,5 +212,16 @@ object CurrentOrigin { def get: Origin = value.get() def set(o: Origin): Unit = value.set(o) def reset(): Unit = value.set(Origin()) + + def withOrigin[A](o: Origin)(f: => A): A = { + // remember the previous one so it can be reset to this + // way withOrigin can be recursive + val previous = get + set(o) + val ret = + try f + finally { set(previous) } + ret + } } /* Apache Spark copy end */ diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.scala new file mode 100644 index 0000000000000..f69e5d92038ed --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonViewCommands.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.spark.sql.catalyst.parser.extensions + +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.spark.catalyst.plans.logical.{CreatePaimonView, DropPaimonView, ResolvedIdentifier, ShowPaimonViews} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{CTESubstitution, ResolvedNamespace, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{CatalogManager, LookupCatalog} + +case class RewritePaimonViewCommands(spark: SparkSession) + extends Rule[LogicalPlan] + with LookupCatalog { + + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + + // A new member was added to CreatePaimonView since spark4.0, + // unapply pattern matching is not used here to ensure compatibility across multiple spark versions. + case c: CreateView => + ResolvedIdent + .unapply(c.child) + .map { + resolved => + CreatePaimonView( + child = resolved, + queryText = c.originalText.get, + query = CTESubstitution.apply(c.query), + columnAliases = c.userSpecifiedColumns.map(_._1), + columnComments = c.userSpecifiedColumns.map(_._2.orElse(None)), + comment = c.comment, + properties = c.properties, + allowExisting = c.allowExisting, + replace = c.replace + ) + } + .getOrElse(c) + + case DropView(ResolvedIdent(resolved), ifExists: Boolean) => + DropPaimonView(resolved, ifExists) + + case ShowViews(_, pattern, output) if catalogManager.currentCatalog.isInstanceOf[SupportView] => + ShowPaimonViews( + ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + pattern, + output) + } + + private object ResolvedIdent { + def unapply(unresolved: Any): Option[ResolvedIdentifier] = unresolved match { + case UnresolvedIdentifier(CatalogAndIdentifier(viewCatalog: SupportView, ident), _) => + Some(ResolvedIdentifier(viewCatalog, ident)) + case _ => + None + } + } +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogImplicits.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogImplicits.scala new file mode 100644 index 0000000000000..f1f20fb6fb31d --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogImplicits.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connector.catalog + +object PaimonCatalogImplicits { + + import CatalogV2Implicits._ + + implicit class PaimonCatalogHelper(plugin: CatalogPlugin) extends CatalogHelper(plugin) + + implicit class PaimonNamespaceHelper(namespace: Array[String]) extends NamespaceHelper(namespace) + +// implicit class PaimonTableHelper(table: Table) extends TableHelper(table) +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala index 265c828661952..5db6894ba093b 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/connector/catalog/PaimonCatalogUtils.scala @@ -22,11 +22,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.ExternalCatalog +import org.apache.spark.sql.connector.catalog.CatalogV2Util import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION -import org.apache.spark.util.Utils - -import scala.reflect.ClassTag -import scala.util.control.NonFatal +import org.apache.spark.sql.paimon.ReflectUtils object PaimonCatalogUtils { @@ -37,22 +35,12 @@ object PaimonCatalogUtils { } else { "org.apache.spark.sql.catalyst.catalog.InMemoryCatalog" } - reflect[ExternalCatalog, SparkConf, Configuration](externalCatalogClassName, conf, hadoopConf) + ReflectUtils.reflect[ExternalCatalog, SparkConf, Configuration]( + externalCatalogClassName, + conf, + hadoopConf) } - private def reflect[T, Arg1 <: AnyRef, Arg2 <: AnyRef]( - className: String, - ctorArg1: Arg1, - ctorArg2: Arg2)(implicit ctorArgTag1: ClassTag[Arg1], ctorArgTag2: ClassTag[Arg2]): T = { - try { - val clazz = Utils.classForName(className) - val ctor = clazz.getDeclaredConstructor(ctorArgTag1.runtimeClass, ctorArgTag2.runtimeClass) - val args = Array[AnyRef](ctorArg1, ctorArg2) - ctor.newInstance(args: _*).asInstanceOf[T] - } catch { - case NonFatal(e) => - throw new IllegalArgumentException(s"Error while instantiating '$className':", e) - } - } + val TABLE_RESERVED_PROPERTIES: Seq[String] = CatalogV2Util.TABLE_RESERVED_PROPERTIES } diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/ReflectUtils.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/ReflectUtils.scala new file mode 100644 index 0000000000000..bedac542ab8bc --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/ReflectUtils.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.spark.sql.paimon + +import org.apache.spark.util.Utils + +import scala.reflect.ClassTag +import scala.util.control.NonFatal + +object ReflectUtils { + + def reflect[T, Arg1 <: AnyRef, Arg2 <: AnyRef](className: String, ctorArg1: Arg1, ctorArg2: Arg2)( + implicit + ctorArgTag1: ClassTag[Arg1], + ctorArgTag2: ClassTag[Arg2]): T = { + try { + val clazz = Utils.classForName(className) + val ctor = clazz.getDeclaredConstructor(ctorArgTag1.runtimeClass, ctorArgTag2.runtimeClass) + val args = Array[AnyRef](ctorArg1, ctorArg2) + ctor.newInstance(args: _*).asInstanceOf[T] + } catch { + case NonFatal(e) => + throw new IllegalArgumentException(s"Error while instantiating '$className':", e) + } + } + +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala new file mode 100644 index 0000000000000..334bd6e93180e --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShim.scala @@ -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.spark.sql.paimon.shims + +import org.apache.paimon.spark.data.{SparkArrayData, SparkInternalRow} +import org.apache.paimon.types.{DataType, RowType} + +import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.StructType + +import java.util.{Map => JMap} + +/** + * A spark shim trait. It declare methods which have incompatible implementations between Spark 3 + * and Spark 4. The specific SparkShim implementation will be loaded through Service Provider + * Interface. + */ +trait SparkShim { + + def createSparkParser(delegate: ParserInterface): ParserInterface + + def createCustomResolution(spark: SparkSession): Rule[LogicalPlan] + + def createSparkInternalRow(rowType: RowType): SparkInternalRow + + def createSparkArrayData(elementType: DataType): SparkArrayData + + def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean + + def createTable( + tableCatalog: TableCatalog, + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: JMap[String, String]): Table + + def column(expr: Expression): Column + + def convertToExpression(spark: SparkSession, column: Column): Expression + +} diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.scala new file mode 100644 index 0000000000000..920896547a1e2 --- /dev/null +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/spark/sql/paimon/shims/SparkShimLoader.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.spark.sql.paimon.shims + +import java.util.ServiceLoader + +import scala.collection.JavaConverters._ + +/** Load a [[SparkShim]]'s implementation. */ +object SparkShimLoader { + + private lazy val sparkShim: SparkShim = loadSparkShim() + + def getSparkShim: SparkShim = { + sparkShim + } + + private def loadSparkShim(): SparkShim = { + val shims = ServiceLoader.load(classOf[SparkShim]).asScala + if (shims.isEmpty) { + throw new IllegalStateException("No available spark shim here.") + } else if (shims.size > 1) { + throw new IllegalStateException("Found more than one spark shim here.") + } + shims.head + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala b/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala deleted file mode 100644 index 2a689b631acd3..0000000000000 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.paimon.spark.sql - -import org.apache.paimon.spark.PaimonSparkTestBase - -import org.apache.spark.sql.{DataFrame, Row} - -class TableValuedFunctionsTest extends PaimonSparkTestBase { - - withPk.foreach { - hasPk => - bucketModes.foreach { - bucket => - test(s"incremental query: hasPk: $hasPk, bucket: $bucket") { - val prop = if (hasPk) { - s"'primary-key'='a,b', 'bucket' = '$bucket' " - } else if (bucket != -1) { - s"'bucket-key'='b', 'bucket' = '$bucket' " - } else { - "'write-only'='true'" - } - - spark.sql(s""" - |CREATE TABLE T (a INT, b INT, c STRING) - |USING paimon - |TBLPROPERTIES ($prop) - |PARTITIONED BY (a) - |""".stripMargin) - - spark.sql("INSERT INTO T values (1, 1, '1'), (2, 2, '2')") - spark.sql("INSERT INTO T VALUES (1, 3, '3'), (2, 4, '4')") - spark.sql("INSERT INTO T VALUES (1, 5, '5'), (1, 7, '7')") - - checkAnswer( - incrementalDF("T", 0, 1).orderBy("a", "b"), - Row(1, 1, "1") :: Row(2, 2, "2") :: Nil) - checkAnswer( - spark.sql("SELECT * FROM paimon_incremental_query('T', '0', '1') ORDER BY a, b"), - Row(1, 1, "1") :: Row(2, 2, "2") :: Nil) - - checkAnswer( - incrementalDF("T", 1, 2).orderBy("a", "b"), - Row(1, 3, "3") :: Row(2, 4, "4") :: Nil) - checkAnswer( - spark.sql("SELECT * FROM paimon_incremental_query('T', '1', '2') ORDER BY a, b"), - Row(1, 3, "3") :: Row(2, 4, "4") :: Nil) - - checkAnswer( - incrementalDF("T", 2, 3).orderBy("a", "b"), - Row(1, 5, "5") :: Row(1, 7, "7") :: Nil) - checkAnswer( - spark.sql("SELECT * FROM paimon_incremental_query('T', '2', '3') ORDER BY a, b"), - Row(1, 5, "5") :: Row(1, 7, "7") :: Nil) - - checkAnswer( - incrementalDF("T", 1, 3).orderBy("a", "b"), - Row(1, 3, "3") :: Row(1, 5, "5") :: Row(1, 7, "7") :: Row(2, 4, "4") :: Nil - ) - checkAnswer( - spark.sql("SELECT * FROM paimon_incremental_query('T', '1', '3') ORDER BY a, b"), - Row(1, 3, "3") :: Row(1, 5, "5") :: Row(1, 7, "7") :: Row(2, 4, "4") :: Nil) - } - } - } - - private def incrementalDF(tableIdent: String, start: Int, end: Int): DataFrame = { - spark.read - .format("paimon") - .option("incremental-between", s"$start,$end") - .table(tableIdent) - } -} diff --git a/paimon-spark/paimon-spark-ut/pom.xml b/paimon-spark/paimon-spark-ut/pom.xml new file mode 100644 index 0000000000000..0a18405964875 --- /dev/null +++ b/paimon-spark/paimon-spark-ut/pom.xml @@ -0,0 +1,180 @@ + + + + 4.0.0 + + + org.apache.paimon + paimon-spark + 1.0-SNAPSHOT + + + paimon-spark-ut + Paimon : Spark : UT + + + ${paimon-spark-common.spark.version} + ${paimon.shade.jackson.version} + + + + + org.apache.paimon + ${paimon-sparkx-common} + ${project.version} + test + + + + org.apache.paimon + paimon-spark-common_${scala.binary.version} + ${project.version} + test + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + test + + + com.fasterxml.jackson.core + * + + + com.fasterxml.jackson.module + * + + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + test + + + com.fasterxml.jackson.core + * + + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + tests + test + + + com.fasterxml.jackson.core + * + + + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${spark.version} + tests + test + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + tests + test + + + com.fasterxml.jackson.core + * + + + com.fasterxml.jackson.module + * + + + + + + org.apache.spark + spark-hive_${scala.binary.version} + ${spark.version} + test + + + com.fasterxml.jackson.core + * + + + com.google.protobuf + protobuf-java + + + + + + org.apache.spark + spark-avro_${scala.binary.version} + ${spark.version} + test + + + + com.fasterxml.jackson.module + jackson-module-scala_${scala.binary.version} + ${jackson.version} + test + + + + com.google.protobuf + protobuf-java + ${protobuf-java.version} + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + test-compile + + test-jar + + + + + + + + diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java similarity index 74% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java index 68cf91b8ec7b7..45ccd06479f2b 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkCatalogWithHiveTest.java @@ -19,7 +19,9 @@ package org.apache.paimon.spark; import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.hive.TestHiveMetastore; +import org.apache.paimon.table.FileStoreTableFactory; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; @@ -32,6 +34,7 @@ import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Base tests for spark read. */ @@ -136,4 +139,46 @@ public void testSpecifyHiveConfDir(@TempDir java.nio.file.Path tempDir) { spark.close(); } + + @Test + public void testCreateExternalTable(@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.warehouse", + warehousePath.toString()) + .master("local[2]") + .getOrCreate(); + + spark.sql("CREATE DATABASE IF NOT EXISTS test_db"); + spark.sql("USE spark_catalog.test_db"); + + // create hive external table + spark.sql("CREATE EXTERNAL TABLE t1 (a INT, bb INT, c STRING)"); + + // drop hive external table + spark.sql("DROP TABLE t1"); + + // file system table exists + assertThatCode( + () -> + FileStoreTableFactory.create( + LocalFileIO.create(), + new Path( + warehousePath, + String.format("%s.db/%s", "test_db", "t1")))) + .doesNotThrowAnyException(); + + spark.close(); + } } diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java similarity index 99% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java index 7da8f2e6b7fe0..0360def685b6b 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java @@ -160,7 +160,8 @@ protected void foreachIndexReader(Consumer consumer) CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + null); Table table = fileSystemCatalog.getTable(Identifier.create("db", "T")); ReadBuilder readBuilder = table.newReadBuilder(); diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFilterConverterTest.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogTest.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkGenericCatalogWithHiveTest.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java similarity index 95% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java index b98213c0e662b..1117ad58c7376 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkInternalRowTest.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.data.Timestamp; +import org.apache.paimon.spark.data.SparkInternalRow; import org.apache.paimon.utils.DateTimeUtils; import org.apache.spark.sql.catalyst.CatalystTypeConverters; @@ -95,7 +96,7 @@ public void test() { SparkTypeUtils.fromPaimonType(ALL_TYPES))); org.apache.spark.sql.Row sparkRow = (org.apache.spark.sql.Row) - sparkConverter.apply(new SparkInternalRow(ALL_TYPES).replace(rowData)); + sparkConverter.apply(SparkInternalRow.create(ALL_TYPES).replace(rowData)); String expected = "1," @@ -122,7 +123,8 @@ public void test() { SparkRow sparkRowData = new SparkRow(ALL_TYPES, sparkRow); sparkRow = (org.apache.spark.sql.Row) - sparkConverter.apply(new SparkInternalRow(ALL_TYPES).replace(sparkRowData)); + sparkConverter.apply( + SparkInternalRow.create(ALL_TYPES).replace(sparkRowData)); assertThat(sparkRowToString(sparkRow)).isEqualTo(expected); TimeZone.setDefault(tz); } diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadITCase.java similarity index 96% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadITCase.java index b4565447c6fc7..b00267410a7fd 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadITCase.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadITCase.java @@ -190,17 +190,20 @@ public void testCreateTableAs() { spark.sql("INSERT INTO partitionedTable VALUES(1,'aaa','bbb')"); spark.sql( "CREATE TABLE partitionedTableAs PARTITIONED BY (a) AS SELECT * FROM partitionedTable"); + Path tablePath = new Path(warehousePath, "default.db/partitionedTableAs"); assertThat(spark.sql("SHOW CREATE TABLE partitionedTableAs").collectAsList().toString()) .isEqualTo( String.format( "[[%s" + "PARTITIONED BY (a)\n" + + "LOCATION '%s'\n" + "TBLPROPERTIES (\n" + " 'path' = '%s')\n" + "]]", showCreateString( "partitionedTableAs", "a BIGINT", "b STRING", "c STRING"), - new Path(warehousePath, "default.db/partitionedTableAs"))); + tablePath, + tablePath)); List resultPartition = spark.sql("SELECT * FROM partitionedTableAs").collectAsList(); assertThat(resultPartition.stream().map(Row::toString)) .containsExactlyInAnyOrder("[1,aaa,bbb]"); @@ -217,17 +220,20 @@ public void testCreateTableAs() { spark.sql("INSERT INTO testTable VALUES(1,'a','b')"); spark.sql( "CREATE TABLE testTableAs TBLPROPERTIES ('file.format' = 'parquet') AS SELECT * FROM testTable"); + tablePath = new Path(warehousePath, "default.db/testTableAs"); assertThat(spark.sql("SHOW CREATE TABLE testTableAs").collectAsList().toString()) .isEqualTo( String.format( "[[%s" + + "LOCATION '%s'\n" + "TBLPROPERTIES (\n" + " 'file.format' = 'parquet',\n" + " 'path' = '%s')\n" + "]]", showCreateString( "testTableAs", "a BIGINT", "b VARCHAR(10)", "c CHAR(10)"), - new Path(warehousePath, "default.db/testTableAs"))); + tablePath, + tablePath)); List resultProp = spark.sql("SELECT * FROM testTableAs").collectAsList(); assertThat(resultProp.stream().map(Row::toString)) @@ -245,13 +251,17 @@ public void testCreateTableAs() { + "COMMENT 'table comment'"); spark.sql("INSERT INTO t_pk VALUES(1,'aaa','bbb')"); spark.sql("CREATE TABLE t_pk_as TBLPROPERTIES ('primary-key' = 'a') AS SELECT * FROM t_pk"); + tablePath = new Path(warehousePath, "default.db/t_pk_as"); assertThat(spark.sql("SHOW CREATE TABLE t_pk_as").collectAsList().toString()) .isEqualTo( String.format( - "[[%sTBLPROPERTIES (\n 'path' = '%s',\n 'primary-key' = 'a')\n]]", + "[[%s" + + "LOCATION '%s'\n" + + "TBLPROPERTIES (\n 'path' = '%s',\n 'primary-key' = 'a')\n]]", showCreateString( "t_pk_as", "a BIGINT NOT NULL", "b STRING", "c STRING"), - new Path(warehousePath, "default.db/t_pk_as"))); + tablePath, + tablePath)); List resultPk = spark.sql("SELECT * FROM t_pk_as").collectAsList(); assertThat(resultPk.stream().map(Row::toString)).containsExactlyInAnyOrder("[1,aaa,bbb]"); @@ -270,11 +280,13 @@ public void testCreateTableAs() { spark.sql("INSERT INTO t_all VALUES(1,2,'bbb','2020-01-01','12')"); spark.sql( "CREATE TABLE t_all_as PARTITIONED BY (dt) TBLPROPERTIES ('primary-key' = 'dt,hh') AS SELECT * FROM t_all"); + tablePath = new Path(warehousePath, "default.db/t_all_as"); assertThat(spark.sql("SHOW CREATE TABLE t_all_as").collectAsList().toString()) .isEqualTo( String.format( "[[%s" + "PARTITIONED BY (dt)\n" + + "LOCATION '%s'\n" + "TBLPROPERTIES (\n" + " 'path' = '%s',\n" + " 'primary-key' = 'dt,hh')\n" @@ -286,7 +298,8 @@ public void testCreateTableAs() { "behavior STRING", "dt STRING NOT NULL", "hh STRING NOT NULL"), - new Path(warehousePath, "default.db/t_all_as"))); + tablePath, + tablePath)); List resultAll = spark.sql("SELECT * FROM t_all_as").collectAsList(); assertThat(resultAll.stream().map(Row::toString)) .containsExactlyInAnyOrder("[1,2,bbb,2020-01-01,12]"); @@ -363,12 +376,14 @@ public void testShowCreateTable() { + " 'k1' = 'v1'\n" + ")"); + Path tablePath = new Path(warehousePath, "default.db/tbl"); assertThat(spark.sql("SHOW CREATE TABLE tbl").collectAsList().toString()) .isEqualTo( String.format( "[[%s" + "PARTITIONED BY (b)\n" + "COMMENT 'tbl comment'\n" + + "LOCATION '%s'\n" + "TBLPROPERTIES (\n" + " 'k1' = 'v1',\n" + " 'path' = '%s',\n" @@ -377,7 +392,8 @@ public void testShowCreateTable() { "tbl", "a INT NOT NULL COMMENT 'a comment'", "b STRING NOT NULL"), - new Path(warehousePath, "default.db/tbl"))); + tablePath, + tablePath)); } @Test diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkReadTestBase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkS3ITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java similarity index 78% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java index 771ddc62878d1..fb4dab38ed940 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkSchemaEvolutionITCase.java @@ -789,6 +789,89 @@ public void testAddAndDropNestedColumn(String formatType) { "[5,[53,[503,500.03,5003],five]]"); } + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testAddAndDropNestedColumnInArray(String formatType) { + String tableName = "testAddNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v ARRAY>) " + + "TBLPROPERTIES ('bucket' = '1', 'primary-key' = 'k', 'file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, ARRAY(STRUCT('apple', 100), STRUCT('banana', 101))), " + + "(2, ARRAY(STRUCT('cat', 200), STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,WrappedArray([apple,100], [banana,101])]", + "[2,WrappedArray([cat,200], [dog,201])]"); + + spark.sql( + "ALTER TABLE paimon.default." + + tableName + + " ADD COLUMN v.element.f3 STRING AFTER f2"); + spark.sql("ALTER TABLE paimon.default." + tableName + " DROP COLUMN v.element.f1"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, ARRAY(STRUCT(110, 'APPLE'), STRUCT(111, 'BANANA'))), " + + "(3, ARRAY(STRUCT(310, 'FLOWER')))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,WrappedArray([110,APPLE], [111,BANANA])]", + "[2,WrappedArray([200,null], [201,null])]", + "[3,WrappedArray([310,FLOWER])]"); + } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testAddAndDropNestedColumnInMap(String formatType) { + String tableName = "testAddNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v MAP>) " + + "TBLPROPERTIES ('bucket' = '1', 'primary-key' = 'k', 'file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, MAP(10, STRUCT('apple', 100), 20, STRUCT('banana', 101))), " + + "(2, MAP(10, STRUCT('cat', 200), 20, STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT k, v[10].f1, v[10].f2 FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[1,apple,100]", "[2,cat,200]"); + + spark.sql( + "ALTER TABLE paimon.default." + + tableName + + " ADD COLUMN v.value.f3 STRING AFTER f2"); + spark.sql("ALTER TABLE paimon.default." + tableName + " DROP COLUMN v.value.f1"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, MAP(10, STRUCT(110, 'APPLE'), 20, STRUCT(111, 'BANANA'))), " + + "(3, MAP(10, STRUCT(310, 'FLOWER')))"); + assertThat( + spark.sql("SELECT k, v[10].f2, v[10].f3 FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[1,110,APPLE]", "[2,200,null]", "[3,310,FLOWER]"); + } + @ParameterizedTest() @ValueSource(strings = {"orc", "avro", "parquet"}) public void testRenameNestedColumn(String formatType) { @@ -818,6 +901,67 @@ public void testRenameNestedColumn(String formatType) { .containsExactlyInAnyOrder("[apple,1]", "[banana,2]"); } + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testRenameNestedColumnInArray(String formatType) { + String tableName = "testRenameNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v ARRAY>) " + + "TBLPROPERTIES ('file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, ARRAY(STRUCT('apple', 100), STRUCT('banana', 101))), " + + "(2, ARRAY(STRUCT('cat', 200), STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT v[0].f1, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[cat,2]"); + + spark.sql( + "ALTER TABLE paimon.default." + tableName + " RENAME COLUMN v.element.f1 to f100"); + assertThat( + spark.sql("SELECT v[0].f100, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[cat,2]"); + } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testRenameNestedColumnInMap(String formatType) { + String tableName = "testRenameNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v MAP>) " + + "TBLPROPERTIES ('file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, MAP(10, STRUCT('apple', 100), 20, STRUCT('banana', 101))), " + + "(2, MAP(10, STRUCT('cat', 200), 20, STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT v[10].f1, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[cat,2]"); + + spark.sql("ALTER TABLE paimon.default." + tableName + " RENAME COLUMN v.value.f1 to f100"); + assertThat( + spark.sql("SELECT v[10].f100, k FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[apple,1]", "[cat,2]"); + } + @ParameterizedTest() @ValueSource(strings = {"orc", "avro", "parquet"}) public void testUpdateNestedColumnType(String formatType) { @@ -850,4 +994,84 @@ public void testUpdateNestedColumnType(String formatType) { .map(Row::toString)) .containsExactlyInAnyOrder("[101,1]", "[200,2]", "[3000000000000,3]"); } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testUpdateNestedColumnTypeInArray(String formatType) { + String tableName = "testRenameNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v ARRAY>) " + + "TBLPROPERTIES ('bucket' = '1', 'primary-key' = 'k', 'file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, ARRAY(STRUCT('apple', 100), STRUCT('banana', 101))), " + + "(2, ARRAY(STRUCT('cat', 200), STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,WrappedArray([apple,100], [banana,101])]", + "[2,WrappedArray([cat,200], [dog,201])]"); + + spark.sql( + "ALTER TABLE paimon.default." + + tableName + + " CHANGE COLUMN v.element.f2 f2 BIGINT"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, ARRAY(STRUCT('APPLE', 1000000000000), STRUCT('BANANA', 111))), " + + "(3, ARRAY(STRUCT('FLOWER', 3000000000000)))"); + assertThat( + spark.sql("SELECT * FROM paimon.default." + tableName).collectAsList() + .stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,WrappedArray([APPLE,1000000000000], [BANANA,111])]", + "[2,WrappedArray([cat,200], [dog,201])]", + "[3,WrappedArray([FLOWER,3000000000000])]"); + } + + @ParameterizedTest() + @ValueSource(strings = {"orc", "avro", "parquet"}) + public void testUpdateNestedColumnTypeInMap(String formatType) { + String tableName = "testRenameNestedColumnTable"; + spark.sql( + "CREATE TABLE paimon.default." + + tableName + + " (k INT NOT NULL, v MAP>) " + + "TBLPROPERTIES ('bucket' = '1', 'primary-key' = 'k', 'file.format' = '" + + formatType + + "')"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, MAP(10, STRUCT('apple', 100), 20, STRUCT('banana', 101))), " + + "(2, MAP(10, STRUCT('cat', 200), 20, STRUCT('dog', 201)))"); + assertThat( + spark.sql("SELECT k, v[10].f1, v[10].f2 FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder("[1,apple,100]", "[2,cat,200]"); + + spark.sql( + "ALTER TABLE paimon.default." + tableName + " CHANGE COLUMN v.value.f2 f2 BIGINT"); + spark.sql( + "INSERT INTO paimon.default." + + tableName + + " VALUES (1, MAP(10, STRUCT('APPLE', 1000000000000), 20, STRUCT('BANANA', 111))), " + + "(3, MAP(10, STRUCT('FLOWER', 3000000000000)))"); + assertThat( + spark.sql("SELECT k, v[10].f1, v[10].f2 FROM paimon.default." + tableName) + .collectAsList().stream() + .map(Row::toString)) + .containsExactlyInAnyOrder( + "[1,APPLE,1000000000000]", "[2,cat,200]", "[3,FLOWER,3000000000000]"); + } } diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTimeTravelITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTimeTravelITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTimeTravelITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTimeTravelITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTimeTravelWithDataFrameITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTimeTravelWithDataFrameITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTimeTravelWithDataFrameITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTimeTravelWithDataFrameITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTypeTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTypeTest.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkTypeTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkTypeTest.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteWithKyroITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteWithKyroITCase.java similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/SparkWriteWithKyroITCase.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteWithKyroITCase.java diff --git a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java similarity index 80% rename from paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java rename to paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java index 61e06016cbd34..e4e571e96bc95 100644 --- a/paimon-spark/paimon-spark-common/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/extensions/CallStatementParserTest.java @@ -79,14 +79,37 @@ public void stopSparkSession() { } } + @Test + public void testDelegateUnsupportedProcedure() { + assertThatThrownBy(() -> parser.parsePlan("CALL cat.d.t()")) + .isInstanceOf(ParseException.class) + .satisfies( + exception -> { + ParseException parseException = (ParseException) exception; + assertThat(parseException.getErrorClass()) + .isEqualTo("PARSE_SYNTAX_ERROR"); + assertThat(parseException.getMessageParameters().get("error")) + .isEqualTo("'CALL'"); + }); + } + + @Test + public void testCallWithBackticks() throws ParseException { + PaimonCallStatement call = + (PaimonCallStatement) parser.parsePlan("CALL cat.`sys`.`rollback`()"); + assertThat(JavaConverters.seqAsJavaList(call.name())) + .isEqualTo(Arrays.asList("cat", "sys", "rollback")); + assertThat(call.args().size()).isEqualTo(0); + } + @Test public void testCallWithNamedArguments() throws ParseException { PaimonCallStatement callStatement = (PaimonCallStatement) parser.parsePlan( - "CALL catalog.system.named_args_func(arg1 => 1, arg2 => 'test', arg3 => true)"); + "CALL catalog.sys.rollback(arg1 => 1, arg2 => 'test', arg3 => true)"); assertThat(JavaConverters.seqAsJavaList(callStatement.name())) - .isEqualTo(Arrays.asList("catalog", "system", "named_args_func")); + .isEqualTo(Arrays.asList("catalog", "sys", "rollback")); assertThat(callStatement.args().size()).isEqualTo(3); assertArgument(callStatement, 0, "arg1", 1, DataTypes.IntegerType); assertArgument(callStatement, 1, "arg2", "test", DataTypes.StringType); @@ -98,11 +121,11 @@ public void testCallWithPositionalArguments() throws ParseException { PaimonCallStatement callStatement = (PaimonCallStatement) parser.parsePlan( - "CALL catalog.system.positional_args_func(1, '${spark.sql.extensions}', 2L, true, 3.0D, 4" + "CALL catalog.sys.rollback(1, '${spark.sql.extensions}', 2L, true, 3.0D, 4" + ".0e1,500e-1BD, " + "TIMESTAMP '2017-02-03T10:37:30.00Z')"); assertThat(JavaConverters.seqAsJavaList(callStatement.name())) - .isEqualTo(Arrays.asList("catalog", "system", "positional_args_func")); + .isEqualTo(Arrays.asList("catalog", "sys", "rollback")); assertThat(callStatement.args().size()).isEqualTo(8); assertArgument(callStatement, 0, 1, DataTypes.IntegerType); assertArgument( @@ -127,9 +150,9 @@ public void testCallWithPositionalArguments() throws ParseException { public void testCallWithMixedArguments() throws ParseException { PaimonCallStatement callStatement = (PaimonCallStatement) - parser.parsePlan("CALL catalog.system.mixed_function(arg1 => 1, 'test')"); + parser.parsePlan("CALL catalog.sys.rollback(arg1 => 1, 'test')"); assertThat(JavaConverters.seqAsJavaList(callStatement.name())) - .isEqualTo(Arrays.asList("catalog", "system", "mixed_function")); + .isEqualTo(Arrays.asList("catalog", "sys", "rollback")); assertThat(callStatement.args().size()).isEqualTo(2); assertArgument(callStatement, 0, "arg1", 1, DataTypes.IntegerType); assertArgument(callStatement, 1, "test", DataTypes.StringType); @@ -137,9 +160,9 @@ public void testCallWithMixedArguments() throws ParseException { @Test public void testCallWithParseException() { - assertThatThrownBy(() -> parser.parsePlan("CALL catalog.system func abc")) + assertThatThrownBy(() -> parser.parsePlan("CALL catalog.sys.rollback abc")) .isInstanceOf(PaimonParseException.class) - .hasMessageContaining("missing '(' at 'func'"); + .hasMessageContaining("missing '(' at 'abc'"); } private void assertArgument( diff --git a/paimon-spark/paimon-spark-common/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/paimon-spark/paimon-spark-ut/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension rename to paimon-spark/paimon-spark-ut/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension diff --git a/paimon-spark/paimon-spark-common/src/test/resources/hive-site.xml b/paimon-spark/paimon-spark-ut/src/test/resources/hive-site.xml similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/resources/hive-site.xml rename to paimon-spark/paimon-spark-ut/src/test/resources/hive-site.xml diff --git a/paimon-spark/paimon-spark-common/src/test/resources/log4j2-test.properties b/paimon-spark/paimon-spark-ut/src/test/resources/log4j2-test.properties similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/resources/log4j2-test.properties rename to paimon-spark/paimon-spark-ut/src/test/resources/log4j2-test.properties diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala index ccd705e269675..6d2ffea04df54 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonHiveTestBase.scala @@ -18,12 +18,11 @@ package org.apache.paimon.spark -import org.apache.paimon.Snapshot import org.apache.paimon.hive.TestHiveMetastore import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf -import org.apache.spark.paimon.Utils +import org.apache.spark.sql.paimon.Utils import java.io.File diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala similarity index 98% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala index 63203122ac40a..61bf5524942df 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala @@ -18,6 +18,7 @@ package org.apache.paimon.spark +import org.apache.spark.SparkConf import org.apache.spark.sql.{Dataset, Row} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions.{col, mean, window} @@ -27,6 +28,10 @@ import java.sql.Date class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.sql.catalog.paimon.cache-enabled", "false") + } + import testImplicits._ test("Paimon Sink: forEachBatch") { diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala similarity index 79% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala index 9b4a344259a93..867b3e5e33379 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSparkTestBase.scala @@ -25,11 +25,12 @@ import org.apache.paimon.spark.sql.{SparkVersionSupport, WithTableOptions} import org.apache.paimon.table.FileStoreTable import org.apache.spark.SparkConf -import org.apache.spark.paimon.Utils import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{Identifier => SparkIdentifier} import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.paimon.Utils import org.apache.spark.sql.test.SharedSparkSession import org.scalactic.source.Position import org.scalatest.Tag @@ -65,7 +66,6 @@ class PaimonSparkTestBase super.sparkConf .set("spark.sql.catalog.paimon", classOf[SparkCatalog].getName) .set("spark.sql.catalog.paimon.warehouse", tempDBDir.getCanonicalPath) - .set("spark.sql.catalog.paimon.cache-enabled", "false") .set("spark.sql.extensions", classOf[PaimonSparkSessionExtensions].getName) .set("spark.serializer", serializer) } @@ -105,7 +105,7 @@ class PaimonSparkTestBase } protected def withTimeZone(timeZone: String)(f: => Unit): Unit = { - withSQLConf("spark.sql.session.timeZone" -> timeZone) { + withSparkSQLConf("spark.sql.session.timeZone" -> timeZone) { val originTimeZone = TimeZone.getDefault try { TimeZone.setDefault(TimeZone.getTimeZone(timeZone)) @@ -116,6 +116,40 @@ class PaimonSparkTestBase } } + // Since SPARK-46227 has changed the definition of withSQLConf that resulted in + // incompatibility between the Spark3.x and Spark4.x, So Paimon declare a separate method + // to provide the same function. + protected def withSparkSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + withSparkSQLConf0(pairs: _*)(f) + } + + private def withSparkSQLConf0(pairs: (String, String)*)(f: => Unit): Unit = { + val conf = SQLConf.get + val (keys, values) = pairs.unzip + val currentValues = keys.map { + key => + if (conf.contains(key)) { + Some(conf.getConfString(key)) + } else { + None + } + } + (keys, values).zipped.foreach { + (k, v) => + if (SQLConf.isStaticConfigKey(k)) { + throw new RuntimeException(s"Cannot modify the value of a static config: $k") + } + conf.setConfString(k, v) + } + try f + finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => conf.setConfString(key, value) + case (key, None) => conf.unsetConf(key) + } + } + } + override def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit pos: Position): Unit = { println(testName) diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonTableTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonTableTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonTableTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala index fc787246f9f12..a3223446f644b 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala @@ -26,7 +26,6 @@ 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._ @@ -35,7 +34,7 @@ import scala.collection.mutable class ScanHelperTest extends PaimonSparkTestBase { test("Paimon: reshuffle splits") { - withSQLConf(("spark.sql.leafNodeDefaultParallelism", "20")) { + withSparkSQLConf(("spark.sql.leafNodeDefaultParallelism", "20")) { val splitNum = 5 val fileNum = 100 diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactManifestProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala similarity index 93% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala index 130860c8351e8..31f78f61c20d9 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala @@ -39,6 +39,56 @@ abstract class CompactProcedureTestBase extends PaimonSparkTestBase with StreamT import testImplicits._ + // ----------------------- Minor Compact ----------------------- + + test("Paimon Procedure: compact aware bucket pk table with minor compact strategy") { + withTable("T") { + spark.sql(s""" + |CREATE TABLE T (id INT, value STRING, pt STRING) + |TBLPROPERTIES ('primary-key'='id, pt', 'bucket'='1', 'write-only'='true') + |PARTITIONED BY (pt) + |""".stripMargin) + + val table = loadTable("T") + + spark.sql(s"INSERT INTO T VALUES (1, 'a', 'p1'), (2, 'b', 'p2')") + spark.sql(s"INSERT INTO T VALUES (3, 'c', 'p1'), (4, 'd', 'p2')") + + Assertions.assertThat(lastSnapshotCommand(table).equals(CommitKind.APPEND)).isTrue + Assertions.assertThat(lastSnapshotId(table)).isEqualTo(2) + + spark.sql( + "CALL sys.compact(table => 'T', compact_strategy => 'minor'," + + "options => 'num-sorted-run.compaction-trigger=3')") + + // Due to the limitation of parameter 'num-sorted-run.compaction-trigger' = 3, so compact is not + // performed. + Assertions.assertThat(lastSnapshotCommand(table).equals(CommitKind.APPEND)).isTrue + Assertions.assertThat(lastSnapshotId(table)).isEqualTo(2) + + // Make par-p1 has 3 datafile and par-p2 has 2 datafile, so par-p2 will not be picked out to + // compact. + spark.sql(s"INSERT INTO T VALUES (1, 'a', 'p1')") + + spark.sql( + "CALL sys.compact(table => 'T', compact_strategy => 'minor'," + + "options => 'num-sorted-run.compaction-trigger=3')") + + Assertions.assertThat(lastSnapshotId(table)).isEqualTo(4) + Assertions.assertThat(lastSnapshotCommand(table).equals(CommitKind.COMPACT)).isTrue + + val splits = table.newSnapshotReader.read.dataSplits + splits.forEach( + split => { + Assertions + .assertThat(split.dataFiles.size) + .isEqualTo(if (split.partition().getString(0).toString == "p2") 2 else 1) + }) + } + } + + // ----------------------- Sort Compact ----------------------- + test("Paimon Procedure: sort compact") { failAfter(streamingTimeout) { withTempDir { diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala similarity index 90% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala index 4561e532f538d..9f0d23dc93794 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala @@ -551,4 +551,69 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest } } } + + test("Paimon Procedure: expire partitions with default num") { + failAfter(streamingTimeout) { + withTempDir { + checkpointDir => + spark.sql( + s""" + |CREATE TABLE T (k STRING, pt STRING) + |TBLPROPERTIES ('primary-key'='k,pt', 'bucket'='1', 'partition.expiration-max-num'='2') + |PARTITIONED BY (pt) + |""".stripMargin) + val location = loadTable("T").location().toString + + val inputData = MemoryStream[(String, String)] + val stream = inputData + .toDS() + .toDF("k", "pt") + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .foreachBatch { + (batch: Dataset[Row], _: Long) => + batch.write.format("paimon").mode("append").save(location) + } + .start() + + val query = () => spark.sql("SELECT * FROM T") + + try { + // snapshot-1 + inputData.addData(("a", "2024-06-01")) + stream.processAllAvailable() + + // snapshot-2 + inputData.addData(("b", "2024-06-02")) + stream.processAllAvailable() + + // snapshot-3 + inputData.addData(("c", "2024-06-03")) + stream.processAllAvailable() + + // This partition never expires. + inputData.addData(("Never-expire", "9999-09-09")) + stream.processAllAvailable() + + checkAnswer( + query(), + Row("a", "2024-06-01") :: Row("b", "2024-06-02") :: Row("c", "2024-06-03") :: Row( + "Never-expire", + "9999-09-09") :: Nil) + // call expire_partitions. + checkAnswer( + spark.sql( + "CALL paimon.sys.expire_partitions(table => 'test.T', expiration_time => '1 d'" + + ", timestamp_formatter => 'yyyy-MM-dd')"), + Row("pt=2024-06-01") :: Row("pt=2024-06-02") :: Nil + ) + + checkAnswer(query(), Row("c", "2024-06-03") :: Row("Never-expire", "9999-09-09") :: 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-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireTagsProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/FastForwardProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/FastForwardProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/FastForwardProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/FastForwardProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateDatabaseProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateFileProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateFileProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateFileProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateFileProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateTableProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateTableProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/MigrateTableProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MigrateTableProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala similarity index 93% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala index f3cb7fa266657..a5f9f3ffa01b2 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTestBase.scala @@ -19,8 +19,8 @@ package org.apache.paimon.spark.procedure import org.apache.paimon.spark.PaimonSparkTestBase -import org.apache.paimon.spark.analysis.NoSuchProcedureException +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.parser.extensions.PaimonParseException import org.assertj.core.api.Assertions.assertThatThrownBy @@ -32,7 +32,7 @@ abstract class ProcedureTestBase extends PaimonSparkTestBase { |""".stripMargin) assertThatThrownBy(() => spark.sql("CALL sys.unknown_procedure(table => 'test.T')")) - .isInstanceOf(classOf[NoSuchProcedureException]) + .isInstanceOf(classOf[ParseException]) } test(s"test parse exception") { diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala similarity index 94% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala index d9d73811266dd..3ffe7fba264f0 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala @@ -52,7 +52,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { fileIO.tryToWriteAtomic(orphanFile2, "b") // by default, no file deleted - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) val orphanFile2ModTime = fileIO.getFileStatus(orphanFile2).getModificationTime val older_than1 = DateTimeUtils.formatLocalDateTime( @@ -63,7 +63,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql(s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than1')"), - Row(1) :: Nil) + Row(1, 1) :: Nil) val older_than2 = DateTimeUtils.formatLocalDateTime( DateTimeUtils.toLocalDateTime(System.currentTimeMillis()), @@ -71,9 +71,9 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql(s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than2')"), - Row(1) :: Nil) + Row(1, 1) :: Nil) - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) } test("Paimon procedure: dry run remove orphan files") { @@ -97,7 +97,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { fileIO.writeFile(orphanFile2, "b", true) // by default, no file deleted - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) val older_than = DateTimeUtils.formatLocalDateTime( DateTimeUtils.toLocalDateTime(System.currentTimeMillis()), @@ -106,10 +106,10 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql( s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than', dry_run => true)"), - Row(2) :: Nil + Row(2, 2) :: Nil ) - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) } test("Paimon procedure: remove database orphan files") { @@ -146,7 +146,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { fileIO2.tryToWriteAtomic(orphanFile22, "b") // by default, no file deleted - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*')"), Row(0, 0) :: Nil) val orphanFile12ModTime = fileIO1.getFileStatus(orphanFile12).getModificationTime val older_than1 = DateTimeUtils.formatLocalDateTime( @@ -157,7 +157,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*', older_than => '$older_than1')"), - Row(2) :: Nil + Row(2, 2) :: Nil ) val older_than2 = DateTimeUtils.formatLocalDateTime( @@ -166,10 +166,10 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*', older_than => '$older_than2')"), - Row(2) :: Nil + Row(2, 2) :: Nil ) - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'test.*')"), Row(0, 0) :: Nil) } test("Paimon procedure: remove orphan files with mode") { @@ -193,7 +193,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { fileIO.tryToWriteAtomic(orphanFile2, "b") // by default, no file deleted - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 0) :: Nil) val orphanFile2ModTime = fileIO.getFileStatus(orphanFile2).getModificationTime val older_than1 = DateTimeUtils.formatLocalDateTime( @@ -205,7 +205,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql( s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than1', mode => 'diSTributed')"), - Row(1) :: Nil) + Row(1, 1) :: Nil) val older_than2 = DateTimeUtils.formatLocalDateTime( DateTimeUtils.toLocalDateTime(System.currentTimeMillis()), @@ -214,9 +214,9 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { checkAnswer( spark.sql( s"CALL sys.remove_orphan_files(table => 'T', older_than => '$older_than2', mode => 'local')"), - Row(1) :: Nil) + Row(1, 1) :: Nil) - checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0) :: Nil) + checkAnswer(spark.sql(s"CALL sys.remove_orphan_files(table => 'T')"), Row(0, 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-ut/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ReplaceTagProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala index 238dd039969a8..4f8ccae22dd55 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala @@ -86,7 +86,7 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS") - withSQLConf("spark.paimon.scan.timestamp-millis" -> System.currentTimeMillis.toString) { + withSparkSQLConf("spark.paimon.scan.timestamp-millis" -> System.currentTimeMillis.toString) { checkAnswer( sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), Row(2, 0, 2, "{ }")) @@ -97,7 +97,7 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS") - withSQLConf("spark.paimon.scan.timestamp-millis" -> System.currentTimeMillis.toString) { + withSparkSQLConf("spark.paimon.scan.timestamp-millis" -> System.currentTimeMillis.toString) { checkAnswer( sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), Row(5, 0, 4, "{ }")) @@ -111,31 +111,31 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { 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") { + withSparkSQLConf("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") { + withSparkSQLConf("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") { + withSparkSQLConf("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") { + withSparkSQLConf("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") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "6") { checkAnswer( sql("SELECT snapshot_id, schema_id, mergedRecordCount, colstat FROM `T$statistics`"), Row(5, 0, 4, "{ }")) diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala index afc70bf9105df..35931924c4875 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/BucketedTableQueryTest.scala @@ -29,12 +29,12 @@ class BucketedTableQueryTest extends PaimonSparkTestBase with AdaptiveSparkPlanH private def checkAnswerAndShuffleSorts(query: String, numShuffles: Int, numSorts: Int): Unit = { var expectedResult: Array[Row] = null // avoid config default value change in future, so specify it manually - withSQLConf( + withSparkSQLConf( "spark.sql.sources.v2.bucketing.enabled" -> "false", "spark.sql.autoBroadcastJoinThreshold" -> "-1") { expectedResult = spark.sql(query).collect() } - withSQLConf( + withSparkSQLConf( "spark.sql.sources.v2.bucketing.enabled" -> "true", "spark.sql.autoBroadcastJoinThreshold" -> "-1") { val df = spark.sql(query) @@ -162,10 +162,10 @@ class BucketedTableQueryTest extends PaimonSparkTestBase with AdaptiveSparkPlanH 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") { + withSparkSQLConf("spark.sql.requireAllClusterKeysForDistribution" -> "false") { checkAnswerAndShuffleSorts("SELECT id, c, count(*) FROM t1 GROUP BY id, c", 0, 0) } - withSQLConf("spark.sql.requireAllClusterKeysForDistribution" -> "true") { + withSparkSQLConf("spark.sql.requireAllClusterKeysForDistribution" -> "true") { checkAnswerAndShuffleSorts("SELECT id, c, count(*) FROM t1 GROUP BY id, c", 1, 0) } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala similarity index 93% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala index cf1a71d51fccb..3ed2c98306fb3 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala @@ -161,7 +161,7 @@ abstract class DDLTestBase extends PaimonSparkTestBase { test("Paimon DDL: create table without using paimon") { withTable("paimon_tbl") { sql("CREATE TABLE paimon_tbl (id int)") - assert(loadTable("paimon_tbl").options().get("provider").equals("paimon")) + assert(!loadTable("paimon_tbl").options().containsKey("provider")) } } @@ -238,21 +238,21 @@ abstract class DDLTestBase extends PaimonSparkTestBase { |USING PAIMON |""".stripMargin) - withSQLConf("spark.sql.legacy.charVarcharAsString" -> "true") { + withSparkSQLConf("spark.sql.legacy.charVarcharAsString" -> "true") { sql("INSERT INTO paimon_tbl VALUES (1, 'ab')") } - withSQLConf("spark.sql.legacy.charVarcharAsString" -> "false") { + withSparkSQLConf("spark.sql.legacy.charVarcharAsString" -> "false") { sql("INSERT INTO paimon_tbl VALUES (2, 'ab')") } if (gteqSpark3_4) { - withSQLConf("spark.sql.readSideCharPadding" -> "true") { + withSparkSQLConf("spark.sql.readSideCharPadding" -> "true") { checkAnswer( spark.sql("SELECT c FROM paimon_tbl ORDER BY id"), Row("ab ") :: Row("ab ") :: Nil) } - withSQLConf("spark.sql.readSideCharPadding" -> "false") { + withSparkSQLConf("spark.sql.readSideCharPadding" -> "false") { checkAnswer( spark.sql("SELECT c FROM paimon_tbl ORDER BY id"), Row("ab") :: Row("ab ") :: Nil) @@ -270,7 +270,8 @@ abstract class DDLTestBase extends PaimonSparkTestBase { format => Seq(true, false).foreach { datetimeJava8APIEnabled => - withSQLConf("spark.sql.datetime.java8API.enabled" -> datetimeJava8APIEnabled.toString) { + withSparkSQLConf( + "spark.sql.datetime.java8API.enabled" -> datetimeJava8APIEnabled.toString) { withTimeZone("Asia/Shanghai") { withTable("paimon_tbl") { // Spark support create table with timestamp_ntz since 3.4 @@ -430,7 +431,8 @@ abstract class DDLTestBase extends PaimonSparkTestBase { test("Paimon DDL: select table with timestamp and timestamp_ntz with filter") { Seq(true, false).foreach { datetimeJava8APIEnabled => - withSQLConf("spark.sql.datetime.java8API.enabled" -> datetimeJava8APIEnabled.toString) { + withSparkSQLConf( + "spark.sql.datetime.java8API.enabled" -> datetimeJava8APIEnabled.toString) { withTable("paimon_tbl") { // Spark support create table with timestamp_ntz since 3.4 if (gteqSpark3_4) { @@ -546,4 +548,28 @@ abstract class DDLTestBase extends PaimonSparkTestBase { } } } + + test("Paimon DDL: create and drop external / managed table") { + withTempDir { + tbLocation => + withTable("external_tbl", "managed_tbl") { + // create external table + val error = intercept[UnsupportedOperationException] { + sql( + s"CREATE TABLE external_tbl (id INT) USING paimon LOCATION '${tbLocation.getCanonicalPath}'") + }.getMessage + assert(error.contains("not support")) + + // create managed table + sql("CREATE TABLE managed_tbl (id INT) USING paimon") + val table = loadTable("managed_tbl") + val fileIO = table.fileIO() + val tableLocation = table.location() + + // drop managed table + sql("DROP TABLE managed_tbl") + assert(!fileIO.exists(tableLocation)) + } + } + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala similarity index 66% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 7478f9628cfbf..1189f1f2906b3 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -189,6 +189,7 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { val props = getDatabaseProps("paimon_db") Assertions.assertEquals(props("k1"), "v1") Assertions.assertEquals(props("k2"), "v2") + Assertions.assertTrue(getDatabaseOwner("paimon_db").nonEmpty) } } } @@ -240,12 +241,12 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { spark.sql(s"CREATE DATABASE paimon_db") spark.sql(s"USE paimon_db") spark.sql(s"CREATE TABLE paimon_tbl (id int, name string, dt string) using paimon") - // Currently, only spark_catalog supports create other table or view + // Only spark_catalog supports create other table if (catalogName.equals(sparkCatalogName)) { spark.sql(s"CREATE TABLE parquet_tbl (id int, name string, dt string) using parquet") spark.sql(s"CREATE VIEW parquet_tbl_view AS SELECT * FROM parquet_tbl") - spark.sql(s"CREATE VIEW paimon_tbl_view AS SELECT * FROM paimon_tbl") } + spark.sql(s"CREATE VIEW paimon_tbl_view AS SELECT * FROM paimon_tbl") spark.sql(s"USE default") spark.sql(s"DROP DATABASE paimon_db CASCADE") } @@ -296,29 +297,160 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } } - def getDatabaseLocation(dbName: String): String = { - spark - .sql(s"DESC DATABASE $dbName") - .filter("info_name == 'Location'") - .head() - .getAs[String]("info_value") - .split(":")(1) + test("Paimon DDL with hive catalog: create and drop external / managed table") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + withTempDir { + tbLocation => + withDatabase("paimon_db") { + spark.sql(s"CREATE DATABASE IF NOT EXISTS paimon_db") + spark.sql(s"USE paimon_db") + withTable("external_tbl", "managed_tbl") { + val expertTbLocation = tbLocation.getCanonicalPath + // create external table + spark.sql( + s"CREATE TABLE external_tbl (id INT) USING paimon LOCATION '$expertTbLocation'") + spark.sql("INSERT INTO external_tbl VALUES (1)") + checkAnswer(spark.sql("SELECT * FROM external_tbl"), Row(1)) + val table = loadTable("paimon_db", "external_tbl") + val fileIO = table.fileIO() + val actualTbLocation = table.location() + assert(actualTbLocation.toString.split(':').apply(1).equals(expertTbLocation)) + + // drop external table + spark.sql("DROP TABLE external_tbl") + assert(fileIO.exists(actualTbLocation)) + + // create external table again using the same location + spark.sql( + s"CREATE TABLE external_tbl (id INT) USING paimon LOCATION '$expertTbLocation'") + checkAnswer(spark.sql("SELECT * FROM external_tbl"), Row(1)) + assert(getActualTableLocation("paimon_db", "external_tbl").equals(expertTbLocation)) + + // create managed table + spark.sql(s"CREATE TABLE managed_tbl (id INT) USING paimon") + val managedTbLocation = loadTable("paimon_db", "managed_tbl").location() + + // drop managed table + spark.sql("DROP TABLE managed_tbl") + assert(!fileIO.exists(managedTbLocation)) + } + } + } + } + } + + test("Paimon DDL with hive catalog: rename external / managed table") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + withTempDir { + tbLocation => + withDatabase("paimon_db") { + spark.sql(s"CREATE DATABASE paimon_db") + spark.sql(s"USE paimon_db") + withTable( + "external_tbl", + "managed_tbl", + "external_tbl_renamed", + "managed_tbl_renamed") { + val expertTbLocation = tbLocation.getCanonicalPath + // create external table + spark.sql( + s"CREATE TABLE external_tbl (id INT) USING paimon LOCATION '$expertTbLocation'") + spark.sql("INSERT INTO external_tbl VALUES (1)") + val actualTbLocation = loadTable("paimon_db", "external_tbl").location() + assert(actualTbLocation.toString.split(':').apply(1).equals(expertTbLocation)) + + // rename external table, location should not change + spark.sql("ALTER TABLE external_tbl RENAME TO external_tbl_renamed") + checkAnswer(spark.sql("SELECT * FROM external_tbl_renamed"), Row(1)) + assert( + getActualTableLocation("paimon_db", "external_tbl_renamed").equals( + expertTbLocation)) + + // create managed table + spark.sql(s"CREATE TABLE managed_tbl (id INT) USING paimon") + spark.sql("INSERT INTO managed_tbl VALUES (1)") + val managedTbLocation = loadTable("paimon_db", "managed_tbl").location() + + // rename managed table, location should change + spark.sql("ALTER TABLE managed_tbl RENAME TO managed_tbl_renamed") + checkAnswer(spark.sql("SELECT * FROM managed_tbl_renamed"), Row(1)) + assert( + !getActualTableLocation("paimon_db", "managed_tbl_renamed").equals( + managedTbLocation.toString)) + } + } + } + } } - def getDatabaseComment(dbName: String): String = { + test("Paimon DDL with hive catalog: create external table without schema") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + spark.sql(s"USE $catalogName") + withTempDir { + tbLocation => + withDatabase("paimon_db") { + spark.sql(s"CREATE DATABASE IF NOT EXISTS paimon_db") + spark.sql(s"USE paimon_db") + withTable("t1", "t2", "t3", "t4", "t5") { + val expertTbLocation = tbLocation.getCanonicalPath + spark.sql(s""" + |CREATE TABLE t1 (id INT, pt INT) USING paimon + |PARTITIONED BY (pt) + |TBLPROPERTIES('primary-key' = 'id', 'k1' = 'v1') + |LOCATION '$expertTbLocation' + |""".stripMargin) + spark.sql("INSERT INTO t1 VALUES (1, 1)") + + // create table without schema + spark.sql(s"CREATE TABLE t2 USING paimon LOCATION '$expertTbLocation'") + checkAnswer(spark.sql("SELECT * FROM t2"), Row(1, 1)) + assert(getActualTableLocation("paimon_db", "t2").equals(expertTbLocation)) + + // create table with wrong schema + intercept[Exception] { + spark.sql( + s"CREATE TABLE t3 (fake_col INT) USING paimon LOCATION '$expertTbLocation'") + } + + // create table with exists props + spark.sql( + s"CREATE TABLE t4 USING paimon TBLPROPERTIES ('k1' = 'v1') LOCATION '$expertTbLocation'") + checkAnswer(spark.sql("SELECT * FROM t4"), Row(1, 1)) + assert(getActualTableLocation("paimon_db", "t4").equals(expertTbLocation)) + + // create table with new props + intercept[Exception] { + spark.sql( + s"CREATE TABLE t5 USING paimon TBLPROPERTIES ('k2' = 'v2') LOCATION '$expertTbLocation'") + } + } + } + } + } + } + + def getDatabaseProp(dbName: String, propertyName: String): String = { spark - .sql(s"DESC DATABASE $dbName") - .filter("info_name == 'Comment'") + .sql(s"DESC DATABASE EXTENDED $dbName") + .filter(s"info_name == '$propertyName'") .head() .getAs[String]("info_value") } + def getDatabaseLocation(dbName: String): String = + getDatabaseProp(dbName, "Location").split(":")(1) + + def getDatabaseComment(dbName: String): String = getDatabaseProp(dbName, "Comment") + + def getDatabaseOwner(dbName: String): String = getDatabaseProp(dbName, "Owner") + def getDatabaseProps(dbName: String): Map[String, String] = { - val dbPropsStr = spark - .sql(s"DESC DATABASE EXTENDED $dbName") - .filter("info_name == 'Properties'") - .head() - .getAs[String]("info_value") + val dbPropsStr = getDatabaseProp(dbName, "Properties") val pattern = "\\(([^,]+),([^)]+)\\)".r pattern .findAllIn(dbPropsStr.drop(1).dropRight(1)) @@ -346,4 +478,8 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { .toMap tableProps("path").split(":")(1) } + + def getActualTableLocation(dbName: String, tblName: String): String = { + loadTable(dbName, tblName).location().toString.split(':').apply(1) + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala similarity index 99% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala index a0a94afacfb95..edd092c85ce8b 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala @@ -20,6 +20,7 @@ package org.apache.paimon.spark.sql import org.apache.paimon.spark.PaimonSparkTestBase +import org.apache.spark.SparkConf import org.apache.spark.sql.Row import org.apache.spark.sql.types.DecimalType import org.junit.jupiter.api.Assertions @@ -27,6 +28,11 @@ import org.junit.jupiter.api.Assertions import java.sql.{Date, Timestamp} class DataFrameWriteTest extends PaimonSparkTestBase { + + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.sql.catalog.paimon.cache-enabled", "false") + } + import testImplicits._ test("Paimon: DataFrameWrite.saveAsTable") { diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala index e944429e42184..46a423b9d699e 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DeletionVectorTest.scala @@ -25,13 +25,13 @@ 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.paimon.Utils import org.apache.spark.sql.util.QueryExecutionListener import org.junit.jupiter.api.Assertions @@ -631,6 +631,27 @@ class DeletionVectorTest extends PaimonSparkTestBase with AdaptiveSparkPlanHelpe ) } + test("Paimon deletionVector: get cardinality") { + sql(s""" + |CREATE TABLE T (id INT) + |TBLPROPERTIES ( + | 'deletion-vectors.enabled' = 'true', + | 'bucket-key' = 'id', + | 'bucket' = '1' + |) + |""".stripMargin) + + sql("INSERT INTO T SELECT /*+ REPARTITION(1) */ id FROM range (1, 50000)") + sql("DELETE FROM T WHERE id >= 111 and id <= 444") + + val fileStore = loadTable("T").store() + val indexManifest = fileStore.snapshotManager().latestSnapshot().indexManifest() + val entry = fileStore.newIndexFileHandler().readManifest(indexManifest).get(0) + val dvMeta = entry.indexFile().deletionVectorMetas().values().iterator().next() + + assert(dvMeta.cardinality() == 334) + } + private def getPathName(path: String): String = { new Path(path).getName } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala similarity index 51% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala index 528dcd3cd107a..ae538fa48c4e2 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala @@ -27,6 +27,76 @@ import java.util.Objects class DescribeTableTest extends PaimonSparkTestBase { + test("Paimon show: show table extended") { + val testDB = "test_show" + withDatabase(testDB) { + spark.sql("CREATE TABLE s1 (id INT)") + + spark.sql(s"CREATE DATABASE $testDB") + spark.sql(s"USE $testDB") + spark.sql("CREATE TABLE s2 (id INT, pt STRING) PARTITIONED BY (pt)") + spark.sql("CREATE TABLE s3 (id INT, pt1 STRING, pt2 STRING) PARTITIONED BY (pt1, pt2)") + + spark.sql("INSERT INTO s2 VALUES (1, '2024'), (2, '2024'), (3, '2025'), (4, '2026')") + spark.sql(""" + |INSERT INTO s3 + |VALUES + |(1, '2024', '11'), (2, '2024', '12'), (3, '2025', '11'), (4, '2025', '12') + |""".stripMargin) + + // SHOW TABL EXTENDED will give four columns: namespace, tableName, isTemporary, information. + checkAnswer( + sql(s"SHOW TABLE EXTENDED IN $dbName0 LIKE '*'") + .select("namespace", "tableName", "isTemporary"), + Row("test", "s1", false)) + checkAnswer( + sql(s"SHOW TABLE EXTENDED IN $testDB LIKE '*'") + .select("namespace", "tableName", "isTemporary"), + Row(testDB, "s2", false) :: Row(testDB, "s3", false) :: Nil + ) + + // check table s1 + val res1 = spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's2'").select("information") + Assertions.assertEquals(1, res1.count()) + val information1 = res1 + .collect() + .head + .getString(0) + .split("\n") + .map { + line => + val kv = line.split(": ", 2) + kv(0) -> kv(1) + } + .toMap + Assertions.assertEquals(information1("Catalog"), "paimon") + Assertions.assertEquals(information1("Namespace"), testDB) + Assertions.assertEquals(information1("Table"), "s2") + Assertions.assertEquals(information1("Provider"), "paimon") + Assertions.assertEquals(information1("Location"), loadTable(testDB, "s2").location().toString) + + // check table s2 partition info + val error1 = intercept[Exception] { + spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's2' PARTITION(pt='2022')") + }.getMessage + assert(error1.contains("PARTITIONS_NOT_FOUND")) + + val error2 = intercept[Exception] { + spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's3' PARTITION(pt1='2024')") + }.getMessage + assert(error2.contains("Partition spec is invalid")) + + val res2 = + spark.sql(s"SHOW TABLE EXTENDED IN $testDB LIKE 's3' PARTITION(pt1 = '2024', pt2 = 11)") + checkAnswer( + res2.select("namespace", "tableName", "isTemporary"), + Row(testDB, "s3", false) + ) + Assertions.assertTrue( + res2.select("information").collect().head.getString(0).contains("Partition Values")) + } + } + test(s"Paimon describe: describe table comment") { var comment = "test comment" spark.sql(s""" diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala similarity index 97% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala index 70339bd7cac3e..f47d40285aa9a 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DisableUnnecessaryPaimonBucketedScanSuite.scala @@ -48,13 +48,13 @@ class DisableUnnecessaryPaimonBucketedScanSuite assert(bucketedScan.length == expectedNumBucketedScan, query) } - withSQLConf("spark.sql.sources.v2.bucketing.enabled" -> "true") { - withSQLConf("spark.sql.sources.bucketing.autoBucketedScan.enabled" -> "true") { + withSparkSQLConf("spark.sql.sources.v2.bucketing.enabled" -> "true") { + withSparkSQLConf("spark.sql.sources.bucketing.autoBucketedScan.enabled" -> "true") { val df = sql(query) val result = df.collect() checkNumBucketedScan(df, expectedNumScanWithAutoScanEnabled) - withSQLConf("spark.sql.sources.bucketing.autoBucketedScan.enabled" -> "false") { + withSparkSQLConf("spark.sql.sources.bucketing.autoBucketedScan.enabled" -> "false") { val expected = sql(query) checkAnswer(expected, result) checkNumBucketedScan(expected, expectedNumScanWithAutoScanDisabled) diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DynamicBucketTableTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala similarity index 88% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala index 674b45fda68b2..977b74707069b 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTestBase.scala @@ -346,7 +346,7 @@ abstract class InsertOverwriteTableTestBase extends PaimonSparkTestBase { spark.sql("SELECT * FROM T ORDER BY a, b"), Row(1, 3, "3") :: Row(2, 4, "4") :: Nil) - withSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { + withSparkSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { // dynamic overwrite the a=1 partition spark.sql("INSERT OVERWRITE T VALUES (1, 5, '5'), (1, 7, '7')") checkAnswer( @@ -387,7 +387,7 @@ abstract class InsertOverwriteTableTestBase extends PaimonSparkTestBase { "ptv2", 22) :: Nil) - withSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { + withSparkSQLConf("spark.sql.sources.partitionOverwriteMode" -> "dynamic") { // dynamic overwrite the pt2=22 partition spark.sql( "INSERT OVERWRITE T PARTITION (pt2 = 22) VALUES (3, 'c2', 'ptv1'), (4, 'd2', 'ptv3')") @@ -508,4 +508,56 @@ abstract class InsertOverwriteTableTestBase extends PaimonSparkTestBase { ) :: Nil ) } + + test("Paimon Insert: insert with column list") { + sql("CREATE TABLE T (name String, student_id INT) PARTITIONED BY (address STRING)") + + // insert with a column list + sql("INSERT INTO T (name, student_id, address) VALUES ('a', '1', 'Hangzhou')") + // Since Spark 3.4, INSERT INTO commands with explicit column lists comprising fewer columns than the target + // table will automatically add the corresponding default values for the remaining columns (or NULL for any column + // lacking an explicitly-assigned default value). In Spark 3.3 or earlier, these commands would have failed. + // See https://issues.apache.org/jira/browse/SPARK-42521 + if (gteqSpark3_4) { + sql("INSERT INTO T (name) VALUES ('b')") + sql("INSERT INTO T (address, name) VALUES ('Hangzhou', 'c')") + } else { + sql("INSERT INTO T (name, student_id, address) VALUES ('b', null, null)") + sql("INSERT INTO T (name, student_id, address) VALUES ('c', null, 'Hangzhou')") + } + + // insert with both a partition spec and a column list + if (gteqSpark3_4) { + sql("INSERT INTO T PARTITION (address='Beijing') (name) VALUES ('d')") + } else { + sql("INSERT INTO T PARTITION (address='Beijing') (name, student_id) VALUES ('d', null)") + } + sql("INSERT INTO T PARTITION (address='Hangzhou') (student_id, name) VALUES (5, 'e')") + + checkAnswer( + sql("SELECT * FROM T ORDER BY name"), + Seq( + Row("a", 1, "Hangzhou"), + Row("b", null, null), + Row("c", null, "Hangzhou"), + Row("d", null, "Beijing"), + Row("e", 5, "Hangzhou")) + ) + + // insert overwrite with a column list + if (gteqSpark3_4) { + sql("INSERT OVERWRITE T (name, address) VALUES ('f', 'Shanghai')") + } else { + sql("INSERT OVERWRITE T (name, student_id, address) VALUES ('f', null, 'Shanghai')") + } + checkAnswer(sql("SELECT * FROM T ORDER BY name"), Row("f", null, "Shanghai")) + + // insert overwrite with both a partition spec and a column list + if (gteqSpark3_4) { + sql("INSERT OVERWRITE T PARTITION (address='Shanghai') (name) VALUES ('g')") + } else { + sql("INSERT OVERWRITE T PARTITION (address='Shanghai') (name, student_id) VALUES ('g', null)") + } + checkAnswer(sql("SELECT * FROM T ORDER BY name"), Row("g", null, "Shanghai")) + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoNotMatchedBySourceTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/ObjectTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/ObjectTableTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/ObjectTableTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/ObjectTableTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonFunctionTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonMetricTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonMetricTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonMetricTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonMetricTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala similarity index 90% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala index 78e8905fa969a..87f4c94486192 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTestBase.scala @@ -20,6 +20,7 @@ package org.apache.paimon.spark.sql import org.apache.paimon.Snapshot.CommitKind import org.apache.paimon.spark.PaimonSparkTestBase +import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper import org.apache.paimon.spark.catalyst.optimizer.MergePaimonScalarSubqueries import org.apache.spark.sql.Row @@ -27,11 +28,12 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateNamedStruct, import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, LogicalPlan, OneRowRelation, WithCTE} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.functions._ +import org.apache.spark.sql.paimon.Utils import org.junit.jupiter.api.Assertions import scala.collection.immutable -abstract class PaimonOptimizationTestBase extends PaimonSparkTestBase { +abstract class PaimonOptimizationTestBase extends PaimonSparkTestBase with ExpressionHelper { import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -59,24 +61,25 @@ abstract class PaimonOptimizationTestBase extends PaimonSparkTestBase { |""".stripMargin) val optimizedPlan = Optimize.execute(query.queryExecution.analyzed) - val relation = createRelationV2("T") - val mergedSubquery = relation + val df = Utils.createDataFrame(spark, createRelationV2("T")) + val mergedSubquery = df .select( - count(Literal(1)).as("cnt"), - sum(col("a").expr).as("sum_a"), - avg(col("b").expr).as("avg_b") + toColumn(count(Literal(1))).as("cnt"), + toColumn(sum(toExpression(spark, col("a")))).as("sum_a"), + toColumn(avg(toExpression(spark, col("b"))).as("avg_b")) ) .select( - CreateNamedStruct( - Seq( - Literal("cnt"), - 'cnt, - Literal("sum_a"), - 'sum_a, - Literal("avg_b"), - 'avg_b - )).as("mergedValue")) - val analyzedMergedSubquery = mergedSubquery.analyze + toColumn( + CreateNamedStruct( + Seq( + Literal("cnt"), + 'cnt, + Literal("sum_a"), + 'sum_a, + Literal("avg_b"), + 'avg_b + )).as("mergedValue"))) + val analyzedMergedSubquery = mergedSubquery.queryExecution.analyzed val correctAnswer = WithCTE( OneRowRelation() .select( diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala similarity index 91% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala index f74d6959b9f16..44df3e54ca724 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala @@ -29,7 +29,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { import testImplicits._ test("Paimon Option: create table with sql conf") { - withSQLConf("spark.paimon.scan.snapshot-id" -> "2") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "2") { sql("CREATE TABLE T (id INT)") val table = loadTable("T") // check options in schema file directly @@ -39,7 +39,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { } test("Paimon Option: create table by dataframe with sql conf") { - withSQLConf("spark.paimon.scan.snapshot-id" -> "2") { + withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "2") { Seq((1L, "x1"), (2L, "x2")) .toDF("a", "b") .write @@ -61,13 +61,13 @@ class PaimonOptionTest extends PaimonSparkTestBase { val table = loadTable("T") // query with mutable option - withSQLConf("spark.paimon.scan.snapshot-id" -> "1") { + withSparkSQLConf("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 immutable option - withSQLConf("spark.paimon.bucket" -> "1") { + withSparkSQLConf("spark.paimon.bucket" -> "1") { assertThrows[UnsupportedOperationException] { sql("SELECT * FROM T ORDER BY id") } @@ -85,19 +85,19 @@ class PaimonOptionTest extends PaimonSparkTestBase { val table = loadTable("T") // query with global options - withSQLConf("spark.paimon.scan.snapshot-id" -> "1") { + withSparkSQLConf("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") { + withSparkSQLConf("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( + withSparkSQLConf( "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) @@ -122,7 +122,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { val table2 = loadTable("T1") // query with global options - withSQLConf("spark.paimon.scan.snapshot-id" -> "1") { + withSparkSQLConf("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 @@ -134,7 +134,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { } // query with table options - withSQLConf("spark.paimon.*.*.*.scan.snapshot-id" -> "1") { + withSparkSQLConf("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 @@ -146,7 +146,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { } // query with both global and table options - withSQLConf( + withSparkSQLConf( "spark.paimon.scan.snapshot-id" -> "1", "spark.paimon.*.*.*.scan.snapshot-id" -> "2") { checkAnswer( @@ -161,7 +161,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { ) } - withSQLConf( + withSparkSQLConf( "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)) @@ -174,7 +174,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { ) } - withSQLConf( + withSparkSQLConf( "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)) @@ -187,7 +187,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { ) } - withSQLConf( + withSparkSQLConf( "spark.paimon.*.*.T1.scan.snapshot-id" -> "2", "spark.paimon.*.*.T2.scan.snapshot-id" -> "2") { checkAnswer( diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPartitionManagementTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPartitionManagementTest.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPartitionManagementTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPartitionManagementTest.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala similarity index 87% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala index ba314e3afa816..503f1c8e3e9d6 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala @@ -18,7 +18,7 @@ package org.apache.paimon.spark.sql -import org.apache.paimon.spark.{PaimonBatch, PaimonInputPartition, PaimonScan, PaimonSparkTestBase, SparkTable} +import org.apache.paimon.spark.{PaimonScan, PaimonSparkTestBase, SparkTable} import org.apache.paimon.table.source.DataSplit import org.apache.spark.sql.Row @@ -29,8 +29,6 @@ import org.apache.spark.sql.connector.read.{ScanBuilder, SupportsPushDownLimit} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.junit.jupiter.api.Assertions -import scala.collection.JavaConverters._ - class PaimonPushDownTest extends PaimonSparkTestBase { import testImplicits._ @@ -64,7 +62,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { checkAnswer(spark.sql(q), Row(1, "a", "p1") :: Nil) // case 2 - // filter "id = '1' or pt = 'p1'" can't push down completely, it still need to be evaluated after scanning + // filter "id = '1' or pt = 'p1'" can't push down completely, it still needs to be evaluated after scanning q = "SELECT * FROM T WHERE id = '1' or pt = 'p1'" Assertions.assertTrue(checkEqualToFilterExists(q, "pt", Literal("p1"))) checkAnswer(spark.sql(q), Row(1, "a", "p1") :: Row(2, "b", "p1") :: Nil) @@ -121,7 +119,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { val dataSplitsWithoutLimit = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits Assertions.assertTrue(dataSplitsWithoutLimit.length >= 2) - // It still return false even it can push down limit. + // 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) @@ -169,12 +167,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { // 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) + dataSplitsWithoutLimit2.count(split => { split.asInstanceOf[DataSplit].rawConvertible() })) // Return 2 dataSplits. Assertions.assertFalse(scanBuilder2.asInstanceOf[SupportsPushDownLimit].pushLimit(2)) @@ -206,7 +199,40 @@ class PaimonPushDownTest extends PaimonSparkTestBase { // Need to scan all dataSplits. Assertions.assertEquals(4, dataSplitsWithLimit3.length) Assertions.assertEquals(1, spark.sql("SELECT * FROM T LIMIT 1").count()) + } + test("Paimon pushDown: limit for table with deletion vector") { + Seq(true, false).foreach( + deletionVectorsEnabled => { + Seq(true, false).foreach( + primaryKeyTable => { + withTable("T") { + sql(s""" + |CREATE TABLE T (id INT) + |TBLPROPERTIES ( + | 'deletion-vectors.enabled' = $deletionVectorsEnabled, + | '${if (primaryKeyTable) "primary-key" else "bucket-key"}' = 'id', + | 'bucket' = '10' + |) + |""".stripMargin) + + sql("INSERT INTO T SELECT id FROM range (1, 50000)") + sql("DELETE FROM T WHERE id % 13 = 0") + + val withoutLimit = getScanBuilder().build().asInstanceOf[PaimonScan].getOriginSplits + assert(withoutLimit.length == 10) + + val scanBuilder = getScanBuilder().asInstanceOf[SupportsPushDownLimit] + scanBuilder.pushLimit(1) + val withLimit = scanBuilder.build().asInstanceOf[PaimonScan].getOriginSplits + if (deletionVectorsEnabled || !primaryKeyTable) { + assert(withLimit.length == 1) + } else { + assert(withLimit.length == 10) + } + } + }) + }) } test("Paimon pushDown: runtime filter") { @@ -250,8 +276,7 @@ class PaimonPushDownTest extends PaimonSparkTestBase { } private def getScanBuilder(tableName: String = "T"): ScanBuilder = { - new SparkTable(loadTable(tableName)) - .newScanBuilder(CaseInsensitiveStringMap.empty()) + SparkTable(loadTable(tableName)).newScanBuilder(CaseInsensitiveStringMap.empty()) } private def checkFilterExists(sql: String): Boolean = { @@ -272,5 +297,4 @@ class PaimonPushDownTest extends PaimonSparkTestBase { case _ => false } } - } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala similarity index 99% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala index beea19c35e925..08f5275f01b50 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala @@ -281,7 +281,7 @@ class PaimonQueryTest extends PaimonSparkTestBase { // 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") { + withSparkSQLConf("spark.sql.ansi.enabled" -> "false") { checkAnswer( sql(s""" |SELECT diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonShowColumnsTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonShowColumnsTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonShowColumnsTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonShowColumnsTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala similarity index 88% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala index 64baf6232fd8f..7baa57a54d909 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonSystemTableTest.scala @@ -81,4 +81,20 @@ class PaimonSystemTableTest extends PaimonSparkTestBase { 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) } + + test("system table: binlog table") { + sql(""" + |CREATE TABLE T (a INT, b INT) + |TBLPROPERTIES ('primary-key'='a', 'changelog-producer' = 'lookup', 'bucket' = '2') + |""".stripMargin) + + sql("INSERT INTO T VALUES (1, 2)") + sql("INSERT INTO T VALUES (1, 3)") + sql("INSERT INTO T VALUES (2, 2)") + + checkAnswer( + sql("SELECT * FROM `T$binlog`"), + Seq(Row("+I", Array(1), Array(3)), Row("+I", Array(2), Array(2))) + ) + } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonTagDdlTestBase.scala diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala new file mode 100644 index 0000000000000..00f5566ed47af --- /dev/null +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTestBase.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.sql + +import org.apache.paimon.spark.PaimonHiveTestBase + +import org.apache.spark.sql.Row + +abstract class PaimonViewTestBase extends PaimonHiveTestBase { + + test("Paimon View: create and drop view") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + { + sql(s"USE $catalogName") + withDatabase("test_db") { + sql("CREATE DATABASE test_db") + sql("USE test_db") + withTable("t") { + withView("v1") { + sql("CREATE TABLE t (id INT) USING paimon") + sql("INSERT INTO t VALUES (1), (2)") + + sql("CREATE VIEW v1 AS SELECT * FROM t") + checkAnswer(sql("SHOW VIEWS"), Seq(Row("test_db", "v1", false))) + checkAnswer(sql("SELECT * FROM v1"), Seq(Row(1), Row(2))) + checkAnswer( + sql("SELECT * FROM v1 WHERE id >= (SELECT max(id) FROM v1)"), + Seq(Row(2))) + + // test drop view + sql("DROP VIEW IF EXISTS v1") + checkAnswer(sql("SHOW VIEWS"), Seq()) + sql("CREATE VIEW v1 AS SELECT * FROM t WHERE id > 1") + checkAnswer(sql("SHOW VIEWS"), Seq(Row("test_db", "v1", false))) + checkAnswer(sql("SELECT * FROM v1"), Seq(Row(2))) + + // test create or replace view + intercept[Exception] { + sql("CREATE VIEW v1 AS SELECT * FROM t WHERE id < 2") + } + sql("CREATE OR REPLACE VIEW v1 AS SELECT * FROM t WHERE id < 2") + checkAnswer(sql("SELECT * FROM v1"), Seq(Row(1))) + } + } + } + } + } + } + + test("Paimon View: show views") { + Seq(sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + { + sql(s"USE $catalogName") + withDatabase("test_db") { + sql("CREATE DATABASE test_db") + sql("USE test_db") + withTable("t") { + withView("va", "vab", "vc") { + sql("CREATE TABLE t (id INT) USING paimon") + sql("CREATE VIEW va AS SELECT * FROM t") + sql("CREATE VIEW vab AS SELECT * FROM t") + sql("CREATE VIEW vc AS SELECT * FROM t") + checkAnswer( + sql("SHOW VIEWS"), + Seq( + Row("test_db", "va", false), + Row("test_db", "vab", false), + Row("test_db", "vc", false))) + checkAnswer( + sql("SHOW VIEWS LIKE 'va*'"), + Seq(Row("test_db", "va", false), Row("test_db", "vab", false))) + } + } + } + } + } + } + + test("Paimon View: show create view") { + sql(s"USE $paimonHiveCatalogName") + withDatabase("test_db") { + sql("CREATE DATABASE test_db") + sql("USE test_db") + withTable("t") { + withView("v") { + sql("CREATE TABLE t (id INT, c STRING) USING paimon") + sql(""" + |CREATE VIEW v + |COMMENT 'test comment' + |TBLPROPERTIES ('k1' = 'v1') + |AS SELECT * FROM t + |""".stripMargin) + + val s = sql("SHOW CREATE TABLE v").collectAsList().get(0).get(0).toString + val r = """ + |CREATE VIEW test_db.v \( + | id, + | c\) + |COMMENT 'test comment' + |TBLPROPERTIES \( + | 'k1' = 'v1', + | 'transient_lastDdlTime' = '\d+'\) + |AS + |SELECT \* FROM t + |""".stripMargin.replace("\n", "").r + assert(r.findFirstIn(s.replace("\n", "")).isDefined) + } + } + } + } + + test("Paimon View: describe [extended] view") { + sql(s"USE $paimonHiveCatalogName") + withDatabase("test_db") { + sql("CREATE DATABASE test_db") + sql("USE test_db") + withTable("t") { + withView("v") { + sql("CREATE TABLE t (id INT, c STRING) USING paimon") + sql(""" + |CREATE VIEW v + |COMMENT 'test comment' + |TBLPROPERTIES ('k1' = 'v1') + |AS SELECT * FROM t + |""".stripMargin) + + checkAnswer(sql("DESC TABLE v"), Seq(Row("id", "INT", null), Row("c", "STRING", null))) + + val rows = sql("DESC TABLE EXTENDED v").collectAsList() + assert(rows.get(3).toString().equals("[# Detailed View Information,,]")) + assert(rows.get(4).toString().equals("[Name,test_db.v,]")) + assert(rows.get(5).toString().equals("[Comment,test comment,]")) + assert(rows.get(6).toString().equals("[View Text,SELECT * FROM t,]")) + assert(rows.get(7).toString().equals("[View Query Output Columns,[id, c],]")) + assert(rows.get(8).toString().contains("[View Properties,[k1=v1")) + } + } + } + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala similarity index 72% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala index 501e7bfb4a515..78c02644a7ced 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PushDownAggregatesTest.scala @@ -117,22 +117,58 @@ class PushDownAggregatesTest extends PaimonSparkTestBase with AdaptiveSparkPlanH } } - 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 - primary key table with deletion vector") { + Seq(true, false).foreach( + deletionVectorsEnabled => { + withTable("T") { + spark.sql(s""" + |CREATE TABLE T (c1 INT, c2 STRING) + |TBLPROPERTIES ( + |'primary-key' = 'c1', + |'deletion-vectors.enabled' = $deletionVectorsEnabled + |) + |""".stripMargin) + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(0) :: Nil, 0) + + spark.sql("INSERT INTO T VALUES(1, 'x'), (2, 'x'), (3, 'x'), (3, 'x')") + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 0) + + spark.sql("INSERT INTO T VALUES(1, 'x_1')") + if (deletionVectorsEnabled) { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Row(3) :: Nil, 0) + } else { + 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) - } + test("Push down aggregate - table with deletion vector") { + Seq(true, false).foreach( + deletionVectorsEnabled => { + Seq(true, false).foreach( + primaryKeyTable => { + withTable("T") { + sql(s""" + |CREATE TABLE T (id INT) + |TBLPROPERTIES ( + | 'deletion-vectors.enabled' = $deletionVectorsEnabled, + | '${if (primaryKeyTable) "primary-key" else "bucket-key"}' = 'id', + | 'bucket' = '1' + |) + |""".stripMargin) + + sql("INSERT INTO T SELECT id FROM range (0, 5000)") + runAndCheckAggregate("SELECT COUNT(*) FROM T", Seq(Row(5000)), 0) + + sql("DELETE FROM T WHERE id > 100 and id <= 400") + if (deletionVectorsEnabled || !primaryKeyTable) { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Seq(Row(4700)), 0) + } else { + runAndCheckAggregate("SELECT COUNT(*) FROM T", Seq(Row(4700)), 2) + } + } + }) + }) } } diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/SparkVersionSupport.scala diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala new file mode 100644 index 0000000000000..b9c187b83a254 --- /dev/null +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/TableValuedFunctionsTest.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.sql + +import org.apache.paimon.spark.PaimonHiveTestBase + +import org.apache.spark.sql.{DataFrame, Row} + +class TableValuedFunctionsTest extends PaimonHiveTestBase { + + withPk.foreach { + hasPk => + bucketModes.foreach { + bucket => + test(s"incremental query: hasPk: $hasPk, bucket: $bucket") { + Seq("paimon", sparkCatalogName, paimonHiveCatalogName).foreach { + catalogName => + sql(s"use $catalogName") + + withTable("t") { + val prop = if (hasPk) { + s"'primary-key'='a,b', 'bucket' = '$bucket' " + } else if (bucket != -1) { + s"'bucket-key'='b', 'bucket' = '$bucket' " + } else { + "'write-only'='true'" + } + + spark.sql(s""" + |CREATE TABLE t (a INT, b INT, c STRING) + |USING paimon + |TBLPROPERTIES ($prop) + |PARTITIONED BY (a) + |""".stripMargin) + + spark.sql("INSERT INTO t values (1, 1, '1'), (2, 2, '2')") + spark.sql("INSERT INTO t VALUES (1, 3, '3'), (2, 4, '4')") + spark.sql("INSERT INTO t VALUES (1, 5, '5'), (1, 7, '7')") + + checkAnswer( + incrementalDF("t", 0, 1).orderBy("a", "b"), + Row(1, 1, "1") :: Row(2, 2, "2") :: Nil) + checkAnswer( + spark.sql( + "SELECT * FROM paimon_incremental_query('t', '0', '1') ORDER BY a, b"), + Row(1, 1, "1") :: Row(2, 2, "2") :: Nil) + + checkAnswer( + incrementalDF("t", 1, 2).orderBy("a", "b"), + Row(1, 3, "3") :: Row(2, 4, "4") :: Nil) + checkAnswer( + spark.sql( + "SELECT * FROM paimon_incremental_query('t', '1', '2') ORDER BY a, b"), + Row(1, 3, "3") :: Row(2, 4, "4") :: Nil) + + checkAnswer( + incrementalDF("t", 2, 3).orderBy("a", "b"), + Row(1, 5, "5") :: Row(1, 7, "7") :: Nil) + checkAnswer( + spark.sql( + "SELECT * FROM paimon_incremental_query('t', '2', '3') ORDER BY a, b"), + Row(1, 5, "5") :: Row(1, 7, "7") :: Nil) + + checkAnswer( + incrementalDF("t", 1, 3).orderBy("a", "b"), + Row(1, 3, "3") :: Row(1, 5, "5") :: Row(1, 7, "7") :: Row(2, 4, "4") :: Nil + ) + checkAnswer( + spark.sql( + "SELECT * FROM paimon_incremental_query('t', '1', '3') ORDER BY a, b"), + Row(1, 3, "3") :: Row(1, 5, "5") :: Row(1, 7, "7") :: Row(2, 4, "4") :: Nil) + } + } + } + } + } + + private def incrementalDF(tableIdent: String, start: Int, end: Int): DataFrame = { + spark.read + .format("paimon") + .option("incremental-between", s"$start,$end") + .table(tableIdent) + } +} diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTestBase.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala similarity index 100% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/WithTableOptions.scala diff --git a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/spark/paimon/Utils.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/spark/sql/paimon/Utils.scala similarity index 74% rename from paimon-spark/paimon-spark-common/src/test/scala/org/apache/spark/paimon/Utils.scala rename to paimon-spark/paimon-spark-ut/src/test/scala/org/apache/spark/sql/paimon/Utils.scala index 5ea2dd861e197..03f1c7706efbd 100644 --- a/paimon-spark/paimon-spark-common/src/test/scala/org/apache/spark/paimon/Utils.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/spark/sql/paimon/Utils.scala @@ -16,9 +16,10 @@ * limitations under the License. */ -package org.apache.spark.paimon +package org.apache.spark.sql.paimon -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.util.{Utils => SparkUtils} import java.io.File @@ -28,9 +29,14 @@ import java.io.File */ object Utils { - def createTempDir: File = SparkUtils.createTempDir() + def createTempDir: File = SparkUtils.createTempDir(System.getProperty("java.io.tmpdir"), "spark") def waitUntilEventEmpty(spark: SparkSession): Unit = { spark.sparkContext.listenerBus.waitUntilEmpty() } + + def createDataFrame(sparkSession: SparkSession, plan: LogicalPlan): DataFrame = { + Dataset.ofRows(sparkSession, plan) + } + } diff --git a/paimon-spark/paimon-spark3-common/pom.xml b/paimon-spark/paimon-spark3-common/pom.xml index 03d29ea05b3ad..5fd869f1b3938 100644 --- a/paimon-spark/paimon-spark3-common/pom.xml +++ b/paimon-spark/paimon-spark3-common/pom.xml @@ -39,9 +39,35 @@ under the License. - org.apache.spark - spark-sql_${scala.binary.version} - ${spark.version} + org.apache.paimon + paimon-spark-common_${scala.binary.version} + ${project.version} + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-paimon + package + + shade + + + + + org.apache.paimon:paimon-bundle + org.apache.paimon:paimon-spark-common_${scala.binary.version} + + + + + + + + \ No newline at end of file diff --git a/paimon-spark/paimon-spark3-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim b/paimon-spark/paimon-spark3-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim new file mode 100644 index 0000000000000..b79ef54f6e309 --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.spark.sql.paimon.shims.Spark3Shim \ No newline at end of file diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark3ResolutionRules.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark3ResolutionRules.scala new file mode 100644 index 0000000000000..924df2d1e3206 --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark3ResolutionRules.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.commands.{PaimonShowTablePartitionCommand, PaimonShowTablesExtendedCommand} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.analysis.{PartitionSpec, ResolvedNamespace, UnresolvedPartitionSpec} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ShowTableExtended} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.Identifier + +case class Spark3ResolutionRules(session: SparkSession) + extends Rule[LogicalPlan] + with SQLConfHelper { + + import org.apache.spark.sql.connector.catalog.PaimonCatalogImplicits._ + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown { + case ShowTableExtended( + ResolvedNamespace(catalog, ns), + pattern, + partitionSpec @ (None | Some(UnresolvedPartitionSpec(_, _))), + output) => + partitionSpec + .map { + spec: PartitionSpec => + val table = Identifier.of(ns.toArray, pattern) + val resolvedSpec = + PaimonResolvePartitionSpec.resolve(catalog.asTableCatalog, table, spec) + PaimonShowTablePartitionCommand(output, catalog.asTableCatalog, table, resolvedSpec) + } + .getOrElse { + PaimonShowTablesExtendedCommand(catalog.asTableCatalog, ns, pattern, output) + } + + } + +} diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark3SqlExtensionsParser.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark3SqlExtensionsParser.scala new file mode 100644 index 0000000000000..07481b6f639f8 --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark3SqlExtensionsParser.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.catalyst.parser.extensions + +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.parser.extensions.AbstractPaimonSparkSqlExtensionsParser + +class PaimonSpark3SqlExtensionsParser(override val delegate: ParserInterface) + extends AbstractPaimonSparkSqlExtensionsParser(delegate) {} diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablePartitionCommand.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablePartitionCommand.scala new file mode 100644 index 0000000000000..32f94985859cf --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablePartitionCommand.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.commands + +import org.apache.paimon.spark.leafnode.PaimonLeafRunnableCommand + +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.analysis.ResolvedPartitionSpec +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName +import org.apache.spark.sql.catalyst.expressions.{Attribute, ToPrettyString} +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.connector.catalog.{Identifier, SupportsPartitionManagement, TableCatalog} +import org.apache.spark.sql.connector.catalog.PaimonCatalogImplicits._ +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +case class PaimonShowTablePartitionCommand( + override val output: Seq[Attribute], + catalog: TableCatalog, + tableIndent: Identifier, + partSpec: ResolvedPartitionSpec) + extends PaimonLeafRunnableCommand { + override def run(sparkSession: SparkSession): Seq[Row] = { + val rows = new mutable.ArrayBuffer[Row]() + val table = catalog.loadTable(tableIndent) + val information = getTablePartitionDetails(tableIndent, table.asPartitionable, partSpec) + rows += Row(tableIndent.namespace.quoted, tableIndent.name(), false, s"$information\n") + + rows.toSeq + } + + private def getTablePartitionDetails( + tableIdent: Identifier, + partitionTable: SupportsPartitionManagement, + partSpec: ResolvedPartitionSpec): String = { + val results = new mutable.LinkedHashMap[String, String]() + + // "Partition Values" + val partitionSchema = partitionTable.partitionSchema() + val (names, ident) = (partSpec.names, partSpec.ident) + val partitionIdentifiers = partitionTable.listPartitionIdentifiers(names.toArray, ident) + if (partitionIdentifiers.isEmpty) { + val part = ident + .toSeq(partitionSchema) + .zip(partitionSchema.map(_.name)) + .map(kv => s"${kv._2}" + s" = ${kv._1}") + .mkString(", ") + throw new RuntimeException( + s""" + |[PARTITIONS_NOT_FOUND] The partition(s) PARTITION ($part) cannot be found in table ${tableIdent.toString}. + |Verify the partition specification and table name. + |""".stripMargin) + } + assert(partitionIdentifiers.length == 1) + val row = partitionIdentifiers.head + val len = partitionSchema.length + val partitions = new Array[String](len) + val timeZoneId = conf.sessionLocalTimeZone + for (i <- 0 until len) { + val dataType = partitionSchema(i).dataType + val partValueUTF8String = + ToPrettyString(Literal(row.get(i, dataType), dataType), Some(timeZoneId)).eval(null) + val partValueStr = if (partValueUTF8String == null) "null" else partValueUTF8String.toString + partitions(i) = escapePathName(partitionSchema(i).name) + "=" + escapePathName(partValueStr) + } + val partitionValues = partitions.mkString("[", ", ", "]") + results.put("Partition Values", s"$partitionValues") + + // TODO "Partition Parameters", "Created Time", "Last Access", "Partition Statistics" + + results + .map { + case (key, value) => + if (value.isEmpty) key else s"$key: $value" + } + .mkString("", "\n", "") + } +} diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablesExtendedCommand.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablesExtendedCommand.scala new file mode 100644 index 0000000000000..b393982e25d3e --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/commands/PaimonShowTablesExtendedCommand.scala @@ -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.spark.commands + +import org.apache.paimon.spark.leafnode.PaimonLeafRunnableCommand + +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.{QuotingUtils, StringUtils} +import org.apache.spark.sql.connector.catalog.{Identifier, PaimonCatalogUtils, SupportsPartitionManagement, Table, TableCatalog} +import org.apache.spark.sql.connector.catalog.PaimonCatalogImplicits._ +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +case class PaimonShowTablesExtendedCommand( + catalog: TableCatalog, + namespace: Seq[String], + pattern: String, + override val output: Seq[Attribute], + isExtended: Boolean = false, + partitionSpec: Option[TablePartitionSpec] = None) + extends PaimonLeafRunnableCommand { + + override def run(spark: SparkSession): Seq[Row] = { + val rows = new mutable.ArrayBuffer[Row]() + + val tables = catalog.listTables(namespace.toArray) + tables.map { + tableIdent: Identifier => + if (StringUtils.filterPattern(Seq(tableIdent.name()), pattern).nonEmpty) { + val table = catalog.loadTable(tableIdent) + val information = getTableDetails(catalog.name, tableIdent, table) + rows += Row(tableIdent.namespace().quoted, tableIdent.name(), false, s"$information\n") + } + } + + // TODO: view + + rows.toSeq + } + + private def getTableDetails(catalogName: String, identifier: Identifier, table: Table): String = { + val results = new mutable.LinkedHashMap[String, String]() + + results.put("Catalog", catalogName) + results.put("Namespace", identifier.namespace().quoted) + results.put("Table", identifier.name()) + val tableType = if (table.properties().containsKey(TableCatalog.PROP_EXTERNAL)) { + CatalogTableType.EXTERNAL + } else { + CatalogTableType.MANAGED + } + results.put("Type", tableType.name) + + PaimonCatalogUtils.TABLE_RESERVED_PROPERTIES + .filterNot(_ == TableCatalog.PROP_EXTERNAL) + .foreach( + propKey => { + if (table.properties.containsKey(propKey)) { + results.put(propKey.capitalize, table.properties.get(propKey)) + } + }) + + val properties: Seq[String] = + conf + .redactOptions(table.properties.asScala.toMap) + .toList + .filter(kv => !PaimonCatalogUtils.TABLE_RESERVED_PROPERTIES.contains(kv._1)) + .sortBy(_._1) + .map { case (key, value) => key + "=" + value } + if (!table.properties().isEmpty) { + results.put("Table Properties", properties.mkString("[", ", ", "]")) + } + + // Partition Provider & Partition Columns + if (supportsPartitions(table) && table.asPartitionable.partitionSchema().nonEmpty) { + results.put("Partition Provider", "Catalog") + results.put( + "Partition Columns", + table.asPartitionable + .partitionSchema() + .map(field => QuotingUtils.quoteIdentifier(field.name)) + .mkString("[", ", ", "]")) + } + + if (table.schema().nonEmpty) { + results.put("Schema", table.schema().treeString) + } + + results + .map { + case (key, value) => + if (value.isEmpty) key else s"$key: $value" + } + .mkString("", "\n", "") + } + + private def supportsPartitions(table: Table): Boolean = table match { + case _: SupportsPartitionManagement => true + case _ => false + } + +} diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3ArrayData.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3ArrayData.scala new file mode 100644 index 0000000000000..cb393d928dcbc --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3ArrayData.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.data + +import org.apache.paimon.types.DataType + +class Spark3ArrayData(override val elementType: DataType) extends AbstractSparkArrayData {} diff --git a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRow.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRow.scala new file mode 100644 index 0000000000000..9c9a1c6bac959 --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/paimon/spark/data/Spark3InternalRow.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.data + +import org.apache.paimon.spark.AbstractSparkInternalRow +import org.apache.paimon.types.RowType + +class Spark3InternalRow(rowType: RowType) extends AbstractSparkInternalRow(rowType) {} 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 deleted file mode 100644 index 13ade3f3c5ac6..0000000000000 --- a/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala +++ /dev/null @@ -1,72 +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.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-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala new file mode 100644 index 0000000000000..f508e2605cbc1 --- /dev/null +++ b/paimon-spark/paimon-spark3-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark3Shim.scala @@ -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.spark.sql.paimon.shims + +import org.apache.paimon.spark.catalyst.analysis.Spark3ResolutionRules +import org.apache.paimon.spark.catalyst.parser.extensions.PaimonSpark3SqlExtensionsParser +import org.apache.paimon.spark.data.{Spark3ArrayData, Spark3InternalRow, SparkArrayData, SparkInternalRow} +import org.apache.paimon.types.{DataType, RowType} + +import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.StructType + +import java.util.{Map => JMap} + +class Spark3Shim extends SparkShim { + + override def createSparkParser(delegate: ParserInterface): ParserInterface = { + new PaimonSpark3SqlExtensionsParser(delegate) + } + + override def createCustomResolution(spark: SparkSession): Rule[LogicalPlan] = { + Spark3ResolutionRules(spark) + } + + override def createSparkInternalRow(rowType: RowType): SparkInternalRow = { + new Spark3InternalRow(rowType) + } + + override def createSparkArrayData(elementType: DataType): SparkArrayData = { + new Spark3ArrayData(elementType) + } + + override def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean = { + Aggregate.supportsHashAggregate(aggregateBufferAttributes) + } + + override def createTable( + tableCatalog: TableCatalog, + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: JMap[String, String]): Table = { + tableCatalog.createTable(ident, schema, partitions, properties) + } + + override def column(expr: Expression): Column = new Column(expr) + + override 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 index dcc5b370d59aa..d160b984fa056 100644 --- a/paimon-spark/paimon-spark4-common/pom.xml +++ b/paimon-spark/paimon-spark4-common/pom.xml @@ -38,10 +38,57 @@ under the License. + + org.apache.paimon + paimon-spark-common_${scala.binary.version} + ${project.version} + + org.apache.spark - spark-sql_${scala.binary.version} + spark-sql-api_2.13 ${spark.version} + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + org.apache.logging.log4j + log4j-slf4j2-impl + + + - \ No newline at end of file + + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-paimon + package + + shade + + + + + org.apache.paimon:paimon-bundle + org.apache.paimon:paimon-spark-common_${scala.binary.version} + + + + + + + + + diff --git a/paimon-spark/paimon-spark4-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim b/paimon-spark/paimon-spark4-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim new file mode 100644 index 0000000000000..b0df8c67cf9a4 --- /dev/null +++ b/paimon-spark/paimon-spark4-common/src/main/resources/META-INF/services/org.apache.spark.sql.paimon.shims.SparkShim @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.spark.sql.paimon.shims.Spark4Shim \ No newline at end of file diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/DataLineageEntity.java b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark4ResolutionRules.scala similarity index 70% rename from paimon-common/src/main/java/org/apache/paimon/lineage/DataLineageEntity.java rename to paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark4ResolutionRules.scala index e7401a9be3b76..461cbd0c938ac 100644 --- a/paimon-common/src/main/java/org/apache/paimon/lineage/DataLineageEntity.java +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/analysis/Spark4ResolutionRules.scala @@ -16,18 +16,12 @@ * limitations under the License. */ -package org.apache.paimon.lineage; +package org.apache.paimon.spark.catalyst.analysis -import org.apache.paimon.data.Timestamp; +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule -/** - * Data lineage entity with table lineage, barrier id and snapshot id for table source and sink - * lineage. - */ -public interface DataLineageEntity extends TableLineageEntity { - long getBarrierId(); - - long getSnapshotId(); - - Timestamp getCreateTime(); +case class Spark4ResolutionRules(session: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala similarity index 63% rename from paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java rename to paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala index 6eabd1db7f382..ef1f5763d27b1 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/flink/table/catalog/CatalogMaterializedTable.java +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/catalyst/parser/extensions/PaimonSpark4SqlExtensionsParser.scala @@ -16,19 +16,13 @@ * limitations under the License. */ -package org.apache.flink.table.catalog; +package org.apache.paimon.spark.catalyst.parser.extensions -/** - * Dummy placeholder to resolve compatibility issue of CatalogMaterializedTable(introduced in flink - * 1.20). - */ -public interface CatalogMaterializedTable extends CatalogBaseTable { - /** Dummy LogicalRefreshMode placeholder. */ - enum LogicalRefreshMode {} +import org.apache.spark.sql.catalyst.parser.{CompoundBody, ParserInterface} +import org.apache.spark.sql.catalyst.parser.extensions.AbstractPaimonSparkSqlExtensionsParser - /** Dummy RefreshMode placeholder. */ - enum RefreshMode {} +class PaimonSpark4SqlExtensionsParser(override val delegate: ParserInterface) + extends AbstractPaimonSparkSqlExtensionsParser(delegate) { - /** Dummy RefreshStatus placeholder. */ - enum RefreshStatus {} + def parseScript(sqlScriptText: String): CompoundBody = delegate.parseScript(sqlScriptText) } diff --git a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntity.java b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4ArrayData.scala similarity index 72% rename from paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntity.java rename to paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4ArrayData.scala index c4312c4eb080f..be319c0a9c232 100644 --- a/paimon-common/src/main/java/org/apache/paimon/lineage/TableLineageEntity.java +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4ArrayData.scala @@ -16,17 +16,14 @@ * limitations under the License. */ -package org.apache.paimon.lineage; +package org.apache.paimon.spark.data -import org.apache.paimon.data.Timestamp; +import org.apache.paimon.types.DataType -/** Table lineage entity with database, table and job for table source and sink lineage. */ -public interface TableLineageEntity { - String getDatabase(); +import org.apache.spark.unsafe.types.VariantVal - String getTable(); +class Spark4ArrayData(override val elementType: DataType) extends AbstractSparkArrayData { - String getJob(); + override def getVariant(ordinal: Int): VariantVal = throw new UnsupportedOperationException - Timestamp getCreateTime(); } diff --git a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRow.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRow.scala new file mode 100644 index 0000000000000..54b0f420ea931 --- /dev/null +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/paimon/spark/data/Spark4InternalRow.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.spark.data + +import org.apache.paimon.spark.AbstractSparkInternalRow +import org.apache.paimon.types.RowType + +import org.apache.spark.unsafe.types.VariantVal + +class Spark4InternalRow(rowType: RowType) extends AbstractSparkInternalRow(rowType) { + override def getVariant(i: Int): VariantVal = throw new UnsupportedOperationException +} 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 deleted file mode 100644 index ee6c9ad35857f..0000000000000 --- a/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims.scala +++ /dev/null @@ -1,86 +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.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/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala new file mode 100644 index 0000000000000..eefddafdbfb89 --- /dev/null +++ b/paimon-spark/paimon-spark4-common/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.paimon.shims + +import org.apache.paimon.spark.catalyst.analysis.Spark4ResolutionRules +import org.apache.paimon.spark.catalyst.parser.extensions.PaimonSpark4SqlExtensionsParser +import org.apache.paimon.spark.data.{Spark4ArrayData, Spark4InternalRow, SparkArrayData, SparkInternalRow} +import org.apache.paimon.types.{DataType, RowType} + +import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, Table, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.internal.ExpressionUtils +import org.apache.spark.sql.types.StructType + +import java.util.{Map => JMap} + +class Spark4Shim extends SparkShim { + + override def createSparkParser(delegate: ParserInterface): ParserInterface = { + new PaimonSpark4SqlExtensionsParser(delegate) + } + + override def createCustomResolution(spark: SparkSession): Rule[LogicalPlan] = { + Spark4ResolutionRules(spark) + } + + override def createSparkInternalRow(rowType: RowType): SparkInternalRow = { + new Spark4InternalRow(rowType) + } + + override def createSparkArrayData(elementType: DataType): SparkArrayData = { + new Spark4ArrayData(elementType) + } + + def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean = { + Aggregate.supportsHashAggregate(aggregateBufferAttributes, groupingExpression) + } + + def createTable( + tableCatalog: TableCatalog, + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: JMap[String, String]): Table = { + val columns = CatalogV2Util.structTypeToV2Columns(schema) + tableCatalog.createTable(ident, columns, partitions, properties) + } + + def column(expr: Expression): Column = ExpressionUtils.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 aac73baa5fec4..61ecd20a0500a 100644 --- a/paimon-spark/pom.xml +++ b/paimon-spark/pom.xml @@ -39,6 +39,7 @@ under the License. paimon-spark-common + paimon-spark-ut @@ -114,6 +115,21 @@ under the License. + + org.apache.spark + spark-hive_${scala.binary.version} + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + org.apache.paimon paimon-bundle @@ -180,22 +196,6 @@ under the License. - - - org.apache.spark - spark-hive_${scala.binary.version} - test - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - @@ -278,6 +278,12 @@ under the License. aws-java-sdk-core ${aws.version} test + + + com.fasterxml.jackson.core + * + + diff --git a/pom.xml b/pom.xml index 24f2923319508..dbef98af06b21 100644 --- a/pom.xml +++ b/pom.xml @@ -69,6 +69,7 @@ under the License. paimon-test-utils paimon-arrow tools/ci/paimon-ci-tools + paimon-open-api @@ -115,6 +116,12 @@ under the License. 3.0.1-1.18 8.0.27 + + paimon-spark3-common + 3.5.3 + 3.3 + 3.3.0 + 1.5.5-11 3.0.11 3.4.6 @@ -522,6 +529,7 @@ under the License. release/** paimon-common/src/main/antlr4/** + paimon-core/src/test/resources/compatibility/** diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml index d5db52cb03df6..80e785353526f 100644 --- a/tools/maven/checkstyle.xml +++ b/tools/maven/checkstyle.xml @@ -74,7 +74,7 @@ This file is based on the checkstyle file of Apache Beam. --> - +