You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@paimon.apache.org by ni...@apache.org on 2023/03/30 06:29:50 UTC

[incubator-paimon] branch master updated: [hive] Remove bundled Hive classes from Paimon Hive catalog and bundle Hive catalog to distributed jars of all engines (#764)

This is an automated email from the ASF dual-hosted git repository.

nicholasjiang pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-paimon.git


The following commit(s) were added to refs/heads/master by this push:
     new 00b09f2c1 [hive] Remove bundled Hive classes from Paimon Hive catalog and bundle Hive catalog to distributed jars of all engines (#764)
00b09f2c1 is described below

commit 00b09f2c198914428d62eb3f1392384b9800e9f5
Author: tsreaper <ts...@gmail.com>
AuthorDate: Thu Mar 30 14:29:46 2023 +0800

    [hive] Remove bundled Hive classes from Paimon Hive catalog and bundle Hive catalog to distributed jars of all engines (#764)
---
 docs/content/engines/hive.md                       |   2 +-
 docs/content/how-to/creating-catalogs.md           |  40 +-
 paimon-e2e-tests/pom.xml                           |  31 +-
 .../test/resources-filtered/docker-compose.yaml    |  20 +-
 paimon-flink/paimon-flink-common/pom.xml           |   4 +
 paimon-flink/pom.xml                               |  10 +
 paimon-hive/paimon-hive-catalog/pom.xml            | 483 --------------------
 .../java/org/apache/paimon/hive/HiveCatalog.java   |  54 ++-
 .../org/apache/paimon/hive/HiveCatalogFactory.java |   7 +-
 .../hive/CustomHiveMetastoreClientITCase.java      | 100 -----
 paimon-hive/paimon-hive-connector-2.3/pom.xml      | 500 +++++++++++++++++++++
 .../paimon/hive/AlterFailHiveMetaStoreClient.java  |   1 +
 .../paimon/hive/CreateFailHiveMetaStoreClient.java |   1 +
 .../apache/paimon/hive/Hive23CatalogITCase.java    | 157 +++++++
 .../paimon/hive/TestHiveMetaStoreClient.java       |   0
 paimon-hive/paimon-hive-connector-3.1/pom.xml      | 476 ++++++++++++++++++++
 .../paimon/hive/AlterFailHiveMetaStoreClient.java  |   9 +-
 .../paimon/hive/CreateFailHiveMetaStoreClient.java |   9 +-
 .../apache/paimon/hive/Hive31CatalogITCase.java    | 157 +++++++
 .../paimon/hive/TestHiveMetaStoreClient.java       |  11 +-
 paimon-hive/paimon-hive-connector-common/pom.xml   | 181 +++++---
 .../apache/paimon/hive/HiveCatalogITCaseBase.java} | 104 +----
 .../paimon/hive/PaimonStorageHandlerITCase.java    |   4 +
 paimon-shade/pom.xml                               |  17 +
 paimon-spark/paimon-spark-common/pom.xml           |  21 +
 25 files changed, 1577 insertions(+), 822 deletions(-)

diff --git a/docs/content/engines/hive.md b/docs/content/engines/hive.md
index ee6056951..cb4db04fa 100644
--- a/docs/content/engines/hive.md
+++ b/docs/content/engines/hive.md
@@ -84,7 +84,7 @@ NOTE: If you are using HDFS, make sure that the environment variable `HADOOP_HOM
 
 By using paimon Hive catalog, you can create, drop and insert into paimon tables from Flink. These operations directly affect the corresponding Hive metastore. Tables created in this way can also be accessed directly from Hive.
 
-**Step 1: Prepare Paimon Hive Catalog Jar File for Flink**
+**Step 1: Prepare Flink Hive Connector Bundled Jar**
 
 See [creating a catalog with Hive metastore]({{< ref "how-to/creating-catalogs#creating-a-catalog-with-hive-metastore" >}}).
 
diff --git a/docs/content/how-to/creating-catalogs.md b/docs/content/how-to/creating-catalogs.md
index 05b4e4e14..00ecbe6d7 100644
--- a/docs/content/how-to/creating-catalogs.md
+++ b/docs/content/how-to/creating-catalogs.md
@@ -76,45 +76,11 @@ USE paimon.default;
 
 By using Paimon Hive catalog, changes to the catalog will directly affect the corresponding Hive metastore. Tables created in such catalog can also be accessed directly from Hive.
 
-### Preparing Paimon Hive Catalog Jar File
-
-Download the jar file with corresponding version.
-
-{{< stable >}}
-
-| Version    | Jar                                                                                                                                                                                    |
-|------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| Hive 2 & 3 | [paimon-hive-catalog-{{< version >}}.jar](https://www.apache.org/dyn/closer.lua/flink/paimon-{{< version >}}/paimon-hive-catalog-{{< version >}}.jar) |
-
-{{< /stable >}}
-
-{{< unstable >}}
-
-| Version    | Jar                                                                                                                                                                                    |
-|------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| Hive 2 & 3 | [paimon-hive-catalog-{{< version >}}.jar](https://repository.apache.org/snapshots/org/apache/paimon/paimon-hive-catalog/{{< version >}}/) |
-
-{{< /unstable >}}
-
-You can also manually build bundled jar from the source code.
-
-To build from source code, [clone the git repository]({{< github_repo >}}).
-
-Build bundled jar with the following command.
-`mvn clean install -Dmaven.test.skip=true`
-
-You can find Hive catalog jar in `./paimon-hive/paimon-hive-catalog/target/paimon-hive-catalog-{{< version >}}.jar`.
-
-### Registering Hive Catalog
-
 {{< tabs "hive-metastore-example" >}}
 
 {{< tab "Flink" >}}
 
-To enable Paimon Hive catalog support in Flink, you can pick one of the following two methods.
-
-* Copy Paimon Hive catalog jar file into the `lib` directory of your Flink installation directory. Note that this must be done before starting your Flink cluster.
-* If you're using Flink's SQL client, append `--jar /path/to/paimon-hive-catalog-{{< version >}}.jar` to the starting command of SQL client.
+Paimon Hive catalog in Flink relies on Flink Hive connector bundled jar. You should first download Flink Hive connector bundled jar and add it to classpath. See [here](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/table/hive/overview/#using-bundled-hive-jar) for more info.
 
 The following Flink SQL registers and uses a Paimon Hive catalog named `my_hive`. Metadata and table files are stored under `hdfs://path/to/warehouse`. In addition, metadata is also stored in Hive metastore.
 
@@ -133,9 +99,9 @@ USE CATALOG my_hive;
 
 {{< tab "Spark3" >}}
 
-To enable Paimon Hive catalog support in Spark3, append the path of Paimon Hive catalog jar file to `--jars` argument when starting spark.
+Your Spark installation should be able to detect, or already contains Hive dependencies. See [here](https://spark.apache.org/docs/latest/sql-data-sources-hive-tables.html) for more information.
 
-The following shell command registers a Table tore Hive catalog named `paimon`. Metadata and table files are stored under `hdfs://path/to/warehouse`. In addition, metadata is also stored in Hive metastore.
+The following shell command registers a Paimon Hive catalog named `paimon`. Metadata and table files are stored under `hdfs://path/to/warehouse`. In addition, metadata is also stored in Hive metastore.
 
 ```bash
 spark-sql ... \
diff --git a/paimon-e2e-tests/pom.xml b/paimon-e2e-tests/pom.xml
index bc6681985..5ae584b49 100644
--- a/paimon-e2e-tests/pom.xml
+++ b/paimon-e2e-tests/pom.xml
@@ -34,6 +34,7 @@ under the License.
     <properties>
         <flink.shaded.hadoop.version>2.8.3-10.0</flink.shaded.hadoop.version>
         <flink.sql.connector.kafka>flink-sql-connector-kafka</flink.sql.connector.kafka>
+        <flink.sql.connector.hive>flink-sql-connector-hive-2.3.9_${scala.binary.version}</flink.sql.connector.hive>
     </properties>
 
     <dependencies>
@@ -46,13 +47,6 @@ under the License.
             <scope>runtime</scope>
         </dependency>
 
-        <dependency>
-            <groupId>org.apache.paimon</groupId>
-            <artifactId>paimon-hive-catalog</artifactId>
-            <version>${project.version}</version>
-            <scope>runtime</scope>
-        </dependency>
-
         <dependency>
             <groupId>org.apache.paimon</groupId>
             <artifactId>paimon-hive-connector-common</artifactId>
@@ -110,16 +104,6 @@ under the License.
                             <outputDirectory>/tmp/paimon-e2e-tests-jars
                             </outputDirectory>
                         </artifactItem>
-                        <artifactItem>
-                            <groupId>org.apache.paimon</groupId>
-                            <artifactId>paimon-hive-catalog</artifactId>
-                            <version>${project.version}</version>
-                            <destFileName>paimon-hive-catalog.jar</destFileName>
-                            <type>jar</type>
-                            <overWrite>true</overWrite>
-                            <outputDirectory>/tmp/paimon-e2e-tests-jars
-                            </outputDirectory>
-                        </artifactItem>
                         <artifactItem>
                             <groupId>org.apache.paimon</groupId>
                             <artifactId>paimon-hive-connector-common</artifactId>
@@ -161,6 +145,17 @@ under the License.
                             <outputDirectory>/tmp/paimon-e2e-tests-jars
                             </outputDirectory>
                         </artifactItem>
+                        <!-- flink hive catalog needs flink hive sql jar -->
+                        <artifactItem>
+                            <groupId>org.apache.flink</groupId>
+                            <artifactId>${flink.sql.connector.hive}</artifactId>
+                            <version>${test.flink.version}</version>
+                            <destFileName>flink-sql-connector-hive.jar</destFileName>
+                            <type>jar</type>
+                            <overWrite>true</overWrite>
+                            <outputDirectory>/tmp/paimon-e2e-tests-jars
+                            </outputDirectory>
+                        </artifactItem>
                     </artifactItems>
                 </configuration>
             </plugin>
@@ -185,6 +180,7 @@ under the License.
                 <test.flink.main.version>1.14</test.flink.main.version>
                 <test.flink.version>1.14.6</test.flink.version>
                 <flink.sql.connector.kafka>flink-sql-connector-kafka_${scala.binary.version}</flink.sql.connector.kafka>
+                <flink.sql.connector.hive>flink-sql-connector-hive-2.3.6_${scala.binary.version}</flink.sql.connector.hive>
             </properties>
         </profile>
 
@@ -194,6 +190,7 @@ under the License.
                 <test.flink.main.version>1.15</test.flink.main.version>
                 <test.flink.version>1.15.3</test.flink.version>
                 <flink.sql.connector.kafka>flink-sql-connector-kafka</flink.sql.connector.kafka>
+                <flink.sql.connector.hive>flink-sql-connector-hive-2.3.6_${scala.binary.version}</flink.sql.connector.hive>
             </properties>
         </profile>
     </profiles>
diff --git a/paimon-e2e-tests/src/test/resources-filtered/docker-compose.yaml b/paimon-e2e-tests/src/test/resources-filtered/docker-compose.yaml
index 536882a61..88d9c6857 100644
--- a/paimon-e2e-tests/src/test/resources-filtered/docker-compose.yaml
+++ b/paimon-e2e-tests/src/test/resources-filtered/docker-compose.yaml
@@ -29,7 +29,15 @@ services:
     volumes:
       - testdata:/test-data
       - /tmp/paimon-e2e-tests-jars:/jars
-    entrypoint: /bin/bash -c "cp /jars/paimon-flink.jar /jars/bundled-hadoop.jar /jars/paimon-hive-catalog.jar /jars/flink-sql-connector-kafka.jar /opt/flink/lib && /docker-entrypoint.sh jobmanager"
+    entrypoint: >
+      /bin/bash -c "
+      cp /jars/paimon-flink.jar /jars/bundled-hadoop.jar
+      /jars/flink-sql-connector-kafka.jar /jars/flink-sql-connector-hive.jar /opt/flink/lib ;
+      echo 'See FLINK-31659 for why we need the following two steps' ;
+      mv /opt/flink/opt/flink-table-planner*.jar /opt/flink/lib/ ;
+      mv /opt/flink/lib/flink-table-planner-loader-*.jar /opt/flink/opt/ ;
+      /docker-entrypoint.sh jobmanager
+      "
     env_file:
       - ./flink.env
     networks:
@@ -44,7 +52,15 @@ services:
     volumes:
       - testdata:/test-data
       - /tmp/paimon-e2e-tests-jars:/jars
-    entrypoint: /bin/bash -c "cp /jars/paimon-flink.jar /jars/bundled-hadoop.jar /jars/paimon-hive-catalog.jar /jars/flink-sql-connector-kafka.jar /opt/flink/lib && /docker-entrypoint.sh taskmanager"
+    entrypoint: >
+      /bin/bash -c "
+      cp /jars/paimon-flink.jar /jars/bundled-hadoop.jar
+      /jars/flink-sql-connector-kafka.jar /jars/flink-sql-connector-hive.jar /opt/flink/lib ;
+      echo 'See FLINK-31659 for why we need the following two steps' ;
+      mv /opt/flink/opt/flink-table-planner*.jar /opt/flink/lib/ ;
+      mv /opt/flink/lib/flink-table-planner-loader-*.jar /opt/flink/opt/ ;
+      /docker-entrypoint.sh taskmanager
+      "
     env_file:
       - ./flink.env
     networks:
diff --git a/paimon-flink/paimon-flink-common/pom.xml b/paimon-flink/paimon-flink-common/pom.xml
index e30271efa..19c86e977 100644
--- a/paimon-flink/paimon-flink-common/pom.xml
+++ b/paimon-flink/paimon-flink-common/pom.xml
@@ -250,10 +250,14 @@ under the License.
                         <configuration>
                             <artifactSet>
                                 <includes combine.children="append">
+                                    <!--
+                                    TODO change these to paimon-shade once we get rid of flink-shaded dependencies
+                                    -->
                                     <include>org.apache.paimon:paimon-common</include>
                                     <include>org.apache.paimon:paimon-core</include>
                                     <include>org.apache.paimon:paimon-format</include>
                                     <include>org.apache.paimon:paimon-codegen-loader</include>
+                                    <include>org.apache.paimon:paimon-hive-catalog</include>
                                 </includes>
                             </artifactSet>
                             <relocations>
diff --git a/paimon-flink/pom.xml b/paimon-flink/pom.xml
index e315d55a3..b1919d1a1 100644
--- a/paimon-flink/pom.xml
+++ b/paimon-flink/pom.xml
@@ -41,6 +41,10 @@ under the License.
     </modules>
 
     <dependencies>
+        <!--
+        TODO change these to paimon-shade once we get rid of flink-shaded dependencies
+        -->
+
         <dependency>
             <groupId>org.apache.paimon</groupId>
             <artifactId>paimon-codegen-loader</artifactId>
@@ -65,6 +69,12 @@ under the License.
             <version>${project.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-hive-catalog</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.paimon</groupId>
             <artifactId>paimon-core</artifactId>
diff --git a/paimon-hive/paimon-hive-catalog/pom.xml b/paimon-hive/paimon-hive-catalog/pom.xml
index 7ab6941b5..937a86964 100644
--- a/paimon-hive/paimon-hive-catalog/pom.xml
+++ b/paimon-hive/paimon-hive-catalog/pom.xml
@@ -120,446 +120,6 @@ under the License.
                 </exclusion>
             </exclusions>
         </dependency>
-
-        <!-- test dependencies -->
-
-        <dependency>
-            <groupId>org.apache.paimon</groupId>
-            <artifactId>paimon-flink-common</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.paimon</groupId>
-            <artifactId>paimon-hive-connector-common</artifactId>
-            <version>${project.version}</version>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
-            <version>${test.flink.version}</version>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-table-runtime</artifactId>
-            <version>${test.flink.version}</version>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-connector-hive_${scala.binary.version}</artifactId>
-            <version>${test.flink.version}</version>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-connector-hive_${scala.binary.version}</artifactId>
-            <version>${test.flink.version}</version>
-            <scope>test</scope>
-            <type>test-jar</type>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.flink</groupId>
-            <artifactId>flink-connector-test-utils</artifactId>
-            <version>${test.flink.version}</version>
-            <scope>test</scope>
-            <exclusions>
-                <exclusion>
-                    <groupId>junit</groupId>
-                    <artifactId>junit</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.junit.vintage</groupId>
-                    <artifactId>junit-vintage-engine</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>com.klarna</groupId>
-            <artifactId>hiverunner</artifactId>
-            <version>${hiverunner.version}</version>
-            <scope>test</scope>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.hive</groupId>
-                    <artifactId>hive-serde</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hive</groupId>
-                    <artifactId>hive-jdbc</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hive</groupId>
-                    <artifactId>hive-service</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hive</groupId>
-                    <artifactId>hive-contrib</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hive</groupId>
-                    <artifactId>hive-exec</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hive</groupId>
-                    <artifactId>hive-hcatalog-core</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hive.hcatalog</groupId>
-                    <artifactId>hive-webhcat-java-client</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.tez</groupId>
-                    <artifactId>tez-common</artifactId>
-                </exclusion>
-                <exclusion>
-                    <!-- This dependency is no longer shipped with the JDK since Java 9.-->
-                    <groupId>jdk.tools</groupId>
-                    <artifactId>jdk.tools</artifactId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-common</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-auth</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-annotations</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-hdfs</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-mapreduce-client-core</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-api</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-client</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-common</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-server-common</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-server-web-proxy</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-shim</artifactId>
-                    <groupId>org.apache.tez</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>jms</artifactId>
-                    <groupId>javax.jms</groupId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-log4j12</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.reflections</groupId>
-            <artifactId>reflections</artifactId>
-            <version>${reflections.version}</version>
-            <scope>test</scope>
-            <exclusions>
-                <exclusion>
-                    <groupId>com.google.guava</groupId>
-                    <artifactId>guava</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.hive</groupId>
-            <artifactId>hive-service</artifactId>
-            <version>${hive.version}</version>
-            <scope>test</scope>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.hive</groupId>
-                    <artifactId>hive-exec</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hive</groupId>
-                    <artifactId>hive-metastore</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.google.guava</groupId>
-                    <artifactId>guava</artifactId>
-                </exclusion>
-                <exclusion>
-                    <!-- This dependency is no longer shipped with the JDK since Java 9.-->
-                    <groupId>jdk.tools</groupId>
-                    <artifactId>jdk.tools</artifactId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-common</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-auth</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-client</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-annotations</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-hdfs</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-mapreduce-client-core</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-api</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-common</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-registry</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-server-common</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hbase-hadoop-compat</artifactId>
-                    <groupId>org.apache.hbase</groupId>
-                </exclusion>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-log4j12</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.jamon</groupId>
-                    <artifactId>jamon-runtime</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.hive.hcatalog</groupId>
-            <artifactId>hive-hcatalog-core</artifactId>
-            <version>${hive.version}</version>
-            <scope>test</scope>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.apache.hive</groupId>
-                    <artifactId>hive-exec</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.google.guava</groupId>
-                    <artifactId>guava</artifactId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-common</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-archives</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-annotations</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-hdfs</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-mapreduce-client-core</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hadoop</groupId>
-                    <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>apache-log4j-extras</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-log4j12</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.fasterxml.jackson.core</groupId>
-                    <artifactId>jackson-databind</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.jamon</groupId>
-                    <artifactId>jamon-runtime</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.hive.hcatalog</groupId>
-            <artifactId>hive-webhcat-java-client</artifactId>
-            <version>${hive.version}</version>
-            <scope>test</scope>
-            <exclusions>
-                <exclusion>
-                    <groupId>jdk.tools</groupId>
-                    <artifactId>jdk.tools</artifactId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>jms</artifactId>
-                    <groupId>javax.jms</groupId>
-                </exclusion>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-log4j12</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.pentaho</groupId>
-                    <artifactId>pentaho-aggdesigner-algorithm</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.jamon</groupId>
-                    <artifactId>jamon-runtime</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.hive</groupId>
-                    <artifactId>hive-exec</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <!--
-        Why we need this test dependency:
-        IDEA reads classes from the same project from target/classes of that module,
-        so even though we've packaged and shaded avro classes into paimon-format.jar
-        we still have to include this test dependency here.
-
-        Why do we put this test dependency before the provided hive-exec:
-        hive-exec produces a shaded jar which contains old versioned avro classes,
-        so we need to make sure that our newer avro is loaded first.
-        -->
-        <dependency>
-            <groupId>org.apache.avro</groupId>
-            <artifactId>avro</artifactId>
-            <version>${avro.version}</version>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.hive</groupId>
-            <artifactId>hive-exec</artifactId>
-            <version>${hive.version}</version>
-            <scope>test</scope>
-            <exclusions>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-log4j12</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.fasterxml.jackson.core</groupId>
-                    <artifactId>jackson-annotations</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.fasterxml.jackson.core</groupId>
-                    <artifactId>jackson-core</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.fasterxml.jackson.core</groupId>
-                    <artifactId>jackson-databind</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.orc</groupId>
-                    <artifactId>orc-core</artifactId>
-                </exclusion>
-                <!-- this dependency cannot be fetched from central maven repository anymore -->
-                <exclusion>
-                    <groupId>org.pentaho</groupId>
-                    <artifactId>*</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-            <version>${junit4.version}</version>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.junit.vintage</groupId>
-            <artifactId>junit-vintage-engine</artifactId>
-            <version>${junit5.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>junit</groupId>
-                    <artifactId>junit</artifactId>
-                </exclusion>
-            </exclusions>
-            <scope>test</scope>
-        </dependency>
     </dependencies>
 
     <build>
@@ -578,51 +138,8 @@ under the License.
                             <artifactSet>
                                 <includes>
                                     <include>org.apache.paimon:paimon-hive-common</include>
-                                    <include>org.apache.thrift:libthrift</include>
-                                    <include>org.apache.thrift:libfb303</include>
-                                    <include>com.google.guava:guava</include>
-                                    <include>org.apache.hive:hive-common</include>
-                                    <include>org.apache.hive.shims:hive-shims-common</include>
-                                    <include>org.apache.hive.shims:hive-shims-0.23</include>
-                                    <include>org.apache.hive:hive-serde</include>
-                                    <include>org.apache.hive:hive-metastore</include>
-                                    <include>org.apache.hive:hive-storage-api</include>
                                 </includes>
                             </artifactSet>
-                            <relocations>
-                                <relocation>
-                                    <pattern>com.facebook.fb303</pattern>
-                                    <shadedPattern>org.apache.paimon.shaded.com.facebook.fb303</shadedPattern>
-                                </relocation>
-                                <relocation>
-                                    <pattern>com.google.common</pattern>
-                                    <shadedPattern>org.apache.paimon.shaded.com.google.common</shadedPattern>
-                                </relocation>
-                                <relocation>
-                                    <pattern>org.apache.hadoop.fs.ProxyLocalFileSystem</pattern>
-                                    <shadedPattern>org.apache.paimon.shaded.org.apache.hadoop.fs.ProxyLocalFileSystem</shadedPattern>
-                                </relocation>
-                                <relocation>
-                                    <pattern>org.apache.hadoop.fs.ProxyFileSystem</pattern>
-                                    <shadedPattern>org.apache.paimon.shaded.org.apache.hadoop.fs.ProxyFileSystem</shadedPattern>
-                                </relocation>
-                                <relocation>
-                                    <pattern>org.apache.hadoop.fs.DefaultFileAccess</pattern>
-                                    <shadedPattern>org.apache.paimon.shaded.org.apache.hadoop.fs.DefaultFileAccess</shadedPattern>
-                                </relocation>
-                                <relocation>
-                                    <pattern>org.apache.hadoop.mapred.WebHCatJTShim23</pattern>
-                                    <shadedPattern>org.apache.paimon.shaded.org.apache.hadoop.mapred.WebHCatJTShim23</shadedPattern>
-                                </relocation>
-                                <relocation>
-                                    <pattern>org.apache.hadoop.security.token.delegation.HiveDelegationTokenSupport</pattern>
-                                    <shadedPattern>org.apache.paimon.shaded.org.apache.hadoop.security.token.delegation.HiveDelegationTokenSupport</shadedPattern>
-                                </relocation>
-                                <relocation>
-                                    <pattern>org.apache.thrift</pattern>
-                                    <shadedPattern>org.apache.paimon.shaded.org.apache.thrift</shadedPattern>
-                                </relocation>
-                            </relocations>
                         </configuration>
                     </execution>
                 </executions>
diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
index c75a448d0..d167c02ca 100644
--- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
+++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java
@@ -36,13 +36,13 @@ import org.apache.paimon.utils.StringUtils;
 import org.apache.flink.table.hive.LegacyHiveClasses;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
@@ -53,6 +53,8 @@ import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.lang.reflect.Method;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -470,17 +472,53 @@ public class HiveCatalog extends AbstractCatalog {
         return Lock.fromCatalog(lock, identifier);
     }
 
+    private static final List<Class<?>[]> GET_PROXY_PARAMS =
+            Arrays.asList(
+                    // for hive 2.x
+                    new Class<?>[] {
+                        HiveConf.class,
+                        HiveMetaHookLoader.class,
+                        ConcurrentHashMap.class,
+                        String.class,
+                        Boolean.TYPE
+                    },
+                    // for hive 3.x
+                    new Class<?>[] {
+                        Configuration.class,
+                        HiveMetaHookLoader.class,
+                        ConcurrentHashMap.class,
+                        String.class,
+                        Boolean.TYPE
+                    });
+
     static IMetaStoreClient createClient(HiveConf hiveConf, String clientClassName) {
+        Method getProxy = null;
+        RuntimeException methodNotFound =
+                new RuntimeException(
+                        "Failed to find desired getProxy method from RetryingMetaStoreClient");
+        for (Class<?>[] classes : GET_PROXY_PARAMS) {
+            try {
+                getProxy = RetryingMetaStoreClient.class.getMethod("getProxy", classes);
+            } catch (NoSuchMethodException e) {
+                methodNotFound.addSuppressed(e);
+            }
+        }
+        if (getProxy == null) {
+            throw methodNotFound;
+        }
+
         IMetaStoreClient client;
         try {
             client =
-                    RetryingMetaStoreClient.getProxy(
-                            hiveConf,
-                            tbl -> null,
-                            new ConcurrentHashMap<>(),
-                            clientClassName,
-                            true);
-        } catch (MetaException e) {
+                    (IMetaStoreClient)
+                            getProxy.invoke(
+                                    null,
+                                    hiveConf,
+                                    (HiveMetaHookLoader) (tbl -> null),
+                                    new ConcurrentHashMap<>(),
+                                    clientClassName,
+                                    true);
+        } catch (Exception e) {
             throw new RuntimeException(e);
         }
         return StringUtils.isNullOrWhitespaceOnly(
diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogFactory.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogFactory.java
index 8cc279141..503739ebf 100644
--- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogFactory.java
+++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalogFactory.java
@@ -30,8 +30,6 @@ import org.apache.paimon.utils.Preconditions;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 
 /** Factory to create {@link HiveCatalog}. */
 public class HiveCatalogFactory implements CatalogFactory {
@@ -41,12 +39,11 @@ public class HiveCatalogFactory implements CatalogFactory {
     private static final ConfigOption<String> METASTORE_CLIENT_CLASS =
             ConfigOptions.key("metastore.client.class")
                     .stringType()
-                    .defaultValue(HiveMetaStoreClient.class.getName())
+                    .defaultValue("org.apache.hadoop.hive.metastore.HiveMetaStoreClient")
                     .withDescription(
                             "Class name of Hive metastore client.\n"
                                     + "NOTE: This class must directly implements "
-                                    + IMetaStoreClient.class.getName()
-                                    + ".");
+                                    + "org.apache.hadoop.hive.metastore.IMetaStoreClient.");
 
     @Override
     public String identifier() {
diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/CustomHiveMetastoreClientITCase.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/CustomHiveMetastoreClientITCase.java
deleted file mode 100644
index c7621c92d..000000000
--- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/CustomHiveMetastoreClientITCase.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.paimon.hive;
-
-import com.klarna.hiverunner.HiveShell;
-import com.klarna.hiverunner.annotations.HiveSQL;
-import org.apache.flink.connectors.hive.FlinkEmbeddedHiveRunner;
-import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.TableEnvironment;
-import org.apache.flink.table.api.internal.TableEnvironmentImpl;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.CloseableIterator;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-/** IT cases for {@link HiveCatalog} with custom Hive metastore client. */
-@RunWith(FlinkEmbeddedHiveRunner.class)
-public class CustomHiveMetastoreClientITCase {
-
-    @Rule public TemporaryFolder folder = new TemporaryFolder();
-
-    private String path;
-    private TableEnvironment tEnv;
-
-    @HiveSQL(files = {})
-    private static HiveShell hiveShell;
-
-    @Before
-    public void before() throws Exception {
-        hiveShell.execute("CREATE DATABASE IF NOT EXISTS test_db");
-        hiveShell.execute("USE test_db");
-    }
-
-    @After
-    public void after() {
-        hiveShell.execute("DROP DATABASE IF EXISTS test_db CASCADE");
-    }
-
-    @Test
-    public void testCustomMetastoreClient() throws Exception {
-        path = folder.newFolder().toURI().toString();
-        EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
-        tEnv = TableEnvironmentImpl.create(settings);
-        tEnv.executeSql(
-                        String.join(
-                                "\n",
-                                "CREATE CATALOG my_hive WITH (",
-                                "  'type' = 'paimon',",
-                                "  'metastore' = 'hive',",
-                                "  'uri' = '',",
-                                "  'warehouse' = '" + path + "',",
-                                "  'metastore.client.class' = '"
-                                        + TestHiveMetaStoreClient.class.getName()
-                                        + "'",
-                                ")"))
-                .await();
-        tEnv.executeSql("USE CATALOG my_hive").await();
-        Assert.assertEquals(
-                Arrays.asList(
-                        Row.of("default"),
-                        Row.of("test_db"),
-                        Row.of(TestHiveMetaStoreClient.MOCK_DATABASE)),
-                collect("SHOW DATABASES"));
-    }
-
-    private List<Row> collect(String sql) throws Exception {
-        List<Row> result = new ArrayList<>();
-        try (CloseableIterator<Row> it = tEnv.executeSql(sql).collect()) {
-            while (it.hasNext()) {
-                result.add(it.next());
-            }
-        }
-        return result;
-    }
-}
diff --git a/paimon-hive/paimon-hive-connector-2.3/pom.xml b/paimon-hive/paimon-hive-connector-2.3/pom.xml
index e42fd2809..6589f6d4c 100644
--- a/paimon-hive/paimon-hive-connector-2.3/pom.xml
+++ b/paimon-hive/paimon-hive-connector-2.3/pom.xml
@@ -42,6 +42,506 @@ under the License.
             <groupId>org.apache.paimon</groupId>
             <artifactId>paimon-hive-connector-common</artifactId>
             <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>*</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-hive-connector-common</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-shade</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-flink-common</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-runtime</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-hive_${scala.binary.version}</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-hive_${scala.binary.version}</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-orc</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-test-utils</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.junit.vintage</groupId>
+                    <artifactId>junit-vintage-engine</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>com.klarna</groupId>
+            <artifactId>hiverunner</artifactId>
+            <version>${hiverunner.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-serde</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-jdbc</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-service</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-contrib</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-exec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-hcatalog-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive.hcatalog</groupId>
+                    <artifactId>hive-webhcat-java-client</artifactId>
+                </exclusion>
+                <exclusion>
+                    <!-- This dependency is no longer shipped with the JDK since Java 9.-->
+                    <groupId>jdk.tools</groupId>
+                    <artifactId>jdk.tools</artifactId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-auth</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-annotations</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-hdfs</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-mapreduce-client-core</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-api</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-client</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-server-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-server-web-proxy</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>jms</artifactId>
+                    <groupId>javax.jms</groupId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.reflections</groupId>
+            <artifactId>reflections</artifactId>
+            <version>${reflections.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-service</artifactId>
+            <version>${hive.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-exec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-metastore</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+                <exclusion>
+                    <!-- This dependency is no longer shipped with the JDK since Java 9.-->
+                    <groupId>jdk.tools</groupId>
+                    <artifactId>jdk.tools</artifactId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-auth</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-client</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-annotations</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-hdfs</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-mapreduce-client-core</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-api</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-registry</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-server-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hbase-hadoop-compat</artifactId>
+                    <groupId>org.apache.hbase</groupId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.jamon</groupId>
+                    <artifactId>jamon-runtime</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive.hcatalog</groupId>
+            <artifactId>hive-hcatalog-core</artifactId>
+            <version>${hive.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-exec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-archives</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-annotations</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-hdfs</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-mapreduce-client-core</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>apache-log4j-extras</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-databind</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.jamon</groupId>
+                    <artifactId>jamon-runtime</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive.hcatalog</groupId>
+            <artifactId>hive-webhcat-java-client</artifactId>
+            <version>${hive.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.jamon</groupId>
+                    <artifactId>jamon-runtime</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>jdk.tools</groupId>
+                    <artifactId>jdk.tools</artifactId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>jms</artifactId>
+                    <groupId>javax.jms</groupId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.pentaho</groupId>
+                    <artifactId>pentaho-aggdesigner-algorithm</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <!--
+        Why we need this test dependency:
+        IDEA reads classes from the same project from target/classes of that module,
+        so even though we've packaged and shaded avro classes into paimon-format.jar
+        we still have to include this test dependency here.
+
+        Why do we put this test dependency before the provided hive-exec:
+        hive-exec produces a shaded jar which contains old versioned avro classes,
+        so we need to make sure that our newer avro is loaded first.
+        -->
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>
+            <version>${avro.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <!--
+        hive-exec must stay after flink-connector-hive and avro to avoid conflicts
+        -->
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-exec</artifactId>
+            <version>${hive.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-annotations</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-databind</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.orc</groupId>
+                    <artifactId>orc-core</artifactId>
+                </exclusion>
+                <!-- this dependency cannot be fetched from central maven repository anymore -->
+                <exclusion>
+                    <groupId>org.pentaho</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-druid</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite.avatica</groupId>
+                    <artifactId>avatica</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-avatica</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>${junit4.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.junit.vintage</groupId>
+            <artifactId>junit-vintage-engine</artifactId>
+            <version>${junit5.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                </exclusion>
+            </exclusions>
+            <scope>test</scope>
         </dependency>
     </dependencies>
 
diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java
similarity index 99%
copy from paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java
copy to paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java
index 6f77c0cf8..a34785993 100644
--- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java
+++ b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java
@@ -29,6 +29,7 @@ import org.apache.thrift.TException;
 
 /** A {@link HiveMetaStoreClient} to test altering table failed in Hive metastore client. */
 public class AlterFailHiveMetaStoreClient extends HiveMetaStoreClient implements IMetaStoreClient {
+
     public AlterFailHiveMetaStoreClient(HiveConf conf) throws MetaException {
         super(conf);
     }
diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java
similarity index 99%
copy from paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java
copy to paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java
index 71afbdc0c..aa88acddc 100644
--- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java
+++ b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java
@@ -32,6 +32,7 @@ import org.apache.thrift.TException;
 
 /** A {@link HiveMetaStoreClient} to test creating table failed in Hive metastore client. */
 public class CreateFailHiveMetaStoreClient extends HiveMetaStoreClient implements IMetaStoreClient {
+
     public CreateFailHiveMetaStoreClient(HiveConf conf) throws MetaException {
         super(conf);
     }
diff --git a/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/Hive23CatalogITCase.java b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/Hive23CatalogITCase.java
new file mode 100644
index 000000000..8c143617b
--- /dev/null
+++ b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/Hive23CatalogITCase.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.hive;
+
+import org.apache.paimon.fs.local.LocalFileIO;
+import org.apache.paimon.schema.SchemaManager;
+
+import com.klarna.hiverunner.annotations.HiveRunnerSetup;
+import com.klarna.hiverunner.config.HiveRunnerConfig;
+import org.apache.flink.connectors.hive.FlinkEmbeddedHiveRunner;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import java.util.Arrays;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_IN_TEST;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_TXN_MANAGER;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** IT cases for using Paimon {@link HiveCatalog} together with Paimon Hive 2.3 connector. */
+@RunWith(FlinkEmbeddedHiveRunner.class)
+public class Hive23CatalogITCase extends HiveCatalogITCaseBase {
+
+    @HiveRunnerSetup
+    private static final HiveRunnerConfig CONFIG =
+            new HiveRunnerConfig() {
+                {
+                    // catalog lock needs txn manager
+                    // hive-3.x requires a proper txn manager to create ACID table
+                    getHiveConfSystemOverride()
+                            .put(HIVE_TXN_MANAGER.varname, DbTxnManager.class.getName());
+                    getHiveConfSystemOverride().put(HIVE_SUPPORT_CONCURRENCY.varname, "true");
+                    // tell TxnHandler to prepare txn DB
+                    getHiveConfSystemOverride().put(HIVE_IN_TEST.varname, "true");
+                }
+            };
+
+    @Test
+    public void testCustomMetastoreClient() throws Exception {
+        path = folder.newFolder().toURI().toString();
+        EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
+        tEnv = TableEnvironmentImpl.create(settings);
+        tEnv.executeSql(
+                        String.join(
+                                "\n",
+                                "CREATE CATALOG my_hive WITH (",
+                                "  'type' = 'paimon',",
+                                "  'metastore' = 'hive',",
+                                "  'uri' = '',",
+                                "  'warehouse' = '" + path + "',",
+                                "  'metastore.client.class' = '"
+                                        + TestHiveMetaStoreClient.class.getName()
+                                        + "'",
+                                ")"))
+                .await();
+        tEnv.executeSql("USE CATALOG my_hive").await();
+        Assert.assertEquals(
+                Arrays.asList(
+                        Row.of("default"),
+                        Row.of("test_db"),
+                        Row.of(TestHiveMetaStoreClient.MOCK_DATABASE)),
+                collect("SHOW DATABASES"));
+    }
+
+    @Test
+    public void testCreateExistTableInHive() throws Exception {
+        tEnv.executeSql(
+                String.join(
+                        "\n",
+                        "CREATE CATALOG my_hive_custom_client WITH (",
+                        "  'type' = 'paimon',",
+                        "  'metastore' = 'hive',",
+                        "  'uri' = '',",
+                        "  'warehouse' = '" + path + "',",
+                        "  'metastore.client.class' = '"
+                                + CreateFailHiveMetaStoreClient.class.getName()
+                                + "'",
+                        ")"));
+        tEnv.executeSql("USE CATALOG my_hive_custom_client");
+        assertThatThrownBy(
+                        () ->
+                                tEnv.executeSql(
+                                                "CREATE TABLE hive_table(a INT, b INT, c INT, d INT)")
+                                        .await())
+                .isInstanceOf(TableException.class)
+                .hasMessage(
+                        "Could not execute CreateTable in path `my_hive_custom_client`.`default`.`hive_table`");
+        assertTrue(
+                new SchemaManager(
+                                LocalFileIO.create(),
+                                new org.apache.paimon.fs.Path(path, "default.db/hive_table"))
+                        .listAllIds()
+                        .isEmpty());
+    }
+
+    @Test
+    public void testAlterTableFailedInHive() throws Exception {
+        tEnv.executeSql(
+                        String.join(
+                                "\n",
+                                "CREATE CATALOG my_alter_hive WITH (",
+                                "  'type' = 'paimon',",
+                                "  'metastore' = 'hive',",
+                                "  'uri' = '',",
+                                "  'warehouse' = '" + path + "',",
+                                "  'metastore.client.class' = '"
+                                        + AlterFailHiveMetaStoreClient.class.getName()
+                                        + "'",
+                                ")"))
+                .await();
+        tEnv.executeSql("USE CATALOG my_alter_hive").await();
+        tEnv.executeSql("CREATE TABLE alter_failed_table(a INT, b STRING)").await();
+
+        assertThatThrownBy(() -> tEnv.executeSql("ALTER TABLE alter_failed_table SET ('aa'='bb')"))
+                .isInstanceOf(TableException.class)
+                .hasMessage(
+                        String.format(
+                                "Could not execute "
+                                        + "ALTER TABLE my_alter_hive.default.alter_failed_table "
+                                        + "SET (aa: [bb], path: [%sdefault.db/alter_failed_table])",
+                                path));
+
+        assertTrue(
+                new SchemaManager(
+                                LocalFileIO.create(),
+                                new org.apache.paimon.fs.Path(
+                                        path, "default.db/alter_failed_table"))
+                        .latest()
+                        .get()
+                        .options()
+                        .isEmpty());
+    }
+}
diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/TestHiveMetaStoreClient.java b/paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/TestHiveMetaStoreClient.java
similarity index 100%
copy from paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/TestHiveMetaStoreClient.java
copy to paimon-hive/paimon-hive-connector-2.3/src/test/java/org/apache/paimon/hive/TestHiveMetaStoreClient.java
diff --git a/paimon-hive/paimon-hive-connector-3.1/pom.xml b/paimon-hive/paimon-hive-connector-3.1/pom.xml
index 8c0f9d2bf..6055208d1 100644
--- a/paimon-hive/paimon-hive-connector-3.1/pom.xml
+++ b/paimon-hive/paimon-hive-connector-3.1/pom.xml
@@ -35,15 +35,455 @@ under the License.
 
     <properties>
         <hive.version>3.1.2</hive.version>
+        <jackson.version>2.13.3</jackson.version>
     </properties>
 
     <dependencies>
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.paimon</groupId>
             <artifactId>paimon-hive-connector-common</artifactId>
             <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>*</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-hive-connector-common</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-shade</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-flink-common</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-table-runtime</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-hive_${scala.binary.version}</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-hive_${scala.binary.version}</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+            <type>test-jar</type>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-orc</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-test-utils</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.junit.vintage</groupId>
+                    <artifactId>junit-vintage-engine</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>com.fasterxml.jackson.module</groupId>
+            <artifactId>jackson-module-scala_${scala.binary.version}</artifactId>
+            <version>${jackson.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-common</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>com.klarna</groupId>
+            <artifactId>hiverunner</artifactId>
+            <version>${hiverunner.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-serde</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-jdbc</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-service</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-contrib</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-exec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-hcatalog-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive.hcatalog</groupId>
+                    <artifactId>hive-webhcat-java-client</artifactId>
+                </exclusion>
+                <exclusion>
+                    <!-- This dependency is no longer shipped with the JDK since Java 9.-->
+                    <groupId>jdk.tools</groupId>
+                    <artifactId>jdk.tools</artifactId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-auth</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-annotations</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-hdfs</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-mapreduce-client-core</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-api</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-client</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-server-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-server-web-proxy</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>jms</artifactId>
+                    <groupId>javax.jms</groupId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.reflections</groupId>
+            <artifactId>reflections</artifactId>
+            <version>${reflections.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-service</artifactId>
+            <version>${hive.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-exec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-metastore</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+                <exclusion>
+                    <!-- This dependency is no longer shipped with the JDK since Java 9.-->
+                    <groupId>jdk.tools</groupId>
+                    <artifactId>jdk.tools</artifactId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-auth</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-client</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-annotations</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-hdfs</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-mapreduce-client-core</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-api</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-registry</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-server-applicationhistoryservice</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-server-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hbase-hadoop-compat</artifactId>
+                    <groupId>org.apache.hbase</groupId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.jamon</groupId>
+                    <artifactId>jamon-runtime</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive.hcatalog</groupId>
+            <artifactId>hive-hcatalog-core</artifactId>
+            <version>${hive.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.hive</groupId>
+                    <artifactId>hive-exec</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.google.guava</groupId>
+                    <artifactId>guava</artifactId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-common</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-archives</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-annotations</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-hdfs</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>hadoop-mapreduce-client-core</artifactId>
+                    <groupId>org.apache.hadoop</groupId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-yarn-server-resourcemanager</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>apache-log4j-extras</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-databind</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.jamon</groupId>
+                    <artifactId>jamon-runtime</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hive.hcatalog</groupId>
+            <artifactId>hive-webhcat-java-client</artifactId>
+            <version>${hive.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.jamon</groupId>
+                    <artifactId>jamon-runtime</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>jdk.tools</groupId>
+                    <artifactId>jdk.tools</artifactId>
+                </exclusion>
+                <exclusion>
+                    <artifactId>jms</artifactId>
+                    <groupId>javax.jms</groupId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.pentaho</groupId>
+                    <artifactId>pentaho-aggdesigner-algorithm</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <!--
+        Why we need this test dependency:
+        IDEA reads classes from the same project from target/classes of that module,
+        so even though we've packaged and shaded avro classes into paimon-format.jar
+        we still have to include this test dependency here.
+
+        Why do we put this test dependency before the provided hive-exec:
+        hive-exec produces a shaded jar which contains old versioned avro classes,
+        so we need to make sure that our newer avro is loaded first.
+        -->
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>
+            <version>${avro.version}</version>
+            <scope>test</scope>
         </dependency>
 
+        <!--
+        hive-exec must stay after flink-connector-hive and avro to avoid conflicts
+        -->
         <dependency>
             <groupId>org.apache.hive</groupId>
             <artifactId>hive-exec</artifactId>
@@ -79,7 +519,43 @@ under the License.
                     <groupId>org.pentaho</groupId>
                     <artifactId>*</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-druid</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite.avatica</groupId>
+                    <artifactId>avatica</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-avatica</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <version>${junit4.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.junit.vintage</groupId>
+            <artifactId>junit-vintage-engine</artifactId>
+            <version>${junit5.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                </exclusion>
             </exclusions>
+            <scope>test</scope>
         </dependency>
     </dependencies>
 
diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java
similarity index 85%
rename from paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java
rename to paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java
index 6f77c0cf8..6bdbb38ea 100644
--- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java
+++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/AlterFailHiveMetaStoreClient.java
@@ -18,7 +18,7 @@
 
 package org.apache.paimon.hive;
 
-import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
@@ -29,17 +29,18 @@ import org.apache.thrift.TException;
 
 /** A {@link HiveMetaStoreClient} to test altering table failed in Hive metastore client. */
 public class AlterFailHiveMetaStoreClient extends HiveMetaStoreClient implements IMetaStoreClient {
-    public AlterFailHiveMetaStoreClient(HiveConf conf) throws MetaException {
+
+    public AlterFailHiveMetaStoreClient(Configuration conf) throws MetaException {
         super(conf);
     }
 
-    public AlterFailHiveMetaStoreClient(HiveConf conf, HiveMetaHookLoader hookLoader)
+    public AlterFailHiveMetaStoreClient(Configuration conf, HiveMetaHookLoader hookLoader)
             throws MetaException {
         super(conf, hookLoader);
     }
 
     public AlterFailHiveMetaStoreClient(
-            HiveConf conf, HiveMetaHookLoader hookLoader, Boolean allowEmbedded)
+            Configuration conf, HiveMetaHookLoader hookLoader, Boolean allowEmbedded)
             throws MetaException {
         super(conf, hookLoader, allowEmbedded);
     }
diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java
similarity index 88%
rename from paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java
rename to paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java
index 71afbdc0c..7d1393d37 100644
--- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java
+++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/CreateFailHiveMetaStoreClient.java
@@ -18,7 +18,7 @@
 
 package org.apache.paimon.hive;
 
-import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
@@ -32,17 +32,18 @@ import org.apache.thrift.TException;
 
 /** A {@link HiveMetaStoreClient} to test creating table failed in Hive metastore client. */
 public class CreateFailHiveMetaStoreClient extends HiveMetaStoreClient implements IMetaStoreClient {
-    public CreateFailHiveMetaStoreClient(HiveConf conf) throws MetaException {
+
+    public CreateFailHiveMetaStoreClient(Configuration conf) throws MetaException {
         super(conf);
     }
 
-    public CreateFailHiveMetaStoreClient(HiveConf conf, HiveMetaHookLoader hookLoader)
+    public CreateFailHiveMetaStoreClient(Configuration conf, HiveMetaHookLoader hookLoader)
             throws MetaException {
         super(conf, hookLoader);
     }
 
     public CreateFailHiveMetaStoreClient(
-            HiveConf conf, HiveMetaHookLoader hookLoader, Boolean allowEmbedded)
+            Configuration conf, HiveMetaHookLoader hookLoader, Boolean allowEmbedded)
             throws MetaException {
         super(conf, hookLoader, allowEmbedded);
     }
diff --git a/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/Hive31CatalogITCase.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/Hive31CatalogITCase.java
new file mode 100644
index 000000000..7c6b3f068
--- /dev/null
+++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/Hive31CatalogITCase.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.hive;
+
+import org.apache.paimon.fs.local.LocalFileIO;
+import org.apache.paimon.schema.SchemaManager;
+
+import com.klarna.hiverunner.annotations.HiveRunnerSetup;
+import com.klarna.hiverunner.config.HiveRunnerConfig;
+import org.apache.flink.connectors.hive.FlinkEmbeddedHiveRunner;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.internal.TableEnvironmentImpl;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import java.util.Arrays;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_IN_TEST;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_TXN_MANAGER;
+import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** IT cases for using Paimon {@link HiveCatalog} together with Paimon Hive 3.1 connector. */
+@RunWith(FlinkEmbeddedHiveRunner.class)
+public class Hive31CatalogITCase extends HiveCatalogITCaseBase {
+
+    @HiveRunnerSetup
+    private static final HiveRunnerConfig CONFIG =
+            new HiveRunnerConfig() {
+                {
+                    // catalog lock needs txn manager
+                    // hive-3.x requires a proper txn manager to create ACID table
+                    getHiveConfSystemOverride()
+                            .put(HIVE_TXN_MANAGER.varname, DbTxnManager.class.getName());
+                    getHiveConfSystemOverride().put(HIVE_SUPPORT_CONCURRENCY.varname, "true");
+                    // tell TxnHandler to prepare txn DB
+                    getHiveConfSystemOverride().put(HIVE_IN_TEST.varname, "true");
+                }
+            };
+
+    @Test
+    public void testCustomMetastoreClient() throws Exception {
+        path = folder.newFolder().toURI().toString();
+        EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build();
+        tEnv = TableEnvironmentImpl.create(settings);
+        tEnv.executeSql(
+                        String.join(
+                                "\n",
+                                "CREATE CATALOG my_hive WITH (",
+                                "  'type' = 'paimon',",
+                                "  'metastore' = 'hive',",
+                                "  'uri' = '',",
+                                "  'warehouse' = '" + path + "',",
+                                "  'metastore.client.class' = '"
+                                        + TestHiveMetaStoreClient.class.getName()
+                                        + "'",
+                                ")"))
+                .await();
+        tEnv.executeSql("USE CATALOG my_hive").await();
+        Assert.assertEquals(
+                Arrays.asList(
+                        Row.of("default"),
+                        Row.of("test_db"),
+                        Row.of(TestHiveMetaStoreClient.MOCK_DATABASE)),
+                collect("SHOW DATABASES"));
+    }
+
+    @Test
+    public void testCreateExistTableInHive() throws Exception {
+        tEnv.executeSql(
+                String.join(
+                        "\n",
+                        "CREATE CATALOG my_hive_custom_client WITH (",
+                        "  'type' = 'paimon',",
+                        "  'metastore' = 'hive',",
+                        "  'uri' = '',",
+                        "  'warehouse' = '" + path + "',",
+                        "  'metastore.client.class' = '"
+                                + CreateFailHiveMetaStoreClient.class.getName()
+                                + "'",
+                        ")"));
+        tEnv.executeSql("USE CATALOG my_hive_custom_client");
+        assertThatThrownBy(
+                        () ->
+                                tEnv.executeSql(
+                                                "CREATE TABLE hive_table(a INT, b INT, c INT, d INT)")
+                                        .await())
+                .isInstanceOf(TableException.class)
+                .hasMessage(
+                        "Could not execute CreateTable in path `my_hive_custom_client`.`default`.`hive_table`");
+        assertTrue(
+                new SchemaManager(
+                                LocalFileIO.create(),
+                                new org.apache.paimon.fs.Path(path, "default.db/hive_table"))
+                        .listAllIds()
+                        .isEmpty());
+    }
+
+    @Test
+    public void testAlterTableFailedInHive() throws Exception {
+        tEnv.executeSql(
+                        String.join(
+                                "\n",
+                                "CREATE CATALOG my_alter_hive WITH (",
+                                "  'type' = 'paimon',",
+                                "  'metastore' = 'hive',",
+                                "  'uri' = '',",
+                                "  'warehouse' = '" + path + "',",
+                                "  'metastore.client.class' = '"
+                                        + AlterFailHiveMetaStoreClient.class.getName()
+                                        + "'",
+                                ")"))
+                .await();
+        tEnv.executeSql("USE CATALOG my_alter_hive").await();
+        tEnv.executeSql("CREATE TABLE alter_failed_table(a INT, b STRING)").await();
+
+        assertThatThrownBy(() -> tEnv.executeSql("ALTER TABLE alter_failed_table SET ('aa'='bb')"))
+                .isInstanceOf(TableException.class)
+                .hasMessage(
+                        String.format(
+                                "Could not execute "
+                                        + "ALTER TABLE my_alter_hive.default.alter_failed_table "
+                                        + "SET (aa: [bb], path: [%sdefault.db/alter_failed_table])",
+                                path));
+
+        assertTrue(
+                new SchemaManager(
+                                LocalFileIO.create(),
+                                new org.apache.paimon.fs.Path(
+                                        path, "default.db/alter_failed_table"))
+                        .latest()
+                        .get()
+                        .options()
+                        .isEmpty());
+    }
+}
diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/TestHiveMetaStoreClient.java b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/TestHiveMetaStoreClient.java
similarity index 81%
rename from paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/TestHiveMetaStoreClient.java
rename to paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/TestHiveMetaStoreClient.java
index 9312d6d59..a8a6acf68 100644
--- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/TestHiveMetaStoreClient.java
+++ b/paimon-hive/paimon-hive-connector-3.1/src/test/java/org/apache/paimon/hive/TestHiveMetaStoreClient.java
@@ -18,11 +18,12 @@
 
 package org.apache.paimon.hive;
 
-import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.thrift.TException;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -32,23 +33,23 @@ public class TestHiveMetaStoreClient extends HiveMetaStoreClient implements IMet
 
     public static final String MOCK_DATABASE = "test_mock_database";
 
-    public TestHiveMetaStoreClient(HiveConf conf) throws MetaException {
+    public TestHiveMetaStoreClient(Configuration conf) throws MetaException {
         super(conf);
     }
 
-    public TestHiveMetaStoreClient(HiveConf conf, HiveMetaHookLoader hookLoader)
+    public TestHiveMetaStoreClient(Configuration conf, HiveMetaHookLoader hookLoader)
             throws MetaException {
         super(conf, hookLoader);
     }
 
     public TestHiveMetaStoreClient(
-            HiveConf conf, HiveMetaHookLoader hookLoader, Boolean allowEmbedded)
+            Configuration conf, HiveMetaHookLoader hookLoader, Boolean allowEmbedded)
             throws MetaException {
         super(conf, hookLoader, allowEmbedded);
     }
 
     @Override
-    public List<String> getAllDatabases() throws MetaException {
+    public List<String> getAllDatabases() throws MetaException, TException {
         List<String> result = new ArrayList<>(super.getAllDatabases());
         result.add(MOCK_DATABASE);
         return result;
diff --git a/paimon-hive/paimon-hive-connector-common/pom.xml b/paimon-hive/paimon-hive-connector-common/pom.xml
index ee277056f..fde822019 100644
--- a/paimon-hive/paimon-hive-connector-common/pom.xml
+++ b/paimon-hive/paimon-hive-connector-common/pom.xml
@@ -40,7 +40,6 @@ under the License.
             <version>${project.version}</version>
         </dependency>
 
-        <!-- Flink All dependencies -->
         <dependency>
             <groupId>org.apache.paimon</groupId>
             <artifactId>paimon-shade</artifactId>
@@ -81,61 +80,6 @@ under the License.
             </exclusions>
         </dependency>
 
-        <!--
-        Why we need this test dependency:
-        IDEA reads classes from the same project from target/classes of that module,
-        so even though we've packaged and shaded avro classes into paimon-format.jar
-        we still have to include this test dependency here.
-
-        Why do we put this test dependency before the provided hive-exec:
-        hive-exec produces a shaded jar which contains old versioned avro classes,
-        so we need to make sure that our newer avro is loaded first.
-        -->
-        <dependency>
-            <groupId>org.apache.avro</groupId>
-            <artifactId>avro</artifactId>
-            <version>${avro.version}</version>
-            <scope>test</scope>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.hive</groupId>
-            <artifactId>hive-exec</artifactId>
-            <version>${hive.version}</version>
-            <scope>provided</scope>
-            <exclusions>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.slf4j</groupId>
-                    <artifactId>slf4j-log4j12</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.fasterxml.jackson.core</groupId>
-                    <artifactId>jackson-annotations</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.fasterxml.jackson.core</groupId>
-                    <artifactId>jackson-core</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>com.fasterxml.jackson.core</groupId>
-                    <artifactId>jackson-databind</artifactId>
-                </exclusion>
-                <exclusion>
-                    <groupId>org.apache.orc</groupId>
-                    <artifactId>orc-core</artifactId>
-                </exclusion>
-                <!-- this dependency cannot be fetched from central maven repository anymore -->
-                <exclusion>
-                    <groupId>org.pentaho</groupId>
-                    <artifactId>*</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
         <!-- test dependencies -->
 
         <dependency>
@@ -148,16 +92,23 @@ under the License.
 
         <!-- dependencies for IT cases -->
 
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-flink-common</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-core</artifactId>
+            <artifactId>flink-table-planner_${scala.binary.version}</artifactId>
             <version>${test.flink.version}</version>
             <scope>test</scope>
         </dependency>
 
         <dependency>
             <groupId>org.apache.flink</groupId>
-            <artifactId>flink-table-common</artifactId>
+            <artifactId>flink-table-runtime</artifactId>
             <version>${test.flink.version}</version>
             <scope>test</scope>
         </dependency>
@@ -177,6 +128,34 @@ under the License.
             <type>test-jar</type>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-orc</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.flink</groupId>
+            <artifactId>flink-connector-test-utils</artifactId>
+            <version>${test.flink.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>junit</groupId>
+                    <artifactId>junit</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.junit.vintage</groupId>
+                    <artifactId>junit-vintage-engine</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
         <dependency>
             <groupId>com.klarna</groupId>
             <artifactId>hiverunner</artifactId>
@@ -464,6 +443,80 @@ under the License.
             </exclusions>
         </dependency>
 
+        <!--
+        Why we need this test dependency:
+        IDEA reads classes from the same project from target/classes of that module,
+        so even though we've packaged and shaded avro classes into paimon-format.jar
+        we still have to include this test dependency here.
+
+        Why do we put this test dependency before the provided hive-exec:
+        hive-exec produces a shaded jar which contains old versioned avro classes,
+        so we need to make sure that our newer avro is loaded first.
+        -->
+        <dependency>
+            <groupId>org.apache.avro</groupId>
+            <artifactId>avro</artifactId>
+            <version>${avro.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <!--
+        hive-exec must stay after flink-connector-hive and avro to avoid conflicts
+        -->
+        <dependency>
+            <groupId>org.apache.hive</groupId>
+            <artifactId>hive-exec</artifactId>
+            <version>${hive.version}</version>
+            <scope>provided</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-annotations</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>com.fasterxml.jackson.core</groupId>
+                    <artifactId>jackson-databind</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.orc</groupId>
+                    <artifactId>orc-core</artifactId>
+                </exclusion>
+                <!-- this dependency cannot be fetched from central maven repository anymore -->
+                <exclusion>
+                    <groupId>org.pentaho</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-druid</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite.avatica</groupId>
+                    <artifactId>avatica</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.calcite</groupId>
+                    <artifactId>calcite-avatica</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
         <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
@@ -528,6 +581,18 @@ under the License.
                     </execution>
                 </executions>
             </plugin>
+
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 </project>
diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java
similarity index 83%
rename from paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogITCase.java
rename to paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java
index 4bebe5883..86b7d4995 100644
--- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogITCase.java
+++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/HiveCatalogITCaseBase.java
@@ -24,22 +24,17 @@ import org.apache.paimon.catalog.CatalogLock;
 import org.apache.paimon.catalog.Identifier;
 import org.apache.paimon.flink.FlinkCatalog;
 import org.apache.paimon.fs.local.LocalFileIO;
-import org.apache.paimon.schema.SchemaManager;
 
 import com.klarna.hiverunner.HiveShell;
-import com.klarna.hiverunner.annotations.HiveRunnerSetup;
 import com.klarna.hiverunner.annotations.HiveSQL;
-import com.klarna.hiverunner.config.HiveRunnerConfig;
 import org.apache.flink.connectors.hive.FlinkEmbeddedHiveRunner;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.table.api.EnvironmentSettings;
 import org.apache.flink.table.api.TableEnvironment;
-import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.internal.TableEnvironmentImpl;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.CloseableIterator;
 import org.apache.flink.util.ExceptionUtils;
-import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 import org.assertj.core.api.Assertions;
 import org.junit.After;
 import org.junit.Assert;
@@ -57,38 +52,20 @@ import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_IN_TEST;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_TXN_MANAGER;
 import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
 import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;
-import static org.junit.jupiter.api.Assertions.assertTrue;
 
-/** IT cases for {@link HiveCatalog}. */
+/** IT cases for using Paimon {@link HiveCatalog} together with Paimon Hive connector. */
 @RunWith(FlinkEmbeddedHiveRunner.class)
-public class HiveCatalogITCase {
+public abstract class HiveCatalogITCaseBase {
 
     @Rule public TemporaryFolder folder = new TemporaryFolder();
 
-    private String path;
-    private TableEnvironment tEnv;
+    protected String path;
+    protected TableEnvironment tEnv;
 
     @HiveSQL(files = {})
-    private static HiveShell hiveShell;
-
-    @HiveRunnerSetup
-    private static final HiveRunnerConfig CONFIG =
-            new HiveRunnerConfig() {
-                {
-                    // catalog lock needs txn manager
-                    // hive-3.x requires a proper txn manager to create ACID table
-                    getHiveConfSystemOverride()
-                            .put(HIVE_TXN_MANAGER.varname, DbTxnManager.class.getName());
-                    getHiveConfSystemOverride().put(HIVE_SUPPORT_CONCURRENCY.varname, "true");
-                    // tell TxnHandler to prepare txn DB
-                    getHiveConfSystemOverride().put(HIVE_IN_TEST.varname, "true");
-                }
-            };
+    protected static HiveShell hiveShell;
 
     @Before
     public void before() throws Exception {
@@ -529,76 +506,7 @@ public class HiveCatalogITCase {
         Assert.assertEquals("[]", tables.toString());
     }
 
-    @Test
-    public void testCreateExistTableInHive() throws Exception {
-        tEnv.executeSql(
-                String.join(
-                        "\n",
-                        "CREATE CATALOG my_hive_custom_client WITH (",
-                        "  'type' = 'paimon',",
-                        "  'metastore' = 'hive',",
-                        "  'uri' = '',",
-                        "  'warehouse' = '" + path + "',",
-                        "  'metastore.client.class' = '"
-                                + CreateFailHiveMetaStoreClient.class.getName()
-                                + "'",
-                        ")"));
-        tEnv.executeSql("USE CATALOG my_hive_custom_client");
-        assertThatThrownBy(
-                        () ->
-                                tEnv.executeSql(
-                                                "CREATE TABLE hive_table(a INT, b INT, c INT, d INT)")
-                                        .await())
-                .isInstanceOf(TableException.class)
-                .hasMessage(
-                        "Could not execute CreateTable in path `my_hive_custom_client`.`default`.`hive_table`");
-        assertTrue(
-                new SchemaManager(
-                                LocalFileIO.create(),
-                                new org.apache.paimon.fs.Path(path, "default.db/hive_table"))
-                        .listAllIds()
-                        .isEmpty());
-    }
-
-    @Test
-    public void testAlterTableFailedInHive() throws Exception {
-        tEnv.executeSql(
-                        String.join(
-                                "\n",
-                                "CREATE CATALOG my_alter_hive WITH (",
-                                "  'type' = 'paimon',",
-                                "  'metastore' = 'hive',",
-                                "  'uri' = '',",
-                                "  'warehouse' = '" + path + "',",
-                                "  'metastore.client.class' = '"
-                                        + AlterFailHiveMetaStoreClient.class.getName()
-                                        + "'",
-                                ")"))
-                .await();
-        tEnv.executeSql("USE CATALOG my_alter_hive").await();
-        tEnv.executeSql("CREATE TABLE alter_failed_table(a INT, b STRING)").await();
-
-        assertThatThrownBy(() -> tEnv.executeSql("ALTER TABLE alter_failed_table SET ('aa'='bb')"))
-                .isInstanceOf(TableException.class)
-                .hasMessage(
-                        String.format(
-                                "Could not execute "
-                                        + "ALTER TABLE my_alter_hive.default.alter_failed_table "
-                                        + "SET (aa: [bb], path: [%sdefault.db/alter_failed_table])",
-                                path));
-
-        assertTrue(
-                new SchemaManager(
-                                LocalFileIO.create(),
-                                new org.apache.paimon.fs.Path(
-                                        path, "default.db/alter_failed_table"))
-                        .latest()
-                        .get()
-                        .options()
-                        .isEmpty());
-    }
-
-    private List<Row> collect(String sql) throws Exception {
+    protected List<Row> collect(String sql) throws Exception {
         List<Row> result = new ArrayList<>();
         try (CloseableIterator<Row> it = tEnv.executeSql(sql).collect()) {
             while (it.hasNext()) {
diff --git a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/PaimonStorageHandlerITCase.java b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/PaimonStorageHandlerITCase.java
index c26524620..a7be90ebb 100644
--- a/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/PaimonStorageHandlerITCase.java
+++ b/paimon-hive/paimon-hive-connector-common/src/test/java/org/apache/paimon/hive/PaimonStorageHandlerITCase.java
@@ -92,6 +92,10 @@ public class PaimonStorageHandlerITCase {
             hiveShell.execute("SET tez.local.mode=true");
             hiveShell.execute("SET hive.jar.directory=" + folder.getRoot().getAbsolutePath());
             hiveShell.execute("SET tez.staging-dir=" + folder.getRoot().getAbsolutePath());
+            // JVM will crash if we do not set this and include paimon-flink-common as dependency
+            // not sure why
+            // in real use case there won't be any Flink dependency in Hive's classpath, so it's OK
+            hiveShell.execute("SET hive.tez.exec.inplace.progress=false");
         } else {
             throw new UnsupportedOperationException("Unsupported engine " + engine);
         }
diff --git a/paimon-shade/pom.xml b/paimon-shade/pom.xml
index 4659c7eae..1a6fd5e82 100644
--- a/paimon-shade/pom.xml
+++ b/paimon-shade/pom.xml
@@ -75,6 +75,20 @@ under the License.
             <version>${lz4.version}</version>
         </dependency>
 
+        <!-- paimon catalogs -->
+
+        <dependency>
+            <groupId>org.apache.paimon</groupId>
+            <artifactId>paimon-hive-catalog</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>*</groupId>
+                    <artifactId>*</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+
         <!-- other runtime dependencies -->
 
         <dependency>
@@ -122,6 +136,9 @@ under the License.
                                     <include>org.apache.paimon:paimon-format</include>
                                     <include>org.apache.paimon:paimon-codegen-loader</include>
 
+                                    <!-- Paimon catalogs -->
+                                    <include>org.apache.paimon:paimon-hive-catalog</include>
+
                                     <!-- Others, not shade to reduce conflicting -->
                                     <!-- <include>org.xerial.snappy:snappy-java</include> -->
                                     <!-- <include>org.lz4:lz4-java</include> -->
diff --git a/paimon-spark/paimon-spark-common/pom.xml b/paimon-spark/paimon-spark-common/pom.xml
index 11c73205c..9fba9e027 100644
--- a/paimon-spark/paimon-spark-common/pom.xml
+++ b/paimon-spark/paimon-spark-common/pom.xml
@@ -57,6 +57,27 @@ under the License.
                 </exclusion>
             </exclusions>
         </dependency>
+
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-hive_2.12</artifactId>
+            <version>${spark.version}</version>
+            <scope>test</scope>
+            <exclusions>
+                <exclusion>
+                    <groupId>log4j</groupId>
+                    <artifactId>log4j</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>slf4j-log4j12</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.orc</groupId>
+                    <artifactId>orc-core</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
     </dependencies>
 
     <build>