This is an automated email from the ASF dual-hosted git repository. lzljs3620320 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/incubator-paimon.git
commit 8856b617e904d7286f734ca86ec8dfa49394473e Author: JingsongLi <[email protected]> AuthorDate: Sat Mar 18 09:57:32 2023 +0800 [core] Rename paimon: Table Store to Paimon --- README.md | 6 +++--- docs/README.md | 6 +++--- docs/content/_index.md | 16 ++++++++-------- docs/content/concepts/basic-concepts.md | 4 ++-- docs/content/concepts/external-log-systems.md | 4 ++-- docs/content/concepts/file-layouts.md | 8 ++++---- docs/content/concepts/overview.md | 10 +++++----- docs/content/concepts/primary-key-table.md | 18 +++++++++--------- docs/content/engines/flink.md | 12 ++++++------ docs/content/engines/hive.md | 14 +++++++------- docs/content/engines/overview.md | 2 +- docs/content/engines/spark2.md | 12 ++++++------ docs/content/engines/spark3.md | 16 ++++++++-------- docs/content/filesystems/overview.md | 4 ++-- docs/content/how-to/creating-catalogs.md | 18 +++++++++--------- docs/content/how-to/creating-tables.md | 12 ++++++------ docs/content/how-to/lookup-joins.md | 6 +++--- docs/content/how-to/querying-tables.md | 4 ++-- docs/content/how-to/writing-tables.md | 12 ++++++------ docs/content/maintenance/expiring-snapshots.md | 4 ++-- docs/content/maintenance/manage-partition.md | 2 +- docs/content/maintenance/rescale-bucket.md | 2 +- docs/content/maintenance/write-performance.md | 14 +++++++------- docs/content/versions.md | 2 +- docs/layouts/partials/docs/inject/content-before.html | 4 ++-- .../shortcodes/generated/catalog_configuration.html | 2 +- paimon-benchmark/paimon-cluster-benchmark/README.md | 4 ++-- paimon-benchmark/paimon-micro-benchmarks/pom.xml | 2 +- .../main/java/org/apache/paimon/table/TableType.java | 4 ++-- .../main/java/org/apache/paimon/io/DataFileMeta.java | 4 ++-- .../paimon/operation/AbstractFileStoreScan.java | 2 +- .../paimon/operation/KeyValueFileStoreRead.java | 2 +- paimon-docs/README.md | 2 +- paimon-flink/paimon-flink-1.14/pom.xml | 2 +- .../flink/TableStoreDataStreamScanProvider.java | 2 +- .../flink/TableStoreDataStreamSinkProvider.java | 2 +- paimon-flink/paimon-flink-1.15/pom.xml | 2 +- paimon-flink/paimon-flink-1.16/pom.xml | 2 +- paimon-flink/paimon-flink-common/pom.xml | 8 ++++---- .../java/org/apache/paimon/flink/FlinkCatalog.java | 2 +- .../flink/TableStoreDataStreamScanProvider.java | 2 +- .../flink/TableStoreDataStreamSinkProvider.java | 2 +- .../org/apache/paimon/flink/action/ActionBase.java | 2 +- .../java/org/apache/paimon/flink/sink/FlinkSink.java | 4 ++-- .../flink/ChangelogWithKeyFileStoreTableITCase.java | 19 +++++++++++-------- .../org/apache/paimon/flink/FlinkCatalogTest.java | 2 +- .../org/apache/paimon/flink/ReadWriteTableITCase.java | 2 +- paimon-hive/paimon-hive-catalog/pom.xml | 2 +- paimon-hive/paimon-hive-connector-2.1-cdh-6.3/pom.xml | 2 +- paimon-hive/paimon-hive-connector-2.1/pom.xml | 2 +- paimon-hive/paimon-hive-connector-2.2/pom.xml | 2 +- paimon-hive/paimon-hive-connector-2.3/pom.xml | 2 +- paimon-hive/paimon-hive-connector-3.1/pom.xml | 2 +- paimon-hive/paimon-hive-connector-common/pom.xml | 4 ++-- .../main/java/org/apache/paimon/hive/HiveSchema.java | 4 +--- .../apache/paimon/hive/TableStoreHiveMetaHook.java | 2 +- .../org/apache/paimon/hive/HiveTableSchemaTest.java | 12 ++++++------ .../hive/SearchArgumentToPredicateConverterTest.java | 4 ++-- paimon-shade/pom.xml | 10 +++++----- paimon-spark/paimon-spark-2/pom.xml | 2 +- paimon-spark/paimon-spark-3.1/pom.xml | 2 +- paimon-spark/paimon-spark-3.2/pom.xml | 2 +- paimon-spark/paimon-spark-3.3/pom.xml | 2 +- paimon-spark/paimon-spark-common/pom.xml | 2 +- 64 files changed, 173 insertions(+), 172 deletions(-) diff --git a/README.md b/README.md index b11440d7b..3defa001b 100644 --- a/README.md +++ b/README.md @@ -1,8 +1,8 @@ -# Flink Table Store +# Paimon -Flink Table Store is a data lake storage for streaming updates/deletes changelog ingestion and high-performance queries in real time. +Paimon is a data lake storage for streaming updates/deletes changelog ingestion and high-performance queries in real time. -Flink Table Store is developed under the umbrella of [Apache Flink](https://flink.apache.org/). +Paimon is developed under the umbrella of [Apache Flink](https://flink.apache.org/). ## Documentation & Getting Started diff --git a/docs/README.md b/docs/README.md index 30fcbb5ea..439ab5d14 100644 --- a/docs/README.md +++ b/docs/README.md @@ -1,7 +1,7 @@ This README gives an overview of how to build and contribute to the -documentation of Flink Table Store. +documentation of Paimon. -The documentation is included with the source of Flink Table Store in order to ensure +The documentation is included with the source of Paimon in order to ensure that you always have docs corresponding to your checked-out version. # Requirements @@ -85,7 +85,7 @@ the page: ### ShortCodes -Flink Table Store uses [shortcodes](https://gohugo.io/content-management/shortcodes/) to add +Paimon uses [shortcodes](https://gohugo.io/content-management/shortcodes/) to add custom functionality to its documentation markdown. Its implementation and documentation can be found at diff --git a/docs/content/_index.md b/docs/content/_index.md index d1c142c75..79999f577 100644 --- a/docs/content/_index.md +++ b/docs/content/_index.md @@ -1,5 +1,5 @@ --- -title: Apache Flink Table Store +title: Apache Paimon type: docs bookToc: false --- @@ -22,26 +22,26 @@ specific language governing permissions and limitations under the License. --> -# Apache Flink Table Store +# Apache Paimon -Flink Table Store is a unified storage to build dynamic tables for both streaming and +Paimon is a unified storage to build dynamic tables for both streaming and batch processing in Flink, supporting high-speed data ingestion and timely data query. -Table Store offers the following core capabilities: +Paimon offers the following core capabilities: - Support storage of large datasets and allow read/write in both batch and streaming mode. - Support streaming queries with minimum latency down to milliseconds. - Support Batch/OLAP queries with minimum latency down to the second level. - Support incremental snapshots for stream consumption by default. So users do not need to combine different pipelines by themself. {{< columns >}} -## Try Table Store +## Try Paimon -If you’re interested in playing around with Flink Table Store, check out our +If you’re interested in playing around with Paimon, check out our quick start guide with [Flink]({{< ref "engines/flink" >}}), [Spark]({{< ref "engines/spark3" >}}) or [Hive]({{< ref "engines/hive" >}}). It provides a step by step introduction to the APIs and guides you through real applications. <---> -## Get Help with Table Store +## Get Help with Paimon If you get stuck, check out our [community support resources](https://flink.apache.org/community.html). In particular, Apache @@ -50,5 +50,5 @@ any Apache project, and is a great way to get help quickly. {{< /columns >}} -Flink Table Store is developed under the umbrella of +Paimon is developed under the umbrella of [Apache Flink](https://flink.apache.org/). diff --git a/docs/content/concepts/basic-concepts.md b/docs/content/concepts/basic-concepts.md index 4c9801065..b54db0e5f 100644 --- a/docs/content/concepts/basic-concepts.md +++ b/docs/content/concepts/basic-concepts.md @@ -32,7 +32,7 @@ A snapshot captures the state of a table at some point in time. Users can access ## Partition -Table Store adopts the same partitioning concept as Apache Hive to separate data. +Paimon adopts the same partitioning concept as Apache Hive to separate data. Partitioning is an optional way of dividing a table into related parts based on the values of particular columns like date, city, and department. Each table can have one or more partition keys to identify a particular partition. @@ -56,6 +56,6 @@ See [file layouts]({{< ref "concepts/file-layouts" >}}) for how files are divide ## Consistency Guarantees -Table Store writers uses two-phase commit protocol to atomically commit a batch of records to the table. Each commit produces at most two [snapshots]({{< ref "concepts/basic-concepts#snapshot" >}}) at commit time. +Paimon writers uses two-phase commit protocol to atomically commit a batch of records to the table. Each commit produces at most two [snapshots]({{< ref "concepts/basic-concepts#snapshot" >}}) at commit time. For any two writers modifying a table at the same time, as long as they do not modify the same bucket, their commits are serializable. If they modify the same bucket, only snapshot isolation is guaranteed. That is, the final table state may be a mix of the two commits, but no changes are lost. diff --git a/docs/content/concepts/external-log-systems.md b/docs/content/concepts/external-log-systems.md index 345b76f51..a7f6435d1 100644 --- a/docs/content/concepts/external-log-systems.md +++ b/docs/content/concepts/external-log-systems.md @@ -26,7 +26,7 @@ under the License. # External Log Systems -Aside from [underlying table files]({{< ref "concepts/primary-key-table#changelog-producers" >}}), changelog of Table Store can also be stored into or consumed from an external log system, such as Kafka. By specifying `log.system` table property, users can choose which external log system to use. +Aside from [underlying table files]({{< ref "concepts/primary-key-table#changelog-producers" >}}), changelog of Paimon can also be stored into or consumed from an external log system, such as Kafka. By specifying `log.system` table property, users can choose which external log system to use. If an external log system is used, all records written into table files will also be written into the log system. Changes produced by the streaming queries will thus come from the log system instead of table files. @@ -36,7 +36,7 @@ By default, changes in the log systems are visible to consumers only after a sna However, users can also specify the table property `'log.consistency' = 'eventual'` so that changelog written into the log system can be immediately consumed by the consumers, without waiting for the next snapshot. This behavior decreases the latency of changelog, but it can only guarantee the at-least-once semantics (that is, consumers might see duplicated records) due to possible failures. -If `'log.consistency' = 'eventual'` is set, in order to achieve correct results, Table Store source in Flink will automatically adds a "normalize" operator for deduplication. This operator persists the values of each key in states. As one can easily tell, this operator will be very costly and should be avoided. +If `'log.consistency' = 'eventual'` is set, in order to achieve correct results, Paimon source in Flink will automatically adds a "normalize" operator for deduplication. This operator persists the values of each key in states. As one can easily tell, this operator will be very costly and should be avoided. ## Supported Log Systems diff --git a/docs/content/concepts/file-layouts.md b/docs/content/concepts/file-layouts.md index b5b081d9c..f00016ae4 100644 --- a/docs/content/concepts/file-layouts.md +++ b/docs/content/concepts/file-layouts.md @@ -26,7 +26,7 @@ under the License. # File Layouts -All files of a table are stored under one base directory. Table Store files are organized in a layered style. The following image illustrates the file layout. Starting from a snapshot file, Table Store readers can recursively access all records from the table. +All files of a table are stored under one base directory. Paimon files are organized in a layered style. The following image illustrates the file layout. Starting from a snapshot file, Paimon readers can recursively access all records from the table. {{< img src="/img/file-layout.png">}} @@ -53,7 +53,7 @@ Data files are grouped by partitions and buckets. Each bucket directory contains ## LSM Trees -Table Store adapts the LSM tree (log-structured merge-tree) as the data structure for file storage. This documentation briefly introduces the concepts about LSM trees. +Paimon adapts the LSM tree (log-structured merge-tree) as the data structure for file storage. This documentation briefly introduces the concepts about LSM trees. ### Sorted Runs @@ -73,6 +73,6 @@ When more and more records are written into the LSM tree, the number of sorted r To limit the number of sorted runs, we have to merge several sorted runs into one big sorted run once in a while. This procedure is called compaction. -However, compaction is a resource intensive procedure which consumes a certain amount of CPU time and disk IO, so too frequent compaction may in turn result in slower writes. It is a trade-off between query and write performance. Table Store currently adapts a compaction strategy similar to Rocksdb's [universal compaction](https://github.com/facebook/rocksdb/wiki/Universal-Compaction). +However, compaction is a resource intensive procedure which consumes a certain amount of CPU time and disk IO, so too frequent compaction may in turn result in slower writes. It is a trade-off between query and write performance. Paimon currently adapts a compaction strategy similar to Rocksdb's [universal compaction](https://github.com/facebook/rocksdb/wiki/Universal-Compaction). -By default, when Table Store writers append records to the LSM tree, they'll also perform compactions as needed. Users can also choose to perform all compactions in a dedicated compaction job. See [dedicated compaction job]({{< ref "maintenance/write-performance#dedicated-compaction-job" >}}) for more info. +By default, when Paimon writers append records to the LSM tree, they'll also perform compactions as needed. Users can also choose to perform all compactions in a dedicated compaction job. See [dedicated compaction job]({{< ref "maintenance/write-performance#dedicated-compaction-job" >}}) for more info. diff --git a/docs/content/concepts/overview.md b/docs/content/concepts/overview.md index 203d9a7b8..3a1d8857d 100644 --- a/docs/content/concepts/overview.md +++ b/docs/content/concepts/overview.md @@ -26,7 +26,7 @@ under the License. # Overview -Flink Table Store is a unified storage to build dynamic tables for both streaming and +Paimon is a unified storage to build dynamic tables for both streaming and batch processing in Flink, supporting high-speed data ingestion and timely data query. ## Architecture @@ -35,7 +35,7 @@ batch processing in Flink, supporting high-speed data ingestion and timely data As shown in the architecture above: -**Read/Write:** Table Store supports a versatile way to read/write data and perform OLAP queries. +**Read/Write:** Paimon supports a versatile way to read/write data and perform OLAP queries. - For reads, it supports consuming data - from historical snapshots (in batch mode), - from the latest offset (in streaming mode), or @@ -43,10 +43,10 @@ As shown in the architecture above: - For writes, it supports streaming synchronization from the changelog of databases (CDC) or batch insert/overwrite from offline data. -**Ecosystem:** In addition to Apache Flink, Table Store also supports read by other computation +**Ecosystem:** In addition to Apache Flink, Paimon also supports read by other computation engines like Apache Hive, Apache Spark and Trino. -**Internal:** Under the hood, Table Store uses a hybrid storage architecture with a lake format to store +**Internal:** Under the hood, Paimon uses a hybrid storage architecture with a lake format to store historical data and a queue system to store incremental data. The former stores the columnar files on the filesystem/object-store and uses the LSM tree structure to support a large volume of data updates and high-performance queries. The latter uses Apache Kafka to capture data in real-time. @@ -62,7 +62,7 @@ There are three types of connectors in Flink SQL. - Batch storage, such as Apache Hive, it supports various operations of the traditional batch processing, including `INSERT OVERWRITE`. -Flink Table Store provides table abstraction. It is used in a way that +Paimon provides table abstraction. It is used in a way that does not differ from the traditional database: - In Flink `batch` execution mode, it acts like a Hive table and supports various operations of Batch SQL. Query it to see the diff --git a/docs/content/concepts/primary-key-table.md b/docs/content/concepts/primary-key-table.md index 1ae10cb4b..2cf5586af 100644 --- a/docs/content/concepts/primary-key-table.md +++ b/docs/content/concepts/primary-key-table.md @@ -28,13 +28,13 @@ under the License. Changelog table is the default table type when creating a table. Users can insert, update or delete records in the table. -Primary keys are a set of columns that are unique for each record. Table Store imposes an ordering of data, which means the system will sort the primary key within each bucket. Using this feature, users can achieve high performance by adding filter conditions on the primary key. +Primary keys are a set of columns that are unique for each record. Paimon imposes an ordering of data, which means the system will sort the primary key within each bucket. Using this feature, users can achieve high performance by adding filter conditions on the primary key. By [defining primary keys]({{< ref "how-to/creating-tables#tables-with-primary-keys" >}}) on a changelog table, users can access the following features. ## Merge Engines -When Table Store sink receives two or more records with the same primary keys, it will merge them into one record to keep primary keys unique. By specifying the `merge-engine` table property, users can choose how records are merged together. +When Paimon sink receives two or more records with the same primary keys, it will merge them into one record to keep primary keys unique. By specifying the `merge-engine` table property, users can choose how records are merged together. {{< hint info >}} Set `table.exec.sink.upsert-materialize` to `NONE` always in Flink SQL TableConfig, sink upsert-materialize may @@ -44,7 +44,7 @@ result in strange behavior. When the input is out of order, we recommend that yo ### Deduplicate -`deduplicate` merge engine is the default merge engine. Table Store will only keep the latest record and throw away other records with the same primary keys. +`deduplicate` merge engine is the default merge engine. Paimon will only keep the latest record and throw away other records with the same primary keys. Specifically, if the latest record is a `DELETE` record, all records with the same primary keys will be deleted. @@ -52,7 +52,7 @@ Specifically, if the latest record is a `DELETE` record, all records with the sa By specifying `'merge-engine' = 'partial-update'`, users can set columns of a record across multiple updates and finally get a complete record. Specifically, value fields are updated to the latest data one by one under the same primary key, but null values are not overwritten. -For example, let's say Table Store receives three records: +For example, let's say Paimon receives three records: - `<1, 23.0, 10, NULL>`- - `<1, NULL, NULL, 'This is a book'>` - `<1, 25.2, NULL, NULL>` @@ -126,7 +126,7 @@ The `changelog-producer` table property only affects changelog from files. It do ### None -By default, no extra changelog producer will be applied to the writer of table. Table Store source can only see the merged changes across snapshots, like what keys are removed and what are the new values of some keys. +By default, no extra changelog producer will be applied to the writer of table. Paimon source can only see the merged changes across snapshots, like what keys are removed and what are the new values of some keys. However, these merged changes cannot form a complete changelog, because we can't read the old values of the keys directly from them. Merged changes require the consumers to "remember" the values of each key and to rewrite the values without seeing the old ones. Some consumers, however, need the old values to ensure correctness or efficiency. @@ -138,9 +138,9 @@ To conclude, `none` changelog producers are best suited for consumers such as a ### Input -By specifying `'changelog-producer' = 'input'`, Table Store writers rely on their inputs as a source of complete changelog. All input records will be saved in separated [changelog files]({{< ref "concepts/file-layouts" >}}) and will be given to the consumers by Table Store sources. +By specifying `'changelog-producer' = 'input'`, Paimon writers rely on their inputs as a source of complete changelog. All input records will be saved in separated [changelog files]({{< ref "concepts/file-layouts" >}}) and will be given to the consumers by Paimon sources. -`input` changelog producer can be used when Table Store writers' inputs are complete changelog, such as from a database CDC, or generated by Flink stateful computation. +`input` changelog producer can be used when Paimon writers' inputs are complete changelog, such as from a database CDC, or generated by Flink stateful computation. {{< img src="/img/changelog-producer-input.png">}} @@ -152,7 +152,7 @@ This is an experimental feature. 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'`, Table Store 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. {{< img src="/img/changelog-producer-lookup.png">}} @@ -194,7 +194,7 @@ Lookup will cache data on the memory and local disk, you can use the following o 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). -By specifying `'changelog-producer' = 'full-compaction'`, Table Store 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. +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. By specifying `changelog-producer.compaction-interval` table property (default value `0s`), users can define the maximum interval between two full compactions to ensure latency. This is set to 0 by default, so each checkpoint will have a full compression and generate a change log. diff --git a/docs/content/engines/flink.md b/docs/content/engines/flink.md index ec7ddce69..913e5dc8d 100644 --- a/docs/content/engines/flink.md +++ b/docs/content/engines/flink.md @@ -26,11 +26,11 @@ under the License. # Flink -This documentation is a guide for using Table Store in Flink. +This documentation is a guide for using Paimon in Flink. -## Preparing Table Store Jar File +## Preparing Paimon Jar File -Table Store currently supports Flink 1.16, 1.15 and 1.14. We recommend the latest Flink version for a better experience. +Paimon currently supports Flink 1.16, 1.15 and 1.14. We recommend the latest Flink version for a better experience. {{< stable >}} @@ -48,7 +48,7 @@ You can also manually build bundled jar from the source code. {{< unstable >}} -You are using an unreleased version of Table Store so you need to manually build bundled jar from the source code. +You are using an unreleased version of Paimon so you need to manually build bundled jar from the source code. {{< /unstable >}} @@ -69,7 +69,7 @@ If you haven't downloaded Flink, you can [download Flink 1.16](https://flink.apa tar -xzf flink-*.tgz ``` -**Step 2: Copy Table Store Bundled Jar** +**Step 2: Copy Paimon Bundled Jar** Copy paimon bundled jar to the `lib` directory of your Flink home. @@ -111,7 +111,7 @@ You can now start Flink SQL client to execute SQL scripts. **Step 5: Create a Catalog and a Table** ```sql --- if you're trying out Table Store in a distributed environment, +-- if you're trying out Paimon in a distributed environment, -- warehouse path should be set to a shared file system, such as HDFS or OSS CREATE CATALOG my_catalog WITH ( 'type'='paimon', diff --git a/docs/content/engines/hive.md b/docs/content/engines/hive.md index ac7488740..5ae357cbd 100644 --- a/docs/content/engines/hive.md +++ b/docs/content/engines/hive.md @@ -26,15 +26,15 @@ under the License. # Hive -This documentation is a guide for using Table Store in Hive. +This documentation is a guide for using Paimon in Hive. ## Version -Table Store currently supports Hive 2.1, 2.1-cdh-6.3, 2.2, 2.3 and 3.1. +Paimon currently supports Hive 2.1, 2.1-cdh-6.3, 2.2, 2.3 and 3.1. ## Execution Engine -Table Store currently supports MR and Tez execution engine for Hive. +Paimon currently supports MR and Tez execution engine for Hive. ## Installation @@ -56,7 +56,7 @@ You can also manually build bundled jar from the source code. {{< unstable >}} -You are using an unreleased version of Table Store so you need to manually build bundled jar from the source code. +You are using an unreleased version of Paimon so you need to manually build bundled jar from the source code. {{< /unstable >}} @@ -74,17 +74,17 @@ There are several ways to add this jar to Hive. NOTE: If you are using HDFS, make sure that the environment variable `HADOOP_HOME` or `HADOOP_CONF_DIR` is set. -## Quick Start with Table Store Hive Catalog +## Quick Start with Paimon Hive Catalog By using paimon Hive catalog, you can create, drop and insert into paimon tables from Flink. These operations directly affect the corresponding Hive metastore. Tables created in this way can also be accessed directly from Hive. -**Step 1: Prepare Table Store Hive Catalog Jar File for Flink** +**Step 1: Prepare Paimon Hive Catalog Jar File for Flink** See [creating a catalog with Hive metastore]({{< ref "how-to/creating-catalogs#creating-a-catalog-with-hive-metastore" >}}). **Step 2: Create Test Data with Flink SQL** -Execute the following Flink SQL script in Flink SQL client to define a Table Store Hive catalog and create a table. +Execute the following Flink SQL script in Flink SQL client to define a Paimon Hive catalog and create a table. ```sql -- Flink SQL CLI diff --git a/docs/content/engines/overview.md b/docs/content/engines/overview.md index 7166512b5..ee13025f7 100644 --- a/docs/content/engines/overview.md +++ b/docs/content/engines/overview.md @@ -26,7 +26,7 @@ under the License. # Overview -Table Store not only supports Flink SQL writes and queries natively, +Paimon not only supports Flink SQL writes and queries natively, but also provides queries from other popular engines, such as Apache Spark and Apache Hive. diff --git a/docs/content/engines/spark2.md b/docs/content/engines/spark2.md index ffafa8eeb..00872a3fe 100644 --- a/docs/content/engines/spark2.md +++ b/docs/content/engines/spark2.md @@ -26,13 +26,13 @@ under the License. # Spark2 -This documentation is a guide for using Table Store in Spark2. +This documentation is a guide for using Paimon in Spark2. ## Version -Table Store supports Spark 2.4+. It is highly recommended to use Spark 2.4+ version with many improvements. +Paimon supports Spark 2.4+. It is highly recommended to use Spark 2.4+ version with many improvements. -## Preparing Table Store Jar File +## Preparing Paimon Jar File {{< stable >}} @@ -44,7 +44,7 @@ You can also manually build bundled jar from the source code. {{< unstable >}} -You are using an unreleased version of Table Store so you need to manually build bundled jar from the source code. +You are using an unreleased version of Paimon so you need to manually build bundled jar from the source code. {{< /unstable >}} @@ -68,11 +68,11 @@ If you are using HDFS, make sure that the environment variable `HADOOP_HOME` or **Step 1: Prepare Test Data** -Table Store currently only supports reading tables through Spark2. To create a Table Store table with records, please follow our [Flink quick start guide]({{< ref "engines/flink#quick-start" >}}). +Paimon currently only supports reading tables through Spark2. To create a Paimon table with records, please follow our [Flink quick start guide]({{< ref "engines/flink#quick-start" >}}). After the guide, all table files should be stored under the path `/tmp/table_store`, or the warehouse path you've specified. -**Step 2: Specify Table Store Jar File** +**Step 2: Specify Paimon Jar File** You can append path to paimon jar file to the `--jars` argument when starting `spark-shell`. diff --git a/docs/content/engines/spark3.md b/docs/content/engines/spark3.md index c685cb60e..c9d013d69 100644 --- a/docs/content/engines/spark3.md +++ b/docs/content/engines/spark3.md @@ -26,11 +26,11 @@ under the License. # Spark3 -This documentation is a guide for using Table Store in Spark3. +This documentation is a guide for using Paimon in Spark3. -## Preparing Table Store Jar File +## Preparing Paimon Jar File -Table Store currently supports Spark 3.3, 3.2 and 3.1. We recommend the latest Spark version for a better experience. +Paimon currently supports Spark 3.3, 3.2 and 3.1. We recommend the latest Spark version for a better experience. {{< stable >}} @@ -48,7 +48,7 @@ You can also manually build bundled jar from the source code. {{< unstable >}} -You are using an unreleased version of Table Store so you need to manually build bundled jar from the source code. +You are using an unreleased version of Paimon so you need to manually build bundled jar from the source code. {{< /unstable >}} @@ -70,7 +70,7 @@ If you are using HDFS, make sure that the environment variable `HADOOP_HOME` or {{< /hint >}} -**Step 1: Specify Table Store Jar File** +**Step 1: Specify Paimon Jar File** Append path to paimon jar file to the `--jars` argument when starting `spark-sql`. @@ -80,9 +80,9 @@ spark-sql ... --jars /path/to/paimon-spark-3.3-{{< version >}}.jar Alternatively, you can copy `paimon-spark-3.3-{{< version >}}.jar` under `spark/jars` in your Spark installation directory. -**Step 2: Specify Table Store Catalog** +**Step 2: Specify Paimon Catalog** -When starting `spark-sql`, use the following command to register Table Store’s Spark catalog with the name `paimon`. Table files of the warehouse is stored under `/tmp/table_store`. +When starting `spark-sql`, use the following command to register Paimon’s Spark catalog with the name `paimon`. Table files of the warehouse is stored under `/tmp/table_store`. ```bash spark-sql ... \ @@ -135,7 +135,7 @@ SELECT * FROM my_table; **Step 6: Query Table with Scala API** -If you don't want to use Table Store catalog, you can also run `spark-shell` and query the table with Scala API. +If you don't want to use Paimon catalog, you can also run `spark-shell` and query the table with Scala API. ```bash spark-shell ... --jars /path/to/paimon-spark-3.3-{{< version >}}.jar diff --git a/docs/content/filesystems/overview.md b/docs/content/filesystems/overview.md index 82be82fbb..de908ad6b 100644 --- a/docs/content/filesystems/overview.md +++ b/docs/content/filesystems/overview.md @@ -26,11 +26,11 @@ under the License. # Overview -Apache Flink Table Store utilizes the same pluggable file systems as Apache Flink. Users can follow the +Apache Paimon utilizes the same pluggable file systems as Apache Flink. Users can follow the [standard plugin mechanism](https://nightlies.apache.org/flink/flink-docs-stable/docs/deployment/filesystems/plugins/) to configure the plugin structure if using Flink as compute engine. However, for other engines like Spark or Hive, the provided opt jars (by Flink) may get conflicts and cannot be used directly. It is not convenient -for users to fix class conflicts, thus Flink Table Store provides the self-contained and engine-unified +for users to fix class conflicts, thus Paimon provides the self-contained and engine-unified FileSystem pluggable jars for user to query tables from Spark/Hive side. ## Supported FileSystems diff --git a/docs/content/how-to/creating-catalogs.md b/docs/content/how-to/creating-catalogs.md index aaa1d8a67..3be4609b0 100644 --- a/docs/content/how-to/creating-catalogs.md +++ b/docs/content/how-to/creating-catalogs.md @@ -26,7 +26,7 @@ under the License. # Creating Catalogs -Table Store catalogs currently support two types of metastores: +Paimon catalogs currently support two types of metastores: * `filesystem` metastore (default), which stores both metadata and table files in filesystems. * `hive` metastore, which additionally stores metadata in Hive metastore. Users can directly access the tables from Hive. @@ -39,7 +39,7 @@ See [CatalogOptions]({{< ref "maintenance/configurations#catalogoptions" >}}) fo {{< tab "Flink" >}} -The following Flink SQL registers and uses a Table Store catalog named `my_catalog`. Metadata and table files are stored under `hdfs://path/to/warehouse`. +The following Flink SQL registers and uses a Paimon catalog named `my_catalog`. Metadata and table files are stored under `hdfs://path/to/warehouse`. ```sql CREATE CATALOG my_catalog WITH ( @@ -74,9 +74,9 @@ USE paimon.default; ## Creating a Catalog with Hive Metastore -By using Table Store Hive catalog, changes to the catalog will directly affect the corresponding Hive metastore. Tables created in such catalog can also be accessed directly from Hive. +By using Paimon Hive catalog, changes to the catalog will directly affect the corresponding Hive metastore. Tables created in such catalog can also be accessed directly from Hive. -### Preparing Table Store Hive Catalog Jar File +### Preparing Paimon Hive Catalog Jar File {{< stable >}} @@ -92,7 +92,7 @@ You can also manually build bundled jar from the source code. {{< unstable >}} -You are using an unreleased version of Table Store so you need to manually build bundled jar from the source code. +You are using an unreleased version of Paimon so you need to manually build bundled jar from the source code. {{< /unstable >}} @@ -109,12 +109,12 @@ You can find Hive catalog jar in `./paimon-hive/paimon-hive-catalog/target/paimo {{< tab "Flink" >}} -To enable Table Store Hive catalog support in Flink, you can pick one of the following two methods. +To enable Paimon Hive catalog support in Flink, you can pick one of the following two methods. -* Copy Table Store Hive catalog jar file into the `lib` directory of your Flink installation directory. Note that this must be done before starting your Flink cluster. +* Copy Paimon Hive catalog jar file into the `lib` directory of your Flink installation directory. Note that this must be done before starting your Flink cluster. * If you're using Flink's SQL client, append `--jar /path/to/paimon-hive-catalog-{{< version >}}.jar` to the starting command of SQL client. -The following Flink SQL registers and uses a Table Store Hive catalog named `my_hive`. Metadata and table files are stored under `hdfs://path/to/warehouse`. In addition, metadata is also stored in Hive metastore. +The following Flink SQL registers and uses a Paimon Hive catalog named `my_hive`. Metadata and table files are stored under `hdfs://path/to/warehouse`. In addition, metadata is also stored in Hive metastore. ```sql CREATE CATALOG my_hive WITH ( @@ -131,7 +131,7 @@ USE CATALOG my_hive; {{< tab "Spark3" >}} -To enable Table Store Hive catalog support in Spark3, append the path of Table Store Hive catalog jar file to `--jars` argument when starting spark. +To enable Paimon Hive catalog support in Spark3, append the path of Paimon Hive catalog jar file to `--jars` argument when starting spark. The following shell command registers a Table tore Hive catalog named `paimon`. Metadata and table files are stored under `hdfs://path/to/warehouse`. In addition, metadata is also stored in Hive metastore. diff --git a/docs/content/how-to/creating-tables.md b/docs/content/how-to/creating-tables.md index 9d585518f..114e6cb1d 100644 --- a/docs/content/how-to/creating-tables.md +++ b/docs/content/how-to/creating-tables.md @@ -28,9 +28,9 @@ under the License. ## Creating Catalog Managed Tables -Tables created in Table Store [catalogs]({{< ref "how-to/creating-catalogs" >}}) are managed by the catalog. When the table is dropped from catalog, its table files will also be deleted. +Tables created in Paimon [catalogs]({{< ref "how-to/creating-catalogs" >}}) are managed by the catalog. When the table is dropped from catalog, its table files will also be deleted. -The following SQL assumes that you have registered and are using a Table Store catalog. It creates a managed table named `MyTable` with five columns in the catalog's `default` database, where `dt`, `hh` and `user_id` are the primary keys. +The following SQL assumes that you have registered and are using a Paimon catalog. It creates a managed table named `MyTable` with five columns in the catalog's `default` database, where `dt`, `hh` and `user_id` are the primary keys. {{< tabs "primary-keys-example" >}} @@ -263,7 +263,7 @@ CREATE TABLE MyTableLike LIKE MyTable; ### Table Properties -Users can specify table properties to enable features or improve performance of Table Store. For a complete list of such properties, see [configurations]({{< ref "maintenance/configurations" >}}). +Users can specify table properties to enable features or improve performance of Paimon. For a complete list of such properties, see [configurations]({{< ref "maintenance/configurations" >}}). The following SQL creates a table named `MyTable` with five columns partitioned by `dt` and `hh`, where `dt`, `hh` and `user_id` are the primary keys. This table has two properties: `'bucket' = '2'` and `'bucket-key' = 'user_id'`. @@ -311,13 +311,13 @@ CREATE TABLE MyTable ( External tables are recorded but not managed by catalogs. If an external table is dropped, its table files will not be deleted. -Table Store external tables can be used in any catalog. If you do not want to create a Table Store catalog and just want to read / write a table, you can consider external tables. +Paimon external tables can be used in any catalog. If you do not want to create a Paimon catalog and just want to read / write a table, you can consider external tables. {{< tabs "external-table-example" >}} {{< tab "Flink" >}} -Flink SQL supports reading and writing an external table. External Table Store tables are created by specifying the `connector` and `path` table properties. The following SQL creates an external table named `MyTable` with five columns, where the base path of table files is `hdfs://path/to/table`. +Flink SQL supports reading and writing an external table. External Paimon tables are created by specifying the `connector` and `path` table properties. The following SQL creates an external table named `MyTable` with five columns, where the base path of table files is `hdfs://path/to/table`. ```sql CREATE TABLE MyTable ( @@ -379,7 +379,7 @@ LOCATION 'hdfs://path/to/table'; Temporary tables are only supported by Flink. Like external tables, temporary tables are just recorded but not managed by the current Flink SQL session. If the temporary table is dropped, its resources will not be deleted. Temporary tables are also dropped when Flink SQL session is closed. -If you want to use Table Store catalog along with other tables but do not want to store them in other catalogs, you can create a temporary table. The following Flink SQL creates a Table Store catalog and a temporary table and also illustrates how to use both tables together. +If you want to use Paimon catalog along with other tables but do not want to store them in other catalogs, you can create a temporary table. The following Flink SQL creates a Paimon catalog and a temporary table and also illustrates how to use both tables together. ```sql CREATE CATALOG my_catalog WITH ( diff --git a/docs/content/how-to/lookup-joins.md b/docs/content/how-to/lookup-joins.md index 04993afff..0f1475545 100644 --- a/docs/content/how-to/lookup-joins.md +++ b/docs/content/how-to/lookup-joins.md @@ -26,11 +26,11 @@ under the License. # Lookup Joins -[Lookup Joins](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/table/sql/queries/joins/) are a type of join in streaming queries. It is used to enrich a table with data that is queried from Table Store. The join requires one table to have a processing time attribute and the other table to be backed by a lookup source connector. +[Lookup Joins](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/table/sql/queries/joins/) are a type of join in streaming queries. It is used to enrich a table with data that is queried from Paimon. The join requires one table to have a processing time attribute and the other table to be backed by a lookup source connector. -Table Store supports lookup joins on unpartitioned tables with primary keys in Flink. The following example illustrates this feature. +Paimon supports lookup joins on unpartitioned tables with primary keys in Flink. The following example illustrates this feature. -First, let's create a Table Store table and update it in real-time. +First, let's create a Paimon table and update it in real-time. ```sql -- Create a paimon catalog diff --git a/docs/content/how-to/querying-tables.md b/docs/content/how-to/querying-tables.md index 47106e940..5f85b7a4c 100644 --- a/docs/content/how-to/querying-tables.md +++ b/docs/content/how-to/querying-tables.md @@ -26,11 +26,11 @@ under the License. # Querying Tables -Just like all other tables, Table Store tables can be queried with `SELECT` statement. +Just like all other tables, Paimon tables can be queried with `SELECT` statement. ## Scan Mode -By specifying the `scan.mode` table property, users can specify where and how Table Store sources should produce records. +By specifying the `scan.mode` table property, users can specify where and how Paimon sources should produce records. <table class="table table-bordered"> <thead> diff --git a/docs/content/how-to/writing-tables.md b/docs/content/how-to/writing-tables.md index aa5b989bf..5a4fe7a8a 100644 --- a/docs/content/how-to/writing-tables.md +++ b/docs/content/how-to/writing-tables.md @@ -96,7 +96,7 @@ Use `INSERT INTO` to apply records and changes to tables. INSERT INTO MyTable SELECT ... ``` -Table Store supports shuffle data by bucket in sink phase. To improve data skew, Table Store also +Paimon supports shuffle data by bucket in sink phase. To improve data skew, Paimon also supports shuffling data by partition fields. You can add option `sink.partition-shuffle` to the table. {{< /tab >}} @@ -115,7 +115,7 @@ INSERT INTO MyTable SELECT ... ## Overwriting the Whole Table -For unpartitioned tables, Table Store supports overwriting the whole table. +For unpartitioned tables, Paimon supports overwriting the whole table. {{< tabs "insert-overwrite-unpartitioned-example" >}} @@ -133,7 +133,7 @@ INSERT OVERWRITE MyTable SELECT ... ## Overwriting a Partition -For partitioned tables, Table Store supports overwriting a partition. +For partitioned tables, Paimon supports overwriting a partition. {{< tabs "insert-overwrite-partitioned-example" >}} @@ -167,7 +167,7 @@ INSERT OVERWRITE MyTable SELECT * FROM MyTable WHERE false ## Purging Partitions -Currently, Table Store supports two ways to purge partitions. +Currently, Paimon supports two ways to purge partitions. 1. Like purging tables, you can use `INSERT OVERWRITE` to purge data of partitions by inserting empty value to them. @@ -234,7 +234,7 @@ For more information of drop-partition, see ## Deleting from table -Currently, Table Store supports deleting records via submitting the 'delete' job through `flink run`. +Currently, Paimon supports deleting records via submitting the 'delete' job through `flink run`. {{< tabs "delete-from-table" >}} @@ -275,7 +275,7 @@ For more information of 'delete', see ## Merging into table -Table Store supports "MERGE INTO" via submitting the 'merge-into' job through `flink run`. +Paimon supports "MERGE INTO" via submitting the 'merge-into' job through `flink run`. {{< hint info >}} Important table properties setting: diff --git a/docs/content/maintenance/expiring-snapshots.md b/docs/content/maintenance/expiring-snapshots.md index 44938af1a..479316b52 100644 --- a/docs/content/maintenance/expiring-snapshots.md +++ b/docs/content/maintenance/expiring-snapshots.md @@ -26,9 +26,9 @@ under the License. # Expiring Snapshots -Table Store writers generates one or two [snapshots]({{< ref "concepts/basic-concepts#snapshots" >}}) per commit. Each snapshot may add some new data files or mark some old data files as deleted. However, the marked data files are not truly deleted because Table Store also supports time traveling to an earlier snapshot. They are only deleted when the snapshot expires. +Paimon writers generates one or two [snapshots]({{< ref "concepts/basic-concepts#snapshots" >}}) per commit. Each snapshot may add some new data files or mark some old data files as deleted. However, the marked data files are not truly deleted because Paimon also supports time traveling to an earlier snapshot. They are only deleted when the snapshot expires. -Currently, expiration is automatically performed by Table Store writers when committing new changes. By expiring old snapshots, old data files and metadata files that are no longer used can be deleted to release disk space. +Currently, expiration is automatically performed by Paimon writers when committing new changes. By expiring old snapshots, old data files and metadata files that are no longer used can be deleted to release disk space. Snapshot expiration is controlled by the following table properties. diff --git a/docs/content/maintenance/manage-partition.md b/docs/content/maintenance/manage-partition.md index 54d68a316..c02c39113 100644 --- a/docs/content/maintenance/manage-partition.md +++ b/docs/content/maintenance/manage-partition.md @@ -26,7 +26,7 @@ under the License. ## Expiring Partitions -You can set `partition.expiration-time` when creating a partitioned table. Table Store will periodically check +You can set `partition.expiration-time` when creating a partitioned table. Paimon will periodically check the status of partitions and delete expired partitions according to time. How to determine whether a partition has expired: compare the time extracted from the partition with the current diff --git a/docs/content/maintenance/rescale-bucket.md b/docs/content/maintenance/rescale-bucket.md index b3ec92f8d..4878a837f 100644 --- a/docs/content/maintenance/rescale-bucket.md +++ b/docs/content/maintenance/rescale-bucket.md @@ -26,7 +26,7 @@ under the License. # Rescale Bucket -Since the number of total buckets dramatically influences the performance, Table Store allows users to +Since the number of total buckets dramatically influences the performance, Paimon allows users to tune bucket numbers by `ALTER TABLE` command and reorganize data layout by `INSERT OVERWRITE` without recreating the table/partition. When executing overwrite jobs, the framework will automatically scan the data with the old bucket number and hash the record according to the current bucket number. diff --git a/docs/content/maintenance/write-performance.md b/docs/content/maintenance/write-performance.md index a0597c073..c67990d23 100644 --- a/docs/content/maintenance/write-performance.md +++ b/docs/content/maintenance/write-performance.md @@ -26,7 +26,7 @@ under the License. # Write Performance -Performance of Table Store writers are related with the following factors. +Performance of Paimon writers are related with the following factors. ## Parallelism @@ -57,9 +57,9 @@ It is recommended that the parallelism of sink should be less than or equal to t ### Number of Sorted Runs to Trigger Compaction -Table Store uses [LSM tree]({{< ref "concepts/file-layouts#lsm-trees" >}}) which supports a large number of updates. LSM organizes files in several [sorted runs]({{< ref "concepts/file-layouts#lsm-trees#sorted-runs" >}}). When querying records from an LSM tree, all sorted runs must be combined to produce a complete view of all records. +Paimon uses [LSM tree]({{< ref "concepts/file-layouts#lsm-trees" >}}) which supports a large number of updates. LSM organizes files in several [sorted runs]({{< ref "concepts/file-layouts#lsm-trees#sorted-runs" >}}). When querying records from an LSM tree, all sorted runs must be combined to produce a complete view of all records. -One can easily see that too many sorted runs will result in poor query performance. To keep the number of sorted runs in a reasonable range, Table Store writers will automatically perform [compactions]({{< ref "concepts/file-layouts#lsm-trees#compactions" >}}). The following table property determines the minimum number of sorted runs to trigger a compaction. +One can easily see that too many sorted runs will result in poor query performance. To keep the number of sorted runs in a reasonable range, Paimon writers will automatically perform [compactions]({{< ref "concepts/file-layouts#lsm-trees#compactions" >}}). The following table property determines the minimum number of sorted runs to trigger a compaction. <table class="table table-bordered"> <thead> @@ -86,7 +86,7 @@ Compaction will become less frequent when `num-sorted-run.compaction-trigger` be ### Number of Sorted Runs to Pause Writing -When number of sorted runs is small, Table Store writers will perform compaction asynchronously in separated threads, so records can be continuously written into the table. However to avoid unbounded growth of sorted runs, writers will have to pause writing when the number of sorted runs hits the threshold. The following table property determines the threshold. +When number of sorted runs is small, Paimon writers will perform compaction asynchronously in separated threads, so records can be continuously written into the table. However to avoid unbounded growth of sorted runs, writers will have to pause writing when the number of sorted runs hits the threshold. The following table property determines the threshold. <table class="table table-bordered"> <thead> @@ -113,10 +113,10 @@ Write stalls will become less frequent when `num-sorted-run.stop-trigger` become ### Dedicated Compaction Job -By default, Table Store writers will perform compaction as needed when writing records. This is sufficient for most use cases, but there are two downsides: +By default, Paimon writers will perform compaction as needed when writing records. This is sufficient for most use cases, but there are two downsides: * This may result in unstable write throughput because throughput might temporarily drop when performing a compaction. -* Compaction will mark some data files as "deleted" (not really deleted, see [expiring snapshots]({{< ref "maintenance/expiring-snapshots" >}}) for more info). If multiple writers mark the same file a conflict will occur when committing the changes. Table Store will automatically resolve the conflict, but this may result in job restarts. +* Compaction will mark some data files as "deleted" (not really deleted, see [expiring snapshots]({{< ref "maintenance/expiring-snapshots" >}}) for more info). If multiple writers mark the same file a conflict will occur when committing the changes. Paimon will automatically resolve the conflict, but this may result in job restarts. To avoid these downsides, users can also choose to skip compactions in writers, and run a dedicated job only for compaction. As compactions are performed only by the dedicated job, writers can continuously write records without pausing and no conflicts will ever occur. @@ -186,7 +186,7 @@ For more usage of the compact action, see ## Memory -There are three main places in Table Store writer that takes up memory: +There are three main places in Paimon writer that takes up memory: * Writer's memory buffer, shared and preempted by all writers of a single task. This memory value can be adjusted by the `write-buffer-size` table property. * Memory consumed when merging several sorted runs for compaction. Can be adjusted by the `num-sorted-run.compaction-trigger` option to change the number of sorted runs to be merged. diff --git a/docs/content/versions.md b/docs/content/versions.md index 0fa4dbbd9..210d7f00d 100644 --- a/docs/content/versions.md +++ b/docs/content/versions.md @@ -25,6 +25,6 @@ under the License. # Versions -An appendix of hosted documentation for all versions of Apache Flink Table Store. +An appendix of hosted documentation for all versions of Apache Paimon. {{< all_versions >}} diff --git a/docs/layouts/partials/docs/inject/content-before.html b/docs/layouts/partials/docs/inject/content-before.html index f4096b2a0..42aca36ae 100644 --- a/docs/layouts/partials/docs/inject/content-before.html +++ b/docs/layouts/partials/docs/inject/content-before.html @@ -23,14 +23,14 @@ under the License. {{ if $.Site.Params.ShowOutDatedWarning }} <article class="markdown"> <blockquote style="border-color:#f66"> - {{ markdownify "This documentation is for an out-of-date version of Apache Flink Table Store. We recommend you use the latest [stable version](https://ci.apache.org/projects/flink/paimon-docs-stable/)."}} + {{ markdownify "This documentation is for an out-of-date version of Apache Paimon. We recommend you use the latest [stable version](https://ci.apache.org/projects/flink/paimon-docs-stable/)."}} </blockquote> </article> {{ end }} {{ if (not $.Site.Params.IsStable) }} <article class="markdown"> <blockquote style="border-color:#f66"> - {{ markdownify "This documentation is for an unreleased version of Apache Flink Table Store. We recommend you use the latest [stable version](https://ci.apache.org/projects/flink/paimon-docs-stable/)."}} + {{ markdownify "This documentation is for an unreleased version of Apache Paimon. We recommend you use the latest [stable version](https://ci.apache.org/projects/flink/paimon-docs-stable/)."}} </blockquote> </article> {{ end }} diff --git a/docs/layouts/shortcodes/generated/catalog_configuration.html b/docs/layouts/shortcodes/generated/catalog_configuration.html index 614763934..22cca42a4 100644 --- a/docs/layouts/shortcodes/generated/catalog_configuration.html +++ b/docs/layouts/shortcodes/generated/catalog_configuration.html @@ -42,7 +42,7 @@ <td><h5>table.type</h5></td> <td style="word-wrap: break-word;">managed</td> <td><p>Enum</p></td> - <td>Type of table.<br /><br />Possible values:<ul><li>"managed": Table Store owned table where the entire lifecycle of the table data is managed.</li><li>"external": The table where Table Store has loose coupling with the data stored in external locations.</li></ul></td> + <td>Type of table.<br /><br />Possible values:<ul><li>"managed": Paimon owned table where the entire lifecycle of the table data is managed.</li><li>"external": The table where Paimon has loose coupling with the data stored in external locations.</li></ul></td> </tr> <tr> <td><h5>uri</h5></td> diff --git a/paimon-benchmark/paimon-cluster-benchmark/README.md b/paimon-benchmark/paimon-cluster-benchmark/README.md index 5d15b3a61..497f10f65 100644 --- a/paimon-benchmark/paimon-cluster-benchmark/README.md +++ b/paimon-benchmark/paimon-cluster-benchmark/README.md @@ -1,6 +1,6 @@ -# Flink Table Store Benchmark +# Paimon Benchmark -This is the cluster benchmark module for Flink Table Store. Inspired by [Nexmark](https://github.com/nexmark/nexmark). +This is the cluster benchmark module for Paimon. Inspired by [Nexmark](https://github.com/nexmark/nexmark). ## How To Run ### Environment Preparation diff --git a/paimon-benchmark/paimon-micro-benchmarks/pom.xml b/paimon-benchmark/paimon-micro-benchmarks/pom.xml index a242d1c49..d1d5380dd 100644 --- a/paimon-benchmark/paimon-micro-benchmarks/pom.xml +++ b/paimon-benchmark/paimon-micro-benchmarks/pom.xml @@ -116,7 +116,7 @@ under the License. <scope>provided</scope> </dependency> - <!-- Flink Table Store dependencies --> + <!-- Paimon dependencies --> <dependency> <groupId>org.apache.paimon</groupId> diff --git a/paimon-common/src/main/java/org/apache/paimon/table/TableType.java b/paimon-common/src/main/java/org/apache/paimon/table/TableType.java index e466d641e..0a5dda648 100644 --- a/paimon-common/src/main/java/org/apache/paimon/table/TableType.java +++ b/paimon-common/src/main/java/org/apache/paimon/table/TableType.java @@ -27,10 +27,10 @@ import static org.apache.paimon.options.description.TextElement.text; public enum TableType implements DescribedEnum { MANAGED( "managed", - "Table Store owned table where the entire lifecycle of the table data is managed."), + "Paimon owned table where the entire lifecycle of the table data is managed."), EXTERNAL( "external", - "The table where Table Store has loose coupling with the data stored in external locations."); + "The table where Paimon has loose coupling with the data stored in external locations."); private final String value; private final String description; 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 39fbfe036..9b547779e 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 @@ -197,10 +197,10 @@ public class DataFileMeta { * Usage: * * <ul> - * <li>Table Store 0.2 + * <li>Paimon 0.2 * <ul> * <li>Stores changelog files for {@link CoreOptions.ChangelogProducer#INPUT}. Changelog - * files are moved to {@link NewFilesIncrement} since Table Store 0.3. + * files are moved to {@link NewFilesIncrement} since Paimon 0.3. * </ul> * </ul> */ 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 31deda6f3..044355c07 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 @@ -261,7 +261,7 @@ public abstract class AbstractFileStoreScan implements FileStoreScan { return snapshot.changelogManifests(manifestList); } - // compatible with Table Store 0.2, we'll read extraFiles in DataFileMeta + // compatible with Paimon 0.2, we'll read extraFiles in DataFileMeta // see comments on DataFileMeta#extraFiles if (snapshot.commitKind() == Snapshot.CommitKind.APPEND) { return snapshot.deltaManifests(manifestList); diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java index b796b61af..3a056fa8a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java @@ -159,7 +159,7 @@ public class KeyValueFileStoreRead implements FileStoreRead<KeyValue> { for (DataFileMeta file : split.files()) { suppliers.add( () -> { - // We need to check extraFiles to be compatible with Table Store 0.2. + // We need to check extraFiles to be compatible with Paimon 0.2. // See comments on DataFileMeta#extraFiles. String fileName = changelogFile(file).orElse(file.fileName()); return readerFactory.createRecordReader( diff --git a/paimon-docs/README.md b/paimon-docs/README.md index f8fabf8a2..b6fa96fb9 100644 --- a/paimon-docs/README.md +++ b/paimon-docs/README.md @@ -19,7 +19,7 @@ under the License. # Documentation generators -This module contains generators that create HTML files directly from Flink Table Store's source code. +This module contains generators that create HTML files directly from Paimon's source code. ## Configuration documentation diff --git a/paimon-flink/paimon-flink-1.14/pom.xml b/paimon-flink/paimon-flink-1.14/pom.xml index e6dec3289..faf88fca0 100644 --- a/paimon-flink/paimon-flink-1.14/pom.xml +++ b/paimon-flink/paimon-flink-1.14/pom.xml @@ -146,7 +146,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-flink-common</include> + <include>org.apache.paimon:paimon-flink-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-flink/paimon-flink-1.14/src/main/java/org/apache/paimon/flink/TableStoreDataStreamScanProvider.java b/paimon-flink/paimon-flink-1.14/src/main/java/org/apache/paimon/flink/TableStoreDataStreamScanProvider.java index 57b707b09..a742f7f02 100644 --- a/paimon-flink/paimon-flink-1.14/src/main/java/org/apache/paimon/flink/TableStoreDataStreamScanProvider.java +++ b/paimon-flink/paimon-flink-1.14/src/main/java/org/apache/paimon/flink/TableStoreDataStreamScanProvider.java @@ -25,7 +25,7 @@ import org.apache.flink.table.data.RowData; import java.util.function.Function; -/** Table Store {@link DataStreamScanProvider}. */ +/** Paimon {@link DataStreamScanProvider}. */ public class TableStoreDataStreamScanProvider implements DataStreamScanProvider { private final boolean isBounded; diff --git a/paimon-flink/paimon-flink-1.14/src/main/java/org/apache/paimon/flink/TableStoreDataStreamSinkProvider.java b/paimon-flink/paimon-flink-1.14/src/main/java/org/apache/paimon/flink/TableStoreDataStreamSinkProvider.java index f915df85b..6d72489be 100644 --- a/paimon-flink/paimon-flink-1.14/src/main/java/org/apache/paimon/flink/TableStoreDataStreamSinkProvider.java +++ b/paimon-flink/paimon-flink-1.14/src/main/java/org/apache/paimon/flink/TableStoreDataStreamSinkProvider.java @@ -25,7 +25,7 @@ import org.apache.flink.table.data.RowData; import java.util.function.Function; -/** Table Store {@link DataStreamSinkProvider}. */ +/** Paimon {@link DataStreamSinkProvider}. */ public class TableStoreDataStreamSinkProvider implements DataStreamSinkProvider { private final Function<DataStream<RowData>, DataStreamSink<?>> producer; diff --git a/paimon-flink/paimon-flink-1.15/pom.xml b/paimon-flink/paimon-flink-1.15/pom.xml index 11b6c9808..44a81194d 100644 --- a/paimon-flink/paimon-flink-1.15/pom.xml +++ b/paimon-flink/paimon-flink-1.15/pom.xml @@ -153,7 +153,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-flink-common</include> + <include>org.apache.paimon:paimon-flink-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-flink/paimon-flink-1.16/pom.xml b/paimon-flink/paimon-flink-1.16/pom.xml index e091ea14d..c42ae213d 100644 --- a/paimon-flink/paimon-flink-1.16/pom.xml +++ b/paimon-flink/paimon-flink-1.16/pom.xml @@ -60,7 +60,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-flink-common</include> + <include>org.apache.paimon:paimon-flink-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-flink/paimon-flink-common/pom.xml b/paimon-flink/paimon-flink-common/pom.xml index e0d7380fa..54e3ecd28 100644 --- a/paimon-flink/paimon-flink-common/pom.xml +++ b/paimon-flink/paimon-flink-common/pom.xml @@ -256,10 +256,10 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-common</include> - <include>org.apache.flink:paimon-core</include> - <include>org.apache.flink:paimon-format</include> - <include>org.apache.flink:paimon-codegen-loader</include> + <include>org.apache.paimon:paimon-common</include> + <include>org.apache.paimon:paimon-core</include> + <include>org.apache.paimon:paimon-format</include> + <include>org.apache.paimon:paimon-codegen-loader</include> </includes> </artifactSet> <relocations> 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 5b53e2239..f73317c55 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 @@ -204,7 +204,7 @@ public class FlinkCatalog extends AbstractCatalog { if (options.containsKey(CONNECTOR.key())) { throw new CatalogException( String.format( - "Table Store Catalog only supports paimon tables, not '%s' connector." + "Paimon Catalog only supports paimon tables, not '%s' connector." + " You can create TEMPORARY table instead.", options.get(CONNECTOR.key()))); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableStoreDataStreamScanProvider.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableStoreDataStreamScanProvider.java index b76bb9740..7ab932352 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableStoreDataStreamScanProvider.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableStoreDataStreamScanProvider.java @@ -26,7 +26,7 @@ import org.apache.flink.table.data.RowData; import java.util.function.Function; -/** Table Store {@link DataStreamScanProvider}. */ +/** Paimon {@link DataStreamScanProvider}. */ public class TableStoreDataStreamScanProvider implements DataStreamScanProvider { private final boolean isBounded; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableStoreDataStreamSinkProvider.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableStoreDataStreamSinkProvider.java index 2b3e9cc94..fa4c6da7b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableStoreDataStreamSinkProvider.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/TableStoreDataStreamSinkProvider.java @@ -26,7 +26,7 @@ import org.apache.flink.table.data.RowData; import java.util.function.Function; -/** Table Store {@link DataStreamSinkProvider}. */ +/** Paimon {@link DataStreamSinkProvider}. */ public class TableStoreDataStreamSinkProvider implements DataStreamSinkProvider { private final Function<DataStream<RowData>, DataStreamSink<?>> producer; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java index af46bb1d1..7d5e935f9 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ActionBase.java @@ -103,7 +103,7 @@ public abstract class ActionBase implements Action { /** * Extract {@link LogicalType}s from Flink {@link org.apache.flink.table.types.DataType}s and - * convert to Table Store {@link DataType}s. + * convert to Paimon {@link DataType}s. */ protected List<DataType> toTableStoreDataTypes( List<org.apache.flink.table.types.DataType> flinkDataTypes) { 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 488f93269..35aa6fa6f 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 @@ -140,12 +140,12 @@ public abstract class FlinkSink implements Serializable { private void assertCheckpointConfiguration(StreamExecutionEnvironment env) { Preconditions.checkArgument( !env.getCheckpointConfig().isUnalignedCheckpointsEnabled(), - "Table Store sink currently does not support unaligned checkpoints. Please set " + "Paimon sink currently does not support unaligned checkpoints. Please set " + ExecutionCheckpointingOptions.ENABLE_UNALIGNED.key() + " to false."); Preconditions.checkArgument( env.getCheckpointConfig().getCheckpointingMode() == CheckpointingMode.EXACTLY_ONCE, - "Table Store sink currently only supports EXACTLY_ONCE checkpoint mode. Please set " + "Paimon sink currently only supports EXACTLY_ONCE checkpoint mode. Please set " + ExecutionCheckpointingOptions.CHECKPOINTING_MODE.key() + " to exactly-once"); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ChangelogWithKeyFileStoreTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ChangelogWithKeyFileStoreTableITCase.java index e65dabbd4..8aa129e0b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ChangelogWithKeyFileStoreTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ChangelogWithKeyFileStoreTableITCase.java @@ -96,6 +96,7 @@ public class ChangelogWithKeyFileStoreTableITCase extends AbstractTestBase { // ------------------------------------------------------------------------ @Test + @Timeout(600) public void testFullCompactionTriggerInterval() throws Exception { innerTestChangelogProducing( Arrays.asList( @@ -104,6 +105,7 @@ public class ChangelogWithKeyFileStoreTableITCase extends AbstractTestBase { } @Test + @Timeout(600) public void testFullCompactionWithLongCheckpointInterval() throws Exception { // create table TableEnvironment bEnv = createBatchTableEnvironment(); @@ -167,6 +169,7 @@ public class ChangelogWithKeyFileStoreTableITCase extends AbstractTestBase { } @Test + @Timeout(600) public void testLookupChangelog() throws Exception { innerTestChangelogProducing(Collections.singletonList("'changelog-producer' = 'lookup'")); } @@ -245,14 +248,14 @@ public class ChangelogWithKeyFileStoreTableITCase extends AbstractTestBase { // ------------------------------------------------------------------------ @Test - @Timeout(600000) + @Timeout(600) public void testNoChangelogProducerBatchRandom() throws Exception { TableEnvironment bEnv = createBatchTableEnvironment(); testNoChangelogProducerRandom(bEnv, 1, false); } @Test - @Timeout(600000) + @Timeout(600) public void testNoChangelogProducerStreamingRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = createStreamingTableEnvironment(random.nextInt(900) + 100); @@ -260,14 +263,14 @@ public class ChangelogWithKeyFileStoreTableITCase extends AbstractTestBase { } @Test - @Timeout(600000) + @Timeout(600) public void testFullCompactionChangelogProducerBatchRandom() throws Exception { TableEnvironment bEnv = createBatchTableEnvironment(); testFullCompactionChangelogProducerRandom(bEnv, 1, false); } @Test - @Timeout(600000) + @Timeout(600) public void testFullCompactionChangelogProducerStreamingRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = createStreamingTableEnvironment(random.nextInt(900) + 100); @@ -275,7 +278,7 @@ public class ChangelogWithKeyFileStoreTableITCase extends AbstractTestBase { } @Test - @Timeout(600000) + @Timeout(600) public void testStandAloneFullCompactJobRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = createStreamingTableEnvironment(random.nextInt(900) + 100); @@ -283,14 +286,14 @@ public class ChangelogWithKeyFileStoreTableITCase extends AbstractTestBase { } @Test - @Timeout(600000) + @Timeout(600) public void testLookupChangelogProducerBatchRandom() throws Exception { TableEnvironment bEnv = createBatchTableEnvironment(); testLookupChangelogProducerRandom(bEnv, 1, false); } @Test - @Timeout(600000) + @Timeout(600) public void testLookupChangelogProducerStreamingRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = createStreamingTableEnvironment(random.nextInt(900) + 100); @@ -298,7 +301,7 @@ public class ChangelogWithKeyFileStoreTableITCase extends AbstractTestBase { } @Test - @Timeout(600000) + @Timeout(600) public void testStandAloneLookupJobRandom() throws Exception { ThreadLocalRandom random = ThreadLocalRandom.current(); TableEnvironment sEnv = createStreamingTableEnvironment(random.nextInt(900) + 100); 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 cdf5b75fe..45c8d9706 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 @@ -212,7 +212,7 @@ public class FlinkCatalogTest { assertThatThrownBy(() -> catalog.createTable(this.path1, newTable, false)) .isInstanceOf(CatalogException.class) .hasMessageContaining( - "Table Store Catalog only supports paimon tables," + "Paimon Catalog only supports paimon tables," + " not 'filesystem' connector. You can create TEMPORARY table instead."); } 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 952f5ed3c..1d823b378 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 @@ -81,7 +81,7 @@ import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.validateStream import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Table Store reading and writing IT cases. */ +/** Paimon reading and writing IT cases. */ public class ReadWriteTableITCase extends AbstractTestBase { private final Map<String, String> streamingReadOverwrite = diff --git a/paimon-hive/paimon-hive-catalog/pom.xml b/paimon-hive/paimon-hive-catalog/pom.xml index d6426e17b..ae42a3511 100644 --- a/paimon-hive/paimon-hive-catalog/pom.xml +++ b/paimon-hive/paimon-hive-catalog/pom.xml @@ -577,7 +577,7 @@ under the License. <configuration> <artifactSet> <includes> - <include>org.apache.flink:paimon-hive-common</include> + <include>org.apache.paimon:paimon-hive-common</include> <include>org.apache.thrift:libthrift</include> <include>org.apache.thrift:libfb303</include> <include>com.google.guava:guava</include> diff --git a/paimon-hive/paimon-hive-connector-2.1-cdh-6.3/pom.xml b/paimon-hive/paimon-hive-connector-2.1-cdh-6.3/pom.xml index 5bd77abad..37612f6ae 100644 --- a/paimon-hive/paimon-hive-connector-2.1-cdh-6.3/pom.xml +++ b/paimon-hive/paimon-hive-connector-2.1-cdh-6.3/pom.xml @@ -60,7 +60,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-hive-connector-common</include> + <include>org.apache.paimon:paimon-hive-connector-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-hive/paimon-hive-connector-2.1/pom.xml b/paimon-hive/paimon-hive-connector-2.1/pom.xml index 05dde38d5..ba17c2e3f 100644 --- a/paimon-hive/paimon-hive-connector-2.1/pom.xml +++ b/paimon-hive/paimon-hive-connector-2.1/pom.xml @@ -60,7 +60,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-hive-connector-common</include> + <include>org.apache.paimon:paimon-hive-connector-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-hive/paimon-hive-connector-2.2/pom.xml b/paimon-hive/paimon-hive-connector-2.2/pom.xml index 71ad5496c..6a391ce3d 100644 --- a/paimon-hive/paimon-hive-connector-2.2/pom.xml +++ b/paimon-hive/paimon-hive-connector-2.2/pom.xml @@ -60,7 +60,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-hive-connector-common</include> + <include>org.apache.paimon:paimon-hive-connector-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-hive/paimon-hive-connector-2.3/pom.xml b/paimon-hive/paimon-hive-connector-2.3/pom.xml index aa2f41611..1c4e8de9d 100644 --- a/paimon-hive/paimon-hive-connector-2.3/pom.xml +++ b/paimon-hive/paimon-hive-connector-2.3/pom.xml @@ -60,7 +60,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-hive-connector-common</include> + <include>org.apache.paimon:paimon-hive-connector-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-hive/paimon-hive-connector-3.1/pom.xml b/paimon-hive/paimon-hive-connector-3.1/pom.xml index af9578398..f81685668 100644 --- a/paimon-hive/paimon-hive-connector-3.1/pom.xml +++ b/paimon-hive/paimon-hive-connector-3.1/pom.xml @@ -98,7 +98,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-hive-connector-common</include> + <include>org.apache.paimon:paimon-hive-connector-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-hive/paimon-hive-connector-common/pom.xml b/paimon-hive/paimon-hive-connector-common/pom.xml index 199262020..304971121 100644 --- a/paimon-hive/paimon-hive-connector-common/pom.xml +++ b/paimon-hive/paimon-hive-connector-common/pom.xml @@ -520,8 +520,8 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-hive-common</include> - <include>org.apache.flink:paimon-shade</include> + <include>org.apache.paimon:paimon-hive-common</include> + <include>org.apache.paimon:paimon-shade</include> </includes> </artifactSet> </configuration> 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 06e3e77e9..8ac166f9c 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 @@ -142,9 +142,7 @@ public class HiveSchema { : schemaNames.get(i) + " " + schemaTypeInfos.get(i).getTypeName(); mismatched.add( String.format( - "Field #%d\n" - + "Hive DDL : %s\n" - + "Table Store Schema: %s\n", + "Field #%d\n" + "Hive DDL : %s\n" + "Paimon Schema: %s\n", i, ddlField, schemaField)); } } diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/TableStoreHiveMetaHook.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/TableStoreHiveMetaHook.java index c793e9688..3d6e0163b 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/TableStoreHiveMetaHook.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/TableStoreHiveMetaHook.java @@ -35,7 +35,7 @@ public class TableStoreHiveMetaHook implements HiveMetaHook { public void preCreateTable(Table table) throws MetaException { Preconditions.checkArgument( !table.isSetPartitionKeys() || table.getPartitionKeys().isEmpty(), - "Flink Table Store currently does not support creating partitioned table " + "Paimon currently does not support creating partitioned table " + "with PARTITIONED BY clause. If you want to query from a partitioned table, " + "please add partition columns into the ordinary table columns."); 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 0f2f9d775..c5c44a036 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 @@ -119,11 +119,11 @@ public class HiveTableSchemaTest { "Mismatched fields are:", "Field #1", "Hive DDL : mismatched string", - "Table Store Schema: b string", + "Paimon Schema: b string", "--------------------", "Field #2", "Hive DDL : c decimal(6,3)", - "Table Store Schema: c decimal(5,3)"); + "Paimon Schema: c decimal(5,3)"); IllegalArgumentException exception = assertThrows( IllegalArgumentException.class, () -> HiveSchema.extract(null, properties)); @@ -148,11 +148,11 @@ public class HiveTableSchemaTest { "Mismatched fields are:", "Field #1", "Hive DDL : null", - "Table Store Schema: b string", + "Paimon Schema: b string", "--------------------", "Field #2", "Hive DDL : null", - "Table Store Schema: c decimal(5,3)"); + "Paimon Schema: c decimal(5,3)"); IllegalArgumentException exception = assertThrows( IllegalArgumentException.class, () -> HiveSchema.extract(null, properties)); @@ -186,11 +186,11 @@ public class HiveTableSchemaTest { "Mismatched fields are:", "Field #3", "Hive DDL : d int", - "Table Store Schema: null", + "Paimon Schema: null", "--------------------", "Field #4", "Hive DDL : e string", - "Table Store Schema: null"); + "Paimon Schema: null"); IllegalArgumentException exception = assertThrows( IllegalArgumentException.class, () -> HiveSchema.extract(null, properties)); diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/SearchArgumentToPredicateConverterTest.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/SearchArgumentToPredicateConverterTest.java index f9ce13c77..4e095f7e1 100644 --- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/SearchArgumentToPredicateConverterTest.java +++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/SearchArgumentToPredicateConverterTest.java @@ -65,9 +65,9 @@ public class SearchArgumentToPredicateConverterTest { Decimal.fromBigDecimal(new BigDecimal("123456789123456789.123456789"), 27, 9)); testLiteral( PredicateLeaf.Type.STRING, - "Table Store", + "Paimon", DataTypes.STRING(), - BinaryString.fromString("Table Store")); + BinaryString.fromString("Paimon")); testLiteral(PredicateLeaf.Type.DATE, Date.valueOf("1971-01-11"), DataTypes.DATE(), 375); testLiteral( PredicateLeaf.Type.TIMESTAMP, diff --git a/paimon-shade/pom.xml b/paimon-shade/pom.xml index 67451b585..1665d7496 100644 --- a/paimon-shade/pom.xml +++ b/paimon-shade/pom.xml @@ -116,11 +116,11 @@ under the License. <include>org.apache.flink:flink-shaded-guava</include> <include>org.apache.flink:flink-shaded-asm-9</include> - <!-- Flink Table Store --> - <include>org.apache.flink:paimon-common</include> - <include>org.apache.flink:paimon-core</include> - <include>org.apache.flink:paimon-format</include> - <include>org.apache.flink:paimon-codegen-loader</include> + <!-- Paimon --> + <include>org.apache.paimon:paimon-common</include> + <include>org.apache.paimon:paimon-core</include> + <include>org.apache.paimon:paimon-format</include> + <include>org.apache.paimon:paimon-codegen-loader</include> <!-- Others, not shade to reduce conflicting --> <!-- <include>org.xerial.snappy:snappy-java</include> --> diff --git a/paimon-spark/paimon-spark-2/pom.xml b/paimon-spark/paimon-spark-2/pom.xml index 34dd7dc1c..cb1c1b3a1 100644 --- a/paimon-spark/paimon-spark-2/pom.xml +++ b/paimon-spark/paimon-spark-2/pom.xml @@ -101,7 +101,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-spark-common</include> + <include>org.apache.paimon:paimon-spark-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-spark/paimon-spark-3.1/pom.xml b/paimon-spark/paimon-spark-3.1/pom.xml index 6ed901dc2..7ce4d063f 100644 --- a/paimon-spark/paimon-spark-3.1/pom.xml +++ b/paimon-spark/paimon-spark-3.1/pom.xml @@ -74,7 +74,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-spark-common</include> + <include>org.apache.paimon:paimon-spark-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-spark/paimon-spark-3.2/pom.xml b/paimon-spark/paimon-spark-3.2/pom.xml index ee7c85f78..a77367b7c 100644 --- a/paimon-spark/paimon-spark-3.2/pom.xml +++ b/paimon-spark/paimon-spark-3.2/pom.xml @@ -74,7 +74,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-spark-common</include> + <include>org.apache.paimon:paimon-spark-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-spark/paimon-spark-3.3/pom.xml b/paimon-spark/paimon-spark-3.3/pom.xml index a0bc22fb2..44ee2427f 100644 --- a/paimon-spark/paimon-spark-3.3/pom.xml +++ b/paimon-spark/paimon-spark-3.3/pom.xml @@ -74,7 +74,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-spark-common</include> + <include>org.apache.paimon:paimon-spark-common</include> </includes> </artifactSet> </configuration> diff --git a/paimon-spark/paimon-spark-common/pom.xml b/paimon-spark/paimon-spark-common/pom.xml index c898a8f9a..91c4d66b0 100644 --- a/paimon-spark/paimon-spark-common/pom.xml +++ b/paimon-spark/paimon-spark-common/pom.xml @@ -74,7 +74,7 @@ under the License. <configuration> <artifactSet> <includes combine.children="append"> - <include>org.apache.flink:paimon-shade</include> + <include>org.apache.paimon:paimon-shade</include> </includes> </artifactSet> </configuration>
