diff --git a/be/src/service/query-options-test.cc b/be/src/service/query-options-test.cc
index 32feedfa3..aa10bd175 100644
--- a/be/src/service/query-options-test.cc
+++ b/be/src/service/query-options-test.cc
@@ -233,7 +233,7 @@ TEST(QueryOptions, SetEnumOptions) {
(THREE_LEVEL, TWO_LEVEL, TWO_LEVEL_THEN_THREE_LEVEL)), true);
TestEnumCase(options, CASE(default_file_format, THdfsFileFormat,
(TEXT, RC_FILE, SEQUENCE_FILE, AVRO, PARQUET, KUDU, ORC, HUDI_PARQUET, ICEBERG,
- JSON, JDBC)), true);
+ JSON, JDBC, PAIMON)), true);
TestEnumCase(options, CASE(runtime_filter_mode, TRuntimeFilterMode,
(OFF, LOCAL, GLOBAL)), true);
TestEnumCase(options, CASE(kudu_read_mode, TKuduReadMode,
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index 2bce6db8d..7cf9b360a 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -296,7 +296,7 @@ export IMPALA_DBCP2_VERSION=2.9.0
export IMPALA_DROPWIZARD_METRICS_VERSION=4.2.26
export IMPALA_AIRCOMPRESSOR_VERSION=0.27
export IMPALA_DATASKETCHES_VERSION=6.0.0
-
+export IMPALA_PAIMON_VERSION=1.1.1
# When Impala is building docker images on Redhat-based distributions,
# it is useful to be able to customize the base image. Some users will
# want to use open source / free distributions like Centos/Rocky/Alma/etc.
@@ -1242,6 +1242,7 @@ echo "IMPALA_HUDI_VERSION = $IMPALA_HUDI_VERSION"
echo "IMPALA_KUDU_VERSION = $IMPALA_KUDU_VERSION"
echo "IMPALA_RANGER_VERSION = $IMPALA_RANGER_VERSION"
echo "IMPALA_ICEBERG_VERSION = $IMPALA_ICEBERG_VERSION"
+echo "IMPALA_PAIMON_VERSION = $IMPALA_PAIMON_VERSION"
echo "IMPALA_COS_VERSION = $IMPALA_COS_VERSION"
echo "IMPALA_OBS_VERSION = $IMPALA_OBS_VERSION"
echo "IMPALA_SYSTEM_PYTHON2 = $IMPALA_SYSTEM_PYTHON2"
diff --git a/bin/rat_exclude_files.txt b/bin/rat_exclude_files.txt
index 3a9b38861..eb4dfd1b0 100644
--- a/bin/rat_exclude_files.txt
+++ b/bin/rat_exclude_files.txt
@@ -170,6 +170,7 @@ testdata/data/widerow.txt
testdata/data/local_tbl/00000.txt
testdata/data/hudi_parquet/*
testdata/data/iceberg_test/*
+testdata/data/paimon_test/*
testdata/data/json_test/*
testdata/data/sfs_d2.txt
testdata/data/sfs_d4.txt
diff --git a/common/thrift/CatalogObjects.thrift b/common/thrift/CatalogObjects.thrift
index 204b6527a..ab3bd7388 100644
--- a/common/thrift/CatalogObjects.thrift
+++ b/common/thrift/CatalogObjects.thrift
@@ -65,6 +65,7 @@ enum TTableType {
MATERIALIZED_VIEW = 7
// Represents a system table reflecting backend internal state.
SYSTEM_TABLE = 8
+ PAIMON_TABLE = 9
}
// TODO: Separate the storage engines (e.g. Kudu) from the file formats.
@@ -82,6 +83,7 @@ enum THdfsFileFormat {
ICEBERG = 8
JSON = 9
JDBC = 10
+ PAIMON = 11
}
enum TVirtualColumnType {
@@ -90,7 +92,9 @@ enum TVirtualColumnType {
FILE_POSITION,
PARTITION_SPEC_ID,
ICEBERG_PARTITION_SERIALIZED,
- ICEBERG_DATA_SEQUENCE_NUMBER
+ ICEBERG_DATA_SEQUENCE_NUMBER,
+ PARTITION_VALUE_SERIALIZED,
+ BUCKET_ID
}
// TODO: Since compression is also enabled for Kudu columns, we should
@@ -698,6 +702,25 @@ struct TSystemTable {
1: required TSystemTableName table_name
}
+enum TPaimonCatalog {
+ HADOOP_CATALOG = 0
+ HIVE_CATALOG = 1
+}
+
+// Paimon Table kind
+enum TPaimonTableKind {
+ JNI = 0
+ NATIVE = 1
+}
+
+// Represents a Paimon Table
+struct TPaimonTable {
+ // Paimon table kind.
+ 1: required TPaimonTableKind kind
+ // Jni table object.
+ 2: optional binary jni_tbl_obj
+}
+
// Represents a table or view.
struct TTable {
// Name of the parent database. Case insensitive, expected to be stored as lowercase.
@@ -759,6 +782,9 @@ struct TTable {
// Set if this is a system table
19: optional TSystemTable system_table
+
+ // Set if this is a paimon table
+ 20: optional TPaimonTable paimon_table
}
// Represents a database.
diff --git a/common/thrift/Descriptors.thrift b/common/thrift/Descriptors.thrift
index 12633126e..a4066ac7e 100644
--- a/common/thrift/Descriptors.thrift
+++ b/common/thrift/Descriptors.thrift
@@ -91,6 +91,7 @@ struct TTableDescriptor {
10: optional CatalogObjects.TKuduTable kuduTable
11: optional CatalogObjects.TIcebergTable icebergTable
12: optional CatalogObjects.TSystemTable systemTable
+ 13: optional CatalogObjects.TPaimonTable paimonTable
// Unqualified name of table
7: required string tableName
diff --git a/docs/impala.ditamap b/docs/impala.ditamap
index 728979cbc..466bb8f86 100644
--- a/docs/impala.ditamap
+++ b/docs/impala.ditamap
@@ -347,6 +347,7 @@ under the License.
+
diff --git a/docs/impala_keydefs.ditamap b/docs/impala_keydefs.ditamap
index 46ec6b856..919a8ebe1 100644
--- a/docs/impala_keydefs.ditamap
+++ b/docs/impala_keydefs.ditamap
@@ -61,6 +61,10 @@ under the License.
the Apache Iceberg Puffin site
+
+ the Apache Paimon site
+
+
the Apache Ozone site
diff --git a/docs/topics/impala_paimon.xml b/docs/topics/impala_paimon.xml
new file mode 100644
index 000000000..01d3c0201
--- /dev/null
+++ b/docs/topics/impala_paimon.xml
@@ -0,0 +1,352 @@
+
+
+
+
+
+ Using Impala with Paimon Tables
+ Paimon Tables
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ Paimon
+ Impala now adds experimental support for Apache Paimon, which is an open table format for realtime lakehouse.
+ With this functionality, you can access any existing Paimon tables using SQL and perform
+ analytics over them. It now supports Hive catalog and Hadoop catalog.
+
+
+
+ For more information on Paimon, see .
+
+
+
+
+
+
+ Overview of Paimon features
+
+
+
+
+
+
+
+ -
+ Real time updates:
+
+ -
+ Primary key table supports writing of large-scale updates, has very high update performance,
+ typically through Flink Streaming.
+
+ -
+ Support defining Merge Engines, update records however you like.
+ Deduplicate to keep the last row, or partial-update, or aggregate records, or first-row, you decide.
+
+
+
+ -
+ Data Lake Capabilities:
+
+ -
+ Scalable metadata: supports storing Petabyte large-scale datasets and storing a large
+ number of partitions.
+
+ -
+ Supports ACID Transactions & Time Travel & Schema Evolution.
+
+
+
+
+
+
+
+
+
+ Creating Paimon tables with Impala
+
+
+
+
+
+
+
+
+ When you have an existing Paimon table that is not yet present in the Hive Metastore,
+ you can use the CREATE EXTERNAL TABLE command in Impala to add the table to the Hive
+ Metastore and make Impala able to interact with this table. Currently Impala supports
+ HadoopCatalog, and HiveCatalog. If you have an existing table in HiveCatalog,
+ and you are using the same Hive Metastore, you need no further actions.
+
+
+ -
+ HadoopCatalog. A table in HadoopCatalog means that there is a catalog location
+ in the file system under which Paimon tables are stored. Use the following command
+ to add a table in a HadoopCatalog to Impala:
+
+CREATE EXTERNAL TABLE paimon_hadoop_cat
+STORED AS PAIMON
+TBLPROPERTIES('paimon.catalog'='hadoop',
+'paimon.catalog_location'='/path/to/paimon_hadoop_catalog',
+'paimon.table_identifier'='paimondb.paimontable');
+
+
+ -
+ HiveCatalog. User can create managed paimon table in HMS like below :
+
+CREATE TABLE paimon_hive_cat(userid INT,movieId INT)
+STORED AS PAIMON;
+
+
+
+
+ Syntax for creating DDL tables
+
+CREATE [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name
+(
+[col_name data_type ,...]
+[PRIMARY KEY (col1,col2)]
+)
+[PARTITIONED BY (col_name data_type [COMMENT 'col_comment'], ...)]
+STORED AS PAIMON
+[LOCATION 'hdfs_path']
+[TBLPROPERTIES (
+'primary-key'='col1,col2',
+'file.format' = 'orc/parquet',
+'bucket' = '2',
+'bucket-key' = 'col3',
+]
+
+
+
+
+
+
+
+ Dropping Paimon tables
+
+
+ One can use DROP TABLE statement to remove an Paimon table:
+
+DROP TABLE test_create_managed_bucket_paimon_table;
+
+
+
+ When external.table.purge table property is set to true, then the
+ DROP TABLE statement will also delete the data files. This property
+ is set to true when Impala creates the Paimon table via CREATE TABLE.
+ When CREATE EXTERNAL TABLE is used (the table already exists in some
+ catalog) then this external.table.purge is set to false, i.e.
+ DROP TABLE doesn't remove any files, only the table definition
+ in HMS.
+
+
+
+
+
+ Supported Data Types for Paimon Columns
+
+
+
+ You can get information about the supported Paimon data types in
+
+ the Paimon spec.
+
+
+
+ The Paimon data types can be mapped to the following SQL types in Impala:
+
+
+
+
+
+
+ Paimon type
+ SQL type in Impala
+
+
+
+
+ BOOLEAN
+ BOOLEAN
+
+
+ TINYINT
+ TINYINT
+
+
+ SMALLINT
+ SMALLINT
+
+
+ INT
+ INTEGER
+
+
+ BIGINT
+ BIGINT
+
+
+ FLOAT
+ FLOAT
+
+
+ DOUBLE
+ DOUBLE
+
+
+ STRING
+ STRING
+
+
+ DECIMAL(P,S)
+ DECIMAL(P,S)
+
+
+ TIMESTAMP
+ TIMESTAMP
+
+
+ TIMESTAMP(*WITH*TIMEZONE)
+ Not Supported
+
+
+ CHAR(N)
+ CHAR(N)
+
+
+ VARCHAR(N)
+ VARCHAR(N)
+
+
+ BINARY(N)
+ BINARY(N)
+
+
+ VARBINARY(N)
+ BINARY(N)
+
+
+ DATE
+ DATE
+
+
+ TIME
+ Not Supported
+
+
+ Not Supported
+ DATETIME
+
+
+ MULTISET<t>
+ Not Supported
+
+
+ ARRAY<t>
+ Not Supported For Now
+
+
+ MAP<t>
+ Not Supported For Now
+
+
+ ROW<n1 t1,n2 t2>
+ Not Supported For Now
+
+
+
+
+
+
+ note: the unsupported type for paimon and impala is noted as "Not Supported".
+ The item noted 'Not Supported for Now' will be supported later.
+
+
+
+
diff --git a/docs/topics/impala_reserved_words.xml b/docs/topics/impala_reserved_words.xml
index 145b3af68..2eed212d8 100644
--- a/docs/topics/impala_reserved_words.xml
+++ b/docs/topics/impala_reserved_words.xml
@@ -1588,6 +1588,13 @@ See the history, any recent changes, here:
XImpala 4.0 and higher
+
+ paimon
+
+
+ XImpala 5.0 and higher
+
+
identity
X
diff --git a/fe/pom.xml b/fe/pom.xml
index ce32f71a7..530b21bee 100644
--- a/fe/pom.xml
+++ b/fe/pom.xml
@@ -642,6 +642,22 @@ under the License.
2.0.3
test
+
+ org.apache.paimon
+ paimon-hive-connector-3.1
+ ${paimon.version}
+
+
+ lz4-java
+ org.lz4
+
+
+ caffeine
+ com.github.ben-manes.caffeine
+
+
+
+
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index 16c197b6b..109334ff4 100755
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -341,7 +341,7 @@ terminal
KW_NOVALIDATE, KW_NULL, KW_NULLS, KW_OF, KW_OFFSET, KW_ON, KW_OPTIMIZE, KW_OR,
KW_ORC, KW_ORDER, KW_OUTER,
KW_OVER, KW_OVERWRITE, KW_PARQUET, KW_PARQUETFILE, KW_PARTITION, KW_PARTITIONED,
- KW_PARTITIONS, KW_PRECEDING, KW_PREPARE_FN, KW_PRIMARY, KW_PRODUCED, KW_PURGE, KW_QUERY,
+ KW_PARTITIONS, KW_PRECEDING, KW_PREPARE_FN, KW_PAIMON, KW_PRIMARY, KW_PRODUCED, KW_PURGE, KW_QUERY,
KW_RANGE, KW_RCFILE, KW_RECOVER, KW_REFERENCES, KW_REFRESH, KW_REGEXP, KW_RELY,
KW_RENAME, KW_REPEATABLE, KW_REPLACE, KW_REPLICATION, KW_RESTRICT, KW_RETURNS,
KW_REVOKE, KW_RIGHT, KW_RLIKE, KW_ROLE, KW_ROLES, KW_ROLLUP, KW_ROW, KW_ROWS, KW_RWSTORAGE,
@@ -2330,6 +2330,8 @@ file_format_val ::=
{: RESULT = THdfsFileFormat.JSON; :}
| KW_JDBC
{: RESULT = THdfsFileFormat.JDBC; :}
+ | KW_PAIMON
+ {: RESULT = THdfsFileFormat.PAIMON; :}
;
storage_engine_val ::=
@@ -2339,6 +2341,8 @@ storage_engine_val ::=
{: RESULT = THdfsFileFormat.ICEBERG; :}
| KW_JDBC
{: RESULT = THdfsFileFormat.JDBC; :}
+ | KW_PAIMON
+ {: RESULT = THdfsFileFormat.PAIMON; :}
;
tbl_properties ::=
@@ -4598,6 +4602,8 @@ word ::=
{: RESULT = r.toString(); :}
| KW_ICEBERG:r
{: RESULT = r.toString(); :}
+ | KW_PAIMON:r
+ {: RESULT = r.toString(); :}
| KW_IF:r
{: RESULT = r.toString(); :}
| KW_IGNORE:r
diff --git a/fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java b/fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
index 7399af916..df74fa985 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AnalysisUtils.java
@@ -21,28 +21,27 @@ import org.apache.impala.common.AnalysisException;
import java.util.Collection;
-class AnalysisUtils {
+public class AnalysisUtils {
- static void throwIfNotNull(T o, String message) throws AnalysisException {
+ public static void throwIfNotNull(T o, String message) throws AnalysisException {
if (o != null) throw new AnalysisException(message);
}
- static void throwIfNotEmpty(Collection> c, String message)
+ public static void throwIfNotEmpty(Collection> c, String message)
throws AnalysisException {
if (c != null && !c.isEmpty()) throw new AnalysisException(message);
}
- static void throwIfNull(T o, String message) throws AnalysisException {
+ public static void throwIfNull(T o, String message) throws AnalysisException {
if (o == null) throw new AnalysisException(message);
}
- static void throwIfNullOrEmpty(Collection> c, String message)
+ public static void throwIfNullOrEmpty(Collection> c, String message)
throws AnalysisException {
if (c == null || c.isEmpty()) throw new AnalysisException(message);
}
- static void throwIfTrue(boolean cond, String message)
- throws AnalysisException {
+ public static void throwIfTrue(boolean cond, String message) throws AnalysisException {
if (cond) throw new AnalysisException(message);
}
}
diff --git a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
index 1fee13f8d..fdfe2bd21 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
@@ -79,6 +79,7 @@ import org.apache.impala.catalog.VirtualColumn;
import org.apache.impala.catalog.VirtualTable;
import org.apache.impala.catalog.iceberg.IcebergMetadataTable;
import org.apache.impala.catalog.local.LocalKuduTable;
+import org.apache.impala.catalog.paimon.FePaimonTable;
import org.apache.impala.common.AnalysisException;
import org.apache.impala.common.IdGenerator;
import org.apache.impala.common.ImpalaException;
@@ -1136,7 +1137,8 @@ public class Analyzer {
Preconditions.checkState(table instanceof FeFsTable ||
table instanceof FeKuduTable ||
table instanceof FeHBaseTable ||
- table instanceof FeDataSourceTable);
+ table instanceof FeDataSourceTable ||
+ table instanceof FePaimonTable);
return new BaseTableRef(tableRef, resolvedPath);
} else {
return new CollectionTableRef(tableRef, resolvedPath, false);
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
index 8fef3f024..bef7b11f9 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateTableStmt.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.mr.Catalogs;
+import org.apache.impala.analysis.paimon.PaimonAnalyzer;
import org.apache.impala.authorization.AuthorizationConfig;
import org.apache.impala.catalog.DataSourceTable;
import org.apache.impala.catalog.HdfsStorageDescriptor;
@@ -135,12 +136,12 @@ public class CreateTableStmt extends StatementBase implements SingleTableStmt {
public TSortingOrder getSortingOrder() { return tableDef_.getSortingOrder(); }
public String getComment() { return tableDef_.getComment(); }
public Map getTblProperties() { return tableDef_.getTblProperties(); }
- private HdfsCachingOp getCachingOp() { return tableDef_.getCachingOp(); }
+ public HdfsCachingOp getCachingOp() { return tableDef_.getCachingOp(); }
public HdfsUri getLocation() { return tableDef_.getLocation(); }
Map getSerdeProperties() { return tableDef_.getSerdeProperties(); }
public THdfsFileFormat getFileFormat() { return tableDef_.getFileFormat(); }
RowFormat getRowFormat() { return tableDef_.getRowFormat(); }
- private void putGeneratedProperty(String key, String value) {
+ public void putGeneratedProperty(String key, String value) {
tableDef_.putGeneratedProperty(key, value);
}
public Map getGeneratedKuduProperties() {
@@ -272,8 +273,9 @@ public class CreateTableStmt extends StatementBase implements SingleTableStmt {
// Avro tables can have empty column defs because they can infer them from the Avro
// schema. Likewise for external Kudu tables, the schema can be read from Kudu.
if (getColumnDefs().isEmpty() && getFileFormat() != THdfsFileFormat.AVRO
- && getFileFormat() != THdfsFileFormat.KUDU && getFileFormat() !=
- THdfsFileFormat.ICEBERG) {
+ && getFileFormat() != THdfsFileFormat.KUDU
+ && getFileFormat() != THdfsFileFormat.ICEBERG
+ && getFileFormat() != THdfsFileFormat.PAIMON) {
throw new AnalysisException("Table requires at least 1 column");
}
if (getRowFormat() != null) {
@@ -320,6 +322,10 @@ public class CreateTableStmt extends StatementBase implements SingleTableStmt {
analyzeJdbcSchema(analyzer);
}
+ if (getFileFormat() == THdfsFileFormat.PAIMON) {
+ PaimonAnalyzer.analyzeCreateTableStmt(this, analyzer);
+ }
+
// If lineage logging is enabled, compute minimal lineage graph.
if (BackendConfig.INSTANCE.getComputeLineage() || RuntimeEnv.INSTANCE.isTestEnv()) {
computeLineageGraph(analyzer);
@@ -932,7 +938,7 @@ public class CreateTableStmt extends StatementBase implements SingleTableStmt {
/**
* @return true for external tables that don't have "external.table.purge" set to true.
*/
- private boolean isExternalWithNoPurge() {
+ public boolean isExternalWithNoPurge() {
return isExternal() && !Boolean.parseBoolean(getTblProperties().get(
Table.TBL_PROP_EXTERNAL_TABLE_PURGE));
}
diff --git a/fe/src/main/java/org/apache/impala/analysis/ShowFilesStmt.java b/fe/src/main/java/org/apache/impala/analysis/ShowFilesStmt.java
index ab90a5387..f4f2bab66 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ShowFilesStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ShowFilesStmt.java
@@ -22,6 +22,7 @@ import java.util.List;
import org.apache.impala.authorization.Privilege;
import org.apache.impala.catalog.FeFsTable;
import org.apache.impala.catalog.FeTable;
+import org.apache.impala.catalog.paimon.FeShowFileStmtSupport;
import org.apache.impala.common.AnalysisException;
import org.apache.impala.thrift.TShowFilesParams;
import org.apache.impala.thrift.TTableName;
@@ -78,13 +79,23 @@ public class ShowFilesStmt extends StatementBase implements SingleTableStmt {
}
table_ = tableRef.getTable();
Preconditions.checkNotNull(table_);
- if (!(table_ instanceof FeFsTable)) {
- throw new AnalysisException("SHOW FILES is applicable only to a HDFS table");
+ if (!(table_ instanceof FeFsTable) && !(table_ instanceof FeShowFileStmtSupport)) {
+ throw new AnalysisException("SHOW FILES is applicable only to file-based tables.");
}
tableRef.analyze(analyzer);
// Analyze the partition spec, if one was specified.
if (partitionSet_ != null) {
+ if (table_ instanceof FeShowFileStmtSupport) {
+ FeShowFileStmtSupport showFileStmtSupport = (FeShowFileStmtSupport) table_;
+ if (!showFileStmtSupport.supportPartitionFilter()) {
+ throw new AnalysisException(
+ "SHOW FILES with partition filter is not applicable to" +
+ " table type:" +
+ showFileStmtSupport.getTableFormat().name()
+ );
+ }
+ }
partitionSet_.setTableName(table_.getTableName());
partitionSet_.setPartitionShouldExist();
partitionSet_.setPrivilegeRequirement(Privilege.VIEW_METADATA);
diff --git a/fe/src/main/java/org/apache/impala/analysis/ShowStatsStmt.java b/fe/src/main/java/org/apache/impala/analysis/ShowStatsStmt.java
index 3a0427959..22a9ad294 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ShowStatsStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ShowStatsStmt.java
@@ -19,12 +19,14 @@ package org.apache.impala.analysis;
import java.util.List;
+import org.apache.impala.analysis.paimon.PaimonAnalyzer;
import org.apache.impala.authorization.Privilege;
import org.apache.impala.catalog.FeFsTable;
import org.apache.impala.catalog.FeIcebergTable;
import org.apache.impala.catalog.FeKuduTable;
import org.apache.impala.catalog.FeTable;
import org.apache.impala.catalog.FeView;
+import org.apache.impala.catalog.paimon.FePaimonTable;
import org.apache.impala.common.AnalysisException;
import org.apache.impala.thrift.TShowStatsOp;
import org.apache.impala.thrift.TShowStatsParams;
@@ -114,6 +116,8 @@ public class ShowStatsStmt extends StatementBase implements SingleTableStmt {
throw new AnalysisException(getSqlPrefix() + " must target a Kudu table: " +
table_.getFullName());
}
+ } else if (table_ instanceof FePaimonTable) {
+ PaimonAnalyzer.analyzeShowStatStmt(this, (FePaimonTable) table_, analyzer);
} else if (table_ instanceof FeKuduTable) {
FeKuduTable kuduTable = (FeKuduTable) table_;
if ((op_ == TShowStatsOp.RANGE_PARTITIONS || op_ == TShowStatsOp.HASH_SCHEMA) &&
diff --git a/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java b/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
old mode 100755
new mode 100644
index 97bf4fd17..2c0c60c8b
--- a/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
@@ -24,6 +24,7 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
+import java.util.stream.Collectors;
import org.antlr.runtime.ANTLRStringStream;
import org.antlr.runtime.RecognitionException;
@@ -52,7 +53,10 @@ import org.apache.impala.catalog.KuduColumn;
import org.apache.impala.catalog.KuduTable;
import org.apache.impala.catalog.RowFormat;
import org.apache.impala.catalog.Table;
+import org.apache.impala.catalog.paimon.FePaimonTable;
+import org.apache.impala.catalog.paimon.PaimonUtil;
import org.apache.impala.common.Pair;
+import org.apache.impala.service.CatalogOpExecutor;
import org.apache.impala.thrift.TBucketInfo;
import org.apache.impala.thrift.TBucketType;
import org.apache.impala.thrift.TIcebergCatalog;
@@ -61,6 +65,7 @@ import org.apache.impala.util.AcidUtils;
import org.apache.impala.util.BucketUtils;
import org.apache.impala.util.IcebergUtil;
import org.apache.impala.util.KuduUtil;
+import org.apache.paimon.CoreOptions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -470,6 +475,50 @@ public class ToSqlUtils {
} catch (Exception e) {
throw new CatalogException("Could not get primary key/foreign keys sql.", e);
}
+ } else if (table instanceof FePaimonTable) {
+ // format
+ String inputFormat = msTable.getSd().getInputFormat();
+ String serDeLib = msTable.getSd().getSerdeInfo().getSerializationLib();
+ format = HdfsFileFormat.fromHdfsInputFormatClass(inputFormat, serDeLib);
+ storageHandlerClassName = null;
+ isPrimaryKeyUnique = true;
+ properties.remove(CoreOptions.PRIMARY_KEY.key());
+ properties.remove(CoreOptions.PARTITION.key());
+ properties.remove(PaimonUtil.STORAGE_HANDLER);
+ properties.remove(CatalogOpExecutor.CAPABILITIES_KEY);
+ // for synchronized table, show sql like a managed table
+ if (PaimonUtil.isSynchronizedTable(msTable)) {
+ properties.remove("TRANSLATED_TO_EXTERNAL");
+ properties.remove(Table.TBL_PROP_EXTERNAL_TABLE_PURGE);
+ if ((location != null)
+ && location.toLowerCase().endsWith(table.getName().toLowerCase())) {
+ location = null;
+ }
+ isExternal = false;
+ }
+ try {
+ FePaimonTable fePaimonTable = (FePaimonTable) table;
+ primaryKeySql = fePaimonTable.getPaimonApiTable()
+ .primaryKeys()
+ .stream()
+ .map(String::toLowerCase)
+ .collect(Collectors.toList());
+
+ partitionColsSql = new ArrayList<>();
+ for (int i = 0; i < table.getNumClusteringCols(); i++) {
+ Column col = table.getColumns().get(i);
+ partitionColsSql.add(columnToSql(col));
+ }
+
+ colsSql = new ArrayList<>();
+
+ for (int i = table.getNumClusteringCols(); i < table.getColumns().size(); i++) {
+ Column col = table.getColumns().get(i);
+ colsSql.add(columnToSql(col));
+ }
+ } catch (Exception e) {
+ throw new CatalogException("Could not get primary key/foreign keys sql.", e);
+ }
} else if (table instanceof FeDataSourceTable) {
// Mask sensitive table properties for external JDBC table.
Set keysToBeMasked = DataSourceTable.getJdbcTblPropertyMaskKeys();
@@ -641,8 +690,16 @@ public class ToSqlUtils {
return sb.toString();
}
+ private static String encodeColumnName(String name) {
+ if (impalaNeedsQuotes(name)) {
+ return String.format("`%s`", name);
+ } else {
+ return name;
+ }
+ }
+
private static String columnToSql(Column col) {
- StringBuilder sb = new StringBuilder(col.getName());
+ StringBuilder sb = new StringBuilder(encodeColumnName(col.getName()));
if (col.getType() != null) sb.append(" " + col.getType().toSql());
if (col instanceof KuduColumn) {
KuduColumn kuduCol = (KuduColumn) col;
diff --git a/fe/src/main/java/org/apache/impala/analysis/paimon/PaimonAnalyzer.java b/fe/src/main/java/org/apache/impala/analysis/paimon/PaimonAnalyzer.java
new file mode 100644
index 000000000..9b5c74842
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/paimon/PaimonAnalyzer.java
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.impala.analysis.paimon;
+
+import com.google.common.collect.Sets;
+
+import org.apache.impala.analysis.AnalysisUtils;
+import org.apache.impala.analysis.Analyzer;
+import org.apache.impala.analysis.ColumnDef;
+import org.apache.impala.analysis.CreateTableStmt;
+import org.apache.impala.analysis.ShowStatsStmt;
+import org.apache.impala.catalog.HdfsFileFormat;
+import org.apache.impala.catalog.Table;
+import org.apache.impala.catalog.paimon.FePaimonTable;
+import org.apache.impala.catalog.paimon.ImpalaTypeUtils;
+import org.apache.impala.catalog.paimon.PaimonUtil;
+import org.apache.impala.common.AnalysisException;
+import org.apache.impala.thrift.TBucketType;
+import org.apache.impala.thrift.TPaimonCatalog;
+import org.apache.impala.thrift.TShowStatsParams;
+import org.apache.paimon.CoreOptions;
+import org.apache.paimon.options.OptionsUtils;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Paimon analyzer utils to perform paimon-related analyze tasks.
+ *
+ * This class is used to reduce coupling of paimon-related implementation
+ * with {@link Analyzer}.
+ */
+public class PaimonAnalyzer {
+
+ /**
+ * Check Paimon Create Table statement.
+ * @param stmt
+ * @param analyzer
+ *
+ */
+ public static void analyzeCreateTableStmt(CreateTableStmt stmt, Analyzer analyzer)
+ throws AnalysisException {
+
+ // TODO: Not supported for now, try to enable later
+ AnalysisUtils.throwIfNotNull(
+ stmt.getCachingOp(), "A Paimon table cannot be cached in HDFS.");
+
+ AnalysisUtils.throwIfTrue(
+ stmt.getTblProperties().containsKey(CoreOptions.PARTITION.key()),
+ "Can't specify 'partition' table property in Paimon DDL, " +
+ "use PARTITIONED BY clause instead.");
+ AnalysisUtils.throwIfTrue(
+ stmt.geTBucketInfo().bucket_type != TBucketType.NONE,
+ "CLUSTERED BY clause is not support by PAIMON now, " +
+ "use property bucket-key instead.");
+ // primary-key and PRIMARY KEY caused should not be both specified
+ AnalysisUtils.throwIfTrue(
+ stmt.getTblProperties().containsKey(CoreOptions.PRIMARY_KEY.key())
+ && !stmt.getPrimaryKeys().isEmpty(),
+ "Can't specify both PRIMARY KEY clause and 'primary-key' table property " +
+ "in Paimon DDL.");
+ analyzePaimonColumns(stmt, analyzer);
+ analyzePaimonFormat(stmt, analyzer);
+ }
+
+ /**
+ * Check Paimon Show Table Stats statement.
+ * @param statsOp
+ * @param analyzer
+ *
+ */
+ public static void analyzeShowStatStmt(ShowStatsStmt statsOp, FePaimonTable table,
+ Analyzer analyzer) throws AnalysisException {
+ TShowStatsParams params = statsOp.toThrift();
+ switch (params.getOp()) {
+ case TABLE_STATS:
+ case COLUMN_STATS: return;
+ case PARTITIONS:
+ if (!PaimonUtil.hasPartition(table.getPaimonApiTable())) {
+ throw new AnalysisException("Table is not partitioned: " + table.getFullName());
+ }
+ break;
+ default:
+ throw new AnalysisException(
+ statsOp.toSql() + " is not supported for Paimon Table");
+ }
+ }
+
+ /**
+ * Setup paimon related property.
+ */
+ private static void putPaimonProperty(CreateTableStmt stmt, String key, String value) {
+ stmt.putGeneratedProperty(key, value);
+ }
+
+ /**
+ * Check paimon format related setting and update stmt object.
+ */
+ private static void analyzePaimonFormat(CreateTableStmt stmt, Analyzer analyzer)
+ throws AnalysisException {
+ Map tblProperties = stmt.getTblProperties();
+ Map paimonTblProperties = OptionsUtils.convertToPropertiesPrefixKey(
+ tblProperties, PaimonUtil.PAIMON_PROPERTY_PREFIX);
+ boolean isExternal = stmt.isExternal();
+
+ // A managed table cannot have 'external.table.purge' property set
+ if (!isExternal
+ && Boolean.parseBoolean(tblProperties.get(Table.TBL_PROP_EXTERNAL_TABLE_PURGE))) {
+ throw new AnalysisException(String.format("Table property '%s' cannot be set to "
+ + "true with a managed paimon table.",
+ Table.TBL_PROP_EXTERNAL_TABLE_PURGE));
+ }
+
+ // External table with purging is not supported.
+ if (stmt.isExternal()
+ && Boolean.parseBoolean(
+ tblProperties.getOrDefault(Table.TBL_PROP_EXTERNAL_TABLE_PURGE, "false"))) {
+ throw new AnalysisException(" External table with purge is not supported.");
+ }
+
+ // check storage handler, add storage handler for compatibility.
+ String handler = tblProperties.get(PaimonUtil.STORAGE_HANDLER);
+ if (handler != null && !handler.equals(PaimonUtil.PAIMON_STORAGE_HANDLER)) {
+ throw new AnalysisException("Invalid storage handler "
+ + "specified for Paimon format: " + handler);
+ }
+ stmt.putGeneratedProperty(
+ PaimonUtil.STORAGE_HANDLER, PaimonUtil.PAIMON_STORAGE_HANDLER);
+
+ // enable the deletion-vector mode by default if not specified
+ if (!paimonTblProperties.containsKey(CoreOptions.DELETION_VECTORS_ENABLED.key())
+ && !paimonTblProperties.containsKey(
+ CoreOptions.FULL_COMPACTION_DELTA_COMMITS.key())) {
+ putPaimonProperty(stmt, CoreOptions.DELETION_VECTORS_ENABLED.key(), "true");
+ }
+
+ // check format support
+ String fileformat = paimonTblProperties.get(CoreOptions.FILE_FORMAT.key());
+ HdfsFileFormat hdfsFileFormat = PaimonUtil.getPaimonFileFormat(fileformat);
+ if (fileformat != null && hdfsFileFormat == null) {
+ throw new AnalysisException(
+ "Unsupported fileformat for Paimon table: " + fileformat);
+ }
+
+ // Determine the Paimon catalog being used. The default catalog is HiveCatalog.
+ TPaimonCatalog catalog = PaimonUtil.getTPaimonCatalog(tblProperties);
+ switch (catalog) {
+ case HIVE_CATALOG: validateTableInHiveCatalog(stmt, tblProperties); break;
+ case HADOOP_CATALOG: validateTableInHadoopCatalog(stmt, tblProperties); break;
+ default:
+ throw new AnalysisException(
+ String.format("Unknown Paimon catalog type: %s", catalog));
+ }
+
+ // HMS will override 'external.table.purge' to 'TRUE' When 'paimon.catalog' is not
+ // the Hive Catalog for managed tables.
+ if (!isExternal && !catalog.equals(TPaimonCatalog.HIVE_CATALOG)
+ && "false".equalsIgnoreCase(
+ tblProperties.get(Table.TBL_PROP_EXTERNAL_TABLE_PURGE))) {
+ analyzer.addWarning("The table property 'external.table.purge' will be set "
+ + "to 'TRUE' on newly created managed Paimon tables.");
+ }
+ }
+
+ /**
+ * Validate for hive catalog.
+ * @param stmt
+ * @param tblProperties
+ */
+ private static void validateTableInHiveCatalog(
+ CreateTableStmt stmt, Map tblProperties) throws AnalysisException {
+
+ // Check if hadoop catalog related properties are defined in the hive catalog
+ if (tblProperties.get(PaimonUtil.PAIMON_HADOOP_CATALOG_LOCATION) != null) {
+ throw new AnalysisException(String.format("%s cannot be set for Paimon table "
+ + "stored in hive.catalog",
+ PaimonUtil.PAIMON_HADOOP_CATALOG_LOCATION));
+ }
+ if (tblProperties.get(PaimonUtil.PAIMON_TABLE_IDENTIFIER) != null) {
+ throw new AnalysisException(String.format("%s cannot be set for Paimon table "
+ + "stored in hive.catalog",
+ PaimonUtil.PAIMON_TABLE_IDENTIFIER));
+ }
+
+ AnalysisUtils.throwIfTrue(stmt.isExternal() && stmt.getLocation() == null,
+ "Location must be set for external Paimon table stored in hive catalog");
+ }
+
+ /**
+ * Validate for hadoop catalog.
+ * @param stmt
+ * @param tblProperties
+ */
+ private static void validateTableInHadoopCatalog(
+ CreateTableStmt stmt, Map tblProperties) throws AnalysisException {
+ String catalogLoc = tblProperties.get(PaimonUtil.PAIMON_HADOOP_CATALOG_LOCATION);
+ if (catalogLoc == null || catalogLoc.isEmpty()) {
+ throw new AnalysisException(String.format("Table property '%s' is necessary "
+ + "for Paimon table with 'hadoop.catalog'.",
+ PaimonUtil.PAIMON_HADOOP_CATALOG_LOCATION));
+ }
+
+ // Table identifier should be specified for external table
+ AnalysisUtils.throwIfTrue(stmt.isExternal()
+ && tblProperties.get(PaimonUtil.PAIMON_TABLE_IDENTIFIER) == null,
+ String.format(
+ "Table property '%s' is necessary for Paimon table with 'hadoop.catalog'.",
+ PaimonUtil.PAIMON_TABLE_IDENTIFIER));
+ }
+
+ /**
+ * Check column type support for the column definitions.
+ * @param columnDef
+ */
+ private static void throwIfColumnTypeIsNotSupported(ColumnDef columnDef)
+ throws AnalysisException {
+ if (!ImpalaTypeUtils.isSupportedColumnType(columnDef.getType())) {
+ throw new AnalysisException("Tables stored by Paimon do not support the column "
+ + columnDef.getColName() + " type: " + columnDef.getType().toSql());
+ }
+ }
+
+ /**
+ * Check column definitions of paimon table.
+ * @param stmt
+ * @param analyzer
+ */
+ private static void analyzePaimonColumns(CreateTableStmt stmt, Analyzer analyzer)
+ throws AnalysisException {
+ Set colSets = Sets.newHashSet();
+
+ // Check if the columns definitions are supported
+ for (ColumnDef col : stmt.getColumnDefs()) {
+ throwIfColumnTypeIsNotSupported(col);
+ colSets.add(col.getColName().toLowerCase());
+ }
+
+ // Check if the partition columns definitions are supported
+ for (ColumnDef col : stmt.getPartitionColumnDefs()) {
+ throwIfColumnTypeIsNotSupported(col);
+ colSets.add(col.getColName().toLowerCase());
+ }
+
+ // Check if primary keys are in the column definitions
+ if (stmt.getTblProperties().containsKey(CoreOptions.PRIMARY_KEY.key())) {
+ List colNames = PaimonUtil.extractColumnNames(
+ stmt.getTblProperties().get(CoreOptions.PRIMARY_KEY.key()));
+ for (String col : colNames) {
+ AnalysisUtils.throwIfTrue(!colSets.contains(col),
+ String.format(
+ "Invalid col name %s specified in 'primary-key' table properties.", col));
+ }
+ }
+
+ // Check if bucket keys are in the column definitions
+ if (stmt.getTblProperties().containsKey(CoreOptions.BUCKET_KEY.key())) {
+ List parts = PaimonUtil.extractColumnNames(
+ stmt.getTblProperties().get(CoreOptions.BUCKET_KEY.key()));
+ for (String col : parts) {
+ AnalysisUtils.throwIfTrue(!colSets.contains(col),
+ String.format(
+ "Invalid col name %s specified in 'bucket-key' table properties.", col));
+ }
+ }
+
+ // Check rule: Managed table does not support inferring schema from underlying paimon
+ // table.
+ if (stmt.getColumnDefs().isEmpty()) {
+ AnalysisUtils.throwIfTrue(!stmt.isExternal(),
+ "Managed table does not support inferring schema from underlying paimon table");
+ }
+ }
+}
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java b/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
index 79d9a24cd..d518067c9 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsFileFormat.java
@@ -85,8 +85,10 @@ public enum HdfsFileFormat {
"org.apache.iceberg.mr.hive.HiveIcebergSerDe", false, false, false),
JDBC("org.apache.hadoop.hive.jdbc.JdbcInputFormat",
"org.apache.hadoop.hive.jdbc.JdbcOutputFormat",
- "org.apache.hadoop.hive.jdbc.JdbcSerDe", false, false, true);
-
+ "org.apache.hadoop.hive.jdbc.JdbcSerDe", false, false, true),
+ PAIMON("org.apache.paimon.hive.mapred.PaimonInputFormat",
+ "org.apache.paimon.hive.mapred.PaimonOutputFormat",
+ "org.apache.paimon.hive.PaimonSerDe", true, false, true);
private final String inputFormat_;
private final String outputFormat_;
@@ -143,6 +145,7 @@ public enum HdfsFileFormat {
.put(ORC.inputFormat(), ORC)
.put(HUDI_PARQUET.inputFormat(), HUDI_PARQUET)
.put(ICEBERG.inputFormat(), ICEBERG)
+ .put(PAIMON.inputFormat(), PAIMON)
.build();
/**
@@ -197,6 +200,7 @@ public enum HdfsFileFormat {
case ICEBERG: return HdfsFileFormat.ICEBERG;
case JSON: return HdfsFileFormat.JSON;
case JDBC: return HdfsFileFormat.JDBC;
+ case PAIMON: return HdfsFileFormat.PAIMON;
default:
throw new RuntimeException("Unknown THdfsFileFormat: "
+ thriftFormat + " - should never happen!");
@@ -216,6 +220,7 @@ public enum HdfsFileFormat {
case ICEBERG: return THdfsFileFormat.ICEBERG;
case JSON: return THdfsFileFormat.JSON;
case JDBC: return THdfsFileFormat.JDBC;
+ case PAIMON: return THdfsFileFormat.PAIMON;
default:
throw new RuntimeException("Unknown HdfsFormat: "
+ this + " - should never happen!");
@@ -243,6 +248,8 @@ public enum HdfsFileFormat {
case ICEBERG: return "ICEBERG";
case JSON: return "JSONFILE";
case JDBC: return "JDBC";
+ case PAIMON: return "PAIMON";
+
default:
throw new RuntimeException("Unknown HdfsFormat: "
+ this + " - should never happen!");
@@ -308,6 +315,7 @@ public enum HdfsFileFormat {
* Returns true if the format is Parquet, false otherwise.
*/
public boolean isParquetBased() {
- return this == HdfsFileFormat.PARQUET || this == HdfsFileFormat.HUDI_PARQUET;
+ return this == HdfsFileFormat.PARQUET || this == HdfsFileFormat.HUDI_PARQUET
+ || this == HdfsFileFormat.PAIMON;
}
}
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsStorageDescriptor.java b/fe/src/main/java/org/apache/impala/catalog/HdfsStorageDescriptor.java
index 94579847f..e3ae8a55d 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsStorageDescriptor.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsStorageDescriptor.java
@@ -79,7 +79,8 @@ public class HdfsStorageDescriptor {
// support for the new input/output format classes. See IMPALA-4214.
"org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe", // (parquet)
"org.apache.iceberg.mr.hive.HiveIcebergSerDe", // (iceberg)
- "org.apache.hadoop.hive.serde2.JsonSerDe");// (json)
+ "org.apache.hadoop.hive.serde2.JsonSerDe", // (json)
+ "org.apache.paimon.hive.PaimonSerDe"); // (paimon)
private final static Logger LOG = LoggerFactory.getLogger(HdfsStorageDescriptor.class);
diff --git a/fe/src/main/java/org/apache/impala/catalog/Table.java b/fe/src/main/java/org/apache/impala/catalog/Table.java
index d513c982b..315a334f2 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Table.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Table.java
@@ -40,11 +40,13 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType;
import org.apache.impala.analysis.TableName;
import org.apache.impala.catalog.events.InFlightEvents;
import org.apache.impala.catalog.monitor.CatalogMonitor;
-import org.apache.impala.compat.MetastoreShim;
+import org.apache.impala.catalog.paimon.PaimonTable;
+import org.apache.impala.catalog.paimon.PaimonUtil;
import org.apache.impala.common.ImpalaRuntimeException;
import org.apache.impala.common.Metrics;
import org.apache.impala.common.Pair;
import org.apache.impala.common.RuntimeEnv;
+import org.apache.impala.compat.MetastoreShim;
import org.apache.impala.service.MetadataOp;
import org.apache.impala.thrift.TAccessLevel;
import org.apache.impala.thrift.TCatalogObject;
@@ -558,6 +560,8 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
table = new KuduTable(msTbl, db, msTbl.getTableName(), msTbl.getOwner());
} else if (IcebergTable.isIcebergTable(msTbl)) {
table = new IcebergTable(msTbl, db, msTbl.getTableName(), msTbl.getOwner());
+ } else if (PaimonUtil.isPaimonTable(msTbl)) {
+ table = new PaimonTable(msTbl, db, msTbl.getTableName(), msTbl.getOwner());
} else if (DataSourceTable.isDataSourceTable(msTbl)) {
// It's important to check if this is a DataSourceTable before HdfsTable because
// DataSourceTables are still represented by HDFS tables in the metastore but
diff --git a/fe/src/main/java/org/apache/impala/catalog/VirtualColumn.java b/fe/src/main/java/org/apache/impala/catalog/VirtualColumn.java
index dc3547e02..5810bb908 100644
--- a/fe/src/main/java/org/apache/impala/catalog/VirtualColumn.java
+++ b/fe/src/main/java/org/apache/impala/catalog/VirtualColumn.java
@@ -50,6 +50,14 @@ public class VirtualColumn extends Column {
Type.BIGINT,
TVirtualColumnType.ICEBERG_DATA_SEQUENCE_NUMBER);
+ // Paimon-related virtual columns.
+ public static VirtualColumn PARTITION_VALUE_SERIALIZED = new
+ VirtualColumn("PARTITION__VALUE__SERIALIZED", Type.BINARY,
+ TVirtualColumnType.PARTITION_VALUE_SERIALIZED);
+ public static VirtualColumn BUCKET_ID = new VirtualColumn("BUCKET_ID",
+ Type.INT, TVirtualColumnType.BUCKET_ID);
+
+
public static VirtualColumn getVirtualColumn(TVirtualColumnType virtColType) {
switch (virtColType) {
case INPUT_FILE_NAME: return INPUT_FILE_NAME;
@@ -57,6 +65,8 @@ public class VirtualColumn extends Column {
case PARTITION_SPEC_ID: return PARTITION_SPEC_ID;
case ICEBERG_PARTITION_SERIALIZED: return ICEBERG_PARTITION_SERIALIZED;
case ICEBERG_DATA_SEQUENCE_NUMBER: return ICEBERG_DATA_SEQUENCE_NUMBER;
+ case PARTITION_VALUE_SERIALIZED: return PARTITION_VALUE_SERIALIZED;
+ case BUCKET_ID: return BUCKET_ID;
default: break;
}
return null;
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalPaimonTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalPaimonTable.java
new file mode 100644
index 000000000..4c9f0e229
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalPaimonTable.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.impala.catalog.local;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.impala.catalog.TableLoadingException;
+import org.apache.impala.catalog.paimon.FePaimonTable;
+import org.apache.impala.catalog.paimon.PaimonUtil;
+import org.apache.impala.thrift.TTableDescriptor;
+import org.apache.impala.thrift.TTableType;
+import org.apache.log4j.Logger;
+import org.apache.paimon.table.Table;
+
+import java.io.IOException;
+import java.util.Set;
+
+/**
+ * Paimon table for LocalCatalog
+ */
+public class LocalPaimonTable extends LocalTable implements FePaimonTable {
+ private static final Logger LOG = Logger.getLogger(LocalPaimonTable.class);
+ private Table table_;
+
+ public static LocalPaimonTable load(LocalDb db,
+ org.apache.hadoop.hive.metastore.api.Table msTbl, MetaProvider.TableMetaRef ref)
+ throws TableLoadingException {
+ Preconditions.checkNotNull(db);
+ Preconditions.checkNotNull(msTbl);
+ Preconditions.checkNotNull(ref);
+ try {
+ LocalPaimonTable localPaimonTable = new LocalPaimonTable(db, msTbl, ref);
+ return localPaimonTable;
+ } catch (MetaException ex) {
+ throw new TableLoadingException("Failed to load table" + msTbl.getTableName(), ex);
+ }
+ }
+
+ protected LocalPaimonTable(LocalDb db, org.apache.hadoop.hive.metastore.api.Table msTbl,
+ MetaProvider.TableMetaRef ref) throws MetaException {
+ super(db, msTbl, ref);
+ table_ = PaimonUtil.createFileStoreTable(msTbl);
+ applyPaimonTableStatsIfPresent();
+ }
+
+ @Override
+ public Table getPaimonApiTable() {
+ return table_;
+ }
+
+ @Override
+ public TTableDescriptor toThriftDescriptor(
+ int tableId, Set referencedPartitions) {
+ TTableDescriptor tableDescriptor = new TTableDescriptor(tableId,
+ TTableType.PAIMON_TABLE, getTColumnDescriptors(), 0, name_, db_.getName());
+ try {
+ tableDescriptor.setPaimonTable(PaimonUtil.getTPaimonTable(this));
+ } catch (IOException e) { throw new RuntimeException(e); }
+ return tableDescriptor;
+ }
+}
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java
index fe5e39fed..2f780d3f5 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java
@@ -49,6 +49,7 @@ import org.apache.impala.catalog.SystemTable;
import org.apache.impala.catalog.TableLoadingException;
import org.apache.impala.catalog.VirtualColumn;
import org.apache.impala.catalog.local.MetaProvider.TableMetaRef;
+import org.apache.impala.catalog.paimon.PaimonUtil;
import org.apache.impala.common.Pair;
import org.apache.impala.common.RuntimeEnv;
import org.apache.impala.compat.MetastoreShim;
@@ -129,12 +130,13 @@ abstract class LocalTable implements FeTable {
t = LocalKuduTable.loadFromKudu(db, msTbl, ref);
} else if (IcebergTable.isIcebergTable(msTbl)) {
t = LocalIcebergTable.loadIcebergTableViaMetaProvider(db, msTbl, ref);
+ } else if (PaimonUtil.isPaimonTable(msTbl)) {
+ t = LocalPaimonTable.load(db, msTbl, ref);
} else if (DataSourceTable.isDataSourceTable(msTbl)) {
t = LocalDataSourceTable.load(db, msTbl, ref);
} else if (SystemTable.isSystemTable(msTbl)) {
t = LocalSystemTable.load(db, msTbl, ref);
- } else if (HdfsFileFormat.isHdfsInputFormatClass(
- msTbl.getSd().getInputFormat())) {
+ } else if (HdfsFileFormat.isHdfsInputFormatClass(msTbl.getSd().getInputFormat())) {
t = LocalFsTable.load(db, msTbl, ref);
}
diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/FePaimonTable.java b/fe/src/main/java/org/apache/impala/catalog/paimon/FePaimonTable.java
new file mode 100644
index 000000000..3990d07ff
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/paimon/FePaimonTable.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.impala.catalog.paimon;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.impala.catalog.Column;
+import org.apache.impala.catalog.ColumnStats;
+import org.apache.impala.catalog.FeTable;
+import org.apache.impala.catalog.HdfsFileFormat;
+import org.apache.impala.thrift.TResultSet;
+import org.apache.impala.thrift.TShowFilesParams;
+import org.apache.impala.thrift.TShowStatsOp;
+import org.apache.paimon.stats.ColStats;
+import org.apache.paimon.stats.Statistics;
+import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.table.Table;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.types.RowType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Frontend interface for interacting with an Paimon-backed table.
+ */
+public interface FePaimonTable extends FeTable, FeShowFileStmtSupport {
+ final static Logger LOG = LoggerFactory.getLogger(FePaimonTable.class);
+
+ public static JobConf jobConf = new JobConf();
+
+ /**
+ * Returns the cached paimon Table object that stores the metadata loaded by Paimon.
+ */
+ Table getPaimonApiTable();
+
+ default boolean hasSnapshotMetaTable() {
+ Table table = getPaimonApiTable();
+ return table instanceof FileStoreTable;
+ }
+
+ /**
+ * @return the Paimon schema.
+ */
+ default RowType getPaimonSchema() { return getPaimonApiTable().rowType(); }
+
+ default void applyPaimonTableStatsIfPresent() {
+ Preconditions.checkState(getTTableStats() != null);
+ Table table = getPaimonApiTable();
+ Optional stats = table.statistics();
+ if (!stats.isPresent()) return;
+ if (getTTableStats().getNum_rows() < 0) {
+ getTTableStats().setNum_rows(stats.get().mergedRecordCount().orElse(-1));
+ }
+ getTTableStats().setTotal_file_bytes(stats.get().mergedRecordSize().orElse(-1));
+ }
+
+ default void applyPaimonColumnStatsIfPresent() {
+ Preconditions.checkState(getTTableStats() != null);
+ Table table = getPaimonApiTable();
+ Optional stats = table.statistics();
+ if (!stats.isPresent()) return;
+ Map> colStatsMap = stats.get().colStats();
+ for (String colName : colStatsMap.keySet()) {
+ Column col = this.getColumn(colName.toLowerCase());
+ if (null == col) { continue; }
+
+ if (!ColumnStats.isSupportedColType(col.getType())) {
+ LOG.warn(String.format("Statistics for %s, column %s are not supported as " +
+ "column has type %s",
+ getFullName(), col.getName(), col.getType()));
+ continue;
+ }
+
+ ColStats> colStats = colStatsMap.get(colName);
+ DataField dataField = table.rowType().getField(colName);
+ Optional colStatsData =
+ PaimonUtil.convertColStats(colStats, dataField);
+ if (colStatsData.isPresent()) {
+ if (!col.updateStats(colStatsData.get())) {
+ LOG.warn(String.format(
+ "Failed to load column stats for %s, column %s. Stats may be " +
+ "incompatible with column type %s. Consider regenerating " +
+ "statistics for %s.",
+ getFullName(), col.getName(), col.getType(), getFullName()));
+ }
+ }
+ }
+ }
+
+ default TResultSet getTableStats(TShowStatsOp op) {
+ if (TShowStatsOp.TABLE_STATS == op) {
+ return PaimonUtil.doGetTableStats(this);
+ } else if (TShowStatsOp.PARTITIONS == op) {
+ return PaimonUtil.doGetPartitionStats(this);
+ } else {
+ throw new UnsupportedOperationException(
+ "paimon table doesn't support Show Stats Op" + op.name());
+ }
+ }
+
+ default TResultSet doGetTableFiles(TShowFilesParams request) {
+ return PaimonUtil.doGetTableFiles(this, request);
+ }
+
+ default HdfsFileFormat getTableFormat() {
+ return HdfsFileFormat.PAIMON;
+ }
+
+ default boolean supportPartitionFilter() {
+ return false;
+ }
+}
diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/FeShowFileStmtSupport.java b/fe/src/main/java/org/apache/impala/catalog/paimon/FeShowFileStmtSupport.java
new file mode 100644
index 000000000..ccb698afa
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/paimon/FeShowFileStmtSupport.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.impala.catalog.paimon;
+
+import org.apache.impala.analysis.Analyzer;
+import org.apache.impala.catalog.HdfsFileFormat;
+import org.apache.impala.thrift.TResultSet;
+import org.apache.impala.thrift.TShowFilesParams;
+
+/**
+ * Interface for show file statement support.
+ *
+ * This class is used to support Show Files statement
+ * for paimon format.
+ */
+public interface FeShowFileStmtSupport {
+ // Get Table files.
+ TResultSet doGetTableFiles(TShowFilesParams request);
+
+ // Current Table format.
+ HdfsFileFormat getTableFormat();
+
+ // Support partition filter.
+ boolean supportPartitionFilter();
+}
diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/ImpalaTypeUtils.java b/fe/src/main/java/org/apache/impala/catalog/paimon/ImpalaTypeUtils.java
new file mode 100644
index 000000000..a62177741
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/paimon/ImpalaTypeUtils.java
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.impala.catalog.paimon;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.hadoop.hive.common.type.HiveChar;
+import org.apache.hadoop.hive.common.type.HiveVarchar;
+import org.apache.impala.catalog.PrimitiveType;
+import org.apache.impala.catalog.ScalarType;
+import org.apache.impala.catalog.StructField;
+import org.apache.impala.catalog.StructType;
+import org.apache.impala.catalog.Type;
+import org.apache.paimon.types.ArrayType;
+import org.apache.paimon.types.BigIntType;
+import org.apache.paimon.types.BinaryType;
+import org.apache.paimon.types.BooleanType;
+import org.apache.paimon.types.CharType;
+import org.apache.paimon.types.DataType;
+import org.apache.paimon.types.DataTypeDefaultVisitor;
+import org.apache.paimon.types.DataTypes;
+import org.apache.paimon.types.DateType;
+import org.apache.paimon.types.DecimalType;
+import org.apache.paimon.types.DoubleType;
+import org.apache.paimon.types.FloatType;
+import org.apache.paimon.types.IntType;
+import org.apache.paimon.types.LocalZonedTimestampType;
+import org.apache.paimon.types.MapType;
+import org.apache.paimon.types.RowType;
+import org.apache.paimon.types.SmallIntType;
+import org.apache.paimon.types.TimestampType;
+import org.apache.paimon.types.TinyIntType;
+import org.apache.paimon.types.VarBinaryType;
+import org.apache.paimon.types.VarCharType;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** Utils for converting types related classes between Paimon and Impala. */
+public class ImpalaTypeUtils {
+ /**
+ * Convert paimon data type {@link DataType} to impala data type {@link Type}.
+ *
+ * @param logicalType paimon data type.
+ * @return impala type info.
+ */
+ public static Type toImpalaType(DataType logicalType) {
+ return logicalType.accept(PaimonToImpalaTypeVisitor.INSTANCE);
+ }
+
+ /**
+ * Convert impala data type {@link Type} to paimon data type {@link DataType}.
+ *
+ * @param Type hive type info
+ * @return paimon data type
+ */
+ public static DataType fromImpalaType(Type Type) {
+ return ImpalaToPaimonTypeVisitor.visit(Type);
+ }
+
+ private static class PaimonToImpalaTypeVisitor extends DataTypeDefaultVisitor {
+ private static final PaimonToImpalaTypeVisitor INSTANCE =
+ new PaimonToImpalaTypeVisitor();
+
+ @Override
+ public Type visit(BooleanType booleanType) {
+ return Type.BOOLEAN;
+ }
+
+ @Override
+ public Type visit(TinyIntType tinyIntType) {
+ return Type.TINYINT;
+ }
+
+ @Override
+ public Type visit(SmallIntType smallIntType) {
+ return Type.SMALLINT;
+ }
+
+ @Override
+ public Type visit(IntType intType) {
+ return Type.INT;
+ }
+
+ @Override
+ public Type visit(BigIntType bigIntType) {
+ return Type.BIGINT;
+ }
+
+ @Override
+ public Type visit(FloatType floatType) {
+ return Type.FLOAT;
+ }
+
+ @Override
+ public Type visit(DoubleType doubleType) {
+ return Type.DOUBLE;
+ }
+
+ @Override
+ public Type visit(DecimalType decimalType) {
+ return ScalarType.createDecimalType(
+ decimalType.getPrecision(), decimalType.getScale());
+ }
+
+ @Override
+ public Type visit(CharType charType) {
+ if (charType.getLength() > HiveChar.MAX_CHAR_LENGTH) {
+ return Type.STRING;
+ } else {
+ return ScalarType.createCharType(charType.getLength());
+ }
+ }
+
+ @Override
+ public Type visit(VarCharType varCharType) {
+ if (varCharType.getLength() > HiveVarchar.MAX_VARCHAR_LENGTH) {
+ return Type.STRING;
+ } else {
+ return ScalarType.createVarcharType(varCharType.getLength());
+ }
+ }
+
+ @Override
+ public Type visit(BinaryType binaryType) {
+ return Type.BINARY;
+ }
+
+ @Override
+ public Type visit(VarBinaryType varBinaryType) {
+ return Type.BINARY;
+ }
+
+ @Override
+ public Type visit(DateType dateType) {
+ return Type.DATE;
+ }
+
+ @Override
+ public Type visit(TimestampType timestampType) {
+ return Type.TIMESTAMP;
+ }
+
+ @Override
+ public Type visit(LocalZonedTimestampType localZonedTimestampType) {
+ return Type.TIMESTAMP;
+ }
+
+ @Override
+ public Type visit(ArrayType arrayType) {
+ DataType elementType = arrayType.getElementType();
+ return new org.apache.impala.catalog.ArrayType(elementType.accept(this));
+ }
+
+ @Override
+ public Type visit(MapType mapType) {
+ return new org.apache.impala.catalog.MapType(
+ mapType.getKeyType().accept(this), mapType.getValueType().accept(this));
+ }
+
+ @Override
+ public Type visit(RowType rowType) {
+ List structFields =
+ rowType.getFields()
+ .stream()
+ .map(dataField
+ -> new StructField(
+ dataField.name().toLowerCase(), dataField.type().accept(this)))
+ .collect(Collectors.toList());
+
+ return new StructType(structFields);
+ }
+
+ @Override
+ protected Type defaultMethod(org.apache.paimon.types.DataType dataType) {
+ throw new UnsupportedOperationException("Unsupported type: " + dataType);
+ }
+ }
+
+ private static class ImpalaToPaimonTypeVisitor {
+ static DataType visit(Type type) {
+ return visit(type, new ImpalaToPaimonTypeVisitor());
+ }
+
+ static DataType visit(Type type, ImpalaToPaimonTypeVisitor visitor) {
+ if (type.isStructType()) {
+ StructType structTypeInfo = (StructType) type;
+ List fields = structTypeInfo.getFields();
+ RowType.Builder builder = RowType.builder();
+ for (StructField field : fields) {
+ builder.field(field.getName(), visit(field.getType(), visitor));
+ }
+ return builder.build();
+ } else if (type.isMapType()) {
+ org.apache.impala.catalog.MapType mapTypeInfo =
+ (org.apache.impala.catalog.MapType) type;
+ return DataTypes.MAP(visit(mapTypeInfo.getKeyType(), visitor),
+ visit(mapTypeInfo.getValueType(), visitor));
+ } else if (type.isArrayType()) {
+ org.apache.impala.catalog.ArrayType listTypeInfo =
+ (org.apache.impala.catalog.ArrayType) type;
+ return DataTypes.ARRAY(visit(listTypeInfo.getItemType(), visitor));
+ } else {
+ return visitor.atomic(type);
+ }
+ }
+
+ public DataType atomic(Type atomic) {
+ PrimitiveType primitiveType = atomic.getPrimitiveType();
+ switch (primitiveType) {
+ case BOOLEAN: return DataTypes.BOOLEAN();
+ case TINYINT: return DataTypes.TINYINT();
+ case SMALLINT: return DataTypes.SMALLINT();
+ case INT: return DataTypes.INT();
+ case BIGINT: return DataTypes.BIGINT();
+ case FLOAT: return DataTypes.FLOAT();
+ case DOUBLE: return DataTypes.DOUBLE();
+ case DECIMAL:
+ ScalarType scalarType = (ScalarType) atomic;
+ return DataTypes.DECIMAL(
+ scalarType.decimalPrecision(), scalarType.decimalScale());
+ case CHAR: return DataTypes.CHAR(((ScalarType) atomic).getLength());
+ case VARCHAR: return DataTypes.VARCHAR(((ScalarType) atomic).getLength());
+ case STRING: return DataTypes.STRING();
+ case DATE: return DataTypes.DATE();
+ case TIMESTAMP: return DataTypes.TIMESTAMP();
+ case BINARY: return DataTypes.BINARY(BinaryType.MAX_LENGTH);
+ default:
+ throw new UnsupportedOperationException(
+ "Not a supported type: " + atomic);
+ }
+ }
+ }
+
+ /**
+ * Returns true if the primitive type is supported.
+ */
+ public static boolean isSupportedPrimitiveType(PrimitiveType primitiveType) {
+ Preconditions.checkNotNull(primitiveType);
+ switch (primitiveType) {
+ case BIGINT:
+ case INT:
+ case SMALLINT:
+ case TINYINT:
+ case DOUBLE:
+ case FLOAT:
+ case BOOLEAN:
+ case STRING:
+ case TIMESTAMP:
+ case DECIMAL:
+ case DATE:
+ case BINARY:
+ case CHAR:
+ case DATETIME:
+ case VARCHAR: return true;
+ default: return false;
+ }
+ }
+
+ /**
+ * Returns true if the column type is supported.
+ */
+ public static boolean isSupportedColumnType(Type colType) {
+ Preconditions.checkNotNull(colType);
+ return isSupportedPrimitiveType(colType.getPrimitiveType());
+ }
+}
diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonCatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonCatalogOpExecutor.java
new file mode 100644
index 000000000..53228dfcb
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonCatalogOpExecutor.java
@@ -0,0 +1,212 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.catalog.paimon;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.impala.catalog.TableLoadingException;
+import org.apache.impala.common.ImpalaRuntimeException;
+import org.apache.impala.thrift.TColumn;
+import org.apache.impala.thrift.TCreateTableParams;
+import org.apache.impala.thrift.TDropTableOrViewParams;
+import org.apache.impala.util.EventSequence;
+import org.apache.paimon.CoreOptions;
+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.schema.Schema;
+import org.apache.paimon.schema.SchemaManager;
+import org.apache.paimon.schema.TableSchema;
+import org.apache.paimon.table.Table;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.utils.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * This is a helper for the CatalogOpExecutor to provide Paimon related DDL functionality
+ * such as creating and dropping tables from Paimon.
+ */
+public class PaimonCatalogOpExecutor {
+ public static final Logger LOG = LoggerFactory.getLogger(PaimonCatalogOpExecutor.class);
+ public static final String LOADED_PAIMON_TABLE = "Loaded paimon table";
+ public static final String CREATED_PAIMON_TABLE =
+ "Created table using Paimon Catalog ";
+ /**
+ * Create Paimon table by Paimon api
+ * Return value is table object from Paimon
+ */
+ public static String createTable(Identifier identifier, String location,
+ TCreateTableParams params, org.apache.hadoop.hive.metastore.api.Table newTable)
+ throws ImpalaRuntimeException {
+ try {
+ List all_columns = Lists.newArrayList(params.getColumns());
+ List partitionKeys = Lists.newArrayList();
+ List primaryKeys = Lists.newArrayList();
+
+ // handle partition columns
+ if (params.getPartition_columnsSize() > 0) {
+ Preconditions.checkArgument(
+ !params.getTable_properties().containsKey(CoreOptions.PARTITION.key()));
+ all_columns.addAll(params.getPartition_columns());
+ partitionKeys.addAll(params.getPartition_columns()
+ .stream()
+ .map(c -> c.getColumnName().toLowerCase())
+ .collect(Collectors.toList()));
+ params.getTable_properties().remove(CoreOptions.PARTITION.key());
+ }
+ if (params.getTable_properties().containsKey(CoreOptions.PARTITION.key())) {
+ Preconditions.checkArgument(params.getPartition_columnsSize() <= 0);
+ partitionKeys.clear();
+ List newPartKeys = Arrays
+ .stream(params.getTable_properties()
+ .get(CoreOptions.PARTITION.key())
+ .split(","))
+ .collect(Collectors.toList());
+ partitionKeys.addAll(newPartKeys);
+ params.getTable_properties().remove(CoreOptions.PARTITION.key());
+ }
+
+ // handle primary keys
+ if (params.getPrimary_key_column_namesSize() > 0) {
+ primaryKeys.addAll(params.getPrimary_key_column_names());
+ Preconditions.checkArgument(
+ !params.getTable_properties().containsKey(CoreOptions.PRIMARY_KEY.key()));
+ }
+
+ if (params.getTable_properties().containsKey(CoreOptions.PRIMARY_KEY.key())) {
+ primaryKeys.clear();
+ List exist = Arrays
+ .stream(params.getTable_properties()
+ .get(CoreOptions.PRIMARY_KEY.key())
+ .split(","))
+ .collect(Collectors.toList());
+ primaryKeys.addAll(exist);
+ }
+ params.getTable_properties().put(
+ CoreOptions.PRIMARY_KEY.key(), StringUtils.join(primaryKeys, ","));
+
+ Path path = new Path(location);
+ CatalogContext context = PaimonUtil.catalogContext(newTable, location);
+ FileIO fileIO;
+ try {
+ fileIO = FileIO.get(path, context);
+ } catch (IOException e) {
+ throw new ImpalaRuntimeException("Failed to get file IO for paimon table", e);
+ }
+
+ SchemaManager schemaManager = new SchemaManager(fileIO, path);
+ Optional tableSchema = schemaManager.latest();
+ if (!tableSchema.isPresent()) {
+ Schema schema = PaimonUtil.genPaimonSchema(
+ all_columns, partitionKeys, params.getTable_properties());
+ schemaManager.createTable(schema);
+ LOG.info("Create paimon table successful.");
+ } else {
+ throw new AlreadyExistsException(
+ "Can't create paimon table, since the table location is not clean.");
+ }
+ return location;
+ } catch (Exception ex) {
+ throw new ImpalaRuntimeException("Failed to create paimon table", ex);
+ }
+ }
+
+ /**
+ * Populates HMS table schema based on the Paimon table's schema.
+ */
+ public static void populateExternalTableSchemaFromPaimonTable(
+ org.apache.hadoop.hive.metastore.api.Table msTbl, Table tbl)
+ throws TableLoadingException {
+ try {
+ List cols = Lists.newArrayList();
+ List partCols = Lists.newArrayList();
+ Set partSet = tbl.partitionKeys()
+ .stream()
+ .map(String::toLowerCase)
+ .collect(Collectors.toSet());
+ List primaryKeys = tbl.primaryKeys()
+ .stream()
+ .map(String::toLowerCase)
+ .collect(Collectors.toList());
+ List hiveFields = PaimonUtil.convertToHiveSchema(tbl.rowType());
+
+ for (int i = 0; i < tbl.rowType().getFields().size(); i++) {
+ DataField dataField = tbl.rowType().getFields().get(i);
+ if (partSet.contains(dataField.name().toLowerCase())) {
+ partCols.add(hiveFields.get(i));
+ } else {
+ cols.add(hiveFields.get(i));
+ }
+ }
+ msTbl.getSd().setCols(cols);
+ msTbl.setPartitionKeys(partCols);
+ Map parameters = msTbl.getParameters();
+ // Update primary key
+ if (!primaryKeys.isEmpty()) {
+ parameters.put(CoreOptions.PRIMARY_KEY.key(), StringUtils.join(primaryKeys, ","));
+ }
+ } catch (ImpalaRuntimeException e) {
+ throw new TableLoadingException("Error while infer schema from underlying" +
+ " paimon table", e);
+ }
+ }
+
+ /**
+ * Drop Paimon table by Paimon api
+ * Return value is whether requires to drop hms table
+ */
+ public static boolean dropTable(org.apache.hadoop.hive.metastore.api.Table msTbl,
+ org.apache.impala.catalog.Table existingTbl, EventSequence catalogTimeline,
+ TDropTableOrViewParams param) throws ImpalaRuntimeException {
+ if (msTbl == null) { return false; }
+ Preconditions.checkArgument(existingTbl instanceof PaimonTable);
+ boolean isSynchronizedPaimonTable =
+ PaimonUtil.isSynchronizedTable(msTbl) || param.isPurge();
+
+ if (isSynchronizedPaimonTable) {
+ String location = msTbl.getSd().getLocation();
+ Path path = new Path(location);
+ CatalogContext context = PaimonUtil.catalogContext(msTbl, location);
+ try {
+ FileIO fileIO = FileIO.get(path, context);
+ if (fileIO.exists(path)) {
+ // clear the data
+ fileIO.deleteDirectoryQuietly(path);
+ fileIO.mkdirs(path);
+ }
+ } catch (IOException e) {
+ LOG.warn("Delete directory '{}' fail for the paimon table.", path, e);
+ }
+ catalogTimeline.markEvent("Dropped table using Paimon");
+ }
+ return true;
+ }
+}
\ No newline at end of file
diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonTable.java b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonTable.java
new file mode 100644
index 000000000..bf52b22da
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonTable.java
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.impala.catalog.paimon;
+
+import com.codahale.metrics.Timer;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.impala.catalog.Column;
+import org.apache.impala.catalog.Db;
+import org.apache.impala.catalog.Table;
+import org.apache.impala.catalog.TableLoadingException;
+import org.apache.impala.catalog.VirtualColumn;
+import org.apache.impala.common.ImpalaRuntimeException;
+import org.apache.impala.thrift.TCatalogObjectType;
+import org.apache.impala.thrift.TPaimonCatalog;
+import org.apache.impala.thrift.TPaimonTable;
+import org.apache.impala.thrift.TPaimonTableKind;
+import org.apache.impala.thrift.TTable;
+import org.apache.impala.thrift.TTableDescriptor;
+import org.apache.impala.thrift.TTableType;
+import org.apache.impala.util.EventSequence;
+import org.apache.paimon.CoreOptions;
+import org.apache.paimon.types.DataField;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Representation of an Paimon table in the catalog cache.
+ */
+public class PaimonTable extends Table implements FePaimonTable {
+ private static final Set PAIMON_EXCLUDED_PROPERTIES = Sets.newHashSet();
+
+ static {
+ PAIMON_EXCLUDED_PROPERTIES.add(CoreOptions.PATH.key());
+ PAIMON_EXCLUDED_PROPERTIES.add("owner");
+ }
+
+ // Paimon api table.
+ private org.apache.paimon.table.Table table_;
+
+
+
+ public PaimonTable(org.apache.hadoop.hive.metastore.api.Table msTable, Db db,
+ String name, String owner) {
+ super(msTable, db, name, owner);
+ }
+
+ @Override
+ public TTableDescriptor toThriftDescriptor(
+ int tableId, Set referencedPartitions) {
+ TTableDescriptor tableDescriptor =
+ new TTableDescriptor(tableId, TTableType.PAIMON_TABLE, getTColumnDescriptors(),
+ numClusteringCols_, name_, db_.getName());
+ try {
+ tableDescriptor.setPaimonTable(PaimonUtil.getTPaimonTable(this));
+ } catch (IOException e) { throw new RuntimeException(e); }
+ return tableDescriptor;
+ }
+
+ @Override
+ public TTable toThrift() {
+ TTable tTable = super.toThrift();
+ try {
+ tTable.setPaimon_table(PaimonUtil.getTPaimonTable(this));
+ } catch (IOException e) { throw new RuntimeException(e); }
+ return tTable;
+ }
+
+ @Override
+ protected void loadFromThrift(TTable thriftTable) throws TableLoadingException {
+ super.loadFromThrift(thriftTable);
+ TPaimonTable tpaimon = thriftTable.getPaimon_table();
+ try {
+ Preconditions.checkArgument(tpaimon.getKind() == TPaimonTableKind.JNI);
+ table_ = PaimonUtil.deserialize(ByteBuffer.wrap(tpaimon.getJni_tbl_obj()));
+ } catch (Exception e) {
+ throw new TableLoadingException("Failed to load paimon table from" +
+ " thrift data.",e);
+ }
+ }
+
+ @Override
+ public TCatalogObjectType getCatalogObjectType() {
+ return TCatalogObjectType.TABLE;
+ }
+
+ /**
+ * Verify the table metadata.
+ * @throws TableLoadingException when it is unsafe to load the table.
+ */
+ private void verifyTable(org.apache.hadoop.hive.metastore.api.Table msTbl)
+ throws TableLoadingException {
+ TPaimonCatalog catalog = PaimonUtil.getTPaimonCatalog(msTbl);
+ if (catalog == TPaimonCatalog.HADOOP_CATALOG) {
+ if (!msTbl.getParameters()
+ .containsKey(PaimonUtil.PAIMON_HADOOP_CATALOG_LOCATION)) {
+ throw new TableLoadingException(
+ String.format(
+ "%s is required for paimon hadoop catalog table.",
+ PaimonUtil.PAIMON_HADOOP_CATALOG_LOCATION)
+ );
+ }
+ }
+ }
+
+ /**
+ * Load schema and partitioning schemes directly from Paimon.
+ */
+ public void loadSchemaFromPaimon()
+ throws TableLoadingException, ImpalaRuntimeException {
+ loadSchema();
+ addVirtualColumns();
+ }
+
+ /**
+ * Loads the HMS schema by Paimon schema
+ */
+ private void loadSchema() throws TableLoadingException {
+ clearColumns();
+ try {
+ List dataFields = getPaimonSchema().getFields();
+ List partitionKeys = getPaimonApiTable()
+ .partitionKeys()
+ .stream()
+ .map(String::toLowerCase)
+ .collect(Collectors.toList());
+ List hiveFields = PaimonUtil.convertToHiveSchema(getPaimonSchema());
+ List impalaFields = PaimonUtil.convertToImpalaSchema(getPaimonSchema());
+ List hivePartitionedFields = Lists.newArrayList();
+ List hiveNonPartitionedFields = Lists.newArrayList();
+ List impalaNonPartitionedFields = Lists.newArrayList();
+ List impalaPartitionedFields = Lists.newArrayList();
+ // lookup the clustering columns
+ for (String name : partitionKeys) {
+ int colIndex = PaimonUtil.getFieldIndexByNameIgnoreCase(getPaimonSchema(), name);
+ Preconditions.checkArgument(colIndex >= 0);
+ hivePartitionedFields.add(hiveFields.get(colIndex));
+ impalaPartitionedFields.add(impalaFields.get(colIndex));
+ }
+ // put non-clustering columns in natural order
+ for (int i = 0; i < dataFields.size(); i++) {
+ if (!partitionKeys.contains(dataFields.get(i).name().toLowerCase())) {
+ hiveNonPartitionedFields.add(hiveFields.get(i));
+ impalaNonPartitionedFields.add(impalaFields.get(i));
+ }
+ }
+ // update hive ms table metadata
+ if (!hivePartitionedFields.isEmpty()) {
+ msTable_.setPartitionKeys(hivePartitionedFields);
+ }
+ msTable_.getSd().setCols(hiveNonPartitionedFields);
+ // update impala table metadata
+ int colPos = 0;
+ for (Column col : impalaPartitionedFields) {
+ col.setPosition(colPos++);
+ addColumn(col);
+ }
+ for (Column col : impalaNonPartitionedFields) {
+ col.setPosition(colPos++);
+ addColumn(col);
+ }
+ numClusteringCols_ = impalaPartitionedFields.size();
+ // sync table properties from underlying paimon table
+ final Map paimonProps =
+ Maps.newHashMap(getPaimonApiTable().options());
+ for (String key : PAIMON_EXCLUDED_PROPERTIES) { paimonProps.remove(key); }
+ for (String key : paimonProps.keySet()) {
+ msTable_.getParameters().put(key, paimonProps.get(key));
+ }
+ } catch (ImpalaRuntimeException e) {
+ throw new TableLoadingException(e.getMessage(), e);
+ }
+ }
+
+ /**
+ * Add virtual columns for Paimon table, Paimon table has 4 metadata
+ * columns: paimon_file_path,paimon_row_index,__paimon_partition,__paimon_bucket.
+ * in these metadata columns,paimon_file_path,and paimon_row_index,__paimon_partition
+ * will be mapped to existing virtual column INPUT_FILE_NAME, FILE_POSITION.
+ * __paimon_partition,__paimon_bucket will be mapped to newly added
+ * PARTITION_VALUE_SERIALIZED, BUCKET_ID separately.
+ */
+ private void addVirtualColumns() {
+ addVirtualColumn(VirtualColumn.INPUT_FILE_NAME);
+ addVirtualColumn(VirtualColumn.FILE_POSITION);
+ addVirtualColumn(VirtualColumn.PARTITION_VALUE_SERIALIZED);
+ addVirtualColumn(VirtualColumn.BUCKET_ID);
+ }
+
+ /**
+ * Loads the metadata of a Paimon table.
+ *
+ * Schema and partitioning schemes are loaded directly from Paimon. for column stats,
+ * will try to loaded from HMS first, if they are absent, will load from Paimon table.
+ * The function also updates the table schema in HMS in order to
+ * propagate alterations made to the Pqimon table to HMS.
+ *
+ */
+ @Override
+ public void load(boolean reuseMetadata, IMetaStoreClient msClient,
+ org.apache.hadoop.hive.metastore.api.Table msTbl, String reason,
+ EventSequence catalogTimeline) throws TableLoadingException {
+ final Timer.Context context =
+ getMetrics().getTimer(Table.LOAD_DURATION_METRIC).time();
+ verifyTable(msTbl);
+ try {
+ // Copy the table to check later if anything has changed.
+ msTable_ = msTbl;
+ setTableStats(msTable_);
+ // Load metadata from Paimon
+ final Timer.Context ctxStorageLdTime =
+ getMetrics().getTimer(Table.LOAD_DURATION_STORAGE_METADATA).time();
+ try {
+ table_ = PaimonUtil.createFileStoreTable(msTbl);
+ catalogTimeline.markEvent("Loaded Paimon API table");
+ loadSchemaFromPaimon();
+ catalogTimeline.markEvent("Loaded schema from Paimon");
+ applyPaimonTableStatsIfPresent();
+ loadAllColumnStats(msClient, catalogTimeline);
+ applyPaimonColumnStatsIfPresent();
+ catalogTimeline.markEvent("Loaded stats from Paimon");
+ } catch (Exception e) {
+ throw new TableLoadingException(
+ "Error loading metadata for Paimon table " + msTbl.getTableName(), e);
+ } finally {
+ catalogTimeline.markEvent("Loaded all from Paimon");
+ storageMetadataLoadTime_ = ctxStorageLdTime.stop();
+ }
+ refreshLastUsedTime();
+ } finally {
+ context.stop();
+ }
+ }
+
+ @Override
+ public org.apache.paimon.table.Table getPaimonApiTable() {
+ return table_;
+ }
+
+}
diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonTableLoadingException.java b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonTableLoadingException.java
new file mode 100644
index 000000000..50c2ee778
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonTableLoadingException.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.impala.catalog.paimon;
+
+import org.apache.impala.common.ImpalaException;
+
+public class PaimonTableLoadingException extends ImpalaException {
+ public PaimonTableLoadingException(String msg, Throwable cause) { super(msg, cause); }
+}
diff --git a/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonUtil.java b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonUtil.java
new file mode 100644
index 000000000..2db411590
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/catalog/paimon/PaimonUtil.java
@@ -0,0 +1,861 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.impala.catalog.paimon;
+
+import static org.apache.impala.catalog.Table.isExternalPurgeTable;
+import static org.apache.paimon.CoreOptions.PARTITION_DEFAULT_NAME;
+import static org.apache.paimon.CoreOptions.PARTITION_GENERATE_LEGCY_NAME;
+import static org.apache.paimon.utils.HadoopUtils.HADOOP_LOAD_DEFAULT_CONFIG;
+
+import com.google.common.collect.Lists;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
+import org.apache.impala.catalog.Column;
+import org.apache.impala.catalog.DatabaseNotFoundException;
+import org.apache.impala.catalog.HdfsFileFormat;
+import org.apache.impala.catalog.MetaStoreClientPool;
+import org.apache.impala.catalog.Type;
+import org.apache.impala.common.FileSystemUtil;
+import org.apache.impala.common.ImpalaRuntimeException;
+import org.apache.impala.common.PrintUtils;
+import org.apache.impala.compat.MetastoreShim;
+import org.apache.impala.thrift.TColumn;
+import org.apache.impala.thrift.TDescribeHistoryParams;
+import org.apache.impala.thrift.TGetTableHistoryResult;
+import org.apache.impala.thrift.TGetTableHistoryResultItem;
+import org.apache.impala.thrift.TPaimonCatalog;
+import org.apache.impala.thrift.TPaimonTable;
+import org.apache.impala.thrift.TPaimonTableKind;
+import org.apache.impala.thrift.TResultSet;
+import org.apache.impala.thrift.TResultSetMetadata;
+import org.apache.impala.thrift.TShowFilesParams;
+import org.apache.impala.thrift.TTableStats;
+import org.apache.impala.util.TResultRowBuilder;
+import org.apache.paimon.CoreOptions;
+import org.apache.paimon.catalog.AbstractCatalog;
+import org.apache.paimon.catalog.Catalog;
+import org.apache.paimon.catalog.CatalogContext;
+import org.apache.paimon.catalog.CatalogFactory;
+import org.apache.paimon.catalog.CatalogUtils;
+import org.apache.paimon.catalog.Identifier;
+import org.apache.paimon.data.InternalRow;
+import org.apache.paimon.data.Timestamp;
+import org.apache.paimon.fs.FileIO;
+import org.apache.paimon.fs.Path;
+import org.apache.paimon.hive.HiveCatalog;
+import org.apache.paimon.hive.HiveTypeUtils;
+import org.apache.paimon.hive.LocationKeyExtractor;
+import org.apache.paimon.hive.utils.HiveUtils;
+import org.apache.paimon.options.CatalogOptions;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.partition.Partition;
+import org.apache.paimon.predicate.Predicate;
+import org.apache.paimon.predicate.PredicateBuilder;
+import org.apache.paimon.reader.RecordReader;
+import org.apache.paimon.schema.Schema;
+import org.apache.paimon.stats.ColStats;
+import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.table.FileStoreTableFactory;
+import org.apache.paimon.table.Table;
+import org.apache.paimon.table.source.DataSplit;
+import org.apache.paimon.table.source.DeletionFile;
+import org.apache.paimon.table.source.RawFile;
+import org.apache.paimon.table.source.ReadBuilder;
+import org.apache.paimon.table.source.Split;
+import org.apache.paimon.table.source.TableRead;
+import org.apache.paimon.table.source.TableScan;
+import org.apache.paimon.table.system.SystemTableLoader;
+import org.apache.paimon.types.BigIntType;
+import org.apache.paimon.types.BooleanType;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.types.DataTypeFamily;
+import org.apache.paimon.types.DoubleType;
+import org.apache.paimon.types.FloatType;
+import org.apache.paimon.types.IntType;
+import org.apache.paimon.types.RowType;
+import org.apache.paimon.types.SmallIntType;
+import org.apache.paimon.types.TinyIntType;
+import org.apache.paimon.utils.InternalRowPartitionComputer;
+import org.apache.thrift.TException;
+import org.postgresql.shaded.com.ongres.scram.common.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+public class PaimonUtil {
+ final static Logger LOG = LoggerFactory.getLogger(PaimonUtil.class);
+
+ public static final String PAIMON_STORAGE_HANDLER =
+ "org.apache.paimon.hive.PaimonStorageHandler";
+ public static final String STORAGE_HANDLER = "storage_handler";
+ public static final String PAIMON_CATALOG = "paimon.catalog";
+ public static final String HIVE_CATALOG = "hive";
+ public static final String PAIMON_PROPERTY_PREFIX = "";
+ public static final String PAIMON_HADOOP_CATALOG_LOCATION = "paimon.catalog_location";
+ public static final String PAIMON_TABLE_LOCATION = "paimon_location";
+ public static final String PAIMON_TABLE_IDENTIFIER = "paimon.table_identifier";
+
+ private static final HiveConf hiveConf_ = new HiveConf();
+ public static Catalog catalog_ = null;
+ private static final String metastoreClientClass_ =
+ "org.apache.hadoop.hive.metastore.HiveMetaStoreClient";
+
+ /**
+ * Returns true if the given Metastore Table represents an Paimon table.
+ * Versions of Hive/Paimon are inconsistent which Paimon related fields are set
+ * (e.g., HIVE-6548 changed the input format to null).
+ * For maximum compatibility consider all known fields that indicate an Paimon table.
+ */
+ public static boolean isPaimonTable(org.apache.hadoop.hive.metastore.api.Table msTbl) {
+ if (msTbl.getParameters() != null
+ && PAIMON_STORAGE_HANDLER.equals(
+ msTbl.getParameters().getOrDefault(STORAGE_HANDLER, ""))) {
+ return true;
+ }
+ StorageDescriptor sd = msTbl.getSd();
+ if (sd == null) return false;
+ if (sd.getInputFormat() != null
+ && sd.getInputFormat().equals(HdfsFileFormat.PAIMON.inputFormat())) {
+ return true;
+ } else
+ return sd.getSerdeInfo() != null && sd.getSerdeInfo().getSerializationLib() != null
+ && sd.getSerdeInfo().getSerializationLib().equals(
+ HdfsFileFormat.PAIMON.serializationLib());
+ }
+
+ public static ByteBuffer serialize(FePaimonTable paimonTable) throws IOException {
+ return ByteBuffer.wrap(SerializationUtils.serialize(paimonTable.getPaimonApiTable()));
+ }
+
+ public static Table deserialize(ByteBuffer b) throws Exception {
+ return SerializationUtils.deserialize(b.array());
+ }
+
+ /**
+ * Get Thrift object for paimon table.
+ */
+ public static TPaimonTable getTPaimonTable(FePaimonTable paimonTable)
+ throws IOException {
+ TPaimonTable t_ = new TPaimonTable();
+ t_.setKind(TPaimonTableKind.JNI);
+ t_.setJni_tbl_obj(serialize(paimonTable));
+ return t_;
+ }
+
+ /**
+ * Converts Paimon schema to a Hive schema.
+ */
+ public static List convertToHiveSchema(RowType schema)
+ throws ImpalaRuntimeException {
+ List ret = new ArrayList<>();
+ for (DataField dataField : schema.getFields()) {
+ ret.add(new FieldSchema(dataField.name().toLowerCase(),
+ HiveTypeUtils.toTypeInfo(dataField.type()).getTypeName(),
+ dataField.description()));
+ }
+ return ret;
+ }
+
+ /**
+ * Converts Paimon schema to an Impala schema.
+ */
+ public static List convertToImpalaSchema(RowType schema)
+ throws ImpalaRuntimeException {
+ List ret = new ArrayList<>();
+ int pos = 0;
+ for (DataField dataField : schema.getFields()) {
+ Type colType = ImpalaTypeUtils.toImpalaType(dataField.type());
+ ret.add(new Column(dataField.name().toLowerCase(), colType, pos++));
+ }
+ return ret;
+ }
+
+ /**
+ * Generates Paimon schema from given columns.
+ */
+ public static Schema genPaimonSchema(List columns, List partitionKeys,
+ Map options) {
+ Schema.Builder schemaBuilder = Schema.newBuilder();
+ for (TColumn column : columns) {
+ schemaBuilder.column(column.getColumnName().toLowerCase(),
+ ImpalaTypeUtils.fromImpalaType(Type.fromThrift(column.getColumnType())));
+ }
+ if (!partitionKeys.isEmpty()) { schemaBuilder.partitionKeys(partitionKeys); }
+ if (!options.isEmpty()) { schemaBuilder.options(options); }
+ return schemaBuilder.build();
+ }
+
+ /**
+ * Returns the corresponding paimon catalog implementation.
+ * TODO:
+ */
+ public static Catalog getPaimonCatalog(TPaimonCatalog catalog, boolean isExternal,
+ String warehouse_location) throws ImpalaRuntimeException {
+ switch (catalog) {
+ case HADOOP_CATALOG: {
+ Preconditions.checkNotNull(
+ warehouse_location, "warehouse location should not be null");
+ CatalogContext context = CatalogContext.create(new Path(warehouse_location));
+ return CatalogFactory.createCatalog(context);
+ }
+ case HIVE_CATALOG: {
+ try {
+ String location = isExternal ?
+ hiveConf_.get(HiveConf.ConfVars.HIVE_METASTORE_WAREHOUSE_EXTERNAL.varname) :
+ hiveConf_.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname);
+ Path path = new Path(location);
+ Options catalogOptions = new Options();
+ catalogOptions.set(CatalogOptions.WAREHOUSE, location);
+ CatalogContext catalogContext = CatalogContext.create(catalogOptions);
+ FileIO fileIO = FileIO.get(path, catalogContext);
+ HiveCatalog externalWarehouseCatalog =
+ new HiveCatalog(fileIO, hiveConf_, metastoreClientClass_, location);
+ return externalWarehouseCatalog;
+ } catch (Exception ex) {
+ throw new ImpalaRuntimeException("failed to create hive catalog : ", ex);
+ }
+ }
+ default: throw new ImpalaRuntimeException("Unexpected catalog type: " + catalog);
+ }
+ }
+
+ /**
+ * get Paimon Identifier object.
+ */
+ public static Identifier getTableIdentifier(String dbName, String tableName) {
+ return new Identifier(dbName, tableName);
+ }
+
+ /**
+ * Generates Paimon table identifier from HMS table object.
+ */
+ public static Identifier getTableIdentifier(
+ org.apache.hadoop.hive.metastore.api.Table msTable) {
+ String name = msTable.getParameters().get(PAIMON_TABLE_IDENTIFIER);
+ if (name == null || name.isEmpty()) {
+ return getTableIdentifier(
+ msTable.getDbName().toLowerCase(), msTable.getTableName().toLowerCase());
+ } else {
+ if (!name.contains(".")) {
+ return getTableIdentifier(msTable.getDbName(), name);
+ } else {
+ String[] names = name.split("\\.");
+ return getTableIdentifier(names[0], names[1]);
+ }
+ }
+ }
+
+ /**
+ * Convert paimon column stats to HMS ColumnStatisticsData.
+ */
+ public static Optional convertColStats(
+ ColStats> colStats, DataField dataField) {
+ ColumnStatisticsData columnStatisticsData = new ColumnStatisticsData();
+ colStats.deserializeFieldsFromString(dataField.type());
+
+ Set fieldFamilySet = dataField.type().getTypeRoot().getFamilies();
+ if (fieldFamilySet.contains(DataTypeFamily.NUMERIC)) {
+ if (fieldFamilySet.contains(DataTypeFamily.INTEGER_NUMERIC)) {
+ // LONG_STATS
+ LongColumnStatsData longColumnStatsData = new LongColumnStatsData();
+ if (colStats.nullCount().isPresent()) {
+ longColumnStatsData.setNumNulls(colStats.nullCount().getAsLong());
+ }
+ if (dataField.type() instanceof BigIntType) {
+ if (colStats.min().isPresent()) {
+ longColumnStatsData.setLowValue((Long) colStats.min().get());
+ }
+ if (colStats.max().isPresent()) {
+ longColumnStatsData.setHighValue((Long) colStats.max().get());
+ }
+ } else if (dataField.type() instanceof IntType) {
+ if (colStats.min().isPresent()) {
+ longColumnStatsData.setLowValue((Integer) colStats.min().get());
+ }
+ if (colStats.max().isPresent()) {
+ longColumnStatsData.setHighValue((Integer) colStats.max().get());
+ }
+ } else if (dataField.type() instanceof SmallIntType) {
+ if (colStats.min().isPresent()) {
+ longColumnStatsData.setLowValue((Short) colStats.min().get());
+ }
+ if (colStats.max().isPresent()) {
+ longColumnStatsData.setHighValue((Short) colStats.max().get());
+ }
+ } else if (dataField.type() instanceof TinyIntType) {
+ if (colStats.min().isPresent()) {
+ longColumnStatsData.setLowValue((Byte) colStats.min().get());
+ }
+ if (colStats.max().isPresent()) {
+ longColumnStatsData.setHighValue((Byte) colStats.max().get());
+ }
+ } else {
+ LOG.warn(String.format("Column stats doesn't support data type %s",
+ dataField.type().asSQLString()));
+ return Optional.empty();
+ }
+
+ if (colStats.distinctCount().isPresent()) {
+ longColumnStatsData.setNumDVs(colStats.distinctCount().getAsLong());
+ }
+ columnStatisticsData.setLongStats(longColumnStatsData);
+ return Optional.of(columnStatisticsData);
+ } else if (fieldFamilySet.contains(DataTypeFamily.APPROXIMATE_NUMERIC)) {
+ // DOUBLE_STATS
+ DoubleColumnStatsData doubleColumnStatsData = new DoubleColumnStatsData();
+ if (colStats.nullCount().isPresent()) {
+ doubleColumnStatsData.setNumNulls(colStats.nullCount().getAsLong());
+ }
+ if (dataField.type() instanceof DoubleType) {
+ if (colStats.min().isPresent()) {
+ doubleColumnStatsData.setLowValue((Double) colStats.min().get());
+ }
+ if (colStats.max().isPresent()) {
+ doubleColumnStatsData.setHighValue((Double) colStats.max().get());
+ }
+ } else if (dataField.type() instanceof FloatType) {
+ if (colStats.min().isPresent()) {
+ doubleColumnStatsData.setLowValue((Float) colStats.min().get());
+ }
+ if (colStats.max().isPresent()) {
+ doubleColumnStatsData.setHighValue((Float) colStats.max().get());
+ }
+ } else {
+ LOG.warn(String.format("Column stats doesn't support data type %s",
+ dataField.type().asSQLString()));
+ return Optional.empty();
+ }
+ if (colStats.distinctCount().isPresent()) {
+ doubleColumnStatsData.setNumDVs(colStats.distinctCount().getAsLong());
+ }
+ columnStatisticsData.setDoubleStats(doubleColumnStatsData);
+ return Optional.of(columnStatisticsData);
+
+ } else {
+ LOG.warn(String.format("Column stats doesn't support data type %s",
+ dataField.type().asSQLString()));
+ return Optional.empty();
+ }
+ } else if (fieldFamilySet.contains(DataTypeFamily.CHARACTER_STRING)) {
+ // STRING_STATS
+ StringColumnStatsData stringColumnStatsData = new StringColumnStatsData();
+ if (colStats.nullCount().isPresent()) {
+ stringColumnStatsData.setNumNulls(colStats.nullCount().getAsLong());
+ }
+ if (colStats.avgLen().isPresent()) {
+ stringColumnStatsData.setAvgColLen(colStats.avgLen().getAsLong());
+ }
+ if (colStats.maxLen().isPresent()) {
+ stringColumnStatsData.setMaxColLen(colStats.maxLen().getAsLong());
+ }
+ columnStatisticsData.setStringStats(stringColumnStatsData);
+ return Optional.of(columnStatisticsData);
+
+ } else if (fieldFamilySet.contains(DataTypeFamily.BINARY_STRING)) {
+ // BINARY_STATS
+ BinaryColumnStatsData binaryColumnStatsData = new BinaryColumnStatsData();
+ if (colStats.nullCount().isPresent()) {
+ binaryColumnStatsData.setNumNulls(colStats.nullCount().getAsLong());
+ }
+ if (colStats.avgLen().isPresent()) {
+ binaryColumnStatsData.setAvgColLen(colStats.avgLen().getAsLong());
+ }
+ if (colStats.maxLen().isPresent()) {
+ binaryColumnStatsData.setMaxColLen(colStats.maxLen().getAsLong());
+ }
+ columnStatisticsData.setBinaryStats(binaryColumnStatsData);
+ return Optional.of(columnStatisticsData);
+
+ } else if (dataField.type() instanceof BooleanType) {
+ // BOOLEAN_STATS
+ BooleanColumnStatsData booleanColumnStatsData = new BooleanColumnStatsData();
+ if (colStats.nullCount().isPresent()) {
+ booleanColumnStatsData.setNumNulls(colStats.nullCount().getAsLong());
+ }
+ columnStatisticsData.setBooleanStats(booleanColumnStatsData);
+ return Optional.of(columnStatisticsData);
+
+ } else {
+ LOG.warn(String.format("Column stats doesn't support data type %s",
+ dataField.type().asSQLString()));
+ return Optional.empty();
+ }
+ }
+
+ /**
+ * Get FileStore Object for paimon table in HMS.
+ */
+ public static FileStoreTable createFileStoreTable(
+ org.apache.hadoop.hive.metastore.api.Table table) throws MetaException {
+ Options options = HiveUtils.extractCatalogConfig(FePaimonTable.jobConf);
+ options.set(CoreOptions.PATH,
+ LocationKeyExtractor.getPaimonLocation(FePaimonTable.jobConf, table));
+ CatalogContext catalogContext;
+ if (options.get(HADOOP_LOAD_DEFAULT_CONFIG)) {
+ catalogContext = CatalogContext.create(options, FePaimonTable.jobConf);
+ } else {
+ catalogContext = CatalogContext.create(options);
+ }
+ return FileStoreTableFactory.create(catalogContext);
+ }
+
+ /**
+ * Get FileStore Object for specified location
+ */
+ public static FileStoreTable createFileStoreTable(String tableLocation)
+ throws MetaException {
+ Options options = HiveUtils.extractCatalogConfig(FePaimonTable.jobConf);
+ options.set(CoreOptions.PATH, tableLocation);
+ CatalogContext catalogContext;
+ if (options.get(HADOOP_LOAD_DEFAULT_CONFIG)) {
+ catalogContext = CatalogContext.create(options, FePaimonTable.jobConf);
+ } else {
+ catalogContext = CatalogContext.create(options);
+ }
+ return FileStoreTableFactory.create(catalogContext);
+ }
+
+ /**
+ * Return paimon catalog from table properties
+ */
+ public static List fieldNames(RowType rowType) {
+ return rowType.getFields()
+ .stream()
+ .map(DataField::name)
+ .map(String::toLowerCase)
+ .collect(Collectors.toList());
+ }
+
+ /**
+ * check if the given table has primary keys
+ */
+ public static boolean hasPrimaryKey(Table table) {
+ return !table.primaryKeys().isEmpty();
+ }
+
+ /**
+ * check if the given table is partitioned
+ */
+ public static boolean hasPartition(Table table) {
+ return !table.partitionKeys().isEmpty();
+ }
+
+ /**
+ * lookup result for given table, max 1000 rows.
+ */
+ public static List lookupInTable(Table table, List predicates) {
+ return lookupInTable(table, predicates, 1000);
+ }
+
+ /**
+ * lookup result for given table.
+ */
+ protected static List lookupInTable(
+ Table table, List predicates, int maxcount) {
+ ReadBuilder readBuilder = table.newReadBuilder().withFilter(predicates);
+ List splits = readBuilder.newScan().plan().splits();
+ TableRead read = readBuilder.newRead();
+ final List internalRows = Lists.newArrayList();
+ try {
+ try (RecordReader recordReader = read.createReader(splits)) {
+ RecordReader.RecordIterator recordIterator =
+ recordReader.readBatch();
+ InternalRow internalRow = null;
+ while ((internalRow = recordIterator.next()) != null) {
+ internalRows.add(internalRow);
+ if (internalRows.size() >= maxcount) { break; }
+ }
+ }
+ } catch (IOException ex) {
+ LOG.warn("failed to read table", ex);
+ return Lists.newArrayList();
+ }
+ return internalRows;
+ }
+
+ /**
+ * check whether the given table support predicate pushdown.
+ */
+ public static boolean canApplyPredicatePushDown(Table table) {
+ return table instanceof FileStoreTable;
+ }
+
+ /**
+ * query the snapshot history for paimon table.
+ */
+ public static TGetTableHistoryResult getPaimonTableHistory(FePaimonTable feTable,
+ TDescribeHistoryParams params) throws DatabaseNotFoundException {
+ try {
+ // Column indexes for paimon snapshot table.
+ // Used to select required columns from snapshot table.
+ // 5: commit_time
+ // 0: snapshot id
+ // 1: schema_id
+ final int[] SNAPSHOT_TABLE_PROJECTION = {5, 0, 1};
+ TGetTableHistoryResult historyResult = new TGetTableHistoryResult();
+ FileStoreTable table = (FileStoreTable) feTable.getPaimonApiTable();
+ org.apache.paimon.table.Table snapshotTable =
+ SystemTableLoader.load("snapshots", table);
+ PredicateBuilder predicateBuilder = new PredicateBuilder(snapshotTable.rowType());
+ Optional predicataOpt = Optional.empty();
+ if (params.isSetFrom_time()) {
+ // DESCRIBE HISTORY FROM
+ // check if commit-time >= from_time
+ predicataOpt = Optional.of(predicateBuilder.greaterOrEqual(
+ 0, Timestamp.fromEpochMillis(params.getFrom_time())));
+ } else if (params.isSetBetween_start_time() && params.isSetBetween_end_time()) {
+ predicataOpt = Optional.of(PredicateBuilder.and(
+ predicateBuilder.greaterOrEqual(
+ 0, Timestamp.fromEpochMillis(params.getBetween_start_time())),
+ predicateBuilder.lessOrEqual(
+ 0, Timestamp.fromEpochMillis(params.getBetween_end_time()))));
+ }
+ ReadBuilder readBuilder =
+ snapshotTable.newReadBuilder().withProjection(SNAPSHOT_TABLE_PROJECTION);
+ predicataOpt.ifPresent(readBuilder::withFilter);
+ List splits = readBuilder.newScan().plan().splits();
+ RecordReader internalRowRecordReader =
+ readBuilder.newRead().createReader(splits);
+ List result =
+ com.google.common.collect.Lists.newArrayList();
+
+ internalRowRecordReader.forEachRemaining(new Consumer() {
+ @Override
+ public void accept(InternalRow internalRow) {
+ TGetTableHistoryResultItem resultItem = new TGetTableHistoryResultItem();
+ long snapshotId = internalRow.getLong(1);
+ Timestamp timestamp = internalRow.getTimestamp(0, 9);
+ resultItem.setCreation_time(timestamp.getMillisecond());
+ resultItem.setSnapshot_id(snapshotId);
+ // note: parent id and ancestor id is always null
+ result.add(resultItem);
+ }
+ });
+ historyResult.setResult(result);
+ return historyResult;
+ } catch (Exception ex) {
+ throw new DatabaseNotFoundException("Failed to get snapshot: " + ex.getMessage());
+ }
+ }
+
+ /**
+ * Get HdfsFileFormat from a string, usually from table properties.
+ * Returns PARQUET when 'format' is null. Returns null for invalid formats.
+ */
+ public static HdfsFileFormat getPaimonFileFormat(String format) {
+ if ("PARQUET".equalsIgnoreCase(format) || format == null) {
+ return HdfsFileFormat.PARQUET;
+ } else if ("ORC".equalsIgnoreCase(format)) {
+ return HdfsFileFormat.ORC;
+ }
+ return null;
+ }
+
+ /**
+ * A table is synchronized table if its Managed table or if its a external table with
+ * external.table.purge property set to true.
+ * We need to create/drop/etc. synchronized tables through the Paimon APIs as well.
+ */
+ public static boolean isSynchronizedTable(
+ org.apache.hadoop.hive.metastore.api.Table msTbl) {
+ com.google.common.base.Preconditions.checkState(isPaimonTable(msTbl));
+ return isManagedTable(msTbl) || isExternalPurgeTable(msTbl);
+ }
+
+ /**
+ * Returns if this metastore table has managed table type
+ */
+ public static boolean isManagedTable(org.apache.hadoop.hive.metastore.api.Table msTbl) {
+ return msTbl.getTableType().equalsIgnoreCase(TableType.MANAGED_TABLE.toString());
+ }
+
+ /**
+ * get the location for the newly created paimon table.
+ */
+ public static String getPaimonCatalogLocation(
+ MetaStoreClientPool.MetaStoreClient msClient,
+ org.apache.hadoop.hive.metastore.api.Table msTable) throws TException {
+ TPaimonCatalog catalog = getTPaimonCatalog(msTable);
+ if (catalog == TPaimonCatalog.HADOOP_CATALOG) {
+ // Using catalog location to create table
+ // We cannot set location for 'hadoop.catalog' table in SQL
+ String location = msTable.getParameters().get(PAIMON_HADOOP_CATALOG_LOCATION);
+ Identifier table_identifier = getTableIdentifier(msTable);
+ return AbstractCatalog.newTableLocation(location, table_identifier).toString();
+ } else {
+ // Using normal location as 'hadoop.tables' table location and create
+ // table
+ return MetastoreShim.getPathForNewTable(
+ msClient.getHiveClient().getDatabase(msTable.getDbName()), msTable);
+ }
+ }
+
+ /**
+ * Get Paimon table catalog location with 'paimon.catalog_location' when using
+ * 'hadoop.catalog'
+ */
+ public static String getPaimonCatalogLocation(
+ org.apache.hadoop.hive.metastore.api.Table msTable) {
+ return msTable.getParameters().get(PAIMON_HADOOP_CATALOG_LOCATION);
+ }
+
+ /**
+ * Get Paimon table catalog type from hms table properties
+ * use HiveCatalog as default
+ */
+ public static TPaimonCatalog getTPaimonCatalog(
+ org.apache.hadoop.hive.metastore.api.Table msTable) {
+ return getTPaimonCatalog(msTable.getParameters());
+ }
+
+ /**
+ * Get Paimon table catalog type from properties
+ * use HiveCatalog as default
+ */
+ public static TPaimonCatalog getTPaimonCatalog(Map props) {
+ return getTPaimonCatalog(props.get(PAIMON_CATALOG));
+ }
+
+ /**
+ * Get Paimon table catalog type string
+ * use HiveCatalog as default
+ */
+ public static TPaimonCatalog getTPaimonCatalog(String catalog) {
+ if ("hadoop".equalsIgnoreCase(catalog)) {
+ return TPaimonCatalog.HADOOP_CATALOG;
+ } else if (HIVE_CATALOG.equalsIgnoreCase(catalog) || catalog == null) {
+ return TPaimonCatalog.HIVE_CATALOG;
+ } else {
+ return TPaimonCatalog.HIVE_CATALOG;
+ }
+ }
+
+ /**
+ * Extract column names from string
+ */
+ public static List extractColumnNames(String value) {
+ return Arrays.stream(value.split(",")).
+ map(String::toLowerCase).collect(Collectors.toList());
+ }
+
+ /**
+ * Create catalog context from HMS table and location
+ */
+ public static CatalogContext catalogContext(
+ org.apache.hadoop.hive.metastore.api.Table table, String location) {
+ Options options = HiveUtils.extractCatalogConfig(hiveConf_);
+ options.set(CoreOptions.PATH, location);
+ table.getParameters().forEach(options::set);
+ return CatalogContext.create(options, hiveConf_);
+ }
+
+ /**
+ * Create catalog context from HMS table and location
+ */
+ public static String partitionSpecToString(final Map spec) {
+ List speclist = spec.keySet()
+ .stream()
+ .map(k -> String.join("=", k, spec.get(k)))
+ .collect(Collectors.toList());
+ return String.join("/", speclist);
+ }
+
+ /**
+ * Get partition stats for the given fe paimon table.
+ */
+ public static TResultSet doGetTableStats(FePaimonTable table) {
+ TResultSet result = new TResultSet();
+ TResultSetMetadata resultSchema = new TResultSetMetadata();
+ result.setSchema(resultSchema);
+ result.setRows(new ArrayList<>());
+
+ resultSchema.addToColumns(new TColumn("Number Of Rows", Type.BIGINT.toThrift()));
+ resultSchema.addToColumns(new TColumn("Number Of Bytes", Type.BIGINT.toThrift()));
+ TTableStats stats = table.getTTableStats();
+ {
+ TResultRowBuilder builder = new TResultRowBuilder();
+ builder.add(stats.getNum_rows());
+ builder.add(stats.getTotal_file_bytes());
+ result.addToRows(builder.get());
+ }
+ return result;
+ }
+
+ public static long localMillisToUTCMillis(long epochMillis) {
+ ZoneId zone = ZoneId.systemDefault();
+ ZoneOffset offset = zone.getRules().getOffset(Instant.now());
+ return epochMillis + offset.getTotalSeconds() * 1000L;
+ }
+
+ /**
+ * Get partition stats for the given fe paimon table.
+ */
+ public static TResultSet doGetPartitionStats(FePaimonTable table) {
+ TResultSet result = new TResultSet();
+ TResultSetMetadata resultSchema = new TResultSetMetadata();
+ result.setSchema(resultSchema);
+ result.setRows(new ArrayList<>());
+
+ resultSchema.addToColumns(new TColumn("Partition", Type.STRING.toThrift()));
+ resultSchema.addToColumns(new TColumn("Number Of Rows", Type.BIGINT.toThrift()));
+ resultSchema.addToColumns(new TColumn("Number Of Files", Type.BIGINT.toThrift()));
+ resultSchema.addToColumns(new TColumn("Number Of Bytes", Type.BIGINT.toThrift()));
+ resultSchema.addToColumns(new TColumn("Last Creation Time", Type.BIGINT.toThrift()));
+
+ List partitions =
+ CatalogUtils.listPartitionsFromFileSystem(table.getPaimonApiTable());
+ for (Partition partition : partitions) {
+ TResultRowBuilder builder = new TResultRowBuilder();
+ builder.add(partitionSpecToString(partition.spec()));
+ builder.add(partition.recordCount());
+ builder.add(partition.fileCount());
+ builder.add(partition.fileSizeInBytes());
+ // epoch millis obtained via paimon api is from LocalDateTime
+ // with default timezone. Different time zone will yield different values,
+ // will remove default timezone offset for the epoch millis obtained via
+ // piamon api.
+ builder.add(localMillisToUTCMillis(partition.lastFileCreationTime()));
+ result.addToRows(builder.get());
+ }
+ return result;
+ }
+
+ public static int getFieldIndexByNameIgnoreCase(RowType rowType, String fieldName) {
+ for (int i = 0; i < rowType.getFields().size(); ++i) {
+ if (rowType.getFields().get(i).name().equalsIgnoreCase(fieldName)) { return i; }
+ }
+ return -1;
+ }
+
+ private static Optional extractPartitionFilter(
+ Table table, TShowFilesParams request) {
+ Iterator> iter =
+ request.getPartition_setIterator();
+ PredicateBuilder predicateBuilder = new PredicateBuilder(table.rowType());
+ List predicates =
+ request.getPartition_set()
+ .parallelStream()
+ .map(l
+ -> PredicateBuilder.and(
+ l.stream()
+ .map(kv
+ -> predicateBuilder.equal(getFieldIndexByNameIgnoreCase(
+ table.rowType(), kv.getName()),
+ kv.getValue()))
+ .collect(Collectors.toList())))
+ .collect(Collectors.toList());
+ if (!predicates.isEmpty()) {
+ return Optional.of(PredicateBuilder.or(predicates));
+ } else {
+ return Optional.empty();
+ }
+ }
+
+ /**
+ * Get files for the given fe paimon table.
+ */
+ public static TResultSet doGetTableFiles(
+ FePaimonTable paimon_table, TShowFilesParams request) {
+ Table table = paimon_table.getPaimonApiTable();
+ TResultSet result = new TResultSet();
+ TResultSetMetadata resultSchema = new TResultSetMetadata();
+ result.setSchema(resultSchema);
+ resultSchema.addToColumns(new TColumn("Path", Type.STRING.toThrift()));
+ resultSchema.addToColumns(new TColumn("Size", Type.STRING.toThrift()));
+ resultSchema.addToColumns(new TColumn("Partition", Type.STRING.toThrift()));
+ resultSchema.addToColumns(new TColumn("EC Policy", Type.STRING.toThrift()));
+ resultSchema.addToColumns(new TColumn("Type", Type.STRING.toThrift()));
+ ReadBuilder readBuilder = table.newReadBuilder();
+ result.setRows(new ArrayList<>());
+ if (request.isSetPartition_set()) {
+ Optional predicate = extractPartitionFilter(table, request);
+ if (predicate.isPresent()) {
+ readBuilder = readBuilder.withFilter(predicate.get());
+ }
+ }
+
+ TableScan.Plan plan = readBuilder.newScan().plan();
+
+ Options options = Options.fromMap(table.options());
+
+ InternalRowPartitionComputer computer =
+ new InternalRowPartitionComputer(options.get(PARTITION_DEFAULT_NAME),
+ table.rowType(), table.partitionKeys().toArray(new String[0]),
+ options.get(PARTITION_GENERATE_LEGCY_NAME));
+
+ for (Split split : plan.splits()) {
+ if (!(split instanceof DataSplit)) continue;
+ DataSplit dataSplit = (DataSplit) split;
+ Optional> rawFiles = dataSplit.convertToRawFiles();
+ if (rawFiles.isPresent()) {
+ for (RawFile rawFile : rawFiles.get()) {
+ TResultRowBuilder builder = new TResultRowBuilder();
+ builder.add(rawFile.path());
+ builder.add(PrintUtils.printBytes(rawFile.fileSize()));
+ builder.add(
+ partitionSpecToString(computer.generatePartValues(dataSplit.partition())));
+ builder.add(FileSystemUtil.getErasureCodingPolicy(
+ new org.apache.hadoop.fs.Path(rawFile.path())));
+ builder.add("DATA");
+ result.addToRows(builder.get());
+ }
+ }
+ if (split.deletionFiles().isPresent()) {
+ for (DeletionFile deletionFile : split.deletionFiles().get()) {
+ if (deletionFile == null) break;
+ TResultRowBuilder builder = new TResultRowBuilder();
+ builder.add(deletionFile.path());
+ builder.add(PrintUtils.printBytes(deletionFile.length()));
+ builder.add(
+ partitionSpecToString(computer.generatePartValues(dataSplit.partition())));
+ builder.add(FileSystemUtil.getErasureCodingPolicy(
+ new org.apache.hadoop.fs.Path(deletionFile.path())));
+ builder.add("DELETE");
+ result.addToRows(builder.get());
+ }
+ }
+ }
+ return result;
+ }
+}
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 38e946fbd..1ea95efc1 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -157,6 +157,10 @@ import org.apache.impala.catalog.events.MetastoreEventsProcessor;
import org.apache.impala.catalog.events.MetastoreNotificationException;
import org.apache.impala.catalog.monitor.CatalogMonitor;
import org.apache.impala.catalog.monitor.CatalogOperationTracker;
+import org.apache.impala.catalog.paimon.FePaimonTable;
+import org.apache.impala.catalog.paimon.PaimonCatalogOpExecutor;
+import org.apache.impala.catalog.paimon.PaimonTableLoadingException;
+import org.apache.impala.catalog.paimon.PaimonUtil;
import org.apache.impala.common.FileSystemUtil;
import org.apache.impala.common.ImpalaException;
import org.apache.impala.common.ImpalaRuntimeException;
@@ -231,6 +235,7 @@ import org.apache.impala.thrift.TIcebergCatalog;
import org.apache.impala.thrift.TImpalaTableType;
import org.apache.impala.thrift.TIcebergPartitionSpec;
import org.apache.impala.thrift.TKuduPartitionParam;
+import org.apache.impala.thrift.TPaimonCatalog;
import org.apache.impala.thrift.TPartitionDef;
import org.apache.impala.thrift.TPartitionKeyValue;
import org.apache.impala.thrift.TPartitionStats;
@@ -267,6 +272,7 @@ import org.apache.impala.util.MetaStoreUtil;
import org.apache.impala.util.MetaStoreUtil.TableInsertEventInfo;
import org.apache.impala.util.NoOpEventSequence;
import org.apache.impala.util.ThreadNameAnnotator;
+import org.apache.paimon.table.DataTable;
import org.apache.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -3322,7 +3328,10 @@ public class CatalogOpExecutor {
tableName.getTbl()), e);
}
}
-
+ if (existingTbl instanceof FePaimonTable) {
+ needsHmsDropTable = PaimonCatalogOpExecutor.dropTable(
+ msTbl, existingTbl, catalogTimeline, params);
+ }
if (needsHmsDropTable) {
try (MetaStoreClient msClient = catalog_.getMetaStoreClient(catalogTimeline)) {
@@ -3866,6 +3875,9 @@ public class CatalogOpExecutor {
params.if_not_exists, params.getColumns(), params.getPartition_spec(),
params.getPrimary_key_column_names(), params.getTable_properties(),
params.getComment(), debugAction);
+ } else if (PaimonUtil.isPaimonTable(tbl)) {
+ return createPaimonTable(
+ tbl, wantMinimalResult, response, catalogTimeline, params, debugAction);
}
Preconditions.checkState(params.getColumns().size() > 0,
"Empty column list given as argument to Catalog.createTable");
@@ -4354,6 +4366,134 @@ public class CatalogOpExecutor {
return true;
}
+ /**
+ * Creates a new Paimon table.
+ */
+ private boolean createPaimonTable(org.apache.hadoop.hive.metastore.api.Table newTable,
+ boolean wantMinimalResult, TDdlExecResponse response, EventSequence catalogTimeline,
+ TCreateTableParams params, @Nullable String debugAction) throws ImpalaException {
+ Preconditions.checkState(PaimonUtil.isPaimonTable(newTable));
+
+ acquireMetastoreDdlLock(catalogTimeline);
+ try {
+ // Add the table to the HMS and the catalog cache. Acquire metastoreDdlLock_ to
+ // ensure the atomicity of these operations.
+ List events;
+ try (MetaStoreClient msClient = catalog_.getMetaStoreClient(catalogTimeline)) {
+ boolean tableInMetastore = msClient.getHiveClient().tableExists(
+ newTable.getDbName(), newTable.getTableName());
+ catalogTimeline.markEvent(CHECKED_HMS_TABLE_EXISTENCE);
+ if (!tableInMetastore) {
+ long eventId = getCurrentEventId(msClient, catalogTimeline);
+ String location = newTable.getSd().getLocation();
+ // Create table in paimon if necessary
+ if (PaimonUtil.isSynchronizedTable(newTable)) {
+ // Set location here if not been specified in sql
+ if (location == null) {
+ location = PaimonUtil.getPaimonCatalogLocation(msClient, newTable);
+ }
+ if (debugAction != null) {
+ DebugUtils.executeDebugAction(debugAction, DebugUtils.PAIMON_CREATE);
+ }
+ String tableLoc = PaimonCatalogOpExecutor.createTable(
+ PaimonUtil.getTableIdentifier(newTable), location, params, newTable);
+ newTable.getSd().setLocation(tableLoc);
+ catalogTimeline.markEvent(PaimonCatalogOpExecutor.CREATED_PAIMON_TABLE +
+ newTable.getTableName());
+ } else {
+ // If this is not a synchronized table, we assume that the table must be
+ // existing in an Paimon Catalog.
+ TPaimonCatalog underlyingCatalog = PaimonUtil.getTPaimonCatalog(newTable);
+ if (underlyingCatalog != TPaimonCatalog.HADOOP_CATALOG &&
+ underlyingCatalog != TPaimonCatalog.HIVE_CATALOG) {
+ throw new TableLoadingException(
+ "Paimon table only support hadoop catalog and hive catalog.");
+ }
+ String locationToLoadFrom;
+ if (underlyingCatalog == TPaimonCatalog.HIVE_CATALOG) {
+ if (location == null) {
+ addSummary(response,
+ "Location is necessary for external paimon table with hive catalog.");
+ return false;
+ }
+ locationToLoadFrom = location;
+ } else {
+ // For HadoopCatalog tables 'locationToLoadFrom' is the location of the
+ // hadoop catalog. For HiveCatalog tables it remains null.
+ locationToLoadFrom =
+ PaimonUtil.getPaimonCatalogLocation(msClient, newTable);
+ }
+ try {
+ org.apache.paimon.table.Table paimonTable =
+ PaimonUtil.createFileStoreTable(locationToLoadFrom);
+ // Populate the HMS table schema based on the Paimon table's schema because
+ // the Paimon metadata is the source of truth. This also avoids an
+ // unnecessary ALTER TABLE.
+ PaimonCatalogOpExecutor.populateExternalTableSchemaFromPaimonTable(
+ newTable, paimonTable);
+ catalogTimeline.markEvent(PaimonCatalogOpExecutor.LOADED_PAIMON_TABLE);
+ if (location == null) {
+ // Using the location of the loaded Paimon table we can also get the
+ // correct location for tables stored in nested namespaces.
+ newTable.getSd().setLocation(
+ ((DataTable) paimonTable).location().toString());
+ }
+ } catch (Exception ex) {
+ // if failed to load paimon table
+ if (newTable.getSd().getCols().isEmpty()) {
+ // if user doesn't specify schema in table, we should load from underlying
+ // paimon table, but it fails. throw the exception.
+ throw new PaimonTableLoadingException(
+ "Failed to extract paimon schema from underlying paimon table", ex);
+ } else {
+ // user has specify schema in table ddl, try to create a new paimon table
+ // instead.
+ String tableLoc = PaimonCatalogOpExecutor.createTable(
+ PaimonUtil.getTableIdentifier(newTable), location, params, newTable);
+ newTable.getSd().setLocation(tableLoc);
+ }
+ }
+ }
+
+ msClient.getHiveClient().createTable(newTable);
+ catalogTimeline.markEvent(CREATED_HMS_TABLE);
+ events = getNextMetastoreEventsForTableIfEnabled(catalogTimeline, eventId,
+ newTable.getDbName(), newTable.getTableName(), CreateTableEvent.EVENT_TYPE);
+ } else {
+ addSummary(response, "Table already exists.");
+ return false;
+ }
+ }
+ Pair eventTblPair =
+ getTableFromEvents(events, params.if_not_exists);
+ long createEventId = eventTblPair == null ? -1 : eventTblPair.first;
+ org.apache.hadoop.hive.metastore.api.Table msTable =
+ eventTblPair == null ? null : eventTblPair.second;
+ setTableNameAndCreateTimeInResponse(msTable, newTable.getDbName(),
+ newTable.getTableName(), response, catalogTimeline);
+ // Add the table to the catalog cache
+ Table newTbl =
+ catalog_.addIncompleteTable(newTable.getDbName(), newTable.getTableName(),
+ TImpalaTableType.TABLE, params.getComment(), createEventId);
+ catalogTimeline.markEvent(CREATED_CATALOG_TABLE);
+ LOG.debug("Created an paimon table {} in catalog with create event Id {} ",
+ newTbl.getFullName(), createEventId);
+ addTableToCatalogUpdate(newTbl, wantMinimalResult, response.result);
+ } catch (Exception e) {
+ if (params.if_not_exists
+ && (e instanceof AlreadyExistsException
+ || e instanceof org.apache.iceberg.exceptions.AlreadyExistsException)) {
+ addSummary(response, "Table already exists.");
+ return false;
+ }
+ throw new ImpalaRuntimeException(
+ String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
+ } finally { getMetastoreDdlLock().unlock(); }
+
+ addSummary(response, "Table has been created.");
+ return true;
+ }
+
/**
* Creates a new table in the metastore based on the definition of an existing table.
* No data is copied as part of this process, it is a metadata only operation. If the
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index 56acc4f78..c7f5e198f 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -147,6 +147,8 @@ import org.apache.impala.catalog.TableLoadingException;
import org.apache.impala.catalog.Type;
import org.apache.impala.catalog.local.InconsistentMetadataFetchException;
import org.apache.impala.catalog.iceberg.IcebergMetadataTable;
+import org.apache.impala.catalog.paimon.FePaimonTable;
+import org.apache.impala.catalog.paimon.FeShowFileStmtSupport;
import org.apache.impala.common.AnalysisException;
import org.apache.impala.common.UserCancelledException;
import org.apache.impala.common.FileSystemUtil;
@@ -1755,6 +1757,8 @@ public class Frontend {
return ((FeHBaseTable) table).getTableStats();
} else if (table instanceof FeDataSourceTable) {
return ((FeDataSourceTable) table).getTableStats();
+ } else if (table instanceof FePaimonTable) {
+ return ((FePaimonTable) table).getTableStats(op);
} else if (table instanceof FeKuduTable) {
if (op == TShowStatsOp.RANGE_PARTITIONS) {
return FeKuduTable.Utils.getRangePartitions((FeKuduTable) table, false);
@@ -3456,7 +3460,9 @@ public class Frontend {
throws ImpalaException{
FeTable table = getCatalog().getTable(request.getTable_name().getDb_name(),
request.getTable_name().getTable_name());
- if (table instanceof FeFsTable) {
+ if (table instanceof FeShowFileStmtSupport) {
+ return ((FeShowFileStmtSupport) table).doGetTableFiles(request);
+ } else if (table instanceof FeFsTable) {
return FeFsTable.Utils.getFiles((FeFsTable)table, request.getPartition_set());
} else {
throw new InternalException("SHOW FILES only supports Hdfs table. " +
diff --git a/fe/src/main/java/org/apache/impala/util/DebugUtils.java b/fe/src/main/java/org/apache/impala/util/DebugUtils.java
index d5a24fc00..5754263d9 100644
--- a/fe/src/main/java/org/apache/impala/util/DebugUtils.java
+++ b/fe/src/main/java/org/apache/impala/util/DebugUtils.java
@@ -66,6 +66,9 @@ public class DebugUtils {
// debug action label for Iceberg create table.
public static final String ICEBERG_CREATE = "catalogd_iceberg_create";
+ // debug action label for Paimon create table.
+ public static final String PAIMON_CREATE = "catalogd_paimon_create";
+
// debug action label for throwing an exception during loadFileMetadataForPartitions.
public static final String LOAD_FILE_METADATA_THROW_EXCEPTION =
"catalogd_load_file_metadata_throw_exception";
diff --git a/fe/src/main/jflex/sql-scanner.flex b/fe/src/main/jflex/sql-scanner.flex
index be7c86811..a6ccc2892 100644
--- a/fe/src/main/jflex/sql-scanner.flex
+++ b/fe/src/main/jflex/sql-scanner.flex
@@ -158,6 +158,7 @@ import org.apache.impala.thrift.TReservedWordsVersion;
keywordMap.put("having", SqlParserSymbols.KW_HAVING);
keywordMap.put("hudiparquet", SqlParserSymbols.KW_HUDIPARQUET);
keywordMap.put("iceberg", SqlParserSymbols.KW_ICEBERG);
+ keywordMap.put("paimon", SqlParserSymbols.KW_PAIMON);
keywordMap.put("if", SqlParserSymbols.KW_IF);
keywordMap.put("ignore", SqlParserSymbols.KW_IGNORE);
keywordMap.put("ilike", SqlParserSymbols.KW_ILIKE);
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
index cd8ea9691..bd4cbf314 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -1084,7 +1084,7 @@ public class AnalyzeDDLTest extends FrontendTestBase {
"only supported on TEXT file format");
}
String[] unsupportedFileFormats = {
- "parquet", "rcfile", "avro", "iceberg"};
+ "parquet", "rcfile", "avro", "iceberg", "paimon"};
for (String format: unsupportedFileFormats) {
AnalysisError("create table " + tmpTableName + " (id int) with serdeproperties(" +
"'serialization.encoding'='GBK') stored as " + format,
@@ -2446,7 +2446,9 @@ public class AnalyzeDDLTest extends FrontendTestBase {
AnalysisError("create table foo stored as RCFILE as select 1",
"CREATE TABLE AS SELECT does not support the (RCFILE) file format. " +
"Supported formats are: (PARQUET, TEXTFILE, KUDU, ICEBERG)");
-
+ AnalysisError("create table foo stored as PAIMON as select 1",
+ "CREATE TABLE AS SELECT does not support the (PAIMON) file format. " +
+ "Supported formats are: (PARQUET, TEXTFILE, KUDU, ICEBERG)");
// CTAS with a WITH clause and inline view (IMPALA-1100)
AnalyzesOk("create table test_with as with with_1 as (select 1 as int_col from " +
"functional.alltypes as t1 right join (select 1 as int_col from " +
@@ -3050,7 +3052,55 @@ public class AnalyzeDDLTest extends FrontendTestBase {
AnalysisError("CREATE EXTERNAL TABLE Foo (i int) PARTITIONED BY (d decimal) " +
"STORED BY JDBC ",
"PARTITIONED BY cannot be used in a JDBC table.");
-
+ // Create Paimon tables.
+ AnalyzesOk("CREATE TABLE IF NOT EXISTS foo (user_id BIGINT, item_id BIGINT, " +
+ "behavior STRING) STORED AS PAIMON");
+ AnalyzesOk("CREATE TABLE IF NOT EXISTS foo (user_id BIGINT, item_id BIGINT, " +
+ "behavior STRING) PARTITIONED BY (dt STRING, hh STRING) STORED AS PAIMON");
+ AnalyzesOk("CREATE TABLE foo (user_id BIGINT, item_id BIGINT, behavior STRING) " +
+ "PARTITIONED BY (dt STRING, hh STRING) STORED AS PAIMON TBLPROPERTIES " +
+ "('primary-key'='user_id')");
+ AnalyzesOk("CREATE TABLE foo (user_id BIGINT, item_id BIGINT, behavior STRING, " +
+ "PRIMARY KEY(user_id)) PARTITIONED BY (dt STRING, hh STRING) STORED AS PAIMON");
+ AnalyzesOk("CREATE TABLE foo (user_id BIGINT, item_id BIGINT, behavior STRING)" +
+ " STORED AS PAIMON TBLPROPERTIES ('bucket' = '4', 'bucket-key'='behavior')");
+ AnalyzesOk("CREATE TABLE foo (user_id BIGINT, item_id BIGINT, behavior STRING, " +
+ "PRIMARY KEY(user_id)) PARTITIONED BY (dt STRING, hh STRING) STORED AS PAIMON " +
+ "LOCATION 'hdfs:///test-warehouse/test_create_managed_location_paimon_table'");
+ AnalyzesOk("CREATE TABLE foo (`USER_ID` BIGINT, `ITEM_ID` BIGINT, BEHAVIOR STRING, " +
+ "PRIMARY KEY(`USER_ID`)) PARTITIONED BY (`DT` STRING, `HH` STRING) " +
+ "STORED AS PAIMON");
+ AnalyzesOk("CREATE EXTERNAL TABLE foo STORED AS PAIMON LOCATION " +
+ "'hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/" +
+ "paimon_non_partitioned'");
+ AnalyzesOk("CREATE EXTERNAL TABLE foo STORED AS PAIMON LOCATION " +
+ "'hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/" +
+ "paimon_partitioned'");
+ AnalyzesOk("CREATE EXTERNAL TABLE foo STORED AS PAIMON TBLPROPERTIES" +
+ "('paimon.catalog'='hadoop', 'paimon.catalog_location'='hdfs:///test-warehouse" +
+ "/paimon_test/paimon_catalog/warehouse', 'paimon.table_identifier'=" +
+ "'functional.paimon_non_partitioned')");
+ AnalyzesOk("CREATE EXTERNAL TABLE foo STORED AS PAIMON TBLPROPERTIES(" +
+ "'paimon.catalog'='hadoop', 'paimon.catalog_location'='hdfs:///test-warehouse" +
+ "/paimon_test/paimon_catalog/warehouse', 'paimon.table_identifier'=" +
+ "'functional.paimon_partitioned')");
+ AnalyzesOk("CREATE TABLE foo (col_boolean BOOLEAN, col_tinyint TINYINT, " +
+ "col_smallint SMALLINT, col_int INT, col_integer INTEGER, col_bigint BIGINT, " +
+ "col_float FLOAT, col_double DOUBLE, col_decimal DECIMAL(10,2), " +
+ "col_string STRING, col_char CHAR(20), col_varchar VARCHAR(100), " +
+ "col_timestamp TIMESTAMP, col_date DATE, col_binary BINARY) STORED AS PAIMON");
+ AnalysisError("CREATE TABLE impala_paimon_complex_types_array " +
+ "(col_array_string ARRAY) STORED AS PAIMON;",
+ "Tables stored by Paimon do not support the " +
+ "column col_array_string type: ARRAY");
+ AnalysisError("CREATE TABLE impala_paimon_complex_types_map " +
+ "(col_map_string MAP) STORED AS PAIMON;",
+ "Tables stored by Paimon do not support the" +
+ " column col_map_string type: MAP");
+ AnalysisError("CREATE TABLE impala_paimon_complex_types_struct " +
+ "(col_struct_string STRUCT) STORED AS PAIMON;",
+ "Tables stored by Paimon do not support the " +
+ "column col_struct_string type: STRUCT");
// Create table PRODUCED BY DATA SOURCE
final String DATA_SOURCE_NAME = "TestDataSource1";
catalog_.addDataSource(new DataSource(DATA_SOURCE_NAME, "/foo.jar",
@@ -3131,7 +3181,7 @@ public class AnalyzeDDLTest extends FrontendTestBase {
AnalyzesOk("CREATE TABLE functional.bucket (i int COMMENT 'hello', s string) " +
"CLUSTERED BY(i) SORT BY (s) INTO 24 BUCKETS");
- // Bucketed table not supported for Kudu, ICEBERG and JDBC table
+ // Bucketed table not supported for Kudu, ICEBERG, PAIMON and JDBC table
AnalysisError("CREATE TABLE functional.bucket (i int COMMENT 'hello', s string) " +
"CLUSTERED BY (i) INTO 24 BUCKETS STORED BY KUDU", "CLUSTERED BY not " +
"support fileformat: 'KUDU'");
@@ -3141,6 +3191,10 @@ public class AnalyzeDDLTest extends FrontendTestBase {
AnalysisError("CREATE TABLE functional.bucket (i int COMMENT 'hello', s string) " +
"CLUSTERED BY (i) INTO 24 BUCKETS STORED BY JDBC",
"CLUSTERED BY not support fileformat: 'JDBC'");
+ AnalysisError("CREATE TABLE functional.bucket (i int COMMENT 'hello', s string) " +
+ "CLUSTERED BY (i) INTO 24 BUCKETS STORED BY PAIMON",
+ "CLUSTERED BY clause is not support by PAIMON now, use property " +
+ "bucket-key instead.");
// Bucketed columns must not contain partition column and don't duplicate
AnalysisError("CREATE TABLE functional.bucket (i int COMMENT 'hello', s string) " +
"PARTITIONED BY(dt string) CLUSTERED BY (dt) INTO 24 BUCKETS",
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
old mode 100755
new mode 100644
index caf288361..d3f310f27
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -2894,8 +2894,8 @@ public class ParserTest extends FrontendTestBase {
ParsesOk("CREATE TABLE T (i int COMMENT 'hi') PARTITIONED BY (j int COMMENT 'bye')");
// Supported file formats
- String [] supportedFileFormats =
- {"TEXTFILE", "SEQUENCEFILE", "PARQUET", "PARQUETFILE", "RCFILE", "AVRO"};
+ String[] supportedFileFormats = {
+ "TEXTFILE", "SEQUENCEFILE", "PARQUET", "PARQUETFILE", "RCFILE", "AVRO", "PAIMON"};
for (String format: supportedFileFormats) {
ParsesOk("CREATE TABLE Foo (i int, s string) STORED AS " + format);
ParsesOk("CREATE EXTERNAL TABLE Foo (i int, s string) STORED AS " + format);
diff --git a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
index b4c6ca400..9168c1f6e 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
@@ -382,6 +382,48 @@ public class ToSqlTest extends FrontendTestBase {
+ "test_pk_fk.fk ( seq INT, id INT, year STRING, a INT, FOREIGN KEY(id, year) "
+ "REFERENCES functional.parent_table(id, year), FOREIGN KEY(a) REFERENCES "
+ "functional.parent_table_2(a) ) STORED AS TEXTFILE", true);
+ // Test create table for paimon table.
+ testToSql("CREATE TABLE test_create_managed_paimon_table ("
+ + "user_id BIGINT, item_id BIGINT, behavior STRING) STORED AS PAIMON",
+ "default",
+ "CREATE TABLE default.test_create_managed_paimon_table ("
+ + " user_id BIGINT, item_id BIGINT, behavior STRING )"
+ + " STORED AS PAIMON TBLPROPERTIES ('deletion-vectors.enabled'='true', "
+ + "'storage_handler'='org.apache.paimon.hive.PaimonStorageHandler')",
+ true);
+
+ testToSql("CREATE TABLE test_create_managed_part_paimon_table ("
+ + " user_id BIGINT, item_id BIGINT, behavior STRING) PARTITIONED BY ("
+ + "dt STRING, hh STRING ) STORED AS PAIMON", "default",
+ "CREATE TABLE default.test_create_managed_part_paimon_table ("
+ + " user_id BIGINT, item_id BIGINT, behavior STRING ) "
+ + "PARTITIONED BY ( dt STRING, hh STRING ) STORED AS PAIMON "
+ + "TBLPROPERTIES ('deletion-vectors.enabled'='true', "
+ + "'storage_handler'='org.apache.paimon.hive.PaimonStorageHandler')",
+ true);
+
+ testToSql("CREATE TABLE test_create_managed_part_pk_paimon_table ("
+ + "user_id BIGINT, item_id BIGINT, behavior STRING) PARTITIONED BY ("
+ + "dt STRING, hh STRING) STORED AS PAIMON TBLPROPERTIES ("
+ + "'primary-key'='user_id')", "default",
+ "CREATE TABLE default.test_create_managed_part_pk_paimon_table ( "
+ + "user_id BIGINT, item_id BIGINT, behavior STRING ) "
+ + "PARTITIONED BY ( dt STRING, hh STRING ) "
+ + "STORED AS PAIMON TBLPROPERTIES ('deletion-vectors.enabled'='true',"
+ + " 'primary-key'='user_id', "
+ + "'storage_handler'='org.apache.paimon.hive.PaimonStorageHandler')",
+ true);
+
+ testToSql("CREATE TABLE test_create_managed_bucket_paimon_table ("
+ + "user_id BIGINT, item_id BIGINT, behavior STRING) STORED AS PAIMON "
+ + "TBLPROPERTIES ('bucket' = '4', 'bucket-key'='behavior')", "default",
+ "CREATE TABLE default.test_create_managed_bucket_paimon_table ("
+ + " user_id BIGINT, item_id BIGINT, behavior STRING ) "
+ + "STORED AS PAIMON TBLPROPERTIES "
+ + "('bucket'='4', 'bucket-key'='behavior', "
+ + "'deletion-vectors.enabled'='true', "
+ + "'storage_handler'='org.apache.paimon.hive.PaimonStorageHandler')",
+ true);
}
@Test
diff --git a/fe/src/test/java/org/apache/impala/catalog/paimon/ImpalaTypeUtilsTest.java b/fe/src/test/java/org/apache/impala/catalog/paimon/ImpalaTypeUtilsTest.java
new file mode 100644
index 000000000..191e5b44e
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/catalog/paimon/ImpalaTypeUtilsTest.java
@@ -0,0 +1,161 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License. You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied. See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.impala.catalog.paimon;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.impala.catalog.ScalarType;
+import org.apache.impala.catalog.StructField;
+import org.apache.impala.catalog.StructType;
+import org.apache.impala.catalog.Type;
+import org.apache.paimon.types.ArrayType;
+import org.apache.paimon.types.BigIntType;
+import org.apache.paimon.types.BooleanType;
+import org.apache.paimon.types.CharType;
+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.FloatType;
+import org.apache.paimon.types.IntType;
+import org.apache.paimon.types.MapType;
+import org.apache.paimon.types.RowType;
+import org.apache.paimon.types.SmallIntType;
+import org.apache.paimon.types.TimestampType;
+import org.apache.paimon.types.TinyIntType;
+import org.apache.paimon.types.VarCharType;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+public class ImpalaTypeUtilsTest {
+ private interface ConvFunc { public void convertFun(); }
+
+ private static void assertThrowUnexpected(ConvFunc convFunc) {
+ try {
+ convFunc.convertFun();
+ } catch (UnsupportedOperationException ex) {
+ // do nothing
+ }
+ }
+
+ @Test
+ public void testToImpalaType() {
+ // Test primitive types
+ assertEquals(Type.BOOLEAN, ImpalaTypeUtils.toImpalaType(new BooleanType()));
+ assertEquals(Type.TINYINT, ImpalaTypeUtils.toImpalaType(new TinyIntType()));
+ assertEquals(Type.SMALLINT, ImpalaTypeUtils.toImpalaType(new SmallIntType()));
+ assertEquals(Type.INT, ImpalaTypeUtils.toImpalaType(new IntType()));
+ assertEquals(Type.BIGINT, ImpalaTypeUtils.toImpalaType(new BigIntType()));
+ assertEquals(Type.FLOAT, ImpalaTypeUtils.toImpalaType(new FloatType()));
+ assertEquals(Type.DOUBLE, ImpalaTypeUtils.toImpalaType(new DoubleType()));
+ assertEquals(Type.DATE, ImpalaTypeUtils.toImpalaType(DataTypes.DATE()));
+ assertEquals(Type.TIMESTAMP, ImpalaTypeUtils.toImpalaType(new TimestampType()));
+
+ // Test decimal type
+ DecimalType decimalType = new DecimalType(10, 2);
+ assertEquals(
+ ScalarType.createDecimalType(10, 2), ImpalaTypeUtils.toImpalaType(decimalType));
+
+ // Test char and varchar types
+ assertEquals(
+ ScalarType.createCharType(10), ImpalaTypeUtils.toImpalaType(new CharType(10)));
+ assertEquals(ScalarType.createVarcharType(10),
+ ImpalaTypeUtils.toImpalaType(new VarCharType(10)));
+ assertEquals(Type.STRING, ImpalaTypeUtils.toImpalaType(new CharType(256)));
+ assertEquals(Type.STRING, ImpalaTypeUtils.toImpalaType(new VarCharType(65536)));
+ // Test array type
+ ArrayType arrayType = new ArrayType(new IntType());
+ assertEquals(new org.apache.impala.catalog.ArrayType(Type.INT),
+ ImpalaTypeUtils.toImpalaType(arrayType));
+
+ // Test map type
+ MapType mapType = new MapType(new IntType(), DataTypes.STRING());
+ assertEquals(new org.apache.impala.catalog.MapType(Type.INT, Type.STRING),
+ ImpalaTypeUtils.toImpalaType(mapType));
+
+ // Test row type
+ RowType rowType = new RowType(Arrays.asList(new DataField(0, "id", new IntType()),
+ new DataField(1, "name", DataTypes.STRING())));
+ StructType expectedStructType = new StructType(Arrays.asList(
+ new StructField("id", Type.INT), new StructField("name", Type.STRING)));
+ assertEquals(expectedStructType, ImpalaTypeUtils.toImpalaType(rowType));
+
+ // doesn't support time
+ assertThrowUnexpected(() -> ImpalaTypeUtils.toImpalaType(DataTypes.TIME()));
+ // doesn't support multiset
+ assertThrowUnexpected(
+ () -> ImpalaTypeUtils.toImpalaType(DataTypes.MULTISET(DataTypes.INT())));
+ // doesn't support timestamp with local timezone
+ assertThrowUnexpected(
+ () -> ImpalaTypeUtils.toImpalaType(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()));
+ }
+
+ @Test
+ public void testFromImpalaType() {
+ // Test primitive types
+ assertEquals(DataTypes.BOOLEAN(), ImpalaTypeUtils.fromImpalaType(Type.BOOLEAN));
+ assertEquals(DataTypes.TINYINT(), ImpalaTypeUtils.fromImpalaType(Type.TINYINT));
+ assertEquals(DataTypes.SMALLINT(), ImpalaTypeUtils.fromImpalaType(Type.SMALLINT));
+ assertEquals(DataTypes.INT(), ImpalaTypeUtils.fromImpalaType(Type.INT));
+ assertEquals(DataTypes.BIGINT(), ImpalaTypeUtils.fromImpalaType(Type.BIGINT));
+ assertEquals(DataTypes.FLOAT(), ImpalaTypeUtils.fromImpalaType(Type.FLOAT));
+ assertEquals(DataTypes.DOUBLE(), ImpalaTypeUtils.fromImpalaType(Type.DOUBLE));
+ assertEquals(DataTypes.DATE(), ImpalaTypeUtils.fromImpalaType(Type.DATE));
+ assertEquals(DataTypes.TIMESTAMP(), ImpalaTypeUtils.fromImpalaType(Type.TIMESTAMP));
+
+ // Test decimal type
+ ScalarType decimalType = ScalarType.createDecimalType(10, 2);
+ assertEquals(DataTypes.DECIMAL(10, 2), ImpalaTypeUtils.fromImpalaType(decimalType));
+
+ // Test char and varchar types
+ assertEquals(DataTypes.CHAR(10),
+ ImpalaTypeUtils.fromImpalaType(ScalarType.createCharType(10)));
+ assertEquals(DataTypes.VARCHAR(255),
+ ImpalaTypeUtils.fromImpalaType(ScalarType.createVarcharType(255)));
+ // Test array type
+ org.apache.impala.catalog.ArrayType arrayType =
+ new org.apache.impala.catalog.ArrayType(Type.INT);
+ assertEquals(
+ DataTypes.ARRAY(DataTypes.INT()), ImpalaTypeUtils.fromImpalaType(arrayType));
+
+ // Test map type
+ org.apache.impala.catalog.MapType mapType =
+ new org.apache.impala.catalog.MapType(Type.INT, Type.STRING);
+ assertEquals(DataTypes.MAP(DataTypes.INT(), DataTypes.STRING()),
+ ImpalaTypeUtils.fromImpalaType(mapType));
+
+ // Test struct type
+ StructType structType = new StructType(Arrays.asList(
+ new StructField("id", Type.INT), new StructField("name", Type.STRING)));
+ RowType expectedRowType = RowType.of(new DataField(0, "id", DataTypes.INT()),
+ new DataField(1, "name", DataTypes.STRING()));
+ assertEquals(expectedRowType, ImpalaTypeUtils.fromImpalaType(structType));
+
+ // doesn't support datetime
+ assertThrowUnexpected(() -> ImpalaTypeUtils.fromImpalaType(Type.DATETIME));
+ // doesn't support NULL TYPE
+ assertThrowUnexpected(() -> ImpalaTypeUtils.fromImpalaType(Type.NULL));
+ // doesn't support INVALID
+ assertThrowUnexpected(() -> ImpalaTypeUtils.fromImpalaType(Type.INVALID));
+ // doesn't support FIXED_UDA_INTERMEDIATE
+ assertThrowUnexpected(
+ () -> ImpalaTypeUtils.fromImpalaType(Type.FIXED_UDA_INTERMEDIATE));
+ }
+}
\ No newline at end of file
diff --git a/java/pom.xml b/java/pom.xml
index e4c61b837..b828ff478 100644
--- a/java/pom.xml
+++ b/java/pom.xml
@@ -41,6 +41,7 @@ under the License.
${env.IMPALA_HIVE_MAJOR_VERSION}
${env.IMPALA_HIVE_DIST_TYPE}
${env.IMPALA_HUDI_VERSION}
+ ${env.IMPALA_PAIMON_VERSION}
${env.IMPALA_RANGER_VERSION}
${env.IMPALA_POSTGRES_JDBC_DRIVER_VERSION}
${env.IMPALA_HBASE_VERSION}
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-0ca8873e-4b8c-4a80-ac31-3cd5bd95ae01-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-0ca8873e-4b8c-4a80-ac31-3cd5bd95ae01-0.parquet
new file mode 100644
index 000000000..e838cab14
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-0ca8873e-4b8c-4a80-ac31-3cd5bd95ae01-0.parquet differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-488765b8-d0d7-4091-ad1a-2753f9fb0b63-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-488765b8-d0d7-4091-ad1a-2753f9fb0b63-0.parquet
new file mode 100644
index 000000000..3b30d3caf
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-488765b8-d0d7-4091-ad1a-2753f9fb0b63-0.parquet differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-5ab52dc4-cd44-4de4-9b81-fcd30baeee0d-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-5ab52dc4-cd44-4de4-9b81-fcd30baeee0d-0.parquet
new file mode 100644
index 000000000..8e7d6ac88
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-5ab52dc4-cd44-4de4-9b81-fcd30baeee0d-0.parquet differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-6684e5ef-26fa-4f13-a1bd-9c2a5bc9557c-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-6684e5ef-26fa-4f13-a1bd-9c2a5bc9557c-0.parquet
new file mode 100644
index 000000000..8f18dca1c
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-6684e5ef-26fa-4f13-a1bd-9c2a5bc9557c-0.parquet differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-ad75a7e8-4ae0-4803-8f49-92d58fd4a222-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-ad75a7e8-4ae0-4803-8f49-92d58fd4a222-0.parquet
new file mode 100644
index 000000000..8d25bedb0
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/bucket-0/data-ad75a7e8-4ae0-4803-8f49-92d58fd4a222-0.parquet differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/index-manifest-1eecc41b-3dcd-4388-afdb-54527b51be2a-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/index-manifest-1eecc41b-3dcd-4388-afdb-54527b51be2a-0
new file mode 100644
index 000000000..13e47d741
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/index-manifest-1eecc41b-3dcd-4388-afdb-54527b51be2a-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-3505f2c0-e77f-42ff-9a54-368038f7eb3e-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-3505f2c0-e77f-42ff-9a54-368038f7eb3e-0
new file mode 100644
index 000000000..9dba1fc58
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-3505f2c0-e77f-42ff-9a54-368038f7eb3e-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-671ffa6b-2665-4d30-b039-d21a5565d698-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-671ffa6b-2665-4d30-b039-d21a5565d698-0
new file mode 100644
index 000000000..4592c0a8e
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-671ffa6b-2665-4d30-b039-d21a5565d698-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-9e6bcbde-1da9-4b06-a17c-c71579de18a5-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-9e6bcbde-1da9-4b06-a17c-c71579de18a5-0
new file mode 100644
index 000000000..c16cd4629
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-9e6bcbde-1da9-4b06-a17c-c71579de18a5-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-b4b02924-241f-4abf-8ad8-d7a2ac5536e2-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-b4b02924-241f-4abf-8ad8-d7a2ac5536e2-0
new file mode 100644
index 000000000..759e14f1a
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-b4b02924-241f-4abf-8ad8-d7a2ac5536e2-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-c32aa2fd-3d98-4f09-9067-8a269d8115eb-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-c32aa2fd-3d98-4f09-9067-8a269d8115eb-0
new file mode 100644
index 000000000..0f8afb8e0
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-c32aa2fd-3d98-4f09-9067-8a269d8115eb-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-025dfcce-8e34-42c0-a8f7-def795d489df-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-025dfcce-8e34-42c0-a8f7-def795d489df-0
new file mode 100644
index 000000000..d4c6e0ddf
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-025dfcce-8e34-42c0-a8f7-def795d489df-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-025dfcce-8e34-42c0-a8f7-def795d489df-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-025dfcce-8e34-42c0-a8f7-def795d489df-1
new file mode 100644
index 000000000..e80b0ce5a
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-025dfcce-8e34-42c0-a8f7-def795d489df-1 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-5cd761f7-4c95-4181-841e-aa8b405e71be-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-5cd761f7-4c95-4181-841e-aa8b405e71be-0
new file mode 100644
index 000000000..c0aa15467
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-5cd761f7-4c95-4181-841e-aa8b405e71be-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-5cd761f7-4c95-4181-841e-aa8b405e71be-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-5cd761f7-4c95-4181-841e-aa8b405e71be-1
new file mode 100644
index 000000000..243fbd3e0
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-5cd761f7-4c95-4181-841e-aa8b405e71be-1 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-6e11688a-4b7c-4bfd-9118-5e49eac61016-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-6e11688a-4b7c-4bfd-9118-5e49eac61016-0
new file mode 100644
index 000000000..a10ff4890
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-6e11688a-4b7c-4bfd-9118-5e49eac61016-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-6e11688a-4b7c-4bfd-9118-5e49eac61016-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-6e11688a-4b7c-4bfd-9118-5e49eac61016-1
new file mode 100644
index 000000000..5142264ce
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-6e11688a-4b7c-4bfd-9118-5e49eac61016-1 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-d86f1254-05df-4560-8885-693e94b46cbf-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-d86f1254-05df-4560-8885-693e94b46cbf-0
new file mode 100644
index 000000000..736663d4e
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-d86f1254-05df-4560-8885-693e94b46cbf-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-d86f1254-05df-4560-8885-693e94b46cbf-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-d86f1254-05df-4560-8885-693e94b46cbf-1
new file mode 100644
index 000000000..8091f1446
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/manifest/manifest-list-d86f1254-05df-4560-8885-693e94b46cbf-1 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/schema/schema-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/schema/schema-0
new file mode 100644
index 000000000..2d10f9a07
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/schema/schema-0
@@ -0,0 +1,28 @@
+{
+ "version" : 3,
+ "id" : 0,
+ "fields" : [ {
+ "id" : 0,
+ "name" : "userId",
+ "type" : "INT"
+ }, {
+ "id" : 1,
+ "name" : "movieId",
+ "type" : "INT"
+ }, {
+ "id" : 2,
+ "name" : "rating",
+ "type" : "FLOAT"
+ }, {
+ "id" : 3,
+ "name" : "timestamp",
+ "type" : "BIGINT"
+ } ],
+ "highestFieldId" : 3,
+ "partitionKeys" : [ ],
+ "primaryKeys" : [ ],
+ "options" : {
+ "owner" : "jichen"
+ },
+ "timeMillis" : 1737733553684
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/schema/schema-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/schema/schema-1
new file mode 100644
index 000000000..284560b92
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/schema/schema-1
@@ -0,0 +1,29 @@
+{
+ "version" : 3,
+ "id" : 1,
+ "fields" : [ {
+ "id" : 0,
+ "name" : "userId",
+ "type" : "INT"
+ }, {
+ "id" : 1,
+ "name" : "movieId",
+ "type" : "INT"
+ }, {
+ "id" : 2,
+ "name" : "rating",
+ "type" : "FLOAT"
+ }, {
+ "id" : 3,
+ "name" : "timestamp",
+ "type" : "BIGINT"
+ } ],
+ "highestFieldId" : 3,
+ "partitionKeys" : [ ],
+ "primaryKeys" : [ ],
+ "options" : {
+ "owner" : "jichen",
+ "deletion-vectors.enabled" : "true"
+ },
+ "timeMillis" : 1737734737038
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/EARLIEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/EARLIEST
new file mode 100644
index 000000000..f11c82a4c
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/EARLIEST
@@ -0,0 +1 @@
+9
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/LATEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/LATEST
new file mode 100644
index 000000000..3cacc0b93
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/LATEST
@@ -0,0 +1 @@
+12
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-10 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-10
new file mode 100644
index 000000000..982be8f5e
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-10
@@ -0,0 +1,19 @@
+{
+ "version" : 3,
+ "id" : 10,
+ "schemaId" : 1,
+ "baseManifestList" : "manifest-list-5cd761f7-4c95-4181-841e-aa8b405e71be-0",
+ "baseManifestListSize" : 884,
+ "deltaManifestList" : "manifest-list-5cd761f7-4c95-4181-841e-aa8b405e71be-1",
+ "deltaManifestListSize" : 884,
+ "changelogManifestList" : null,
+ "indexManifest" : "index-manifest-1eecc41b-3dcd-4388-afdb-54527b51be2a-0",
+ "commitUser" : "ea690e49-679c-46bb-8307-a6016b7ce27c",
+ "commitIdentifier" : 9223372036854775807,
+ "commitKind" : "COMPACT",
+ "timeMillis" : 1748965436933,
+ "logOffsets" : { },
+ "totalRecordCount" : 0,
+ "deltaRecordCount" : 0,
+ "changelogRecordCount" : 0
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-11 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-11
new file mode 100644
index 000000000..0893f4962
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-11
@@ -0,0 +1,19 @@
+{
+ "version" : 3,
+ "id" : 11,
+ "schemaId" : 1,
+ "baseManifestList" : "manifest-list-6e11688a-4b7c-4bfd-9118-5e49eac61016-0",
+ "baseManifestListSize" : 884,
+ "deltaManifestList" : "manifest-list-6e11688a-4b7c-4bfd-9118-5e49eac61016-1",
+ "deltaManifestListSize" : 989,
+ "changelogManifestList" : null,
+ "indexManifest" : "index-manifest-1eecc41b-3dcd-4388-afdb-54527b51be2a-0",
+ "commitUser" : "b2d1075a-036e-4b86-b864-b13b797fe6bc",
+ "commitIdentifier" : 9223372036854775807,
+ "commitKind" : "OVERWRITE",
+ "timeMillis" : 1748965464523,
+ "logOffsets" : { },
+ "totalRecordCount" : 452,
+ "deltaRecordCount" : 452,
+ "changelogRecordCount" : 0
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-12 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-12
new file mode 100644
index 000000000..ea5807485
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-12
@@ -0,0 +1,20 @@
+{
+ "version" : 3,
+ "id" : 12,
+ "schemaId" : 1,
+ "baseManifestList" : "manifest-list-d86f1254-05df-4560-8885-693e94b46cbf-0",
+ "baseManifestListSize" : 989,
+ "deltaManifestList" : "manifest-list-d86f1254-05df-4560-8885-693e94b46cbf-1",
+ "deltaManifestListSize" : 884,
+ "changelogManifestList" : null,
+ "indexManifest" : "index-manifest-1eecc41b-3dcd-4388-afdb-54527b51be2a-0",
+ "commitUser" : "2e57810a-5006-4dec-8b0f-40166fbec40f",
+ "commitIdentifier" : 9223372036854775807,
+ "commitKind" : "ANALYZE",
+ "timeMillis" : 1748965940065,
+ "logOffsets" : { },
+ "totalRecordCount" : 452,
+ "deltaRecordCount" : 0,
+ "changelogRecordCount" : 0,
+ "statistics" : "stat-eaea66dc-9657-4d89-823a-98315aa27329-0"
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-9 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-9
new file mode 100644
index 000000000..aec5dc1c4
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/snapshot/snapshot-9
@@ -0,0 +1,19 @@
+{
+ "version" : 3,
+ "id" : 9,
+ "schemaId" : 1,
+ "baseManifestList" : "manifest-list-025dfcce-8e34-42c0-a8f7-def795d489df-0",
+ "baseManifestListSize" : 1050,
+ "deltaManifestList" : "manifest-list-025dfcce-8e34-42c0-a8f7-def795d489df-1",
+ "deltaManifestListSize" : 989,
+ "changelogManifestList" : null,
+ "indexManifest" : "index-manifest-1eecc41b-3dcd-4388-afdb-54527b51be2a-0",
+ "commitUser" : "e2925e8d-4a2e-47ef-94a0-06a72d8886cb",
+ "commitIdentifier" : 9223372036854775807,
+ "commitKind" : "OVERWRITE",
+ "timeMillis" : 1748965369869,
+ "logOffsets" : { },
+ "totalRecordCount" : 0,
+ "deltaRecordCount" : -452,
+ "changelogRecordCount" : 0
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/statistics/stat-eaea66dc-9657-4d89-823a-98315aa27329-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/statistics/stat-eaea66dc-9657-4d89-823a-98315aa27329-0
new file mode 100644
index 000000000..fedde6f9b
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned/statistics/stat-eaea66dc-9657-4d89-823a-98315aa27329-0
@@ -0,0 +1,44 @@
+{
+ "snapshotId" : 11,
+ "schemaId" : 1,
+ "mergedRecordCount" : 452,
+ "mergedRecordSize" : 7352,
+ "colStats" : {
+ "rating" : {
+ "colId" : 2,
+ "distinctCount" : 5,
+ "min" : "1.0",
+ "max" : "5.0",
+ "nullCount" : 0,
+ "avgLen" : 4,
+ "maxLen" : 4
+ },
+ "movieId" : {
+ "colId" : 1,
+ "distinctCount" : 386,
+ "min" : "1",
+ "max" : "3893",
+ "nullCount" : 0,
+ "avgLen" : 4,
+ "maxLen" : 4
+ },
+ "userId" : {
+ "colId" : 0,
+ "distinctCount" : 5,
+ "min" : "1",
+ "max" : "5",
+ "nullCount" : 0,
+ "avgLen" : 4,
+ "maxLen" : 4
+ },
+ "timestamp" : {
+ "colId" : 3,
+ "distinctCount" : 228,
+ "min" : "978241072",
+ "max" : "978824351",
+ "nullCount" : 0,
+ "avgLen" : 8,
+ "maxLen" : 8
+ }
+ }
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-1db1d7b8-9b21-4241-a0c6-8189c368a0bc-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-1db1d7b8-9b21-4241-a0c6-8189c368a0bc-0
new file mode 100644
index 000000000..53a90d279
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-1db1d7b8-9b21-4241-a0c6-8189c368a0bc-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-5a2ea60f-e1b6-4126-aee4-088ff0b82f28-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-5a2ea60f-e1b6-4126-aee4-088ff0b82f28-0
new file mode 100644
index 000000000..f1d388d43
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-5a2ea60f-e1b6-4126-aee4-088ff0b82f28-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-9216adfd-451c-486d-a6db-b66884314880-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-9216adfd-451c-486d-a6db-b66884314880-0
new file mode 100644
index 000000000..b53c66281
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-9216adfd-451c-486d-a6db-b66884314880-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-22716ac0-0c0e-4f03-a48c-31c807cbf8d9-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-22716ac0-0c0e-4f03-a48c-31c807cbf8d9-0
new file mode 100644
index 000000000..f36ee5ad3
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-22716ac0-0c0e-4f03-a48c-31c807cbf8d9-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-22716ac0-0c0e-4f03-a48c-31c807cbf8d9-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-22716ac0-0c0e-4f03-a48c-31c807cbf8d9-1
new file mode 100644
index 000000000..057344ac6
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-22716ac0-0c0e-4f03-a48c-31c807cbf8d9-1 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-fececc90-17e5-4cde-bc69-84a98da91b79-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-fececc90-17e5-4cde-bc69-84a98da91b79-0
new file mode 100644
index 000000000..ca33e17ae
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-fececc90-17e5-4cde-bc69-84a98da91b79-0 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-fececc90-17e5-4cde-bc69-84a98da91b79-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-fececc90-17e5-4cde-bc69-84a98da91b79-1
new file mode 100644
index 000000000..3f0e5731c
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/manifest/manifest-list-fececc90-17e5-4cde-bc69-84a98da91b79-1 differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-0
new file mode 100644
index 000000000..e9ec9bff1
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-0
@@ -0,0 +1,28 @@
+{
+ "version" : 3,
+ "id" : 0,
+ "fields" : [ {
+ "id" : 0,
+ "name" : "userId",
+ "type" : "INT"
+ }, {
+ "id" : 1,
+ "name" : "movieId",
+ "type" : "INT"
+ }, {
+ "id" : 2,
+ "name" : "rating",
+ "type" : "FLOAT"
+ }, {
+ "id" : 3,
+ "name" : "ts",
+ "type" : "BIGINT"
+ } ],
+ "highestFieldId" : 3,
+ "partitionKeys" : [ "userId" ],
+ "primaryKeys" : [ ],
+ "options" : {
+ "owner" : "jichen"
+ },
+ "timeMillis" : 1741358917111
+}
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-1 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-1
new file mode 100644
index 000000000..3ba691fda
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-1
@@ -0,0 +1,29 @@
+{
+ "version" : 3,
+ "id" : 1,
+ "fields" : [ {
+ "id" : 0,
+ "name" : "userId",
+ "type" : "INT"
+ }, {
+ "id" : 1,
+ "name" : "movieId",
+ "type" : "INT"
+ }, {
+ "id" : 2,
+ "name" : "rating",
+ "type" : "FLOAT"
+ }, {
+ "id" : 3,
+ "name" : "ts",
+ "type" : "BIGINT"
+ } ],
+ "highestFieldId" : 3,
+ "partitionKeys" : [ "userId" ],
+ "primaryKeys" : [ ],
+ "options" : {
+ "owner" : "jichen",
+ "snapshot.num-retained" : "1"
+ },
+ "timeMillis" : 1747666219242
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-2 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-2
new file mode 100644
index 000000000..24d910452
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-2
@@ -0,0 +1,31 @@
+{
+ "version" : 3,
+ "id" : 2,
+ "fields" : [ {
+ "id" : 0,
+ "name" : "userId",
+ "type" : "INT"
+ }, {
+ "id" : 1,
+ "name" : "movieId",
+ "type" : "INT"
+ }, {
+ "id" : 2,
+ "name" : "rating",
+ "type" : "FLOAT"
+ }, {
+ "id" : 3,
+ "name" : "ts",
+ "type" : "BIGINT"
+ } ],
+ "highestFieldId" : 3,
+ "partitionKeys" : [ "userId" ],
+ "primaryKeys" : [ ],
+ "options" : {
+ "owner" : "jichen",
+ "snapshot.num-retained.max" : "1",
+ "snapshot.num-retained" : "1",
+ "snapshot.num-retained.min" : "1"
+ },
+ "timeMillis" : 1747666351718
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-3 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-3
new file mode 100644
index 000000000..831c780fb
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-3
@@ -0,0 +1,30 @@
+{
+ "version" : 3,
+ "id" : 3,
+ "fields" : [ {
+ "id" : 0,
+ "name" : "userId",
+ "type" : "INT"
+ }, {
+ "id" : 1,
+ "name" : "movieId",
+ "type" : "INT"
+ }, {
+ "id" : 2,
+ "name" : "rating",
+ "type" : "FLOAT"
+ }, {
+ "id" : 3,
+ "name" : "ts",
+ "type" : "BIGINT"
+ } ],
+ "highestFieldId" : 3,
+ "partitionKeys" : [ "userId" ],
+ "primaryKeys" : [ ],
+ "options" : {
+ "owner" : "jichen",
+ "snapshot.num-retained.max" : "1",
+ "snapshot.num-retained.min" : "1"
+ },
+ "timeMillis" : 1747717105767
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-4 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-4
new file mode 100644
index 000000000..06b166577
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-4
@@ -0,0 +1,29 @@
+{
+ "version" : 3,
+ "id" : 4,
+ "fields" : [ {
+ "id" : 0,
+ "name" : "userId",
+ "type" : "INT"
+ }, {
+ "id" : 1,
+ "name" : "movieId",
+ "type" : "INT"
+ }, {
+ "id" : 2,
+ "name" : "rating",
+ "type" : "FLOAT"
+ }, {
+ "id" : 3,
+ "name" : "ts",
+ "type" : "BIGINT"
+ } ],
+ "highestFieldId" : 3,
+ "partitionKeys" : [ "userId" ],
+ "primaryKeys" : [ ],
+ "options" : {
+ "owner" : "jichen",
+ "snapshot.num-retained.min" : "1"
+ },
+ "timeMillis" : 1747717647085
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-5 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-5
new file mode 100644
index 000000000..d08eafbc8
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/schema/schema-5
@@ -0,0 +1,28 @@
+{
+ "version" : 3,
+ "id" : 5,
+ "fields" : [ {
+ "id" : 0,
+ "name" : "userId",
+ "type" : "INT"
+ }, {
+ "id" : 1,
+ "name" : "movieId",
+ "type" : "INT"
+ }, {
+ "id" : 2,
+ "name" : "rating",
+ "type" : "FLOAT"
+ }, {
+ "id" : 3,
+ "name" : "ts",
+ "type" : "BIGINT"
+ } ],
+ "highestFieldId" : 3,
+ "partitionKeys" : [ "userId" ],
+ "primaryKeys" : [ ],
+ "options" : {
+ "owner" : "jichen"
+ },
+ "timeMillis" : 1747717651365
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/EARLIEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/EARLIEST
new file mode 100644
index 000000000..7813681f5
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/EARLIEST
@@ -0,0 +1 @@
+5
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/LATEST b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/LATEST
new file mode 100644
index 000000000..62f945751
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/LATEST
@@ -0,0 +1 @@
+6
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/snapshot-5 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/snapshot-5
new file mode 100644
index 000000000..90a7b627f
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/snapshot-5
@@ -0,0 +1,18 @@
+{
+ "version" : 3,
+ "id" : 5,
+ "schemaId" : 5,
+ "baseManifestList" : "manifest-list-fececc90-17e5-4cde-bc69-84a98da91b79-0",
+ "baseManifestListSize" : 1028,
+ "deltaManifestList" : "manifest-list-fececc90-17e5-4cde-bc69-84a98da91b79-1",
+ "deltaManifestListSize" : 992,
+ "changelogManifestList" : null,
+ "commitUser" : "0bd4c93e-9d0c-466c-9cd6-68cec360df29",
+ "commitIdentifier" : 9223372036854775807,
+ "commitKind" : "OVERWRITE",
+ "timeMillis" : 1748963555332,
+ "logOffsets" : { },
+ "totalRecordCount" : 452,
+ "deltaRecordCount" : 452,
+ "changelogRecordCount" : 0
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/snapshot-6 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/snapshot-6
new file mode 100644
index 000000000..4d31c24a1
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/snapshot/snapshot-6
@@ -0,0 +1,19 @@
+{
+ "version" : 3,
+ "id" : 6,
+ "schemaId" : 5,
+ "baseManifestList" : "manifest-list-22716ac0-0c0e-4f03-a48c-31c807cbf8d9-0",
+ "baseManifestListSize" : 1064,
+ "deltaManifestList" : "manifest-list-22716ac0-0c0e-4f03-a48c-31c807cbf8d9-1",
+ "deltaManifestListSize" : 884,
+ "changelogManifestList" : null,
+ "commitUser" : "d9cb06ed-8061-4a5b-af9c-daffa4e2337e",
+ "commitIdentifier" : 9223372036854775807,
+ "commitKind" : "ANALYZE",
+ "timeMillis" : 1748965930673,
+ "logOffsets" : { },
+ "totalRecordCount" : 452,
+ "deltaRecordCount" : 0,
+ "changelogRecordCount" : 0,
+ "statistics" : "stat-7061d17d-5a4d-4532-b98f-714d66103721-0"
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/statistics/stat-7061d17d-5a4d-4532-b98f-714d66103721-0 b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/statistics/stat-7061d17d-5a4d-4532-b98f-714d66103721-0
new file mode 100644
index 000000000..574766e56
--- /dev/null
+++ b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/statistics/stat-7061d17d-5a4d-4532-b98f-714d66103721-0
@@ -0,0 +1,44 @@
+{
+ "snapshotId" : 5,
+ "schemaId" : 5,
+ "mergedRecordCount" : 452,
+ "mergedRecordSize" : 7282,
+ "colStats" : {
+ "rating" : {
+ "colId" : 2,
+ "distinctCount" : 5,
+ "min" : "1.0",
+ "max" : "5.0",
+ "nullCount" : 0,
+ "avgLen" : 4,
+ "maxLen" : 4
+ },
+ "movieId" : {
+ "colId" : 1,
+ "distinctCount" : 386,
+ "min" : "1",
+ "max" : "3893",
+ "nullCount" : 0,
+ "avgLen" : 4,
+ "maxLen" : 4
+ },
+ "userId" : {
+ "colId" : 0,
+ "distinctCount" : 5,
+ "min" : "1",
+ "max" : "5",
+ "nullCount" : 0,
+ "avgLen" : 4,
+ "maxLen" : 4
+ },
+ "ts" : {
+ "colId" : 3,
+ "distinctCount" : 228,
+ "min" : "978241072",
+ "max" : "978824351",
+ "nullCount" : 0,
+ "avgLen" : 8,
+ "maxLen" : 8
+ }
+ }
+}
\ No newline at end of file
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=1/bucket-0/data-d7e0b3b7-a5e7-407f-b591-033cdf2da158-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=1/bucket-0/data-d7e0b3b7-a5e7-407f-b591-033cdf2da158-0.parquet
new file mode 100644
index 000000000..ec0086c8c
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=1/bucket-0/data-d7e0b3b7-a5e7-407f-b591-033cdf2da158-0.parquet differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=2/bucket-0/data-010c93fa-86ab-4389-aa03-4d76b7ece890-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=2/bucket-0/data-010c93fa-86ab-4389-aa03-4d76b7ece890-0.parquet
new file mode 100644
index 000000000..a86742722
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=2/bucket-0/data-010c93fa-86ab-4389-aa03-4d76b7ece890-0.parquet differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=3/bucket-0/data-4decfe67-c7e1-43f9-96ee-ae5188779a39-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=3/bucket-0/data-4decfe67-c7e1-43f9-96ee-ae5188779a39-0.parquet
new file mode 100644
index 000000000..9bb5aa3b4
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=3/bucket-0/data-4decfe67-c7e1-43f9-96ee-ae5188779a39-0.parquet differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=4/bucket-0/data-ee1eb48c-19ba-4275-9a73-ca788d55c57d-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=4/bucket-0/data-ee1eb48c-19ba-4275-9a73-ca788d55c57d-0.parquet
new file mode 100644
index 000000000..720841a89
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=4/bucket-0/data-ee1eb48c-19ba-4275-9a73-ca788d55c57d-0.parquet differ
diff --git a/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=5/bucket-0/data-86902019-80d0-4395-8bd4-666e1ffe2422-0.parquet b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=5/bucket-0/data-86902019-80d0-4395-8bd4-666e1ffe2422-0.parquet
new file mode 100644
index 000000000..52f34720d
Binary files /dev/null and b/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned/userId=5/bucket-0/data-86902019-80d0-4395-8bd4-666e1ffe2422-0.parquet differ
diff --git a/testdata/datasets/functional/functional_schema_template.sql b/testdata/datasets/functional/functional_schema_template.sql
index 99c26a602..2fb4af44e 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -4800,3 +4800,28 @@ delimited fields terminated by ','
insert overwrite table {db_name}{db_suffix}.{table_name}
select id, concat(cast(timestamp_col as string format 'YYYY-MM-DD HH24:MM:SS'), ' +08:00') from functional.alltypestiny;
====
+
+---- DATASET
+functional
+---- BASE_TABLE_NAME
+paimon_partitioned
+---- CREATE
+CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}
+STORED AS PAIMON
+LOCATION '/test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned';
+---- DEPENDENT_LOAD
+`hadoop fs -mkdir -p /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db && \
+hadoop fs -put -f ${IMPALA_HOME}/testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db
+====
+---- DATASET
+functional
+---- BASE_TABLE_NAME
+paimon_non_partitioned
+---- CREATE
+CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}
+STORED AS PAIMON
+LOCATION '/test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned'
+---- DEPENDENT_LOAD
+`hadoop fs -mkdir -p /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db && \
+hadoop fs -put -f ${IMPALA_HOME}//testdata/data/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned /test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db
+====
\ No newline at end of file
diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv
index c5229ecac..ba384ebf1 100644
--- a/testdata/datasets/functional/schema_constraints.csv
+++ b/testdata/datasets/functional/schema_constraints.csv
@@ -440,3 +440,7 @@ table_name:unique_with_nulls, constraint:restrict_to, table_format:parquet/none/
# The table is used to test casting string values to date type. Don't need to test on
# all formats.
table_name:timestamp_with_tz_str, constraint:restrict_to, table_format:parquet/none/none
+
+# Paimon tests are executed in the PARQUET file format dimension
+table_name:paimon_partitioned, constraint:restrict_to, table_format:parquet/none/none
+table_name:paimon_non_partitioned, constraint:restrict_to, table_format:parquet/none/none
\ No newline at end of file
diff --git a/testdata/workloads/functional-query/queries/QueryTest/paimon-ddl-negative.test b/testdata/workloads/functional-query/queries/QueryTest/paimon-ddl-negative.test
new file mode 100644
index 000000000..814615f8a
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/paimon-ddl-negative.test
@@ -0,0 +1,24 @@
+====
+---- QUERY
+CREATE TABLE impala_paimon_complex_types_array (
+ col_array_string ARRAY
+)
+STORED AS PAIMON;
+---- CATCH
+AnalysisException: Tables stored by Paimon do not support the column col_array_string type: ARRAY
+====
+---- QUERY
+CREATE TABLE impala_paimon_complex_types_map (
+ col_map_string MAP
+)
+STORED AS PAIMON;
+---- CATCH
+AnalysisException: Tables stored by Paimon do not support the column col_map_string type: MAP
+====
+---- QUERY
+CREATE TABLE impala_paimon_complex_types_struct (
+ col_struct_string STRUCT
+)
+STORED AS PAIMON;
+---- CATCH
+AnalysisException: Tables stored by Paimon do not support the column col_struct_string type: STRUCT
diff --git a/testdata/workloads/functional-query/queries/QueryTest/paimon-ddl.test b/testdata/workloads/functional-query/queries/QueryTest/paimon-ddl.test
new file mode 100644
index 000000000..6705de883
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/paimon-ddl.test
@@ -0,0 +1,211 @@
+====
+---- QUERY
+CREATE TABLE IF NOT EXISTS test_create_managed_paimon_table (
+ user_id BIGINT COMMENT 'The user_id field',
+ item_id BIGINT COMMENT 'The item_id field',
+ behavior STRING COMMENT 'The behavior field'
+)
+STORED AS PAIMON;
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE TABLE IF NOT EXISTS test_create_managed_part_paimon_table (
+ user_id BIGINT COMMENT 'The user_id field',
+ item_id BIGINT COMMENT 'The item_id field',
+ behavior STRING COMMENT 'The behavior field'
+)
+PARTITIONED BY (
+ dt STRING COMMENT 'The dt field',
+ hh STRING COMMENT 'The hh field'
+)
+STORED AS PAIMON;
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE TABLE test_create_managed_part_pk_paimon_table (
+ user_id BIGINT COMMENT 'The user_id field',
+ item_id BIGINT COMMENT 'The item_id field',
+ behavior STRING COMMENT 'The behavior field'
+)
+PARTITIONED BY (
+ dt STRING COMMENT 'The dt field',
+ hh STRING COMMENT 'The hh field'
+)
+STORED AS PAIMON
+TBLPROPERTIES (
+ 'primary-key'='user_id'
+);
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE TABLE test_create_managed_part_pkstmt_paimon_table (
+ user_id BIGINT COMMENT 'The user_id field',
+ item_id BIGINT COMMENT 'The item_id field',
+ behavior STRING COMMENT 'The behavior field',
+ PRIMARY KEY(user_id)
+)
+PARTITIONED BY (
+ dt STRING COMMENT 'The dt field',
+ hh STRING COMMENT 'The hh field'
+)
+STORED AS PAIMON;
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE TABLE test_create_managed_bucket_paimon_table (
+ user_id BIGINT COMMENT 'The user_id field',
+ item_id BIGINT COMMENT 'The item_id field',
+ behavior STRING COMMENT 'The behavior field'
+)
+STORED AS PAIMON
+TBLPROPERTIES (
+ 'bucket' = '4',
+ 'bucket-key'='behavior'
+);
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE TABLE test_create_managed_location_paimon_table (
+ user_id BIGINT COMMENT 'The user_id field',
+ item_id BIGINT COMMENT 'The item_id field',
+ behavior STRING COMMENT 'The behavior field',
+ PRIMARY KEY(user_id)
+)
+PARTITIONED BY (
+ dt STRING COMMENT 'The dt field',
+ hh STRING COMMENT 'The hh field'
+)
+STORED AS PAIMON
+LOCATION 'hdfs:///test-warehouse/test_create_managed_location_paimon_table';
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE TABLE TEST_CREATE_MANAGED_UPPERCASE_PAIMON_TABLE (
+ `USER_ID` BIGINT COMMENT 'The user_id field',
+ `ITEM_ID` BIGINT COMMENT 'The item_id field',
+ BEHAVIOR STRING COMMENT 'The behavior field',
+ PRIMARY KEY(`USER_ID`)
+)
+PARTITIONED BY (
+ `DT` STRING COMMENT 'The dt field',
+ `HH` STRING COMMENT 'The hh field'
+)
+STORED AS PAIMON;
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE EXTERNAL TABLE test_create_external_hivecat_nocol_paimon_table
+STORED AS PAIMON
+LOCATION 'hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned';
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE EXTERNAL TABLE test_create_external_hivecat_nocol_paimon_part_table
+STORED AS PAIMON
+LOCATION 'hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned';
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE EXTERNAL TABLE test_create_external_hadoopcat_nocol_paimon_table
+STORED AS PAIMON
+TBLPROPERTIES('paimon.catalog'='hadoop',
+'paimon.catalog_location'='hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse',
+'paimon.table_identifier'='functional.paimon_non_partitioned');
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE EXTERNAL TABLE test_create_external_hadoopcat_nocol_paimon_part_table
+STORED AS PAIMON
+TBLPROPERTIES('paimon.catalog'='hadoop',
+'paimon.catalog_location'='hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse',
+'paimon.table_identifier'='functional.paimon_partitioned');
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+CREATE TABLE impala_paimon_data_types_test (
+ col_boolean BOOLEAN,
+ col_tinyint TINYINT,
+ col_smallint SMALLINT,
+ col_int INT,
+ col_integer INTEGER,
+ col_bigint BIGINT,
+ col_float FLOAT,
+ col_double DOUBLE,
+ col_decimal DECIMAL(10,2),
+ col_string STRING,
+ col_char CHAR(20),
+ col_varchar VARCHAR(100),
+ col_timestamp TIMESTAMP,
+ col_date DATE,
+ col_binary BINARY
+)
+COMMENT 'Table to test all Impala paimon-supported data types'
+STORED AS PAIMON;
+---- RESULTS
+'Table has been created.'
+====
+---- QUERY
+DROP TABLE IF EXISTS test_create_managed_paimon_table;
+---- RESULTS
+'Table has been dropped.'
+====
+---- QUERY
+DROP TABLE IF EXISTS test_create_managed_part_paimon_table;
+---- RESULTS
+'Table has been dropped.'
+====
+---- QUERY
+DROP TABLE IF EXISTS test_create_managed_part_pk_paimon_table;
+---- RESULTS
+'Table has been dropped.'
+====
+---- QUERY
+DROP TABLE IF EXISTS test_create_managed_part_pkstmt_paimon_table;
+---- RESULTS
+'Table has been dropped.'
+====
+---- QUERY
+DROP TABLE IF EXISTS test_create_managed_bucket_paimon_table;
+---- RESULTS
+'Table has been dropped.'
+====
+---- QUERY
+DROP TABLE IF EXISTS test_create_managed_location_paimon_table;
+---- RESULTS
+'Table has been dropped.'
+====
+---- QUERY
+DROP TABLE IF EXISTS TEST_CREATE_MANAGED_UPPERCASE_PAIMON_TABLE;
+---- RESULTS
+'Table has been dropped.'
+====
+---- QUERY
+DROP TABLE IF EXISTS test_create_external_hivecat_nocol_paimon_part_table;
+---- RESULTS
+'Table has been dropped.'
+====
+---- QUERY
+DROP TABLE IF EXISTS test_create_external_hadoopcat_nocol_paimon_table;
+---- RESULTS
+'Table has been dropped.'
+====
+---- QUERY
+DROP TABLE IF EXISTS test_create_external_hadoopcat_nocol_paimon_part_table;
+---- RESULTS
+'Table has been dropped.'
+====
+---- QUERY
+DROP TABLE IF EXISTS impala_paimon_data_types_test;
+---- RESULTS
+'Table has been dropped.'
diff --git a/testdata/workloads/functional-query/queries/QueryTest/paimon-show-stats.test b/testdata/workloads/functional-query/queries/QueryTest/paimon-show-stats.test
new file mode 100644
index 000000000..4e8fcaf36
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/paimon-show-stats.test
@@ -0,0 +1,49 @@
+====
+---- QUERY
+SHOW TABLE STATS functional_parquet.paimon_partitioned;
+---- LABELS
+Number Of Rows,Number Of Bytes
+---- RESULTS
+452,7282
+---- TYPES
+BIGINT, BIGINT
+====
+---- QUERY
+SHOW COLUMN STATS functional_parquet.paimon_partitioned;
+---- LABELS
+Column,Type,#Distinct Values,#Nulls,Max Size,Avg Size,#Trues,#Falses
+---- RESULTS
+'movieid','INT',386,0,4,4.0,-1,-1
+'rating','FLOAT',5,0,4,4.0,-1,-1
+'ts','BIGINT',228,0,8,8.0,-1,-1
+'userid','INT',5,0,4,4.0,-1,-1
+---- TYPES
+STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE, BIGINT, BIGINT
+====
+---- QUERY
+SHOW PARTITIONS functional_parquet.paimon_partitioned;
+---- LABELS
+Partition,Number Of Rows,Number Of Files,Number Of Bytes,Last Creation Time
+---- RESULTS
+'userId=3',51,1,1301,1748992355131
+'userId=2',129,1,1657,1748992355155
+'userId=5',198,1,1922,1748992355177
+'userId=1',53,1,1301,1748992355201
+'userId=4',21,1,1101,1748992355224
+---- TYPES
+STRING, BIGINT, BIGINT, BIGINT, BIGINT
+====
+---- QUERY
+SHOW PARTITIONS functional_parquet.paimon_non_partitioned;
+---- CATCH
+AnalysisException: Table is not partitioned: functional_parquet.paimon_non_partitioned
+====
+---- QUERY
+SHOW RANGE PARTITIONS functional_parquet.paimon_non_partitioned;
+---- CATCH
+AnalysisException: SHOW RANGE PARTITIONS functional_parquet.paimon_non_partitioned is not supported for Paimon Table
+====
+---- QUERY
+SHOW FILES IN functional_parquet.paimon_partitioned PARTITION(userid=1403);
+---- CATCH
+AnalysisException: SHOW FILES with partition filter is not applicable to table type:PAIMON
diff --git a/testdata/workloads/functional-query/queries/QueryTest/set.test b/testdata/workloads/functional-query/queries/QueryTest/set.test
index 68d6941af..1a8dc7283 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/set.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/set.test
@@ -147,7 +147,7 @@ Invalid Kudu read mode: 'bar'. Valid values are DEFAULT(0), READ_LATEST(1), READ
---- QUERY
set default_file_format=bar
---- CATCH
-Invalid default file format: 'bar'. Valid values are TEXT(0), RC_FILE(1), SEQUENCE_FILE(2), AVRO(3), PARQUET(4), KUDU(5), ORC(6), HUDI_PARQUET(7), ICEBERG(8), JSON(9), JDBC(10).
+Invalid default file format: 'bar'. Valid values are TEXT(0), RC_FILE(1), SEQUENCE_FILE(2), AVRO(3), PARQUET(4), KUDU(5), ORC(6), HUDI_PARQUET(7), ICEBERG(8), JSON(9), JDBC(10), PAIMON(11).
====
---- QUERY
set default_transactional_type=bar
diff --git a/testdata/workloads/functional-query/queries/QueryTest/show-create-table-paimon.test b/testdata/workloads/functional-query/queries/QueryTest/show-create-table-paimon.test
new file mode 100644
index 000000000..1b9b38960
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/show-create-table-paimon.test
@@ -0,0 +1,201 @@
+====
+---- CREATE_TABLE
+CREATE TABLE test_create_managed_paimon_table (
+ user_id BIGINT COMMENT 'The user_id field',
+ item_id BIGINT COMMENT 'The item_id field',
+ behavior STRING COMMENT 'The behavior field'
+)
+STORED AS PAIMON;
+---- RESULTS-HIVE
+CREATE TABLE show_create_table_test_db.test_create_managed_paimon_table (
+ user_id BIGINT,
+ item_id BIGINT,
+ behavior STRING
+)
+STORED AS PAIMON
+TBLPROPERTIES ('deletion-vectors.enabled'='true')
+====
+---- CREATE_TABLE
+CREATE TABLE test_create_managed_paimon_table (
+user_id BIGINT COMMENT 'The user_id field',
+item_id BIGINT COMMENT 'The item_id field',
+behavior STRING COMMENT 'The behavior field'
+)
+STORED AS PAIMON;
+---- RESULTS-HIVE
+CREATE TABLE show_create_table_test_db.test_create_managed_paimon_table (
+user_id BIGINT,
+item_id BIGINT,
+behavior STRING
+)
+STORED AS PAIMON
+TBLPROPERTIES ('deletion-vectors.enabled'='true')
+====
+---- CREATE_TABLE
+CREATE TABLE test_create_managed_part_paimon_table (
+user_id BIGINT COMMENT 'The user_id field',
+item_id BIGINT COMMENT 'The item_id field',
+behavior STRING COMMENT 'The behavior field'
+)
+PARTITIONED BY (
+dt STRING COMMENT 'The dt field',
+hh STRING COMMENT 'The hh field'
+)
+STORED AS PAIMON;
+---- RESULTS-HIVE
+CREATE TABLE show_create_table_test_db.test_create_managed_part_paimon_table (
+user_id BIGINT,
+item_id BIGINT,
+behavior STRING
+)
+PARTITIONED BY (
+dt STRING,
+hh STRING
+)
+STORED AS PAIMON
+TBLPROPERTIES ('deletion-vectors.enabled'='true')
+====
+---- CREATE_TABLE
+CREATE TABLE test_create_managed_part_pk_paimon_table (
+user_id BIGINT COMMENT 'The user_id field',
+item_id BIGINT COMMENT 'The item_id field',
+behavior STRING COMMENT 'The behavior field'
+)
+PARTITIONED BY (
+dt STRING COMMENT 'The dt field',
+hh STRING COMMENT 'The hh field'
+)
+STORED AS PAIMON
+TBLPROPERTIES (
+'primary-key'='user_id'
+);
+---- RESULTS-HIVE
+CREATE TABLE show_create_table_test_db.test_create_managed_part_pk_paimon_table (
+user_id BIGINT,
+item_id BIGINT,
+behavior STRING,
+PRIMARY KEY (user_id)
+)
+PARTITIONED BY (
+dt STRING,
+hh STRING
+)
+STORED AS PAIMON
+TBLPROPERTIES ('deletion-vectors.enabled'='true')
+====
+---- CREATE_TABLE
+CREATE TABLE test_create_managed_part_pkstmt_paimon_table (
+user_id BIGINT COMMENT 'The user_id field',
+item_id BIGINT COMMENT 'The item_id field',
+behavior STRING COMMENT 'The behavior field',
+PRIMARY KEY (user_id)
+)
+PARTITIONED BY (
+dt STRING COMMENT 'The dt field',
+hh STRING COMMENT 'The hh field'
+)
+STORED AS PAIMON;
+---- RESULTS-HIVE
+CREATE TABLE show_create_table_test_db.test_create_managed_part_pkstmt_paimon_table (
+user_id BIGINT,
+item_id BIGINT,
+behavior STRING,
+PRIMARY KEY (user_id)
+)
+PARTITIONED BY (
+dt STRING,
+hh STRING
+)
+STORED AS PAIMON
+TBLPROPERTIES ('deletion-vectors.enabled'='true')
+====
+---- CREATE_TABLE
+CREATE TABLE test_create_managed_bucket_paimon_table (
+user_id BIGINT COMMENT 'The user_id field',
+item_id BIGINT COMMENT 'The item_id field',
+behavior STRING COMMENT 'The behavior field'
+)
+STORED AS PAIMON
+TBLPROPERTIES (
+'bucket' = '4',
+'bucket-key'='behavior'
+);
+---- RESULTS-HIVE
+CREATE TABLE show_create_table_test_db.test_create_managed_bucket_paimon_table (
+user_id BIGINT,
+item_id BIGINT,
+behavior STRING
+)
+STORED AS PAIMON
+TBLPROPERTIES ('bucket'='4', 'bucket-key'='behavior', 'deletion-vectors.enabled'='true')
+====
+---- CREATE_TABLE
+CREATE EXTERNAL TABLE test_create_external_hivecat_nocol_paimon_table
+STORED AS PAIMON
+LOCATION 'hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/paimon_non_partitioned';
+---- RESULTS-HIVE
+CREATE EXTERNAL TABLE show_create_table_test_db.test_create_external_hivecat_nocol_paimon_table (
+userid INT,
+movieid INT,
+rating FLOAT,
+`timestamp` BIGINT
+)
+STORED AS PAIMON
+LOCATION '$$location_uri$$'
+TBLPROPERTIES ('deletion-vectors.enabled'='true')
+====
+---- CREATE_TABLE
+CREATE EXTERNAL TABLE test_create_external_hadoopcat_nocol_paimon_table
+STORED AS PAIMON
+TBLPROPERTIES('paimon.catalog'='hadoop',
+'paimon.catalog_location'='hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse',
+'paimon.table_identifier'='functional.paimon_non_partitioned');
+---- RESULTS-HIVE
+CREATE EXTERNAL TABLE show_create_table_test_db.test_create_external_hadoopcat_nocol_paimon_table (
+userid INT,
+movieid INT,
+rating FLOAT,
+`timestamp` BIGINT
+)
+STORED AS PAIMON
+LOCATION '$$location_uri$$'
+TBLPROPERTIES ('deletion-vectors.enabled'='true', 'paimon.catalog'='hadoop', 'paimon.catalog_location'='hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse', 'paimon.table_identifier'='functional.paimon_non_partitioned')
+====
+---- CREATE_TABLE
+CREATE EXTERNAL TABLE test_create_external_hivecat_nocol_paimon_part_table
+STORED AS PAIMON
+LOCATION 'hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse/functional.db/paimon_partitioned';
+---- RESULTS-HIVE
+CREATE EXTERNAL TABLE show_create_table_test_db.test_create_external_hivecat_nocol_paimon_part_table (
+movieid INT,
+rating FLOAT,
+ts BIGINT
+)
+PARTITIONED BY (
+userid INT
+)
+STORED AS PAIMON
+LOCATION '$$location_uri$$'
+TBLPROPERTIES ('deletion-vectors.enabled'='true')
+====
+---- CREATE_TABLE
+CREATE EXTERNAL TABLE test_create_external_hadoopcat_nocol_paimon_part_table
+STORED AS PAIMON
+TBLPROPERTIES('paimon.catalog'='hadoop',
+'paimon.catalog_location'='hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse',
+'paimon.table_identifier'='functional.paimon_partitioned')
+---- RESULTS-HIVE
+CREATE EXTERNAL TABLE show_create_table_test_db.test_create_external_hadoopcat_nocol_paimon_part_table (
+movieid INT,
+rating FLOAT,
+ts BIGINT
+)
+PARTITIONED BY (
+userid INT
+)
+STORED AS PAIMON
+LOCATION '$$location_uri$$'
+TBLPROPERTIES ('deletion-vectors.enabled'='true', 'paimon.catalog'='hadoop', 'paimon.catalog_location'='hdfs:///test-warehouse/paimon_test/paimon_catalog/warehouse', 'paimon.table_identifier'='functional.paimon_partitioned')
+
+
+
diff --git a/tests/metadata/test_show_create_table.py b/tests/metadata/test_show_create_table.py
index 01eaf5bb9..b9dbd6bd2 100644
--- a/tests/metadata/test_show_create_table.py
+++ b/tests/metadata/test_show_create_table.py
@@ -21,7 +21,7 @@ import re
import shlex
from tests.common.impala_test_suite import ImpalaTestSuite
-from tests.common.skip import SkipIfFS, SkipIfHive2
+from tests.common.skip import SkipIf, SkipIfFS, SkipIfHive2
from tests.common.test_dimensions import create_uncompressed_text_dimension
from tests.util.test_file_parser import QueryTestSectionReader, remove_comments
from tests.common.environ import HIVE_MAJOR_VERSION
@@ -88,6 +88,12 @@ class TestShowCreateTable(ImpalaTestSuite):
vector,
unique_database)
+ @SkipIf.not_hdfs
+ def test_show_create_table_paimon(self, vector, unique_database):
+ self.__run_show_create_table_test_case('QueryTest/show-create-table-paimon',
+ vector,
+ unique_database)
+
def __run_show_create_table_test_case(self, test_file_name, vector, unique_db_name):
"""
Runs a show-create-table test file, containing the following sections:
diff --git a/tests/query_test/test_paimon.py b/tests/query_test/test_paimon.py
new file mode 100644
index 000000000..4a5db5d75
--- /dev/null
+++ b/tests/query_test/test_paimon.py
@@ -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.
+
+from __future__ import absolute_import, division, print_function
+
+from tests.common.impala_test_suite import ImpalaTestSuite
+from tests.common.skip import SkipIf
+
+@SkipIf.not_hdfs
+class TestCreatingPaimonTable(ImpalaTestSuite):
+
+ @classmethod
+ def get_workload(cls):
+ return 'functional-query'
+
+ @classmethod
+ def add_test_dimensions(cls):
+ super(TestCreatingPaimonTable, cls).add_test_dimensions()
+ cls.ImpalaTestMatrix.add_constraint(
+ lambda v: v.get_value('table_format').file_format == 'parquet')
+
+ def test_paimon_ddl(self, vector, unique_database):
+ self.run_test_case('QueryTest/paimon-ddl', vector, unique_database)
+
+ def test_paimon_show_stats(self, vector, unique_database):
+ self.run_test_case('QueryTest/paimon-show-stats', vector, unique_database)
+
+ def test_paimon_show_file_stats(self):
+ tables = ["paimon_non_partitioned", "paimon_partitioned"]
+ for table in tables:
+ result = self.client.execute("show files in functional_parquet." + table)
+ assert len(result.data) == 5, "Expected 5 files"
+
+ def test_create_paimon_ddl_negative(self, vector, unique_database):
+ self.run_test_case('QueryTest/paimon-ddl-negative',
+ vector, unique_database)