You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2023/01/23 04:11:49 UTC

[doris] branch branch-1.2-lts updated (96a4125a94 -> bfc414ee53)

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

morningman pushed a change to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git


    from 96a4125a94 [feature wip](multi catalog)Support iceberg schema evolution. (#15836)
     new 6b2c6460fb [Feature](ES): catalog support mapping es _id #15943
     new c9e68b7481 [Feature](multi-catalog)Add support for JuiceFS (#15969)
     new 1db3e975cd [fix](DOE) only return first batch data in ES 8.x (#16025)
     new a553bc6d7b [Fix][regression-test] Fix test_hdfs_tvf.groovy by update HDFS conf URI to uri and better error msg handling. (#16029)
     new 641c49d925 [fix](jdbc) fix jdbc driver bug and external datasource p2 test case issue (#16033)
     new e10346ba53 [fix](multi-catalog)switching catalogs after dropping will get NPE. (#16067)
     new 88096e0993 [fix](multi-catalog)Make ES catalog and resource compatible (#16096)
     new e35ef74f72 [fix](oracle catalog) oracle catalog support `TIMESTAMP` dateType of oracle (#16113)
     new c7d1535dba [feature](jdbc catalog) support SQLServer jdbc catalog (#16093)
     new c32a372b3c [Fix](Oracle External Table) fix that oracle external table can not insert batch values (#16117)
     new bfc414ee53 [refactor](fe) refactor and upgrade dependency tree of FE and support AWS glue catalog (#16046)

The 11 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/exec/es/es_scan_reader.cpp                  |    5 +-
 be/src/exec/table_connector.cpp                    |   41 +-
 be/src/exec/table_connector.h                      |    9 +
 be/src/util/s3_uri.cpp                             |    7 +
 be/src/util/s3_uri.h                               |    1 +
 be/src/vec/exec/scan/new_jdbc_scanner.cpp          |   15 +-
 be/src/vec/exec/scan/new_jdbc_scanner.h            |    2 +
 be/src/vec/exec/vjdbc_connector.cpp                |   25 +-
 be/src/vec/exec/vjdbc_connector.h                  |   16 +-
 bin/start_fe.sh                                    |    9 +
 .../docker-compose/oracle/init/03-create-table.sql |   10 +
 .../docker-compose/oracle/init/04-insert.sql       |    8 +
 .../sqlserver/init/01-drop-database.sql            |   18 +
 .../sqlserver/init/02-create-database.sql          |   18 +
 .../sqlserver/init/03-create-table.sql             |   89 +
 .../docker-compose/sqlserver/init/04-insert.sql    |   52 +
 .../docker-compose/sqlserver/sqlserver.env         |   19 +
 .../docker-compose/sqlserver/sqlserver.yaml        |   47 +
 docker/thirdparties/start-thirdparties-docker.sh   |    7 +
 docker/thirdparties/stop-thirdparties-docker.sh    |    3 +
 docs/en/docs/advanced/broker.md                    |   15 +-
 .../Create/CREATE-CATALOG.md                       |   24 +
 docs/zh-CN/docs/advanced/broker.md                 |   13 +
 .../Create/CREATE-CATALOG.md                       |   24 +
 fe/check/checkstyle/suppressions.xml               |    5 +
 .../java/org/apache/doris/sparkdpp}/DppResult.java |    4 +-
 .../org/apache/doris/sparkdpp}/EtlJobConfig.java   |    2 +-
 fe/fe-core/pom.xml                                 |   62 +-
 .../metastore/hive2/ProxyMetaStoreClient.java      | 2193 ++++++++++++
 .../converters/BaseCatalogToHiveConverter.java     |  541 +++
 .../catalog/converters/CatalogToHiveConverter.java |   58 +
 .../converters/CatalogToHiveConverterFactory.java  |   54 +
 .../glue/catalog/converters/ConverterUtils.java    |   49 +
 .../catalog/converters/GlueInputConverter.java     |  116 +
 .../converters/Hive3CatalogToHiveConverter.java    |   70 +
 .../catalog/converters/HiveToCatalogConverter.java |  372 ++
 .../catalog/converters/PartitionNameParser.java    |  143 +
 .../ConfigurationAWSCredentialsProvider.java       |   60 +
 ...onfigurationAWSCredentialsProviderFactory.java} |   14 +-
 .../exceptions/InvalidPartitionNameException.java} |   18 +-
 .../exceptions/LakeFormationException.java}        |   18 +-
 .../metastore/AWSCatalogMetastoreClient.java       | 2481 ++++++++++++++
 .../metastore/AWSCredentialsProviderFactory.java}  |   16 +-
 .../catalog/metastore/AWSGlueClientFactory.java    |  157 +
 .../catalog/metastore/AWSGlueDecoratorBase.java    | 1153 +++++++
 .../glue/catalog/metastore/AWSGlueMetastore.java   |  133 +
 .../metastore/AWSGlueMetastoreBaseDecorator.java   |  198 ++
 .../metastore/AWSGlueMetastoreCacheDecorator.java  |  185 +
 .../catalog/metastore/AWSGlueMetastoreFactory.java |   47 +
 .../metastore/AWSGlueMultipleCatalogDecorator.java |  370 ++
 .../DefaultAWSCredentialsProviderFactory.java}     |   22 +-
 .../catalog/metastore/DefaultAWSGlueMetastore.java |  662 ++++
 .../metastore/DefaultExecutorServiceFactory.java   |   43 +
 .../catalog/metastore/ExecutorServiceFactory.java} |   18 +-
 .../glue/catalog/metastore/GlueClientFactory.java} |   19 +-
 .../metastore/GlueMetastoreClientDelegate.java     | 1843 ++++++++++
 .../SessionCredentialsProviderFactory.java         |   56 +
 .../amazonaws/glue/catalog/util/AWSGlueConfig.java |   64 +
 .../catalog/util/BatchCreatePartitionsHelper.java  |  153 +
 .../catalog/util/BatchDeletePartitionsHelper.java  |  147 +
 .../glue/catalog/util/ExpressionHelper.java        |  242 ++
 .../glue/catalog/util/HiveTableValidator.java      |   86 +
 .../amazonaws/glue/catalog/util/LoggingHelper.java |   57 +
 .../glue/catalog/util/MetastoreClientUtils.java    |  141 +
 .../amazonaws/glue/catalog/util/PartitionKey.java  |   60 +
 .../glue/catalog/util/PartitionUtils.java          |   57 +
 .../java/org/apache/doris/analysis/BrokerDesc.java |    1 +
 .../java/org/apache/doris/analysis/ExportStmt.java |    5 +-
 .../org/apache/doris/analysis/StorageBackend.java  |    3 +
 .../java/org/apache/doris/backup/BlobStorage.java  |    6 +-
 .../main/java/org/apache/doris/catalog/Env.java    |    5 +-
 .../java/org/apache/doris/catalog/EsResource.java  |   20 +-
 .../java/org/apache/doris/catalog/EsTable.java     |    2 +-
 .../java/org/apache/doris/catalog/HMSResource.java |   54 +-
 .../doris/catalog/HiveMetaStoreClientHelper.java   |    3 +-
 .../doris/catalog/InternalSchemaInitializer.java   |   16 +-
 .../org/apache/doris/catalog/JdbcResource.java     |   10 +-
 .../java/org/apache/doris/catalog/OdbcTable.java   |   12 +-
 .../java/org/apache/doris/catalog/S3Resource.java  |   12 +-
 .../doris/catalog/external/HMSExternalTable.java   |    4 -
 .../java/org/apache/doris/common/FeConstants.java  |    1 +
 .../apache/doris/datasource/CatalogProperty.java   |    2 +
 .../apache/doris/datasource/EsExternalCatalog.java |   13 +-
 .../doris/datasource/HMSExternalCatalog.java       |    3 +-
 .../apache/doris/datasource/InternalCatalog.java   |    5 +-
 .../doris/datasource/hive/HiveMetaStoreClient.java | 2758 ---------------
 .../datasource/hive/PooledHiveMetaStoreClient.java |    5 +
 .../doris/external/elasticsearch/EsRestClient.java |    2 +
 .../doris/external/elasticsearch/EsUtil.java       |   14 +-
 .../org/apache/doris/external/jdbc/JdbcClient.java |   96 +-
 .../main/java/org/apache/doris/load/EtlStatus.java |    2 +-
 .../doris/load/loadv2/SparkEtlJobHandler.java      |    9 +-
 .../org/apache/doris/load/loadv2/SparkLoadJob.java |    4 +-
 .../doris/load/loadv2/SparkLoadPendingTask.java    |   22 +-
 .../org/apache/doris/planner/BrokerScanNode.java   |    8 +-
 .../org/apache/doris/planner/HiveScanNode.java     |    7 +-
 .../org/apache/doris/planner/HudiScanNode.java     |  344 --
 .../apache/doris/planner/PredicatePushDown.java    |    4 +-
 .../apache/doris/planner/SingleNodePlanner.java    |    5 +-
 .../doris/planner/external/HiveScanProvider.java   |    6 +-
 .../tablefunction/HdfsTableValuedFunction.java     |    3 +
 .../doris/tablefunction/S3TableValuedFunction.java |   13 +-
 .../hadoop/hive/metastore/HiveMetaStoreClient.java | 3552 ++++++++++++++++++++
 .../catalog/HiveMetaStoreClientHelperTest.java     |  275 --
 .../doris/load/loadv2/SparkEtlJobHandlerTest.java  |    2 +-
 .../apache/doris/load/loadv2/SparkLoadJobTest.java |    2 +-
 .../load/loadv2/SparkLoadPendingTaskTest.java      |   12 +-
 fe/hive-udf/pom.xml                                |   24 -
 fe/java-udf/pom.xml                                |    6 +
 .../java/org/apache/doris/udf/JdbcExecutor.java    |   20 +-
 fe/pom.xml                                         |   97 +-
 fe/spark-dpp/pom.xml                               |    1 -
 .../apache/doris/load/loadv2/dpp/ColumnParser.java |    2 +-
 .../load/loadv2/dpp/DorisRangePartitioner.java     |    2 +-
 .../org/apache/doris/load/loadv2/dpp/DppUtils.java |    2 +-
 .../dpp/MinimumCoverageRollupTreeBuilder.java      |    2 +-
 .../doris/load/loadv2/dpp/RollupTreeBuilder.java   |    2 +-
 .../doris/load/loadv2/dpp/RollupTreeNode.java      |    2 +-
 .../org/apache/doris/load/loadv2/dpp/SparkDpp.java |    3 +-
 .../doris/load/loadv2/dpp/SparkRDDAggregator.java  |    2 +-
 .../apache/doris/load/loadv2/etl/SparkEtlJob.java  |   11 +-
 .../doris/load/loadv2/dpp/ColumnParserTest.java    |    2 +-
 .../load/loadv2/dpp/DorisRangePartitionerTest.java |    2 +-
 .../apache/doris/load/loadv2/dpp/DppUtilsTest.java |    2 +-
 .../dpp/MinimumCoverageRollupTreeBuilderTest.java  |    2 +-
 .../apache/doris/load/loadv2/dpp/SparkDppTest.java |    2 +-
 .../doris/load/loadv2/etl/SparkEtlJobTest.java     |   17 +-
 .../doris/broker/hdfs/FileSystemManager.java       |  132 +-
 gensrc/thrift/Types.thrift                         |    1 +
 regression-test/conf/regression-conf.groovy        |    1 +
 .../jdbc_catalog_p0/test_oracle_jdbc_catalog.out   |   12 +-
 .../test_sqlserver_jdbc_catalog.out                |   43 +
 .../table_valued_function/test_hdfs_tvf.groovy     |    4 +-
 .../test_oracle_jdbc_catalog.groovy                |    1 +
 .../jdbc_catalog_p0/test_pg_jdbc_catalog.groovy    |   10 +-
 ...g.groovy => test_sqlserver_jdbc_catalog.groovy} |   58 +-
 136 files changed, 16719 insertions(+), 3734 deletions(-)
 create mode 100644 docker/thirdparties/docker-compose/sqlserver/init/01-drop-database.sql
 create mode 100644 docker/thirdparties/docker-compose/sqlserver/init/02-create-database.sql
 create mode 100644 docker/thirdparties/docker-compose/sqlserver/init/03-create-table.sql
 create mode 100644 docker/thirdparties/docker-compose/sqlserver/init/04-insert.sql
 create mode 100644 docker/thirdparties/docker-compose/sqlserver/sqlserver.env
 create mode 100644 docker/thirdparties/docker-compose/sqlserver/sqlserver.yaml
 rename fe/{spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp => fe-common/src/main/java/org/apache/doris/sparkdpp}/DppResult.java (96%)
 rename fe/{spark-dpp/src/main/java/org/apache/doris/load/loadv2/etl => fe-common/src/main/java/org/apache/doris/sparkdpp}/EtlJobConfig.java (99%)
 create mode 100644 fe/fe-core/src/main/java/com/aliyun/datalake/metastore/hive2/ProxyMetaStoreClient.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/BaseCatalogToHiveConverter.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverter.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverterFactory.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/ConverterUtils.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/GlueInputConverter.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/Hive3CatalogToHiveConverter.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/HiveToCatalogConverter.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/PartitionNameParser.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/credentials/ConfigurationAWSCredentialsProvider.java
 copy fe/{spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java => fe-core/src/main/java/com/amazonaws/glue/catalog/credentials/ConfigurationAWSCredentialsProviderFactory.java} (62%)
 copy fe/{spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java => fe-core/src/main/java/com/amazonaws/glue/catalog/exceptions/InvalidPartitionNameException.java} (65%)
 copy fe/{spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java => fe-core/src/main/java/com/amazonaws/glue/catalog/exceptions/LakeFormationException.java} (66%)
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCatalogMetastoreClient.java
 copy fe/{spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java => fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCredentialsProviderFactory.java} (67%)
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueClientFactory.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueDecoratorBase.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastore.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreBaseDecorator.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreCacheDecorator.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreFactory.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMultipleCatalogDecorator.java
 copy fe/{spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java => fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSCredentialsProviderFactory.java} (57%)
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSGlueMetastore.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultExecutorServiceFactory.java
 copy fe/{spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java => fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/ExecutorServiceFactory.java} (65%)
 copy fe/{spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java => fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/GlueClientFactory.java} (66%)
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/GlueMetastoreClientDelegate.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/SessionCredentialsProviderFactory.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/util/AWSGlueConfig.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/util/BatchCreatePartitionsHelper.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/util/BatchDeletePartitionsHelper.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/util/ExpressionHelper.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/util/HiveTableValidator.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/util/LoggingHelper.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/util/MetastoreClientUtils.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/util/PartitionKey.java
 create mode 100644 fe/fe-core/src/main/java/com/amazonaws/glue/catalog/util/PartitionUtils.java
 delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetaStoreClient.java
 delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/planner/HudiScanNode.java
 create mode 100644 fe/fe-core/src/main/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
 delete mode 100644 fe/fe-core/src/test/java/org/apache/doris/catalog/HiveMetaStoreClientHelperTest.java
 create mode 100644 regression-test/data/jdbc_catalog_p0/test_sqlserver_jdbc_catalog.out
 copy regression-test/suites/jdbc_catalog_p0/{test_oracle_jdbc_catalog.groovy => test_sqlserver_jdbc_catalog.groovy} (51%)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 05/11: [fix](jdbc) fix jdbc driver bug and external datasource p2 test case issue (#16033)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 641c49d925d7c6d06a72f5dd359e9b5f397b5f5f
Author: Mingyu Chen <mo...@163.com>
AuthorDate: Wed Jan 18 17:48:06 2023 +0800

    [fix](jdbc) fix jdbc driver bug and external datasource p2 test case issue (#16033)
    
    Fix bug that when create jdbc resource with only jdbc driver file name, it will failed to do checksum
    This is because we forgot the pass the full driver url to JdbcClient.
    
    Add ResultSet.FETCH_FORWARD and set AutoCommit to false to jdbc connection, so to avoid OOM when fetching large amount of data
    
    set useCursorFetch in jdbc url for both MySQL and PostgreSQL.
    
    Fix some p2 external datasource bug
---
 .../main/java/org/apache/doris/catalog/JdbcResource.java    |  8 +++++---
 .../java/org/apache/doris/external/jdbc/JdbcClient.java     | 13 ++-----------
 .../src/main/java/org/apache/doris/udf/JdbcExecutor.java    |  5 ++++-
 3 files changed, 11 insertions(+), 15 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java
index add3009de8..afdd099702 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java
@@ -189,10 +189,10 @@ public class JdbcResource extends Resource {
             // skip checking checksum when running ut
             return "";
         }
-        String fullDriverPath = getRealDriverPath(driverPath);
+        String fullDriverUrl = getFullDriverUrl(driverPath);
         InputStream inputStream = null;
         try {
-            inputStream = Util.getInputStreamFromUrl(fullDriverPath, null, HTTP_TIMEOUT_MS, HTTP_TIMEOUT_MS);
+            inputStream = Util.getInputStreamFromUrl(fullDriverUrl, null, HTTP_TIMEOUT_MS, HTTP_TIMEOUT_MS);
             MessageDigest digest = MessageDigest.getInstance("MD5");
             byte[] buf = new byte[4096];
             int bytesRead = 0;
@@ -213,7 +213,7 @@ public class JdbcResource extends Resource {
         }
     }
 
-    private static String getRealDriverPath(String driverUrl) {
+    public static String getFullDriverUrl(String driverUrl) {
         try {
             URI uri = new URI(driverUrl);
             String schema = uri.getScheme();
@@ -254,6 +254,8 @@ public class JdbcResource extends Resource {
             // it will convert to Doris tinyint, not bit.
             newJdbcUrl = checkJdbcUrlParam(newJdbcUrl, "yearIsDateType", "true", "false");
             newJdbcUrl = checkJdbcUrlParam(newJdbcUrl, "tinyInt1isBit", "true", "false");
+        }
+        if (dbType.equals(MYSQL) || dbType.equals(POSTGRESQL)) {
             newJdbcUrl = checkJdbcUrlParam(newJdbcUrl, "useCursorFetch", "false", "true");
         }
         return newJdbcUrl;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
index cc8fc80b3d..6f420246c5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
@@ -50,29 +50,20 @@ public class JdbcClient {
 
     private String dbType;
     private String jdbcUser;
-    private String jdbcPasswd;
-    private String jdbcUrl;
-    private String driverUrl;
-    private String driverClass;
 
     private URLClassLoader classLoader = null;
 
     private HikariDataSource dataSource = null;
 
-
     public JdbcClient(String user, String password, String jdbcUrl, String driverUrl, String driverClass) {
         this.jdbcUser = user;
-        this.jdbcPasswd = password;
-        this.jdbcUrl = jdbcUrl;
         this.dbType = parseDbType(jdbcUrl);
-        this.driverUrl = driverUrl;
-        this.driverClass = driverClass;
 
         ClassLoader oldClassLoader = Thread.currentThread().getContextClassLoader();
         try {
             // TODO(ftw): The problem here is that the jar package is handled by FE
             //  and URLClassLoader may load the jar package directly into memory
-            URL[] urls = {new URL(driverUrl)};
+            URL[] urls = {new URL(JdbcResource.getFullDriverUrl(driverUrl))};
             // set parent ClassLoader to null, we can achieve class loading isolation.
             classLoader = URLClassLoader.newInstance(urls, null);
             Thread.currentThread().setContextClassLoader(classLoader);
@@ -80,7 +71,7 @@ public class JdbcClient {
             config.setDriverClassName(driverClass);
             config.setJdbcUrl(jdbcUrl);
             config.setUsername(jdbcUser);
-            config.setPassword(jdbcPasswd);
+            config.setPassword(password);
             config.setMaximumPoolSize(1);
             dataSource = new HikariDataSource(config);
         } catch (MalformedURLException e) {
diff --git a/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java b/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
index f7273e3718..7f2366b948 100644
--- a/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
+++ b/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
@@ -249,8 +249,10 @@ public class JdbcExecutor {
             dataSource = new HikariDataSource(config);
             conn = dataSource.getConnection();
             if (op == TJdbcOperation.READ) {
+                conn.setAutoCommit(false);
                 Preconditions.checkArgument(sql != null);
-                stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
+                stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY,
+                        ResultSet.FETCH_FORWARD);
                 stmt.setFetchSize(batchSize);
             } else {
                 stmt = conn.createStatement();
@@ -264,3 +266,4 @@ public class JdbcExecutor {
         }
     }
 }
+


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 01/11: [Feature](ES): catalog support mapping es _id #15943

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 6b2c6460fbe44d9aae0e283b26cf9c1ec7f85f81
Author: Stalary <st...@163.com>
AuthorDate: Sat Jan 21 08:08:32 2023 +0800

    [Feature](ES): catalog support mapping es _id #15943
---
 .../src/main/java/org/apache/doris/catalog/EsResource.java |  5 +++++
 .../src/main/java/org/apache/doris/catalog/EsTable.java    |  2 +-
 .../org/apache/doris/datasource/EsExternalCatalog.java     | 13 ++++++-------
 .../org/apache/doris/external/elasticsearch/EsUtil.java    | 14 +++++++++++++-
 4 files changed, 25 insertions(+), 9 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EsResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/EsResource.java
index 131ca70e09..513093d0ae 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EsResource.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EsResource.java
@@ -59,12 +59,14 @@ public class EsResource extends Resource {
     public static final String MAX_DOCVALUE_FIELDS = "max_docvalue_fields";
     public static final String NODES_DISCOVERY = "nodes_discovery";
     public static final String HTTP_SSL_ENABLED = "http_ssl_enabled";
+    public static final String MAPPING_ES_ID = "mapping_es_id";
     public static final String QUERY_DSL = "query_dsl";
 
     public static final String DOC_VALUE_SCAN_DEFAULT_VALUE = "true";
     public static final String KEYWORD_SNIFF_DEFAULT_VALUE = "true";
     public static final String HTTP_SSL_ENABLED_DEFAULT_VALUE = "false";
     public static final String NODES_DISCOVERY_DEFAULT_VALUE = "true";
+    public static final String MAPPING_ES_ID_DEFAULT_VALUE = "false";
     @SerializedName(value = "properties")
     private Map<String, String> properties;
 
@@ -122,6 +124,9 @@ public class EsResource extends Resource {
         if (properties.containsKey(EsResource.NODES_DISCOVERY)) {
             EsUtil.getBoolean(properties, EsResource.NODES_DISCOVERY);
         }
+        if (properties.containsKey(EsResource.MAPPING_ES_ID)) {
+            EsUtil.getBoolean(properties, EsResource.MAPPING_ES_ID);
+        }
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EsTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/EsTable.java
index 0c89a3b0d6..a26a7db7f9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EsTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EsTable.java
@@ -306,6 +306,6 @@ public class EsTable extends Table {
     }
 
     public List<Column> genColumnsFromEs() {
-        return EsUtil.genColumnsFromEs(client, indexName, mappingType);
+        return EsUtil.genColumnsFromEs(client, indexName, mappingType, false);
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/EsExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/EsExternalCatalog.java
index f3441e0c9f..9b219ea6a2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/EsExternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/EsExternalCatalog.java
@@ -67,12 +67,6 @@ public class EsExternalCatalog extends ExternalCatalog {
         if (properties.containsKey("username")) {
             properties.put(EsResource.USER, properties.remove("username"));
         }
-        if (properties.containsKey("doc_value_scan")) {
-            properties.put(EsResource.DOC_VALUE_SCAN, properties.remove("doc_value_scan"));
-        }
-        if (properties.containsKey("keyword_sniff")) {
-            properties.put(EsResource.KEYWORD_SNIFF, properties.remove("keyword_sniff"));
-        }
         return properties;
     }
 
@@ -109,6 +103,11 @@ public class EsExternalCatalog extends ExternalCatalog {
                 EsResource.NODES_DISCOVERY_DEFAULT_VALUE));
     }
 
+    public boolean enableMappingEsId() {
+        return Boolean.parseBoolean(catalogProperty.getOrDefault(EsResource.MAPPING_ES_ID,
+                EsResource.MAPPING_ES_ID_DEFAULT_VALUE));
+    }
+
     @Override
     protected void initLocalObjectsImpl() {
         esRestClient = new EsRestClient(getNodes(), getUsername(), getPassword(), enableSsl());
@@ -161,6 +160,6 @@ public class EsExternalCatalog extends ExternalCatalog {
     @Override
     public List<Column> getSchema(String dbName, String tblName) {
         makeSureInitialized();
-        return EsUtil.genColumnsFromEs(getEsRestClient(), tblName, null);
+        return EsUtil.genColumnsFromEs(getEsRestClient(), tblName, null, enableMappingEsId());
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsUtil.java b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsUtil.java
index 0cde72067d..10c9a675c2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsUtil.java
@@ -189,6 +189,7 @@ public class EsUtil {
 
     /**
      * Remove `dynamic_templates` and check explicit mapping
+     *
      * @param mappings
      */
     private static void checkDynamicTemplates(JSONObject mappings) {
@@ -376,13 +377,24 @@ public class EsUtil {
 
     /**
      * Generate columns from ES Cluster.
+     * Add mappingEsId config in es external catalog.
      **/
-    public static List<Column> genColumnsFromEs(EsRestClient client, String indexName, String mappingType) {
+    public static List<Column> genColumnsFromEs(EsRestClient client, String indexName, String mappingType,
+            boolean mappingEsId) {
         String mapping = client.getMapping(indexName);
         JSONObject mappingProps = getMappingProps(indexName, mapping, mappingType);
         List<String> arrayFields = getArrayFields(mapping);
         Set<String> keys = (Set<String>) mappingProps.keySet();
         List<Column> columns = new ArrayList<>();
+        if (mappingEsId) {
+            Column column = new Column();
+            column.setName("_id");
+            column.setIsKey(true);
+            column.setType(ScalarType.createVarcharType(255));
+            column.setIsAllowNull(true);
+            column.setUniqueId(-1);
+            columns.add(column);
+        }
         for (String key : keys) {
             JSONObject field = (JSONObject) mappingProps.get(key);
             Type type;


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 04/11: [Fix][regression-test] Fix test_hdfs_tvf.groovy by update HDFS conf URI to uri and better error msg handling. (#16029)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit a553bc6d7bb4946ed6e1a2fa2fe5c583857bb6c3
Author: Qi Chen <ka...@gmail.com>
AuthorDate: Fri Jan 20 12:40:25 2023 +0800

    [Fix][regression-test] Fix test_hdfs_tvf.groovy by update HDFS conf URI to uri and better error msg handling. (#16029)
    
    Fix test_hdfs_tvf.groovy by update HDFS conf URI to uri and better error msg handling.
    test_hdfs_tvf.groovy didn't passed.
---
 .../java/org/apache/doris/tablefunction/HdfsTableValuedFunction.java  | 3 +++
 .../suites/correctness_p0/table_valued_function/test_hdfs_tvf.groovy  | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HdfsTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HdfsTableValuedFunction.java
index 542fc46bee..9c845103f9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HdfsTableValuedFunction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/HdfsTableValuedFunction.java
@@ -80,6 +80,9 @@ public class HdfsTableValuedFunction extends ExternalFileTableValuedFunction {
             }
         }
 
+        if (!locationProperties.containsKey(HDFS_URI)) {
+            throw new AnalysisException(String.format("Configuration '%s' is required.", HDFS_URI));
+        }
         ExportStmt.checkPath(locationProperties.get(HDFS_URI), StorageType.HDFS);
         hdfsUri = URI.create(locationProperties.get(HDFS_URI));
         filePath = locationProperties.get(HADOOP_FS_NAME) + hdfsUri.getPath();
diff --git a/regression-test/suites/correctness_p0/table_valued_function/test_hdfs_tvf.groovy b/regression-test/suites/correctness_p0/table_valued_function/test_hdfs_tvf.groovy
index 4c4f940026..5a283eafa2 100644
--- a/regression-test/suites/correctness_p0/table_valued_function/test_hdfs_tvf.groovy
+++ b/regression-test/suites/correctness_p0/table_valued_function/test_hdfs_tvf.groovy
@@ -48,7 +48,7 @@ suite("test_hdfs_tvf") {
             uri = "${defaultFS}" + "/user/doris/preinstalled_data/csv_format_test/array_malformat.csv"
             format = "csv"
             qt_csv_array_malformat """ select * from HDFS(
-                                        "URI" = "${uri}",
+                                        "uri" = "${uri}",
                                         "fs.defaultFS"= "${defaultFS}",
                                         "hadoop.username" = "${hdfsUserName}",
                                         "format" = "csv",
@@ -56,7 +56,7 @@ suite("test_hdfs_tvf") {
 
             uri = "${defaultFS}" + "/user/doris/preinstalled_data/csv_format_test/array_normal.csv"
             format = "csv"
-            qt_csv_array_normal """ select * from HDFS("URI" = "${uri}",
+            qt_csv_array_normal """ select * from HDFS("uri" = "${uri}",
                                     "fs.defaultFS"= "${defaultFS}",
                                     "hadoop.username" = "${hdfsUserName}",
                                     "format" = "csv",


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 06/11: [fix](multi-catalog)switching catalogs after dropping will get NPE. (#16067)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit e10346ba53088f52575da4d01c0d04d033a03a68
Author: qiye <ji...@gmail.com>
AuthorDate: Thu Jan 19 15:13:21 2023 +0800

    [fix](multi-catalog)switching catalogs after dropping will get NPE. (#16067)
    
    Issue Number: close #16066
---
 fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
index 3194f5c350..2cfd39f70a 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
@@ -4447,7 +4447,10 @@ public class Env {
 
         String currentDB = ctx.getDatabase();
         if (StringUtils.isNotEmpty(currentDB)) {
-            catalogMgr.addLastDBOfCatalog(ctx.getCurrentCatalog().getName(), currentDB);
+            // When dropped the current catalog in current context, the current catalog will be null.
+            if (ctx.getCurrentCatalog() != null) {
+                catalogMgr.addLastDBOfCatalog(ctx.getCurrentCatalog().getName(), currentDB);
+            }
         }
         ctx.changeDefaultCatalog(catalogName);
         String lastDb = catalogMgr.getLastDB(catalogName);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 11/11: [refactor](fe) refactor and upgrade dependency tree of FE and support AWS glue catalog (#16046)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit bfc414ee538351473c114ff470b2bc49587fb417
Author: Mingyu Chen <mo...@163.com>
AuthorDate: Fri Jan 20 14:42:16 2023 +0800

    [refactor](fe) refactor and upgrade dependency tree of FE and support AWS glue catalog (#16046)
    
    1. Spark dpp
    
            Move `DppResult` and `EtlJobConfig` to sparkdpp package in `fe-common` module.
            So taht `fe-core` is longer depends on `spark-dpp` module, so that the `spark-dpp.jar`
            will not be moved into `fe/lib`, which reduce the size of FE output.
    
    2. Modify start_fe.sh
    
            Modify the CLASSPATH to make sure that doris-fe.jar is at front, so that
            when loading classes with same qualified name, it will be got from doris-fe.jar firstly.
    
    3. Upgrade hadoop and hive version
    
            hadoop: 2.10.2 -> 3.3.3
            hive: 2.3.7 -> 3.1.3
    
    4. Override the IHiveMetastoreClient implementations from dependency
    
            `ProxyMetaStoreClient.java` for Aliyun DLF.
            `HiveMetaStoreClient.java` for origin Apache Hive metastore.
    
            Because I need to modified some of their method to make them compatible with
            different version of Hive.
    
    5. Exclude some unused dependencies to reduce the size of FE output
    
            Now it is only 370MB (Before is 600MB)
    
    6. Upgrade aws-java-sdk version to 1.12.31
    
    7. Support AWS Glue Data Catalog
    
    8. Remove HudiScanNode(no longer support)
---
 be/src/util/s3_uri.cpp                             |    7 +
 be/src/util/s3_uri.h                               |    1 +
 bin/start_fe.sh                                    |    9 +
 fe/check/checkstyle/suppressions.xml               |    5 +
 .../java/org/apache/doris/sparkdpp}/DppResult.java |    4 +-
 .../org/apache/doris/sparkdpp}/EtlJobConfig.java   |    2 +-
 fe/fe-core/pom.xml                                 |   62 +-
 .../metastore/hive2/ProxyMetaStoreClient.java      | 2193 ++++++++++++
 .../converters/BaseCatalogToHiveConverter.java     |  541 +++
 .../catalog/converters/CatalogToHiveConverter.java |   58 +
 .../converters/CatalogToHiveConverterFactory.java  |   54 +
 .../glue/catalog/converters/ConverterUtils.java    |   49 +
 .../catalog/converters/GlueInputConverter.java     |  116 +
 .../converters/Hive3CatalogToHiveConverter.java    |   70 +
 .../catalog/converters/HiveToCatalogConverter.java |  372 ++
 .../catalog/converters/PartitionNameParser.java    |  143 +
 .../ConfigurationAWSCredentialsProvider.java       |   60 +
 ...onfigurationAWSCredentialsProviderFactory.java} |   14 +-
 .../exceptions/InvalidPartitionNameException.java} |   18 +-
 .../exceptions/LakeFormationException.java}        |   18 +-
 .../metastore/AWSCatalogMetastoreClient.java       | 2481 ++++++++++++++
 .../metastore/AWSCredentialsProviderFactory.java}  |   16 +-
 .../catalog/metastore/AWSGlueClientFactory.java    |  157 +
 .../catalog/metastore/AWSGlueDecoratorBase.java    | 1153 +++++++
 .../glue/catalog/metastore/AWSGlueMetastore.java   |  133 +
 .../metastore/AWSGlueMetastoreBaseDecorator.java   |  198 ++
 .../metastore/AWSGlueMetastoreCacheDecorator.java  |  185 +
 .../catalog/metastore/AWSGlueMetastoreFactory.java |   47 +
 .../metastore/AWSGlueMultipleCatalogDecorator.java |  370 ++
 .../DefaultAWSCredentialsProviderFactory.java}     |   22 +-
 .../catalog/metastore/DefaultAWSGlueMetastore.java |  662 ++++
 .../metastore/DefaultExecutorServiceFactory.java   |   43 +
 .../catalog/metastore/ExecutorServiceFactory.java} |   18 +-
 .../glue/catalog/metastore/GlueClientFactory.java} |   19 +-
 .../metastore/GlueMetastoreClientDelegate.java     | 1843 ++++++++++
 .../SessionCredentialsProviderFactory.java         |   56 +
 .../amazonaws/glue/catalog/util/AWSGlueConfig.java |   64 +
 .../catalog/util/BatchCreatePartitionsHelper.java  |  153 +
 .../catalog/util/BatchDeletePartitionsHelper.java  |  147 +
 .../glue/catalog/util/ExpressionHelper.java        |  242 ++
 .../glue/catalog/util/HiveTableValidator.java      |   86 +
 .../amazonaws/glue/catalog/util/LoggingHelper.java |   57 +
 .../glue/catalog/util/MetastoreClientUtils.java    |  141 +
 .../amazonaws/glue/catalog/util/PartitionKey.java  |   60 +
 .../glue/catalog/util/PartitionUtils.java          |   57 +
 .../java/org/apache/doris/catalog/HMSResource.java |   54 +-
 .../doris/catalog/InternalSchemaInitializer.java   |   16 +-
 .../java/org/apache/doris/catalog/S3Resource.java  |   12 +-
 .../doris/catalog/external/HMSExternalTable.java   |    4 -
 .../apache/doris/datasource/CatalogProperty.java   |    2 +
 .../doris/datasource/HMSExternalCatalog.java       |    3 +-
 .../apache/doris/datasource/InternalCatalog.java   |    5 +-
 .../doris/datasource/hive/HiveMetaStoreClient.java | 2758 ---------------
 .../datasource/hive/PooledHiveMetaStoreClient.java |    5 +
 .../main/java/org/apache/doris/load/EtlStatus.java |    2 +-
 .../doris/load/loadv2/SparkEtlJobHandler.java      |    9 +-
 .../org/apache/doris/load/loadv2/SparkLoadJob.java |    4 +-
 .../doris/load/loadv2/SparkLoadPendingTask.java    |   22 +-
 .../org/apache/doris/planner/BrokerScanNode.java   |    5 -
 .../org/apache/doris/planner/HiveScanNode.java     |    5 +-
 .../org/apache/doris/planner/HudiScanNode.java     |  344 --
 .../apache/doris/planner/PredicatePushDown.java    |    4 +-
 .../apache/doris/planner/SingleNodePlanner.java    |    5 +-
 .../doris/planner/external/HiveScanProvider.java   |    2 +-
 .../doris/tablefunction/S3TableValuedFunction.java |   13 +-
 .../hadoop/hive/metastore/HiveMetaStoreClient.java | 3552 ++++++++++++++++++++
 .../catalog/HiveMetaStoreClientHelperTest.java     |  275 --
 .../doris/load/loadv2/SparkEtlJobHandlerTest.java  |    2 +-
 .../apache/doris/load/loadv2/SparkLoadJobTest.java |    2 +-
 .../load/loadv2/SparkLoadPendingTaskTest.java      |   12 +-
 fe/hive-udf/pom.xml                                |   24 -
 fe/pom.xml                                         |   97 +-
 fe/spark-dpp/pom.xml                               |    1 -
 .../apache/doris/load/loadv2/dpp/ColumnParser.java |    2 +-
 .../load/loadv2/dpp/DorisRangePartitioner.java     |    2 +-
 .../org/apache/doris/load/loadv2/dpp/DppUtils.java |    2 +-
 .../dpp/MinimumCoverageRollupTreeBuilder.java      |    2 +-
 .../doris/load/loadv2/dpp/RollupTreeBuilder.java   |    2 +-
 .../doris/load/loadv2/dpp/RollupTreeNode.java      |    2 +-
 .../org/apache/doris/load/loadv2/dpp/SparkDpp.java |    3 +-
 .../doris/load/loadv2/dpp/SparkRDDAggregator.java  |    2 +-
 .../apache/doris/load/loadv2/etl/SparkEtlJob.java  |   11 +-
 .../doris/load/loadv2/dpp/ColumnParserTest.java    |    2 +-
 .../load/loadv2/dpp/DorisRangePartitionerTest.java |    2 +-
 .../apache/doris/load/loadv2/dpp/DppUtilsTest.java |    2 +-
 .../dpp/MinimumCoverageRollupTreeBuilderTest.java  |    2 +-
 .../apache/doris/load/loadv2/dpp/SparkDppTest.java |    2 +-
 .../doris/load/loadv2/etl/SparkEtlJobTest.java     |   17 +-
 88 files changed, 15888 insertions(+), 3612 deletions(-)

diff --git a/be/src/util/s3_uri.cpp b/be/src/util/s3_uri.cpp
index 882ca021a7..36ceec0eae 100644
--- a/be/src/util/s3_uri.cpp
+++ b/be/src/util/s3_uri.cpp
@@ -64,4 +64,11 @@ bool S3URI::parse() {
     return true;
 }
 
+std::string S3URI::to_string() const {
+    std::stringstream ss;
+    ss << "location: " << _location << ", bucket: " << _bucket << ", key: " << _key
+       << ", schema: " << _scheme;
+    return ss.str();
+}
+
 } // end namespace doris
diff --git a/be/src/util/s3_uri.h b/be/src/util/s3_uri.h
index dc6a07fd23..46e61b3282 100644
--- a/be/src/util/s3_uri.h
+++ b/be/src/util/s3_uri.h
@@ -31,6 +31,7 @@ public:
     const std::string& get_key() const { return _key; }
     const std::string& get_location() const { return _location; }
     const std::string& get_scheme() const { return _scheme; }
+    std::string to_string() const;
 
 private:
     static const std::string _SCHEME_DELIM;
diff --git a/bin/start_fe.sh b/bin/start_fe.sh
index fc134f2cac..fc7ec47e39 100755
--- a/bin/start_fe.sh
+++ b/bin/start_fe.sh
@@ -167,9 +167,18 @@ echo "using java version ${java_version}" >>"${LOG_DIR}/fe.out"
 echo "${final_java_opt}" >>"${LOG_DIR}/fe.out"
 
 # add libs to CLASSPATH
+DORIS_FE_JAR=
 for f in "${DORIS_HOME}/lib"/*.jar; do
+    if [[ "${f}" == *"doris-fe.jar" ]]; then
+        DORIS_FE_JAR="${f}"
+        continue
+    fi
     CLASSPATH="${f}:${CLASSPATH}"
 done
+
+# make sure the doris-fe.jar is at first order, so that some classed
+# with same qualified name can be loaded priority from doris-fe.jar
+CLASSPATH="${DORIS_FE_JAR}:${CLASSPATH}"
 export CLASSPATH="${CLASSPATH}:${DORIS_HOME}/lib:${DORIS_HOME}/conf"
 
 pidfile="${PID_DIR}/fe.pid"
diff --git a/fe/check/checkstyle/suppressions.xml b/fe/check/checkstyle/suppressions.xml
index 55db90b6ee..a95bae1b30 100644
--- a/fe/check/checkstyle/suppressions.xml
+++ b/fe/check/checkstyle/suppressions.xml
@@ -58,4 +58,9 @@ under the License.
     <suppress files="org[\\/]apache[\\/]doris[\\/]common[\\/]ErrorCode\.java" checks="LineLength" />
     <suppress files="org[\\/]apache[\\/]doris[\\/]udf[\\/]UdafExecutor\.java" checks="NoFinalizer" />
     <suppress files="org[\\/]apache[\\/]doris[\\/]udf[\\/]UdfExecutor\.java" checks="NoFinalizer" />
+
+    <!-- ignore hive meta store client copied from hive/aliyun/aws -->
+    <suppress files="HiveMetaStoreClient\.java" checks="[a-zA-Z0-9]*"/>
+    <suppress files="[\\/]com[\\/]amazonaws[\\/]glue[\\/]catalog[\\/]" checks="[a-zA-Z0-9]*"/>
+    <suppress files="[\\/]com[\\/]aliyun[\\/]datalake[\\/]metastore[\\/]hive2[\\/]" checks="[a-zA-Z0-9]*"/>
 </suppressions>
diff --git a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/DppResult.java b/fe/fe-common/src/main/java/org/apache/doris/sparkdpp/DppResult.java
similarity index 96%
rename from fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/DppResult.java
rename to fe/fe-common/src/main/java/org/apache/doris/sparkdpp/DppResult.java
index 63e307f890..a388532776 100644
--- a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/DppResult.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/sparkdpp/DppResult.java
@@ -15,14 +15,14 @@
 // specific language governing permissions and limitations
 // under the License.
 
-package org.apache.doris.load.loadv2.dpp;
+package org.apache.doris.sparkdpp;
 
 import com.google.gson.annotations.SerializedName;
 
 import java.io.Serializable;
 
 public class DppResult implements Serializable {
-    DppResult() {
+    public DppResult() {
         isSuccess = true;
         failedReason = "";
         scannedRows = 0;
diff --git a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/etl/EtlJobConfig.java b/fe/fe-common/src/main/java/org/apache/doris/sparkdpp/EtlJobConfig.java
similarity index 99%
rename from fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/etl/EtlJobConfig.java
rename to fe/fe-common/src/main/java/org/apache/doris/sparkdpp/EtlJobConfig.java
index 7c53c1c7ab..c59901d383 100644
--- a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/etl/EtlJobConfig.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/sparkdpp/EtlJobConfig.java
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-package org.apache.doris.load.loadv2.etl;
+package org.apache.doris.sparkdpp;
 
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableMap;
diff --git a/fe/fe-core/pom.xml b/fe/fe-core/pom.xml
index b26e2c132e..094bbbb94a 100644
--- a/fe/fe-core/pom.xml
+++ b/fe/fe-core/pom.xml
@@ -96,17 +96,6 @@ under the License.
             <artifactId>fe-common</artifactId>
             <version>${project.version}</version>
         </dependency>
-        <dependency>
-            <groupId>${project.groupId}</groupId>
-            <artifactId>spark-dpp</artifactId>
-            <version>${project.version}</version>
-            <exclusions>
-                <exclusion>
-                    <artifactId>netty-all</artifactId>
-                    <groupId>io.netty</groupId>
-                </exclusion>
-            </exclusions>
-        </dependency>
         <dependency>
             <groupId>org.apache.logging.log4j</groupId>
             <artifactId>log4j-web</artifactId>
@@ -211,22 +200,6 @@ under the License.
             <groupId>org.apache.doris</groupId>
             <artifactId>je</artifactId>
         </dependency>
-        <!-- https://mvnrepository.com/artifact/org.mortbay.jetty/jetty -->
-        <dependency>
-            <groupId>org.mortbay.jetty</groupId>
-            <artifactId>jetty</artifactId>
-            <exclusions>
-                <exclusion>
-                    <artifactId>servlet-api-2.5</artifactId>
-                    <groupId>org.mortbay.jetty</groupId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <!-- https://mvnrepository.com/artifact/org.mortbay.jetty/jetty-util -->
-        <dependency>
-            <groupId>org.mortbay.jetty</groupId>
-            <artifactId>jetty-util</artifactId>
-        </dependency>
         <!-- https://mvnrepository.com/artifact/de.jflex/jflex -->
         <dependency>
             <groupId>de.jflex</groupId>
@@ -507,6 +480,14 @@ under the License.
             <groupId>com.amazonaws</groupId>
             <artifactId>aws-java-sdk-s3</artifactId>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-glue</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-dynamodb</artifactId>
+        </dependency>
         <!-- https://mvnrepository.com/artifact/org.springframework.boot/spring-boot-starter-web -->
         <dependency>
             <groupId>org.springframework.boot</groupId>
@@ -630,7 +611,6 @@ under the License.
         <dependency>
             <groupId>io.netty</groupId>
             <artifactId>netty-all</artifactId>
-            <version>${netty.version}</version>
         </dependency>
         <dependency>
             <groupId>org.apache.hive</groupId>
@@ -664,30 +644,6 @@ under the License.
                 </exclusion>
             </exclusions>
         </dependency>
-        <!-- https://mvnrepository.com/artifact/org.apache.calcite.avatica/avatica -->
-        <dependency>
-            <groupId>org.apache.calcite.avatica</groupId>
-            <artifactId>avatica</artifactId>
-            <version>${avatica.version}</version>
-        </dependency>
-        <!-- https://mvnrepository.com/artifact/org.apache.calcite/calcite-core -->
-        <dependency>
-            <groupId>org.apache.calcite</groupId>
-            <artifactId>calcite-core</artifactId>
-            <version>${calcite.version}</version>
-        </dependency>
-        <!-- https://mvnrepository.com/artifact/org.apache.calcite/calcite-linq4j -->
-        <dependency>
-            <groupId>org.apache.calcite</groupId>
-            <artifactId>calcite-linq4j</artifactId>
-            <version>${calcite.version}</version>
-        </dependency>
-        <!-- https://mvnrepository.com/artifact/org.apache.calcite/calcite-druid -->
-        <dependency>
-            <groupId>org.apache.calcite</groupId>
-            <artifactId>calcite-druid</artifactId>
-            <version>${calcite.version}</version>
-        </dependency>
         <dependency>
            <groupId>org.apache.httpcomponents</groupId>
            <artifactId>httpclient</artifactId>
@@ -819,7 +775,7 @@ under the License.
         <!-- for aliyun dlf -->
         <dependency>
             <groupId>com.aliyun.datalake</groupId>
-            <artifactId>metastore-client-hive2</artifactId>
+            <artifactId>metastore-client-hive3</artifactId>
             <exclusions>
                 <exclusion>
                     <groupId>com.aliyun</groupId>
diff --git a/fe/fe-core/src/main/java/com/aliyun/datalake/metastore/hive2/ProxyMetaStoreClient.java b/fe/fe-core/src/main/java/com/aliyun/datalake/metastore/hive2/ProxyMetaStoreClient.java
new file mode 100644
index 0000000000..8ac9f33ee6
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/aliyun/datalake/metastore/hive2/ProxyMetaStoreClient.java
@@ -0,0 +1,2193 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from:
+// https://github.com/aliyun/datalake-catalog-metastore-client/blob/master/metastore-client-hive/metastore-client-hive2/src/main/java/com/aliyun/datalake/metastore/hive2/ProxyMetaStoreClient.java
+// 3c6f5905
+
+package com.aliyun.datalake.metastore.hive2;
+
+import com.aliyun.datalake.metastore.common.DataLakeConfig;
+import com.aliyun.datalake.metastore.common.ProxyMode;
+import com.aliyun.datalake.metastore.common.Version;
+import com.aliyun.datalake.metastore.common.functional.FunctionalUtils;
+import com.aliyun.datalake.metastore.common.functional.ThrowingConsumer;
+import com.aliyun.datalake.metastore.common.functional.ThrowingFunction;
+import com.aliyun.datalake.metastore.common.functional.ThrowingRunnable;
+import com.aliyun.datalake.metastore.common.util.DataLakeUtil;
+import com.aliyun.datalake.metastore.common.util.ProxyLogUtils;
+import com.aliyun.datalake.metastore.hive.common.utils.ClientUtils;
+import com.aliyun.datalake.metastore.hive.common.utils.ConfigUtils;
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.common.ValidWriteIdList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.PartitionDropOptions;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.CheckConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.CmRecycleRequest;
+import org.apache.hadoop.hive.metastore.api.CmRecycleResponse;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DefaultConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.FindSchemasByColsResp;
+import org.apache.hadoop.hive.metastore.api.FindSchemasByColsRqst;
+import org.apache.hadoop.hive.metastore.api.FireEventRequest;
+import org.apache.hadoop.hive.metastore.api.FireEventResponse;
+import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
+import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
+import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest;
+import org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
+import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
+import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+import org.apache.hadoop.hive.metastore.api.ISchema;
+import org.apache.hadoop.hive.metastore.api.InvalidInputException;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.Materialization;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
+import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest;
+import org.apache.hadoop.hive.metastore.api.NotificationEventsCountResponse;
+import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+import org.apache.hadoop.hive.metastore.api.PartitionValuesRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
+import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+import org.apache.hadoop.hive.metastore.api.Role;
+import org.apache.hadoop.hive.metastore.api.RuntimeStat;
+import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
+import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMMapping;
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.hadoop.hive.metastore.utils.ObjectPair;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class ProxyMetaStoreClient implements IMetaStoreClient {
+    private static final Logger logger = LoggerFactory.getLogger(ProxyMetaStoreClient.class);
+    private final static String HIVE_FACTORY_CLASS = "org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClientFactory";
+
+    private final ProxyMode proxyMode;
+
+    // Dlf Client
+    private IMetaStoreClient dlfSessionMetaStoreClient;
+
+    // Hive Client
+    private IMetaStoreClient hiveSessionMetaStoreClient;
+
+    // ReadWrite Client
+    private IMetaStoreClient readWriteClient;
+
+    // Extra Write Client
+    private Optional<IMetaStoreClient> extraClient;
+
+    // Allow failure
+    private boolean allowFailure = false;
+
+    // copy Hive conf
+    private HiveConf hiveConf;
+
+    private final String readWriteClientType;
+
+    public ProxyMetaStoreClient(HiveConf hiveConf) throws MetaException {
+        this(hiveConf, null, false);
+    }
+
+    // morningman: add this constructor to avoid NoSuchMethod exception
+    public ProxyMetaStoreClient(Configuration conf, HiveMetaHookLoader hiveMetaHookLoader, Boolean allowEmbedded)
+            throws MetaException {
+        this((HiveConf) conf, hiveMetaHookLoader, allowEmbedded);
+    }
+
+    public ProxyMetaStoreClient(HiveConf hiveConf, HiveMetaHookLoader hiveMetaHookLoader, Boolean allowEmbedded)
+            throws MetaException {
+        long startTime = System.currentTimeMillis();
+        logger.info("ProxyMetaStoreClient start, datalake-metastore-client-version:{}",
+                Version.DATALAKE_METASTORE_CLIENT_VERSION);
+        this.hiveConf = new HiveConf(hiveConf);
+
+        proxyMode = ConfigUtils.getProxyMode(hiveConf);
+
+        // init logging if needed
+        ProxyLogUtils.initLogUtils(proxyMode, hiveConf.get(DataLakeConfig.CATALOG_PROXY_LOGSTORE,
+                        ConfigUtils.getUserId(hiveConf)), hiveConf.getBoolean(DataLakeConfig.CATALOG_ACTION_LOG_ENABLED,
+                        DataLakeConfig.DEFAULT_CATALOG_ACTION_LOG_ENABLED),
+                hiveConf.getBoolean(DataLakeConfig.CATALOG_LOG_ENABLED, DataLakeConfig.DEFAULT_CATALOG_LOG_ENABLED));
+
+        // init Dlf Client if any
+        createClient(true, () -> initDlfClient(hiveConf, hiveMetaHookLoader, allowEmbedded, new ConcurrentHashMap<>()));
+
+        // init Hive Client if any
+        createClient(false, () -> initHiveClient(hiveConf, hiveMetaHookLoader, allowEmbedded, new ConcurrentHashMap<>()));
+
+        // init extraClient
+        initClientByProxyMode();
+
+        readWriteClientType = this.readWriteClient instanceof DlfSessionMetaStoreClient ? "dlf" : "hive";
+
+        logger.info("ProxyMetaStoreClient end, cost:{}ms", System.currentTimeMillis() - startTime);
+    }
+
+    public static Map<String, org.apache.hadoop.hive.ql.metadata.Table> getTempTablesForDatabase(String dbName) {
+        return getTempTables().get(dbName);
+    }
+
+    public static Map<String, Map<String, org.apache.hadoop.hive.ql.metadata.Table>> getTempTables() {
+        SessionState ss = SessionState.get();
+        if (ss == null) {
+            return Collections.emptyMap();
+        }
+        return ss.getTempTables();
+    }
+
+    public HiveConf getHiveConf() {
+        return hiveConf;
+    }
+
+    public void initClientByProxyMode() throws MetaException {
+        switch (proxyMode) {
+            case METASTORE_ONLY:
+                this.readWriteClient = hiveSessionMetaStoreClient;
+                this.extraClient = Optional.empty();
+                break;
+            case METASTORE_DLF_FAILURE:
+                this.allowFailure = true;
+                this.readWriteClient = hiveSessionMetaStoreClient;
+                this.extraClient = Optional.ofNullable(dlfSessionMetaStoreClient);
+                break;
+            case METASTORE_DLF_SUCCESS:
+                this.readWriteClient = hiveSessionMetaStoreClient;
+                this.extraClient = Optional.of(dlfSessionMetaStoreClient);
+                break;
+            case DLF_METASTORE_SUCCESS:
+                this.readWriteClient = dlfSessionMetaStoreClient;
+                this.extraClient = Optional.of(hiveSessionMetaStoreClient);
+                break;
+            case DLF_METASTORE_FAILURE:
+                this.allowFailure = true;
+                this.readWriteClient = dlfSessionMetaStoreClient;
+                this.extraClient = Optional.ofNullable(hiveSessionMetaStoreClient);
+                break;
+            case DLF_ONLY:
+                this.readWriteClient = dlfSessionMetaStoreClient;
+                this.extraClient = Optional.empty();
+                break;
+            default:
+                throw new IllegalStateException("Unexpected value: " + proxyMode);
+        }
+    }
+
+    private void createClient(boolean isDlf, ThrowingRunnable<MetaException> createClient) throws MetaException {
+        try {
+            createClient.run();
+        } catch (Exception e) {
+            if ((isDlf && proxyMode == ProxyMode.METASTORE_DLF_FAILURE)) {
+                dlfSessionMetaStoreClient = null;
+            } else if (!isDlf && proxyMode == ProxyMode.DLF_METASTORE_FAILURE) {
+                hiveSessionMetaStoreClient = null;
+            } else {
+                throw DataLakeUtil.throwException(new MetaException(e.getMessage()), e);
+            }
+        }
+    }
+
+    public void initHiveClient(HiveConf hiveConf, HiveMetaHookLoader hiveMetaHookLoader, boolean allowEmbedded,
+            ConcurrentHashMap<String, Long> metaCallTimeMap) throws MetaException {
+        switch (proxyMode) {
+            case METASTORE_ONLY:
+            case METASTORE_DLF_FAILURE:
+            case METASTORE_DLF_SUCCESS:
+            case DLF_METASTORE_SUCCESS:
+            case DLF_METASTORE_FAILURE:
+                this.hiveSessionMetaStoreClient = ClientUtils.createMetaStoreClient(HIVE_FACTORY_CLASS,
+                        hiveConf, hiveMetaHookLoader, allowEmbedded, metaCallTimeMap);
+                break;
+            case DLF_ONLY:
+                break;
+            default:
+                throw new IllegalStateException("Unexpected value: " + proxyMode);
+        }
+    }
+
+    public void initDlfClient(HiveConf hiveConf, HiveMetaHookLoader hiveMetaHookLoader, boolean allowEmbedded,
+            ConcurrentHashMap<String, Long> metaCallTimeMap) throws MetaException {
+        switch (proxyMode) {
+            case METASTORE_ONLY:
+                break;
+            case METASTORE_DLF_FAILURE:
+            case METASTORE_DLF_SUCCESS:
+            case DLF_METASTORE_SUCCESS:
+            case DLF_METASTORE_FAILURE:
+            case DLF_ONLY:
+                this.dlfSessionMetaStoreClient = new DlfSessionMetaStoreClient(hiveConf, hiveMetaHookLoader, allowEmbedded);
+                break;
+            default:
+                throw new IllegalStateException("Unexpected value: " + proxyMode);
+        }
+    }
+
+    @Override
+    public boolean isCompatibleWith(Configuration conf) {
+        try {
+            return call(this.readWriteClient, client -> client.isCompatibleWith(conf), "isCompatibleWith", conf);
+        } catch (TException e) {
+            logger.error(e.getMessage(), e);
+        }
+        return false;
+    }
+
+    @Override
+    public void setHiveAddedJars(String s) {
+        try {
+            run(client -> client.setHiveAddedJars(s), "setHiveAddedJars", s);
+        } catch (TException e) {
+            logger.error(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public boolean isLocalMetaStore() {
+        return !extraClient.isPresent() && readWriteClient.isLocalMetaStore();
+    }
+
+    @Override
+    public void reconnect() throws MetaException {
+        if (hiveSessionMetaStoreClient != null) {
+            hiveSessionMetaStoreClient.reconnect();
+        }
+    }
+
+    @Override
+    public void close() {
+        if (hiveSessionMetaStoreClient != null) {
+            hiveSessionMetaStoreClient.close();
+        }
+    }
+
+    @Override
+    public void createDatabase(Database database)
+            throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+        run(client -> client.createDatabase(database), "createDatabase", database);
+    }
+
+    @Override
+    public Database getDatabase(String name) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getDatabase(name), "getDatabase", name);
+    }
+
+    @Override
+    public Database getDatabase(String catalogId, String databaseName) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getDatabase(catalogId, databaseName), "getDatabase", catalogId, databaseName);
+    }
+
+    @Override
+    public List<String> getDatabases(String pattern) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getDatabases(pattern), "getDatabases", pattern);
+    }
+
+    @Override
+    public List<String> getDatabases(String catalogId, String databasePattern) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getDatabases(catalogId, databasePattern), "getDatabases", catalogId, databasePattern);
+    }
+
+    @Override
+    public List<String> getAllDatabases() throws MetaException, TException {
+        return getDatabases(".*");
+    }
+
+    @Override
+    public List<String> getAllDatabases(String catalogId) throws MetaException, TException {
+        return getDatabases(catalogId);
+    }
+
+    @Override
+    public void alterDatabase(String databaseName, Database database)
+            throws NoSuchObjectException, MetaException, TException {
+        run(client -> client.alterDatabase(databaseName, database), "alterDatabase", databaseName, database);
+    }
+
+    @Override
+    public void alterDatabase(String catalogId, String databaseName, Database database) throws NoSuchObjectException, MetaException, TException {
+        run(client -> client.alterDatabase(catalogId, databaseName, database), "alterDatabase", catalogId, databaseName, database);
+    }
+
+    @Override
+    public void dropDatabase(String name)
+            throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+        dropDatabase(name, true, false, false);
+    }
+
+    @Override
+    public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb)
+            throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+        dropDatabase(name, deleteData, ignoreUnknownDb, false);
+    }
+
+    @Override
+    public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb, boolean cascade)
+            throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+        run(client -> client.dropDatabase(name, deleteData, ignoreUnknownDb, cascade), "dropDatabase", name, deleteData,
+                ignoreUnknownDb, cascade);
+    }
+
+    @Override
+    public void dropDatabase(String catalogId, String name, boolean deleteData, boolean ignoreUnknownDb, boolean cascade) throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+        run(client -> client.dropDatabase(catalogId, name, deleteData, ignoreUnknownDb, cascade), "dropDatabase", catalogId, name, deleteData, ignoreUnknownDb, cascade);
+    }
+
+    @Override
+    public Partition add_partition(Partition partition)
+            throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+        return call(client -> client.add_partition(partition), "add_partition", partition);
+    }
+
+    @Override
+    public int add_partitions(List<Partition> partitions)
+            throws InvalidObjectException, AlreadyExistsException, MetaException,
+            TException {
+        return call(client -> client.add_partitions(partitions), "add_partitions", partitions);
+    }
+
+    @Override
+    public List<Partition> add_partitions(
+            List<Partition> partitions,
+            boolean ifNotExists,
+            boolean needResult
+    ) throws TException {
+        return call(client -> client.add_partitions(partitions, ifNotExists, needResult), "add_partitions", partitions, ifNotExists, needResult);
+    }
+
+    @Override
+    public int add_partitions_pspec(PartitionSpecProxy pSpec)
+            throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+        return call(client -> client.add_partitions_pspec(pSpec), "add_partitions_pspec", pSpec);
+    }
+
+    @Override
+    public void alterFunction(String dbName, String functionName, Function newFunction)
+            throws InvalidObjectException, MetaException, TException {
+        run(client -> client.alterFunction(dbName, functionName, newFunction), "alterFunction", dbName, functionName, newFunction);
+    }
+
+    @Override
+    public void alterFunction(String catalogId, String dbName, String functionName, Function newFunction) throws InvalidObjectException, MetaException, TException {
+        run(client -> client.alterFunction(catalogId, dbName, functionName, newFunction), "alterFunction", catalogId, dbName, functionName, newFunction);
+    }
+
+    @Override
+    public void alter_partition(String dbName, String tblName, Partition partition)
+            throws InvalidOperationException, MetaException, TException {
+        run(client -> client.alter_partition(dbName, tblName, partition), "alter_partition", dbName, tblName, partition);
+    }
+
+    @Override
+    public void alter_partition(
+            String dbName,
+            String tblName,
+            Partition partition,
+            EnvironmentContext environmentContext
+    ) throws InvalidOperationException, MetaException, TException {
+        run(client -> client.alter_partition(dbName, tblName, partition, environmentContext), "alter_partition", dbName, tblName, partition, environmentContext);
+    }
+
+    @Override
+    public void alter_partition(String catalogId, String dbName, String tblName, Partition partition, EnvironmentContext environmentContext) throws InvalidOperationException, MetaException, TException {
+        run(client -> client.alter_partition(catalogId, dbName, tblName, partition, environmentContext), "alter_partition", catalogId, dbName, tblName, partition, environmentContext);
+    }
+
+    @Override
+    public void alter_partitions(
+            String dbName,
+            String tblName,
+            List<Partition> partitions
+    ) throws InvalidOperationException, MetaException, TException {
+        run(client -> client.alter_partitions(dbName, tblName, partitions), "alter_partitions", dbName, tblName, partitions);
+    }
+
+    @Override
+    public void alter_partitions(
+            String dbName,
+            String tblName,
+            List<Partition> partitions,
+            EnvironmentContext environmentContext
+    ) throws InvalidOperationException, MetaException, TException {
+        run(client -> client.alter_partitions(dbName, tblName, partitions, environmentContext), "alter_partitions", dbName, tblName, partitions, environmentContext);
+    }
+
+    @Override
+    public void alter_partitions(String catalogId, String dbName, String tblName, List<Partition> partitions, EnvironmentContext environmentContext) throws InvalidOperationException, MetaException, TException {
+        run(client -> client.alter_partitions(catalogId, dbName, tblName, partitions, environmentContext), "alter_partitions", catalogId, dbName, tblName, partitions, environmentContext);
+    }
+
+    @Override
+    public void alter_table(String dbName, String tblName, Table table)
+            throws InvalidOperationException, MetaException, TException {
+        if (table.isTemporary()) {
+            run(this.readWriteClient, client -> client.alter_table(dbName, tblName, table), "alter_table", dbName, tblName, table);
+        } else {
+            run(client -> client.alter_table(dbName, tblName, table), "alter_table", dbName, tblName, table);
+        }
+    }
+
+    @Override
+    public void alter_table(String catalogId, String dbName, String tblName, Table table, EnvironmentContext environmentContext) throws InvalidOperationException, MetaException, TException {
+        if (table.isTemporary()) {
+            run(this.readWriteClient, client -> client.alter_table(catalogId, dbName, tblName, table, environmentContext), "alter_table", catalogId, dbName, tblName, table, environmentContext);
+        } else {
+            run(client -> client.alter_table(catalogId, dbName, tblName, table, environmentContext), "alter_table", catalogId, dbName, tblName, table, environmentContext);
+        }
+    }
+
+    @Override
+    @Deprecated
+    public void alter_table(
+            String dbName,
+            String tblName,
+            Table table,
+            boolean cascade
+    ) throws InvalidOperationException, MetaException, TException {
+        if (table.isTemporary()) {
+            run(this.readWriteClient, client -> client.alter_table(dbName, tblName, table, cascade), "alter_table", dbName, tblName, table, cascade);
+        } else {
+            run(client -> client.alter_table(dbName, tblName, table, cascade), "alter_table", dbName, tblName, table, cascade);
+        }
+    }
+
+    @Override
+    public void alter_table_with_environmentContext(
+            String dbName,
+            String tblName,
+            Table table,
+            EnvironmentContext environmentContext
+    ) throws InvalidOperationException, MetaException, TException {
+        if (table.isTemporary()) {
+            run(this.readWriteClient, client -> client.alter_table_with_environmentContext(dbName, tblName, table, environmentContext), "alter_table_with_environmentContext", dbName, tblName, table, environmentContext);
+        } else {
+            run(client -> client.alter_table_with_environmentContext(dbName, tblName, table, environmentContext), "alter_table_with_environmentContext", dbName,
+                    tblName, table, environmentContext);
+        }
+    }
+
+    @Override
+    public Partition appendPartition(String dbName, String tblName, List<String> values)
+            throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+        return call(client -> client.appendPartition(dbName, tblName, values), "appendPartition", dbName, tblName, values);
+    }
+
+    @Override
+    public Partition appendPartition(String catalogId, String dbName, String tblName, List<String> values) throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+        return call(client -> client.appendPartition(catalogId, dbName, tblName, values), "appendPartition", catalogId, dbName, tblName, values);
+    }
+
+    @Override
+    public Partition appendPartition(String dbName, String tblName, String partitionName)
+            throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+        return call(client -> client.appendPartition(dbName, tblName, partitionName), "appendPartition", dbName, tblName, partitionName);
+    }
+
+    @Override
+    public Partition appendPartition(String catalogId, String dbName, String tblName, String partitionName) throws InvalidObjectException, AlreadyExistsException, MetaException, TException {
+        return call(client -> client.appendPartition(catalogId, dbName, tblName, partitionName), "appendPartition", catalogId, dbName, tblName, partitionName);
+    }
+
+    @Override
+    public boolean create_role(Role role) throws MetaException, TException {
+        return call(client -> client.create_role(role), "create_role", role);
+    }
+
+    @Override
+    public boolean drop_role(String roleName) throws MetaException, TException {
+        return call(client -> client.drop_role(roleName), "drop_role", roleName);
+    }
+
+    @Override
+    public List<Role> list_roles(
+            String principalName, PrincipalType principalType
+    ) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.list_roles(principalName, principalType), "list_roles", principalName, principalType);
+    }
+
+    @Override
+    public List<String> listRoleNames() throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.listRoleNames(), "listRoleNames");
+    }
+
+    @Override
+    public GetPrincipalsInRoleResponse get_principals_in_role(GetPrincipalsInRoleRequest request)
+            throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.get_principals_in_role(request), "get_principals_in_role", request);
+    }
+
+    @Override
+    public GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(
+            GetRoleGrantsForPrincipalRequest request) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.get_role_grants_for_principal(request), "get_role_grants_for_principal", request);
+    }
+
+    @Override
+    public boolean grant_role(
+            String roleName,
+            String userName,
+            PrincipalType principalType,
+            String grantor,
+            PrincipalType grantorType,
+            boolean grantOption
+    ) throws MetaException, TException {
+        return call(client -> client.grant_role(roleName, userName, principalType, grantor, grantorType, grantOption)
+                , "grant_role", roleName, userName, principalType, grantor, grantorType);
+    }
+
+    @Override
+    public boolean revoke_role(
+            String roleName,
+            String userName,
+            PrincipalType principalType,
+            boolean grantOption
+    ) throws MetaException, TException {
+        return call(client -> client.revoke_role(roleName, userName, principalType, grantOption), "revoke_role", roleName, userName,
+                principalType, grantOption);
+    }
+
+    @Override
+    public void cancelDelegationToken(String tokenStrForm) throws MetaException, TException {
+        run(client -> client.cancelDelegationToken(tokenStrForm), "cancelDelegationToken", tokenStrForm);
+    }
+
+    @Override
+    public String getTokenStrForm() throws IOException {
+        try {
+            return call(this.readWriteClient, client -> {
+                try {
+                    return client.getTokenStrForm();
+                } catch (IOException e) {
+                    throw new TException(e.getMessage(), e);
+                }
+            }, "getTokenStrForm");
+        } catch (TException e) {
+            throw new IOException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public boolean addToken(String tokenIdentifier, String delegationToken) throws TException {
+        return call(client -> client.addToken(tokenIdentifier, delegationToken), "addToken", tokenIdentifier, delegationToken);
+    }
+
+    @Override
+    public boolean removeToken(String tokenIdentifier) throws TException {
+        return call(client -> client.removeToken(tokenIdentifier), "removeToken", tokenIdentifier);
+    }
+
+    @Override
+    public String getToken(String tokenIdentifier) throws TException {
+        return call(this.readWriteClient, client -> client.getToken(tokenIdentifier), "getToken", tokenIdentifier);
+    }
+
+    @Override
+    public List<String> getAllTokenIdentifiers() throws TException {
+        return call(this.readWriteClient, client -> client.getAllTokenIdentifiers(), "getAllTokenIdentifiers");
+    }
+
+    @Override
+    public int addMasterKey(String key) throws MetaException, TException {
+        return call(client -> client.addMasterKey(key), "addMasterKey", key);
+    }
+
+    @Override
+    public void updateMasterKey(Integer seqNo, String key)
+            throws NoSuchObjectException, MetaException, TException {
+        run(client -> client.updateMasterKey(seqNo, key), "updateMasterKey", key);
+    }
+
+    @Override
+    public boolean removeMasterKey(Integer keySeq) throws TException {
+        return call(client -> client.removeMasterKey(keySeq), "removeMasterKey", keySeq);
+    }
+
+    @Override
+    public String[] getMasterKeys() throws TException {
+        return call(this.readWriteClient, client -> client.getMasterKeys(), "getMasterKeys");
+    }
+
+    @Override
+    public LockResponse checkLock(long lockId)
+            throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, TException {
+        return call(this.readWriteClient, client -> client.checkLock(lockId), "checkLock", lockId);
+    }
+
+    @Override
+    public void commitTxn(long txnId) throws NoSuchTxnException, TxnAbortedException, TException {
+        run(client -> client.commitTxn(txnId), "commitTxn", txnId);
+    }
+
+    @Override
+    public void replCommitTxn(long srcTxnId, String replPolicy) throws NoSuchTxnException, TxnAbortedException, TException {
+        run(client -> client.replCommitTxn(srcTxnId, replPolicy), "replCommitTxn", srcTxnId, replPolicy);
+    }
+
+    @Override
+    public void abortTxns(List<Long> txnIds) throws TException {
+        run(client -> client.abortTxns(txnIds), "abortTxns", txnIds);
+    }
+
+    @Override
+    public long allocateTableWriteId(long txnId, String dbName, String tableName) throws TException {
+        return call(client -> client.allocateTableWriteId(txnId, dbName, tableName), "allocateTableWriteId", txnId, dbName, tableName);
+    }
+
+    @Override
+    public void replTableWriteIdState(String validWriteIdList, String dbName, String tableName, List<String> partNames) throws TException {
+        run(client -> client.replTableWriteIdState(validWriteIdList, dbName, tableName, partNames), "replTableWriteIdState", validWriteIdList, dbName, tableName, partNames);
+    }
+
+    @Override
+    public List<TxnToWriteId> allocateTableWriteIdsBatch(List<Long> txnIds, String dbName, String tableName) throws TException {
+        return call(client -> client.allocateTableWriteIdsBatch(txnIds, dbName, tableName), "allocateTableWriteIdsBatch", txnIds, dbName, tableName);
+    }
+
+    @Override
+    public List<TxnToWriteId> replAllocateTableWriteIdsBatch(String dbName, String tableName,
+            String replPolicy, List<TxnToWriteId> srcTxnToWriteIdList) throws TException {
+        return call(client -> client.replAllocateTableWriteIdsBatch(dbName, tableName, replPolicy, srcTxnToWriteIdList), "replAllocateTableWriteIdsBatch", dbName, tableName, replPolicy, srcTxnToWriteIdList);
+    }
+
+    @Override
+    @Deprecated
+    public void compact(
+            String dbName,
+            String tblName,
+            String partitionName,
+            CompactionType compactionType
+    ) throws TException {
+        run(client -> client.compact(dbName, tblName, partitionName, compactionType), "compact", dbName, tblName, partitionName, compactionType);
+    }
+
+    @Override
+    @Deprecated
+    public void compact(
+            String dbName,
+            String tblName,
+            String partitionName,
+            CompactionType compactionType,
+            Map<String, String> tblProperties
+    ) throws TException {
+        run(client -> client.compact(dbName, tblName, partitionName, compactionType, tblProperties), "compact", dbName, tblName, partitionName, compactionType, tblProperties);
+    }
+
+    @Override
+    public CompactionResponse compact2(
+            String dbName,
+            String tblName,
+            String partitionName,
+            CompactionType compactionType,
+            Map<String, String> tblProperties
+    ) throws TException {
+        return call(client -> client.compact2(dbName, tblName, partitionName, compactionType, tblProperties), "compact2", dbName, tblName, partitionName, compactionType, tblProperties);
+    }
+
+    @Override
+    public void createFunction(Function function) throws InvalidObjectException, MetaException, TException {
+        run(client -> client.createFunction(function), "createFunction", function);
+    }
+
+    @Override
+    public void createTable(Table tbl)
+            throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, TException {
+        createTable(tbl, null);
+    }
+
+    public void createTable(Table tbl, EnvironmentContext envContext) throws AlreadyExistsException,
+            InvalidObjectException, MetaException, NoSuchObjectException, TException {
+        // Subclasses can override this step (for example, for temporary tables)
+        if (tbl.isTemporary()) {
+            run(this.readWriteClient, client -> client.createTable(tbl), "createTable", tbl);
+        } else {
+            run(client -> client.createTable(tbl), "createTable", tbl);
+        }
+    }
+
+    @Override
+    public boolean deletePartitionColumnStatistics(
+            String dbName, String tableName, String partName, String colName
+    ) throws NoSuchObjectException, MetaException, InvalidObjectException, TException, InvalidInputException {
+        return call(client -> client.deletePartitionColumnStatistics(dbName, tableName, partName, colName), "deletePartitionColumnStatistics", dbName,
+                tableName, partName, colName);
+    }
+
+    @Override
+    public boolean deletePartitionColumnStatistics(String catalogId, String dbName, String tableName, String partName, String colName) throws NoSuchObjectException, MetaException, InvalidObjectException, TException, InvalidInputException {
+        return call(client -> client.deletePartitionColumnStatistics(catalogId, dbName, tableName, partName, colName), "deletePartitionColumnStatistics", catalogId, dbName, tableName, partName, colName);
+    }
+
+    @Override
+    public boolean deleteTableColumnStatistics(String dbName, String tableName, String colName)
+            throws NoSuchObjectException, MetaException, InvalidObjectException,
+            TException, InvalidInputException {
+        if (getTempTable(dbName, tableName) != null) {
+            return call(this.readWriteClient, client -> client.deleteTableColumnStatistics(dbName, tableName, colName), "deleteTableColumnStatistics", dbName, tableName, colName);
+        } else {
+            return call(client -> client.deleteTableColumnStatistics(dbName, tableName, colName), "deleteTableColumnStatistics", dbName, tableName, colName);
+        }
+    }
+
+    @Override
+    public boolean deleteTableColumnStatistics(String catalogId, String dbName, String tableName, String colName) throws NoSuchObjectException, MetaException, InvalidObjectException, TException, InvalidInputException {
+        if (getTempTable(dbName, tableName) != null) {
+            return call(this.readWriteClient, client -> client.deleteTableColumnStatistics(catalogId, dbName, tableName, colName), "deleteTableColumnStatistics", catalogId, dbName, tableName, colName);
+        } else {
+            return call(client -> client.deleteTableColumnStatistics(catalogId, dbName, tableName, colName), "deleteTableColumnStatistics", catalogId, dbName, tableName, colName);
+        }
+    }
+
+    @Override
+    public void dropFunction(String dbName, String functionName) throws MetaException, NoSuchObjectException,
+            InvalidObjectException, InvalidInputException, TException {
+        run(client -> client.dropFunction(dbName, functionName), "dropFunction", dbName, functionName);
+    }
+
+    @Override
+    public void dropFunction(String catalogId, String dbName, String functionName) throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException, TException {
+        run(client -> client.dropFunction(catalogId, dbName, functionName), "dropFunction", catalogId, dbName, functionName);
+    }
+
+    @Override
+    public boolean dropPartition(String dbName, String tblName, List<String> values, boolean deleteData)
+            throws NoSuchObjectException, MetaException, TException {
+        return call(client -> client.dropPartition(dbName, tblName, values, deleteData), "dropPartition", dbName, tblName, values, deleteData);
+    }
+
+    @Override
+    public boolean dropPartition(String catalogId, String dbName, String tblName, List<String> values, boolean deleteData) throws NoSuchObjectException, MetaException, TException {
+        return call(client -> client.dropPartition(catalogId, dbName, tblName, values, deleteData), "dropPartition", catalogId, dbName, tblName, values, deleteData);
+    }
+
+    @Override
+    public boolean dropPartition(String dbName, String tblName, List<String> values, PartitionDropOptions options)
+            throws TException {
+        return call(client -> client.dropPartition(dbName, tblName, values, options), "dropPartition", dbName, tblName, values, options);
+    }
+
+    @Override
+    public boolean dropPartition(String catalogId, String dbName, String tblName, List<String> values, PartitionDropOptions options) throws NoSuchObjectException, MetaException, TException {
+        return call(client -> client.dropPartition(catalogId, dbName, tblName, values, options), "dropPartition", catalogId, dbName, tblName, values, options);
+    }
+
+    @Override
+    public List<Partition> dropPartitions(String dbName, String tblName,
+            List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
+            boolean ifExists) throws NoSuchObjectException, MetaException, TException {
+        return call(client -> client.dropPartitions(dbName, tblName, partExprs, deleteData, ifExists), "dropPartitions", dbName, tblName, partExprs, deleteData, ifExists);
+    }
+
+    @Override
+    public List<Partition> dropPartitions(String dbName, String tblName,
+            List<ObjectPair<Integer, byte[]>> partExprs, boolean deleteData,
+            boolean ifExists, boolean needResult) throws NoSuchObjectException, MetaException, TException {
+        return call(client -> client.dropPartitions(dbName, tblName, partExprs, deleteData, ifExists, needResult), "dropPartitions", dbName, tblName, partExprs, deleteData, ifExists, needResult);
+    }
+
+    @Override
+    public List<Partition> dropPartitions(String dbName, String tblName, List<ObjectPair<Integer, byte[]>> partExprs, PartitionDropOptions partitionDropOptions) throws NoSuchObjectException, MetaException, TException {
+        return call(client -> client.dropPartitions(dbName, tblName, partExprs, partitionDropOptions), "dropPartitions", dbName, tblName, partExprs, partitionDropOptions);
+    }
+
+    @Override
+    public List<Partition> dropPartitions(String catalogId, String dbName, String tblName, List<ObjectPair<Integer, byte[]>> partExprs, PartitionDropOptions partitionDropOptions) throws NoSuchObjectException, MetaException, TException {
+        return call(client -> client.dropPartitions(catalogId, dbName, tblName, partExprs, partitionDropOptions), "dropPartitions", catalogId, dbName, tblName, partExprs, partitionDropOptions);
+    }
+
+    @Override
+    public boolean dropPartition(String dbName, String tblName, String partitionName, boolean deleteData)
+            throws NoSuchObjectException, MetaException, TException {
+        return call(client -> client.dropPartition(dbName, tblName, partitionName, deleteData), "dropPartition", dbName, tblName,
+                partitionName, deleteData);
+    }
+
+    @Override
+    public boolean dropPartition(String catName, String dbName, String tblName, String partitionName,
+            boolean deleteData) throws NoSuchObjectException, MetaException, TException {
+        return call(client -> client.dropPartition(catName, dbName, tblName, partitionName, deleteData), "dropPartition", catName, dbName, tblName, partitionName, deleteData);
+    }
+
+    private Table getTempTable(String dbName, String tableName) {
+        Map<String, org.apache.hadoop.hive.ql.metadata.Table> tables = getTempTablesForDatabase(dbName.toLowerCase());
+        if (tables != null) {
+            org.apache.hadoop.hive.ql.metadata.Table table = tables.get(tableName.toLowerCase());
+            if (table != null) {
+                return table.getTTable();
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public void dropTable(String dbname, String tableName)
+            throws MetaException, TException, NoSuchObjectException {
+        Table table = getTempTable(dbname, tableName);
+        if (table != null) {
+            run(this.readWriteClient, client -> client.dropTable(dbname, tableName), "dropTable", dbname, tableName);
+        } else {
+            run(client -> client.dropTable(dbname, tableName), "dropTable", dbname, tableName);
+        }
+    }
+
+    @Override
+    public void dropTable(String catalogId, String dbname, String tableName, boolean deleteData, boolean ignoreUnknownTab, boolean ifPurge) throws MetaException, NoSuchObjectException, TException {
+        Table table = getTempTable(dbname, tableName);
+        if (table != null) {
+            run(this.readWriteClient, client -> client.dropTable(catalogId, dbname, tableName, deleteData, ignoreUnknownTab, ifPurge), "dropTable", catalogId, dbname, tableName, deleteData, ignoreUnknownTab, ifPurge);
+        } else {
+            run(client -> client.dropTable(catalogId, dbname, tableName, deleteData, ignoreUnknownTab, ifPurge), "dropTable", catalogId, dbname, tableName, deleteData, ignoreUnknownTab, ifPurge);
+        }
+    }
+
+    @Override
+    public void truncateTable(String dbName, String tableName, List<String> partNames) throws MetaException, TException {
+        run(client -> client.truncateTable(dbName, tableName, partNames), "truncateTable", dbName, tableName, partNames);
+    }
+
+    @Override
+    public void truncateTable(String catName, String dbName, String tableName, List<String> partNames) throws MetaException, TException {
+        run(client -> client.truncateTable(catName, dbName, tableName, partNames), "truncateTable", catName, dbName, tableName, partNames);
+    }
+
+    @Override
+    public CmRecycleResponse recycleDirToCmPath(CmRecycleRequest cmRecycleRequest) throws MetaException, TException {
+        return call(client -> client.recycleDirToCmPath(cmRecycleRequest), "recycleDirToCmPath", cmRecycleRequest);
+    }
+
+    @Override
+    public void dropTable(
+            String dbname,
+            String tableName,
+            boolean deleteData,
+            boolean ignoreUnknownTab
+    ) throws MetaException, TException, NoSuchObjectException {
+        Table table = getTempTable(dbname, tableName);
+        if (table != null) {
+            run(this.readWriteClient, client -> client.dropTable(dbname, tableName, deleteData, ignoreUnknownTab), "dropTable", dbname, tableName, deleteData, ignoreUnknownTab);
+        } else {
+            run(client -> client.dropTable(dbname, tableName, deleteData, ignoreUnknownTab), "dropTable", dbname, tableName, deleteData, ignoreUnknownTab);
+        }
+    }
+
+    @Override
+    public void dropTable(
+            String dbname,
+            String tableName,
+            boolean deleteData,
+            boolean ignoreUnknownTable,
+            boolean ifPurge
+    ) throws MetaException, TException, NoSuchObjectException {
+        Table table = getTempTable(dbname, tableName);
+        if (table != null) {
+            run(this.readWriteClient, client -> client.dropTable(dbname, tableName, deleteData, ignoreUnknownTable, ifPurge), "dropTable", dbname, tableName, deleteData, ignoreUnknownTable, ifPurge);
+        } else {
+            run(client -> client.dropTable(dbname, tableName, deleteData, ignoreUnknownTable, ifPurge), "dropTable", dbname, tableName, deleteData, ignoreUnknownTable, ifPurge);
+        }
+    }
+
+    @Override
+    public Partition exchange_partition(
+            Map<String, String> partitionSpecs,
+            String srcDb,
+            String srcTbl,
+            String dstDb,
+            String dstTbl
+    ) throws MetaException, NoSuchObjectException, InvalidObjectException, TException {
+        return call(client -> client.exchange_partition(partitionSpecs, srcDb, srcTbl, dstDb, dstTbl), "exchange_partition", partitionSpecs
+                , srcDb, srcTbl, dstDb, dstTbl);
+    }
+
+    @Override
+    public Partition exchange_partition(Map<String, String> partitionSpecs, String srcCatalogId, String srcDb, String srcTbl, String descCatalogId, String dstDb, String dstTbl) throws MetaException, NoSuchObjectException, InvalidObjectException, TException {
+        return call(client -> client.exchange_partition(partitionSpecs, srcCatalogId, srcDb, srcTbl, descCatalogId, dstDb, dstTbl), "exchange_partition", partitionSpecs, srcCatalogId, srcDb, srcTbl, descCatalogId, dstDb, dstTbl);
+    }
+
+    @Override
+    public List<Partition> exchange_partitions(
+            Map<String, String> partitionSpecs,
+            String sourceDb,
+            String sourceTbl,
+            String destDb,
+            String destTbl
+    ) throws MetaException, NoSuchObjectException, InvalidObjectException, TException {
+        return call(client -> client.exchange_partitions(partitionSpecs, sourceDb, sourceTbl, destDb, destTbl), "exchange_partitions",
+                partitionSpecs, sourceDb, sourceTbl, destDb, destTbl);
+    }
+
+    @Override
+    public List<Partition> exchange_partitions(Map<String, String> partitionSpecs, String srcCatalogId, String sourceDb, String sourceTbl, String dstCatalogId, String destDb, String destTbl) throws MetaException, NoSuchObjectException, InvalidObjectException, TException {
+        return call(client -> client.exchange_partitions(partitionSpecs, srcCatalogId, sourceDb, sourceTbl, dstCatalogId, destDb, destTbl), "exchange_partitions", partitionSpecs, srcCatalogId, sourceDb, sourceTbl, dstCatalogId, destDb, destTbl);
+    }
+
+    @Override
+    public AggrStats getAggrColStatsFor(
+            String dbName,
+            String tblName,
+            List<String> colNames,
+            List<String> partName
+    ) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getAggrColStatsFor(dbName, tblName, colNames, partName), "getAggrColStatsFor", dbName, tblName, colNames, partName);
+    }
+
+    @Override
+    public AggrStats getAggrColStatsFor(
+            String catalogId,
+            String dbName,
+            String tblName,
+            List<String> colNames,
+            List<String> partName
+    ) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getAggrColStatsFor(catalogId, dbName, tblName, colNames, partName), "getAggrColStatsFor", catalogId, dbName, tblName, colNames, partName);
+    }
+
+    @Override
+    public List<String> getAllTables(String dbname)
+            throws MetaException, TException, UnknownDBException {
+        return getTables(dbname, ".*");
+    }
+
+    @Override
+    public List<String> getAllTables(String catalogId, String dbName) throws MetaException, TException, UnknownDBException {
+        return getTables(catalogId, dbName);
+    }
+
+    @Override
+    public String getConfigValue(String name, String defaultValue)
+            throws TException, ConfigValSecurityException {
+        return call(this.readWriteClient, client -> client.getConfigValue(name, defaultValue), "getConfigValue", name, defaultValue);
+    }
+
+    @Override
+    public String getDelegationToken(String owner, String renewerKerberosPrincipalName)
+            throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getDelegationToken(owner, renewerKerberosPrincipalName), "getDelegationToken", owner, renewerKerberosPrincipalName);
+    }
+
+    @Override
+    public List<FieldSchema> getFields(String db, String tableName) throws TException {
+        return call(this.readWriteClient, client -> client.getFields(db, tableName), "getFields", db, tableName);
+    }
+
+    @Override
+    public List<FieldSchema> getFields(String catalogId, String db, String tableName) throws MetaException, TException, UnknownTableException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.getFields(catalogId, db, tableName), "getFields", catalogId, db, tableName);
+    }
+
+    @Override
+    public Function getFunction(String dbName, String functionName) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getFunction(dbName, functionName), "getFunction", dbName, functionName);
+    }
+
+    @Override
+    public Function getFunction(String catalogId, String dbName, String funcName) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getFunction(catalogId, dbName, funcName), "getFunction", catalogId, dbName, funcName);
+    }
+
+    @Override
+    public List<String> getFunctions(String dbName, String pattern) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getFunctions(dbName, pattern), "getFunctions", dbName, pattern);
+    }
+
+    @Override
+    public List<String> getFunctions(String catalogId, String dbName, String pattern) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getFunctions(catalogId, dbName, pattern), "getFunctions", catalogId, dbName, pattern);
+    }
+
+    @Override
+    public GetAllFunctionsResponse getAllFunctions() throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getAllFunctions(), "getAllFunctions");
+    }
+
+    @Override
+    public String getMetaConf(String key) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getMetaConf(key), "getMetaConf", key);
+    }
+
+    @Override
+    public void createCatalog(Catalog catalog) throws AlreadyExistsException, InvalidObjectException, MetaException, TException {
+        run(client -> client.createCatalog(catalog), "createCatalog", catalog);
+    }
+
+    @Override
+    public void alterCatalog(String catalogName, Catalog newCatalog) throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
+        run(client -> client.alterCatalog(catalogName, newCatalog), "alterCatalog", catalogName, newCatalog);
+    }
+
+    @Override
+    public Catalog getCatalog(String catalogId) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getCatalog(catalogId), "getCatalog", catalogId);
+    }
+
+    @Override
+    public List<String> getCatalogs() throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getCatalogs(), "getCatalogs");
+    }
+
+    @Override
+    public void dropCatalog(String catalogId) throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+        run(client -> client.dropCatalog(catalogId), "dropCatalog", catalogId);
+    }
+
+    @Override
+    public Partition getPartition(String dbName, String tblName, List<String> values)
+            throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getPartition(dbName, tblName, values), "getPartition", dbName, tblName, values);
+    }
+
+    @Override
+    public Partition getPartition(String catalogId, String dbName, String tblName, List<String> values) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getPartition(catalogId, dbName, tblName, values), "getPartition", catalogId, dbName, tblName, values);
+    }
+
+    @Override
+    public Partition getPartition(String dbName, String tblName, String partitionName)
+            throws MetaException, UnknownTableException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getPartition(dbName, tblName, partitionName), "getPartition", dbName, tblName, partitionName);
+    }
+
+    @Override
+    public Partition getPartition(String catalogId, String dbName, String tblName, String partitionName) throws MetaException, UnknownTableException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getPartition(catalogId, dbName, tblName, partitionName), "getPartition", catalogId, dbName, tblName, partitionName);
+    }
+
+    @Override
+    public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
+            String dbName,
+            String tableName,
+            List<String> partitionNames,
+            List<String> columnNames
+    ) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getPartitionColumnStatistics(dbName, tableName, partitionNames, columnNames), "getPartitionColumnStatistics", dbName, tableName, partitionNames, columnNames);
+    }
+
+    @Override
+    public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
+            String catalogId,
+            String dbName,
+            String tableName,
+            List<String> partitionNames,
+            List<String> columnNames
+    ) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getPartitionColumnStatistics(catalogId, dbName, tableName, partitionNames, columnNames), "getPartitionColumnStatistics", catalogId, dbName, tableName, partitionNames, columnNames);
+    }
+
+    @Override
+    public Partition getPartitionWithAuthInfo(
+            String databaseName,
+            String tableName,
+            List<String> values,
+            String userName,
+            List<String> groupNames
+    ) throws MetaException, UnknownTableException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getPartitionWithAuthInfo(databaseName, tableName, values, userName, groupNames), "getPartitionWithAuthInfo", databaseName, tableName, values, userName, groupNames);
+    }
+
+    @Override
+    public Partition getPartitionWithAuthInfo(
+            String catalogId,
+            String databaseName,
+            String tableName,
+            List<String> values,
+            String userName,
+            List<String> groupNames) throws MetaException, UnknownTableException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getPartitionWithAuthInfo(catalogId, databaseName, tableName, values, userName, groupNames), "getPartitionWithAuthInfo", catalogId, databaseName, tableName, values, userName, groupNames);
+    }
+
+    @Override
+    public List<Partition> getPartitionsByNames(
+            String databaseName,
+            String tableName,
+            List<String> partitionNames
+    ) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getPartitionsByNames(databaseName, tableName, partitionNames), "getPartitionsByNames", databaseName, tableName, partitionNames);
+    }
+
+    @Override
+    public List<Partition> getPartitionsByNames(
+            String catalogId,
+            String databaseName,
+            String tableName,
+            List<String> partitionNames
+    ) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getPartitionsByNames(catalogId, databaseName, tableName, partitionNames), "getPartitionsByNames", catalogId, databaseName, tableName, partitionNames);
+    }
+
+    @Override
+    public List<FieldSchema> getSchema(String db, String tableName) throws TException {
+        return call(this.readWriteClient, client -> client.getSchema(db, tableName), "getSchema", db, tableName);
+    }
+
+    @Override
+    public List<FieldSchema> getSchema(String catalogId, String db, String tableName) throws MetaException, TException, UnknownTableException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.getSchema(catalogId, db, tableName), "getSchema", catalogId, db, tableName);
+    }
+
+    @Override
+    public Table getTable(String dbName, String tableName)
+            throws MetaException, TException, NoSuchObjectException {
+        return call(this.readWriteClient, client -> client.getTable(dbName, tableName), "getTable", dbName, tableName);
+    }
+
+    @Override
+    public Table getTable(String catalogId, String dbName, String tableName) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getTable(catalogId, dbName, tableName), "getTable", catalogId, dbName, tableName);
+    }
+
+    @Override
+    public List<ColumnStatisticsObj> getTableColumnStatistics(
+            String dbName,
+            String tableName,
+            List<String> colNames
+    ) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getTableColumnStatistics(dbName, tableName, colNames), "getTableColumnStatistics", dbName, tableName, colNames);
+    }
+
+    @Override
+    public List<ColumnStatisticsObj> getTableColumnStatistics(
+            String catalogId,
+            String dbName,
+            String tableName,
+            List<String> colNames
+    ) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getTableColumnStatistics(catalogId, dbName, tableName, colNames), "getTableColumnStatistics", catalogId, dbName, tableName, colNames);
+    }
+
+    @Override
+    public List<Table> getTableObjectsByName(String dbName, List<String> tableNames)
+            throws MetaException, InvalidOperationException, UnknownDBException, TException {
+        return call(this.readWriteClient, client -> client.getTableObjectsByName(dbName, tableNames), "getTableObjectsByName", dbName, tableNames);
+    }
+
+    @Override
+    public List<Table> getTableObjectsByName(String catalogId, String dbName, List<String> tableNames) throws MetaException, InvalidOperationException, UnknownDBException, TException {
+        return call(this.readWriteClient, client -> client.getTableObjectsByName(catalogId, dbName, tableNames), "getTableObjectsByName", catalogId, dbName, tableNames);
+    }
+
+    @Override
+    public Materialization getMaterializationInvalidationInfo(CreationMetadata creationMetadata, String validTxnList) throws MetaException, InvalidOperationException, UnknownDBException, TException {
+        return call(this.readWriteClient, client -> client.getMaterializationInvalidationInfo(creationMetadata, validTxnList), "getMaterializationInvalidationInfo", creationMetadata, validTxnList);
+    }
+
+    @Override
+    public void updateCreationMetadata(String dbName, String tableName, CreationMetadata cm) throws MetaException, TException {
+        run(client -> client.updateCreationMetadata(dbName, tableName, cm), "updateCreationMetadata", dbName, tableName, cm);
+    }
+
+    @Override
+    public void updateCreationMetadata(String catalogId, String dbName, String tableName, CreationMetadata cm) throws MetaException, TException {
+        run(client -> client.updateCreationMetadata(catalogId, dbName, tableName, cm), "updateCreationMetadata", catalogId, dbName, tableName, cm);
+    }
+
+    @Override
+    public List<String> getTables(String dbname, String tablePattern)
+            throws MetaException, TException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.getTables(dbname, tablePattern), "getTables", dbname, tablePattern);
+    }
+
+    @Override
+    public List<String> getTables(String catalogId, String dbname, String tablePattern) throws MetaException, TException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.getTables(catalogId, dbname, tablePattern), "getTables", catalogId, dbname, tablePattern);
+    }
+
+    @Override
+    public List<String> getTables(String dbname, String tablePattern, TableType tableType)
+            throws MetaException, TException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.getTables(dbname, tablePattern, tableType), "getTables", dbname, tablePattern, tableType);
+    }
+
+    @Override
+    public List<String> getTables(String catalogId, String dbname, String tablePattern, TableType tableType) throws MetaException, TException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.getTables(catalogId, dbname, tablePattern, tableType), "getTables", catalogId, dbname, tablePattern, tableType);
+    }
+
+    @Override
+    public List<String> getMaterializedViewsForRewriting(String dbName) throws MetaException, TException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.getMaterializedViewsForRewriting(dbName), "getMaterializedViewsForRewriting", dbName);
+    }
+
+    @Override
+    public List<String> getMaterializedViewsForRewriting(String catalogId, String dbName) throws MetaException, TException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.getMaterializedViewsForRewriting(catalogId, dbName), "getMaterializedViewsForRewriting", catalogId, dbName);
+    }
+
+    @Override
+    public List<TableMeta> getTableMeta(
+            String dbPatterns,
+            String tablePatterns,
+            List<String> tableTypes
+    ) throws MetaException, TException, UnknownDBException, UnsupportedOperationException {
+        return call(this.readWriteClient, client -> client.getTableMeta(dbPatterns, tablePatterns, tableTypes), "getTableMeta", dbPatterns, tablePatterns, tableTypes);
+    }
+
+    @Override
+    public List<TableMeta> getTableMeta(String catalogId, String dbPatterns, String tablePatterns, List<String> tableTypes) throws MetaException, TException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.getTableMeta(catalogId, dbPatterns, tablePatterns, tableTypes), "getTableMeta", catalogId, dbPatterns, tablePatterns, tableTypes);
+    }
+
+    @Override
+    public ValidTxnList getValidTxns() throws TException {
+        return call(this.readWriteClient, client -> client.getValidTxns(), "getValidTxns");
+    }
+
+    @Override
+    public ValidTxnList getValidTxns(long currentTxn) throws TException {
+        return call(this.readWriteClient, client -> client.getValidTxns(currentTxn), "getValidTxns", currentTxn);
+    }
+
+    @Override
+    public ValidWriteIdList getValidWriteIds(String fullTableName) throws TException {
+        return call(this.readWriteClient, client -> client.getValidWriteIds(fullTableName), "getValidWriteIds", fullTableName);
+    }
+
+    @Override
+    public List<TableValidWriteIds> getValidWriteIds(List<String> tablesList, String validTxnList) throws TException {
+        return call(this.readWriteClient, client -> client.getValidWriteIds(tablesList, validTxnList), "getValidWriteIds", tablesList, validTxnList);
+    }
+
+    @Override
+    public PrincipalPrivilegeSet get_privilege_set(HiveObjectRef obj, String user, List<String> groups)
+            throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.get_privilege_set(obj, user, groups), "get_privilege_set", obj, user, groups);
+    }
+
+    @Override
+    public boolean grant_privileges(PrivilegeBag privileges)
+            throws MetaException, TException {
+        return call(client -> client.grant_privileges(privileges), "grant_privileges", privileges);
+    }
+
+    @Override
+    public boolean revoke_privileges(PrivilegeBag privileges, boolean grantOption)
+            throws MetaException, TException {
+        return call(client -> client.revoke_privileges(privileges, grantOption), "revoke_privileges", privileges, grantOption);
+    }
+
+    @Override
+    public boolean refresh_privileges(HiveObjectRef objToRefresh, String authorizer, PrivilegeBag grantPrivileges) throws MetaException, TException {
+        return call(client -> client.refresh_privileges(objToRefresh, authorizer, grantPrivileges), "refresh_privileges", objToRefresh, authorizer, grantPrivileges);
+    }
+
+    @Override
+    public void heartbeat(long txnId, long lockId)
+            throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, TException {
+        run(client -> client.heartbeat(txnId, lockId), "heartbeat", txnId, lockId);
+    }
+
+    @Override
+    public HeartbeatTxnRangeResponse heartbeatTxnRange(long min, long max) throws TException {
+        return call(client -> client.heartbeatTxnRange(min, max), "heartbeatTxnRange", min, max);
+    }
+
+    @Override
+    public boolean isPartitionMarkedForEvent(
+            String dbName,
+            String tblName,
+            Map<String, String> partKVs,
+            PartitionEventType eventType
+    ) throws MetaException, NoSuchObjectException, TException, UnknownTableException, UnknownDBException,
+            UnknownPartitionException, InvalidPartitionException {
+        return call(this.readWriteClient, client -> client.isPartitionMarkedForEvent(dbName, tblName, partKVs, eventType), "isPartitionMarkedForEvent", dbName, tblName, partKVs, eventType);
+    }
+
+    @Override
+    public boolean isPartitionMarkedForEvent(String catalogId, String db_name, String tbl_name, Map<String, String> partKVs, PartitionEventType eventType) throws MetaException, NoSuchObjectException, TException, UnknownTableException, UnknownDBException, UnknownPartitionException, InvalidPartitionException {
+        return call(this.readWriteClient, client -> client.isPartitionMarkedForEvent(catalogId, db_name, tbl_name, partKVs, eventType), "isPartitionMarkedForEvent", catalogId, db_name, tbl_name, partKVs, eventType);
+    }
+
+    @Override
+    public List<String> listPartitionNames(String dbName, String tblName, short max)
+            throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.listPartitionNames(dbName, tblName, max), "listPartitionNames", dbName, tblName, max);
+    }
+
+    @Override
+    public List<String> listPartitionNames(String catalogId, String dbName, String tblName, int max) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.listPartitionNames(catalogId, dbName, tblName, max), "listPartitionNames", catalogId, dbName, tblName, max);
+    }
+
+    @Override
+    public List<String> listPartitionNames(
+            String databaseName,
+            String tableName,
+            List<String> values,
+            short max
+    ) throws MetaException, TException, NoSuchObjectException {
+        return call(this.readWriteClient, client -> client.listPartitionNames(databaseName, tableName, values, max), "listPartitionNames", databaseName, tableName, values, max);
+    }
+
+    @Override
+    public List<String> listPartitionNames(
+            String catalogId,
+            String databaseName,
+            String tableName,
+            List<String> values,
+            int max
+    ) throws MetaException, TException, NoSuchObjectException {
+        return call(this.readWriteClient, client -> client.listPartitionNames(catalogId, databaseName, tableName, values, max), "listPartitionNames", catalogId, databaseName, tableName, values, max);
+    }
+
+    @Override
+    public PartitionValuesResponse listPartitionValues(PartitionValuesRequest partitionValuesRequest) throws MetaException, TException, NoSuchObjectException {
+        return call(this.readWriteClient, client -> client.listPartitionValues(partitionValuesRequest), "listPartitionValues", partitionValuesRequest);
+    }
+
+    @Override
+    public int getNumPartitionsByFilter(
+            String dbName,
+            String tableName,
+            String filter
+    ) throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getNumPartitionsByFilter(dbName, tableName, filter), "getNumPartitionsByFilter", dbName, tableName, filter);
+    }
+
+    @Override
+    public int getNumPartitionsByFilter(
+            String catalogId,
+            String dbName,
+            String tableName,
+            String filter
+    ) throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getNumPartitionsByFilter(catalogId, dbName, tableName, filter), "getNumPartitionsByFilter", catalogId, dbName, tableName, filter);
+    }
+
+    @Override
+    public PartitionSpecProxy listPartitionSpecs(
+            String dbName,
+            String tblName,
+            int max
+    ) throws TException {
+        return call(this.readWriteClient, client -> client.listPartitionSpecs(dbName, tblName, max), "listPartitionSpecs", dbName, tblName, max);
+    }
+
+    @Override
+    public PartitionSpecProxy listPartitionSpecs(
+            String catalogId,
+            String dbName,
+            String tblName,
+            int max
+    ) throws TException {
+        return call(this.readWriteClient, client -> client.listPartitionSpecs(catalogId, dbName, tblName, max), "listPartitionSpecs", catalogId, dbName, tblName, max);
+    }
+
+    @Override
+    public PartitionSpecProxy listPartitionSpecsByFilter(
+            String dbName,
+            String tblName,
+            String filter,
+            int max
+    ) throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.listPartitionSpecsByFilter(dbName, tblName, filter, max), "listPartitionSpecsByFilter", dbName, tblName, filter, max);
+    }
+
+    @Override
+    public PartitionSpecProxy listPartitionSpecsByFilter(
+            String catalogId,
+            String dbName,
+            String tblName,
+            String filter,
+            int max
+    ) throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.listPartitionSpecsByFilter(catalogId, dbName, tblName, filter, max), "listPartitionSpecsByFilter", catalogId, dbName, tblName, filter, max);
+    }
+
+    @Override
+    public List<Partition> listPartitions(String dbName, String tblName, short max)
+            throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.listPartitions(dbName, tblName, max), "listPartitions", dbName, tblName, max);
+    }
+
+    @Override
+    public List<Partition> listPartitions(String catalogId, String dbName, String tblName, int max) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.listPartitions(catalogId, dbName, tblName, max), "listPartitions", catalogId, dbName, tblName, max);
+    }
+
+    @Override
+    public List<Partition> listPartitions(
+            String databaseName,
+            String tableName,
+            List<String> values,
+            short max
+    ) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.listPartitions(databaseName, tableName, values, max), "listPartitions", databaseName, tableName, values, max);
+    }
+
+    @Override
+    public List<Partition> listPartitions(String catalogId, String databaseName, String tableName, List<String> values, int max) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.listPartitions(catalogId, databaseName, tableName, values, max), "listPartitions", catalogId, databaseName, tableName, values, max);
+    }
+
+    @Override
+    public boolean listPartitionsByExpr(
+            String databaseName,
+            String tableName,
+            byte[] expr,
+            String defaultPartitionName,
+            short max,
+            List<Partition> result
+    ) throws TException {
+        return call(this.readWriteClient, client -> client.listPartitionsByExpr(databaseName, tableName, expr, defaultPartitionName, max, result), "listPartitionsByExpr", databaseName, tableName, expr, defaultPartitionName, max, result);
+    }
+
+    @Override
+    public boolean listPartitionsByExpr(
+            String catalogId,
+            String databaseName,
+            String tableName,
+            byte[] expr,
+            String defaultPartitionName,
+            int max,
+            List<Partition> result
+    ) throws TException {
+        return call(this.readWriteClient, client -> client.listPartitionsByExpr(catalogId, databaseName, tableName, expr, defaultPartitionName, max, result), "listPartitionsByExpr", catalogId, databaseName, tableName, expr, defaultPartitionName, max, result);
+    }
+
+    @Override
+    public List<Partition> listPartitionsByFilter(
+            String databaseName,
+            String tableName,
+            String filter,
+            short max
+    ) throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.listPartitionsByFilter(databaseName, tableName, filter, max), "listPartitionsByFilter", databaseName, tableName, filter, max);
+    }
+
+    @Override
+    public List<Partition> listPartitionsByFilter(
+            String catalogId,
+            String databaseName,
+            String tableName,
+            String filter,
+            int max
+    ) throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.listPartitionsByFilter(catalogId, databaseName, tableName, filter, max), "listPartitionsByFilter", catalogId, databaseName, tableName, filter, max);
+    }
+
+    @Override
+    public List<Partition> listPartitionsWithAuthInfo(
+            String database,
+            String table,
+            short maxParts,
+            String user,
+            List<String> groups
+    ) throws MetaException, TException, NoSuchObjectException {
+        return call(this.readWriteClient, client -> client.listPartitionsWithAuthInfo(database, table, maxParts, user, groups), "listPartitionsWithAuthInfo", database, table, maxParts, user, groups);
+    }
+
+    @Override
+    public List<Partition> listPartitionsWithAuthInfo(
+            String catalogId,
+            String database,
+            String table,
+            int maxParts,
+            String user,
+            List<String> groups
+    ) throws MetaException, TException, NoSuchObjectException {
+        return call(this.readWriteClient, client -> client.listPartitionsWithAuthInfo(catalogId, database, table, maxParts, user, groups), "listPartitionsWithAuthInfo", catalogId, database, table, maxParts, user, groups);
+    }
+
+    @Override
+    public List<Partition> listPartitionsWithAuthInfo(
+            String database,
+            String table,
+            List<String> partVals,
+            short maxParts,
+            String user,
+            List<String> groups
+    ) throws MetaException, TException, NoSuchObjectException {
+        return call(this.readWriteClient, client -> client.listPartitionsWithAuthInfo(database, table, partVals, maxParts, user, groups), "listPartitionsWithAuthInfo", database, table, partVals, maxParts, user, groups);
+    }
+
+    @Override
+    public List<Partition> listPartitionsWithAuthInfo(String catalogId, String database, String table, List<String> partVals, int maxParts, String user, List<String> groups) throws MetaException, TException, NoSuchObjectException {
+        return call(this.readWriteClient, client -> client.listPartitionsWithAuthInfo(catalogId, database, table, partVals, maxParts, user, groups), "listPartitionsWithAuthInfo", catalogId, database, table, partVals, maxParts, user, groups);
+    }
+
+    @Override
+    public List<String> listTableNamesByFilter(
+            String dbName,
+            String filter,
+            short maxTables
+    ) throws MetaException, TException, InvalidOperationException, UnknownDBException, UnsupportedOperationException {
+        return call(this.readWriteClient, client -> client.listTableNamesByFilter(dbName, filter, maxTables), "listTableNamesByFilter", dbName, filter, maxTables);
+    }
+
+    @Override
+    public List<String> listTableNamesByFilter(String catalogId, String dbName, String filter, int maxTables) throws TException, InvalidOperationException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.listTableNamesByFilter(catalogId, dbName, filter, maxTables), "listTableNamesByFilter", catalogId, dbName, filter, maxTables);
+    }
+
+    @Override
+    public List<HiveObjectPrivilege> list_privileges(
+            String principal,
+            PrincipalType principalType,
+            HiveObjectRef objectRef
+    ) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.list_privileges(principal, principalType, objectRef), "list_privileges", principal, principalType, objectRef);
+    }
+
+    @Override
+    public LockResponse lock(LockRequest lockRequest) throws NoSuchTxnException, TxnAbortedException, TException {
+        return call(client -> client.lock(lockRequest), "lock", lockRequest);
+    }
+
+    @Override
+    public void markPartitionForEvent(
+            String dbName,
+            String tblName,
+            Map<String, String> partKVs,
+            PartitionEventType eventType
+    ) throws MetaException, NoSuchObjectException, TException, UnknownTableException, UnknownDBException,
+            UnknownPartitionException, InvalidPartitionException {
+        run(client -> client.markPartitionForEvent(dbName, tblName, partKVs, eventType), "markPartitionForEvent", dbName, tblName, partKVs, eventType);
+    }
+
+    @Override
+    public void markPartitionForEvent(
+            String catalogId,
+            String dbName,
+            String tblName,
+            Map<String, String> partKVs,
+            PartitionEventType eventType) throws MetaException, NoSuchObjectException, TException, UnknownTableException, UnknownDBException, UnknownPartitionException, InvalidPartitionException {
+        run(client -> client.markPartitionForEvent(catalogId, dbName, tblName, partKVs, eventType), "markPartitionForEvent", catalogId, dbName, tblName, partKVs, eventType);
+    }
+
+    @Override
+    public long openTxn(String user) throws TException {
+        return call(client -> client.openTxn(user), "openTxn", user);
+    }
+
+    @Override
+    public List<Long> replOpenTxn(String replPolicy, List<Long> srcTxnIds, String user) throws TException {
+        return call(client -> client.replOpenTxn(replPolicy, srcTxnIds, user), "replOpenTxn", replPolicy, srcTxnIds, user);
+    }
+
+    @Override
+    public OpenTxnsResponse openTxns(String user, int numTxns) throws TException {
+        return call(client -> client.openTxns(user, numTxns), "openTxns", numTxns);
+    }
+
+    @Override
+    public Map<String, String> partitionNameToSpec(String name) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.partitionNameToSpec(name), "partitionNameToSpec", name);
+    }
+
+    @Override
+    public List<String> partitionNameToVals(String name) throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.partitionNameToVals(name), "partitionNameToVals", name);
+    }
+
+    @Override
+    public void renamePartition(
+            String dbName,
+            String tblName,
+            List<String> partitionValues,
+            Partition newPartition
+    ) throws InvalidOperationException, MetaException, TException {
+        run(client -> client.renamePartition(dbName, tblName, partitionValues, newPartition), "renamePartition", dbName, tblName,
+                partitionValues, newPartition);
+    }
+
+    @Override
+    public void renamePartition(String catalogId, String dbName, String tblName, List<String> partitionValues, Partition newPartition) throws InvalidOperationException, MetaException, TException {
+        run(client -> client.renamePartition(catalogId, dbName, tblName, partitionValues, newPartition), "renamePartition", catalogId, dbName, tblName, partitionValues, newPartition);
+    }
+
+    @Override
+    public long renewDelegationToken(String tokenStrForm) throws MetaException, TException {
+        return call(client -> client.renewDelegationToken(tokenStrForm), "renewDelegationToken", tokenStrForm);
+    }
+
+    @Override
+    public void rollbackTxn(long txnId) throws NoSuchTxnException, TException {
+        run(client -> client.rollbackTxn(txnId), "rollbackTxn", txnId);
+    }
+
+    @Override
+    public void replRollbackTxn(long srcTxnId, String replPolicy) throws NoSuchTxnException, TException {
+        run(client -> client.replRollbackTxn(srcTxnId, replPolicy), "replRollbackTxn", srcTxnId, replPolicy);
+    }
+
+    @Override
+    public void setMetaConf(String key, String value) throws MetaException, TException {
+        run(client -> client.setMetaConf(key, value), "setMetaConf", key, value);
+    }
+
+    @Override
+    public boolean setPartitionColumnStatistics(SetPartitionsStatsRequest request)
+            throws NoSuchObjectException, InvalidObjectException,
+            MetaException, TException, InvalidInputException {
+        if (request.getColStatsSize() == 1) {
+            ColumnStatistics colStats = request.getColStatsIterator().next();
+            ColumnStatisticsDesc desc = colStats.getStatsDesc();
+            String dbName = desc.getDbName().toLowerCase();
+            String tableName = desc.getTableName().toLowerCase();
+            if (getTempTable(dbName, tableName) != null) {
+                return call(this.readWriteClient, client -> client.setPartitionColumnStatistics(request), "setPartitionColumnStatistics", request);
+            }
+        }
+        SetPartitionsStatsRequest deepCopy = request.deepCopy();
+        boolean result = readWriteClient.setPartitionColumnStatistics(deepCopy);
+        if (extraClient.isPresent()) {
+            try {
+                extraClient.get().setPartitionColumnStatistics(request);
+            } catch (Exception e) {
+                FunctionalUtils.collectLogs(e, "setPartitionColumnStatistics", request);
+                if (!allowFailure) {
+                    throw e;
+                }
+            }
+        }
+        return result;
+    }
+
+    @Override
+    public void flushCache() {
+        try {
+            run(client -> client.flushCache(), "flushCache");
+        } catch (TException e) {
+            logger.info(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public Iterable<Map.Entry<Long, ByteBuffer>> getFileMetadata(List<Long> fileIds) throws TException {
+        return call(this.readWriteClient, client -> client.getFileMetadata(fileIds), "getFileMetadata", fileIds);
+    }
+
+    @Override
+    public Iterable<Map.Entry<Long, MetadataPpdResult>> getFileMetadataBySarg(
+            List<Long> fileIds,
+            ByteBuffer sarg,
+            boolean doGetFooters
+    ) throws TException {
+        return call(this.readWriteClient, client -> client.getFileMetadataBySarg(fileIds, sarg, doGetFooters), "getFileMetadataBySarg", fileIds, sarg, doGetFooters);
+    }
+
+    @Override
+    public void clearFileMetadata(List<Long> fileIds) throws TException {
+        run(client -> client.clearFileMetadata(fileIds), "clearFileMetadata", fileIds);
+    }
+
+    @Override
+    public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) throws TException {
+        run(client -> client.putFileMetadata(fileIds, metadata), "putFileMetadata", fileIds, metadata);
+    }
+
+    @Override
+    public boolean isSameConfObj(Configuration conf) {
+        try {
+            return call(this.readWriteClient, client -> client.isSameConfObj(conf), "isSameConfObj", conf);
+        } catch (TException e) {
+            logger.error(e.getMessage(), e);
+        }
+        return false;
+    }
+
+    @Override
+    public boolean cacheFileMetadata(
+            String dbName,
+            String tblName,
+            String partName,
+            boolean allParts
+    ) throws TException {
+        return call(client -> client.cacheFileMetadata(dbName, tblName, partName, allParts), "cacheFileMetadata", dbName, tblName, partName, allParts);
+    }
+
+    @Override
+    public List<SQLPrimaryKey> getPrimaryKeys(PrimaryKeysRequest primaryKeysRequest)
+            throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getPrimaryKeys(primaryKeysRequest), "getPrimaryKeys", primaryKeysRequest);
+    }
+
+    @Override
+    public List<SQLForeignKey> getForeignKeys(ForeignKeysRequest foreignKeysRequest)
+            throws MetaException, NoSuchObjectException, TException {
+        // PrimaryKeys are currently unsupported
+        //return null to allow DESCRIBE (FORMATTED | EXTENDED)
+        return call(this.readWriteClient, client -> client.getForeignKeys(foreignKeysRequest), "getForeignKeys", foreignKeysRequest);
+    }
+
+    @Override
+    public List<SQLUniqueConstraint> getUniqueConstraints(UniqueConstraintsRequest uniqueConstraintsRequest) throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getUniqueConstraints(uniqueConstraintsRequest), "getUniqueConstraints", uniqueConstraintsRequest);
+    }
+
+    @Override
+    public List<SQLNotNullConstraint> getNotNullConstraints(NotNullConstraintsRequest notNullConstraintsRequest) throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getNotNullConstraints(notNullConstraintsRequest), "getNotNullConstraints", notNullConstraintsRequest);
+    }
+
+    @Override
+    public List<SQLDefaultConstraint> getDefaultConstraints(DefaultConstraintsRequest defaultConstraintsRequest) throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getDefaultConstraints(defaultConstraintsRequest), "getDefaultConstraints", defaultConstraintsRequest);
+    }
+
+    @Override
+    public List<SQLCheckConstraint> getCheckConstraints(CheckConstraintsRequest checkConstraintsRequest) throws MetaException, NoSuchObjectException, TException {
+        return call(this.readWriteClient, client -> client.getCheckConstraints(checkConstraintsRequest), "getCheckConstraints", checkConstraintsRequest);
+    }
+
+    @Override
+    public void createTableWithConstraints(
+            Table tbl,
+            List<SQLPrimaryKey> primaryKeys,
+            List<SQLForeignKey> foreignKeys,
+            List<SQLUniqueConstraint> uniqueConstraints,
+            List<SQLNotNullConstraint> notNullConstraints,
+            List<SQLDefaultConstraint> defaultConstraints,
+            List<SQLCheckConstraint> checkConstraints
+    ) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, TException {
+        run(client -> client.createTableWithConstraints(tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints, checkConstraints), "createTableWithConstraints", tbl, primaryKeys, foreignKeys, uniqueConstraints, notNullConstraints, defaultConstraints, checkConstraints);
+    }
+
+    @Override
+    public void dropConstraint(
+            String dbName,
+            String tblName,
+            String constraintName
+    ) throws MetaException, NoSuchObjectException, TException {
+        run(client -> client.dropConstraint(dbName, tblName, constraintName), "dropConstraint", dbName, tblName, constraintName);
+    }
+
+    @Override
+    public void dropConstraint(String catalogId, String dbName, String tableName, String constraintName) throws MetaException, NoSuchObjectException, TException {
+        run(client -> client.dropConstraint(catalogId, dbName, tableName, constraintName), "dropConstraint", catalogId, dbName, tableName, constraintName);
+    }
+
+    @Override
+    public void addPrimaryKey(List<SQLPrimaryKey> primaryKeyCols)
+            throws MetaException, NoSuchObjectException, TException {
+        run(client -> client.addPrimaryKey(primaryKeyCols), "addPrimaryKey", primaryKeyCols);
+    }
+
+    @Override
+    public void addForeignKey(List<SQLForeignKey> foreignKeyCols)
+            throws MetaException, NoSuchObjectException, TException {
+        run(client -> client.addForeignKey(foreignKeyCols), "addForeignKey", foreignKeyCols);
+    }
+
+    @Override
+    public void addUniqueConstraint(List<SQLUniqueConstraint> uniqueConstraintCols) throws MetaException, NoSuchObjectException, TException {
+        run(client -> client.addUniqueConstraint(uniqueConstraintCols), "addUniqueConstraint", uniqueConstraintCols);
+    }
+
+    @Override
+    public void addNotNullConstraint(List<SQLNotNullConstraint> notNullConstraintCols) throws MetaException, NoSuchObjectException, TException {
+        run(client -> client.addNotNullConstraint(notNullConstraintCols), "addNotNullConstraint", notNullConstraintCols);
+    }
+
+    @Override
+    public void addDefaultConstraint(List<SQLDefaultConstraint> defaultConstraints) throws MetaException, NoSuchObjectException, TException {
+        run(client -> client.addDefaultConstraint(defaultConstraints), "addDefaultConstraint", defaultConstraints);
+    }
+
+    @Override
+    public void addCheckConstraint(List<SQLCheckConstraint> checkConstraints) throws MetaException, NoSuchObjectException, TException {
+        run(client -> client.addCheckConstraint(checkConstraints), "addCheckConstraint", checkConstraints);
+    }
+
+    @Override
+    public String getMetastoreDbUuid() throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getMetastoreDbUuid(), "getMetastoreDbUuid");
+    }
+
+    @Override
+    public void createResourcePlan(WMResourcePlan wmResourcePlan, String copyFromName) throws InvalidObjectException, MetaException, TException {
+        run(client -> client.createResourcePlan(wmResourcePlan, copyFromName), "createResourcePlan", wmResourcePlan, copyFromName);
+    }
+
+    @Override
+    public WMFullResourcePlan getResourcePlan(String resourcePlanName) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getResourcePlan(resourcePlanName), "getResourcePlan", resourcePlanName);
+    }
+
+    @Override
+    public List<WMResourcePlan> getAllResourcePlans() throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getAllResourcePlans(), "getAllResourcePlans");
+    }
+
+    @Override
+    public void dropResourcePlan(String resourcePlanName) throws NoSuchObjectException, MetaException, TException {
+        run(client -> client.dropResourcePlan(resourcePlanName), "dropResourcePlan", resourcePlanName);
+    }
+
+    @Override
+    public WMFullResourcePlan alterResourcePlan(String resourcePlanName, WMNullableResourcePlan wmNullableResourcePlan, boolean canActivateDisabled, boolean isForceDeactivate, boolean isReplace) throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
+        return call(client -> client.alterResourcePlan(resourcePlanName, wmNullableResourcePlan, canActivateDisabled, isForceDeactivate, isReplace), "alterResourcePlan", resourcePlanName, wmNullableResourcePlan, canActivateDisabled, isForceDeactivate, isReplace);
+    }
+
+    @Override
+    public WMFullResourcePlan getActiveResourcePlan() throws MetaException, TException {
+        return call(this.readWriteClient, client -> client.getActiveResourcePlan(), "getActiveResourcePlan");
+    }
+
+    @Override
+    public WMValidateResourcePlanResponse validateResourcePlan(String resourcePlanName) throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.validateResourcePlan(resourcePlanName), "validateResourcePlan", resourcePlanName);
+    }
+
+    @Override
+    public void createWMTrigger(WMTrigger wmTrigger) throws InvalidObjectException, MetaException, TException {
+        run(client -> client.createWMTrigger(wmTrigger), "createWMTrigger", wmTrigger);
+    }
+
+    @Override
+    public void alterWMTrigger(WMTrigger wmTrigger) throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
+        run(client -> client.alterWMTrigger(wmTrigger), "alterWMTrigger", wmTrigger);
+    }
+
+    @Override
+    public void dropWMTrigger(String resourcePlanName, String triggerName) throws NoSuchObjectException, MetaException, TException {
+        run(client -> client.dropWMTrigger(resourcePlanName, triggerName), "dropWMTrigger", resourcePlanName, triggerName);
+    }
+
+    @Override
+    public List<WMTrigger> getTriggersForResourcePlan(String resourcePlan) throws NoSuchObjectException, MetaException, TException {
+        return call(this.readWriteClient, client -> client.getTriggersForResourcePlan(resourcePlan), "getTriggersForResourcePlan", resourcePlan);
+    }
+
+    @Override
+    public void createWMPool(WMPool wmPool) throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
+        run(client -> client.createWMPool(wmPool), "createWMPool", wmPool);
+    }
+
+    @Override
+    public void alterWMPool(WMNullablePool wmNullablePool, String poolPath) throws NoSuchObjectException, InvalidObjectException, TException {
+        run(client -> client.alterWMPool(wmNullablePool, poolPath), "alterWMPool", wmNullablePool, poolPath);
+    }
+
+    @Override
+    public void dropWMPool(String resourcePlanName, String poolPath) throws TException {
+        run(client -> client.dropWMPool(resourcePlanName, poolPath), "dropWMPool", resourcePlanName, poolPath);
+    }
+
+    @Override
+    public void createOrUpdateWMMapping(WMMapping wmMapping, boolean isUpdate) throws TException {
+        run(client -> client.createOrUpdateWMMapping(wmMapping, isUpdate), "createOrUpdateWMMapping", wmMapping, isUpdate);
+    }
+
+    @Override
+    public void dropWMMapping(WMMapping wmMapping) throws TException {
+        run(client -> client.dropWMMapping(wmMapping), "dropWMMapping", wmMapping);
+    }
+
+    @Override
+    public void createOrDropTriggerToPoolMapping(String resourcePlanName, String triggerName, String poolPath, boolean shouldDrop) throws AlreadyExistsException, NoSuchObjectException, InvalidObjectException, MetaException, TException {
+        run(client -> client.createOrDropTriggerToPoolMapping(resourcePlanName, triggerName, poolPath, shouldDrop), "createOrDropTriggerToPoolMapping", resourcePlanName, triggerName, poolPath, shouldDrop);
+    }
+
+    @Override
+    public void createISchema(ISchema iSchema) throws TException {
+        run(client -> client.createISchema(iSchema), "createISchema", iSchema);
+    }
+
+    @Override
+    public void alterISchema(String catName, String dbName, String schemaName, ISchema newSchema) throws TException {
+        run(client -> client.alterISchema(catName, dbName, schemaName, newSchema), "alterISchema", catName, dbName, schemaName, newSchema);
+    }
+
+    @Override
+    public ISchema getISchema(String catalogId, String dbName, String name) throws TException {
+        return call(this.readWriteClient, client -> client.getISchema(catalogId, dbName, name), "getISchema", catalogId, dbName, name);
+    }
+
+    @Override
+    public void dropISchema(String catalogId, String dbName, String name) throws TException {
+        run(client -> client.dropISchema(catalogId, dbName, name), "dropISchema", catalogId, dbName, name);
+    }
+
+    @Override
+    public void addSchemaVersion(SchemaVersion schemaVersion) throws TException {
+        run(client -> client.addSchemaVersion(schemaVersion), "addSchemaVersion", schemaVersion);
+    }
+
+    @Override
+    public SchemaVersion getSchemaVersion(String catalogId, String dbName, String schemaName, int version) throws TException {
+        return call(this.readWriteClient, client -> client.getSchemaVersion(catalogId, dbName, schemaName, version), "getSchemaVersion", catalogId, dbName, schemaName, version);
+    }
+
+    @Override
+    public SchemaVersion getSchemaLatestVersion(String catalogId, String dbName, String schemaName) throws TException {
+        return call(this.readWriteClient, client -> client.getSchemaLatestVersion(catalogId, dbName, schemaName), "getSchemaLatestVersion", catalogId, dbName, schemaName);
+    }
+
+    @Override
+    public List<SchemaVersion> getSchemaAllVersions(String catalogId, String dbName, String schemaName) throws TException {
+        return call(this.readWriteClient, client -> client.getSchemaAllVersions(catalogId, dbName, schemaName), "getSchemaAllVersions", catalogId, dbName, schemaName);
+    }
+
+    @Override
+    public void dropSchemaVersion(String catalogId, String dbName, String schemaName, int version) throws TException {
+        run(client -> client.dropSchemaVersion(catalogId, dbName, schemaName, version), "dropSchemaVersion", catalogId, dbName, schemaName, version);
+    }
+
+    @Override
+    public FindSchemasByColsResp getSchemaByCols(FindSchemasByColsRqst findSchemasByColsRqst) throws TException {
+        return call(this.readWriteClient, client -> client.getSchemaByCols(findSchemasByColsRqst), "getSchemaByCols", findSchemasByColsRqst);
+    }
+
+    @Override
+    public void mapSchemaVersionToSerde(String catalogId, String dbName, String schemaName, int version, String serdeName) throws TException {
+        run(client -> client.mapSchemaVersionToSerde(catalogId, dbName, schemaName, version, serdeName), "mapSchemaVersionToSerde", catalogId, dbName, schemaName, version, serdeName);
+    }
+
+    @Override
+    public void setSchemaVersionState(String catalogId, String dbName, String schemaName, int version, SchemaVersionState state) throws TException {
+        run(client -> client.setSchemaVersionState(catalogId, dbName, schemaName, version, state), "setSchemaVersionState", catalogId, dbName, schemaName, version, state);
+    }
+
+    @Override
+    public void addSerDe(SerDeInfo serDeInfo) throws TException {
+        run(client -> client.addSerDe(serDeInfo), "addSerDe", serDeInfo);
+    }
+
+    @Override
+    public SerDeInfo getSerDe(String serDeName) throws TException {
+        return call(this.readWriteClient, client -> client.getSerDe(serDeName), "getSerDe", serDeName);
+    }
+
+    @Override
+    public LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId) throws TException {
+        return call(client -> client.lockMaterializationRebuild(dbName, tableName, txnId), "lockMaterializationRebuild", dbName, tableName, txnId);
+    }
+
+    @Override
+    public boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId) throws TException {
+        return call(client -> client.heartbeatLockMaterializationRebuild(dbName, tableName, txnId), "heartbeatLockMaterializationRebuild", dbName, tableName, txnId);
+    }
+
+    @Override
+    public void addRuntimeStat(RuntimeStat runtimeStat) throws TException {
+        run(client -> client.addRuntimeStat(runtimeStat), "addRuntimeStat", runtimeStat);
+    }
+
+    @Override
+    public List<RuntimeStat> getRuntimeStats(int maxWeight, int maxCreateTime) throws TException {
+        return call(this.readWriteClient, client -> client.getRuntimeStats(maxWeight, maxCreateTime), "getRuntimeStats", maxWeight, maxCreateTime);
+    }
+
+    @Override
+    public ShowCompactResponse showCompactions() throws TException {
+        return call(this.readWriteClient, client -> client.showCompactions(), "showCompactions");
+    }
+
+    @Override
+    public void addDynamicPartitions(long txnId, long writeId, String dbName, String tableName, List<String> partNames) throws TException {
+        run(client -> client.addDynamicPartitions(txnId, writeId, dbName, tableName, partNames), "addDynamicPartitions", txnId, writeId, dbName, tableName, partNames);
+    }
+
+    @Override
+    public void addDynamicPartitions(long txnId, long writeId, String dbName, String tableName, List<String> partNames, DataOperationType operationType) throws TException {
+        run(client -> client.addDynamicPartitions(txnId, writeId, dbName, tableName, partNames, operationType), "addDynamicPartitions", txnId, writeId, dbName, tableName, partNames, operationType);
+    }
+
+    @Override
+    public void insertTable(Table table, boolean overwrite) throws MetaException {
+        try {
+            run(client -> client.insertTable(table, overwrite), "insertTable", table, overwrite);
+        } catch (TException e) {
+            throw DataLakeUtil.throwException(new MetaException(e.getMessage()), e);
+        }
+    }
+
+    @Override
+    public NotificationEventResponse getNextNotification(
+            long lastEventId,
+            int maxEvents,
+            NotificationFilter notificationFilter
+    ) throws TException {
+        return call(this.readWriteClient, client -> client.getNextNotification(lastEventId, maxEvents, notificationFilter), "getNextNotification", lastEventId, maxEvents, notificationFilter);
+    }
+
+    @Override
+    public CurrentNotificationEventId getCurrentNotificationEventId() throws TException {
+        return call(this.readWriteClient, client -> client.getCurrentNotificationEventId(), "getCurrentNotificationEventId");
+    }
+
+    @Override
+    public NotificationEventsCountResponse getNotificationEventsCount(NotificationEventsCountRequest notificationEventsCountRequest) throws TException {
+        return call(this.readWriteClient, client -> client.getNotificationEventsCount(notificationEventsCountRequest), "getNotificationEventsCount", notificationEventsCountRequest);
+    }
+
+    @Override
+    public FireEventResponse fireListenerEvent(FireEventRequest fireEventRequest) throws TException {
+        return call(this.readWriteClient, client -> client.fireListenerEvent(fireEventRequest), "fireListenerEvent", fireEventRequest);
+    }
+
+    @Override
+    @Deprecated
+    public ShowLocksResponse showLocks() throws TException {
+        return call(this.readWriteClient, client -> client.showLocks(), "showLocks");
+    }
+
+    @Override
+    public ShowLocksResponse showLocks(ShowLocksRequest showLocksRequest) throws TException {
+        return call(this.readWriteClient, client -> client.showLocks(showLocksRequest), "showLocks", showLocksRequest);
+    }
+
+    @Override
+    public GetOpenTxnsInfoResponse showTxns() throws TException {
+        return call(this.readWriteClient, client -> client.showTxns(), "showTxns");
+    }
+
+    @Override
+    public boolean tableExists(String databaseName, String tableName)
+            throws MetaException, TException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.tableExists(databaseName, tableName), "tableExists", databaseName, tableName);
+    }
+
+    @Override
+    public boolean tableExists(String catalogId, String databaseName, String tableName) throws MetaException, TException, UnknownDBException {
+        return call(this.readWriteClient, client -> client.tableExists(catalogId, databaseName, tableName), "tableExists", catalogId, databaseName, tableName);
+    }
+
+    @Override
+    public void unlock(long lockId) throws NoSuchLockException, TxnOpenException, TException {
+        run(client -> client.unlock(lockId), "unlock", lockId);
+    }
+
+    @Override
+    public boolean updatePartitionColumnStatistics(ColumnStatistics columnStatistics)
+            throws NoSuchObjectException, InvalidObjectException, MetaException, TException, InvalidInputException {
+        return call(client -> client.updatePartitionColumnStatistics(columnStatistics), "updatePartitionColumnStatistics", columnStatistics);
+    }
+
+    @Override
+    public boolean updateTableColumnStatistics(ColumnStatistics columnStatistics)
+            throws NoSuchObjectException, InvalidObjectException, MetaException, TException, InvalidInputException {
+        if (getTempTable(columnStatistics.getStatsDesc().getDbName(), columnStatistics.getStatsDesc().getTableName()) != null) {
+            return call(this.readWriteClient, client -> client.updateTableColumnStatistics(columnStatistics), "updateTableColumnStatistics", columnStatistics);
+        } else {
+            return call(client -> client.updateTableColumnStatistics(columnStatistics), "updateTableColumnStatistics", columnStatistics);
+        }
+    }
+
+    @Override
+    public void validatePartitionNameCharacters(List<String> part_vals) throws TException, MetaException {
+        run(this.readWriteClient, client -> client.validatePartitionNameCharacters(part_vals), "validatePartitionNameCharacters", part_vals);
+    }
+
+    @VisibleForTesting
+    public IMetaStoreClient getDlfSessionMetaStoreClient() {
+        return dlfSessionMetaStoreClient;
+    }
+
+    @VisibleForTesting
+    public IMetaStoreClient getHiveSessionMetaStoreClient() {
+        return hiveSessionMetaStoreClient;
+    }
+
+    @VisibleForTesting
+    boolean isAllowFailure() {
+        return allowFailure;
+    }
+
+    public void run(ThrowingConsumer<IMetaStoreClient, TException> consumer, String actionName, Object... parameters) throws TException {
+        FunctionalUtils.run(this.readWriteClient, extraClient, allowFailure, consumer, this.readWriteClientType, actionName, parameters);
+    }
+
+    public void run(IMetaStoreClient client, ThrowingConsumer<IMetaStoreClient, TException> consumer,
+            String actionName, Object... parameters) throws TException {
+        FunctionalUtils.run(client, Optional.empty(), allowFailure, consumer, this.readWriteClientType, actionName, parameters);
+    }
+
+    public <R> R call(ThrowingFunction<IMetaStoreClient, R, TException> consumer,
+            String actionName, Object... parameters) throws TException {
+        return FunctionalUtils.call(this.readWriteClient, extraClient, allowFailure, consumer,
+                this.readWriteClientType, actionName, parameters);
+    }
+
+    public <R> R call(IMetaStoreClient client, ThrowingFunction<IMetaStoreClient, R, TException> consumer,
+            String actionName, Object... parameters) throws TException {
+        return FunctionalUtils.call(client, Optional.empty(), allowFailure, consumer, this.readWriteClientType,
+                actionName, parameters);
+    }
+}
\ No newline at end of file
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/BaseCatalogToHiveConverter.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/BaseCatalogToHiveConverter.java
new file mode 100644
index 0000000000..6c788ddeb8
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/BaseCatalogToHiveConverter.java
@@ -0,0 +1,541 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.converters;
+
+import com.amazonaws.services.glue.model.BinaryColumnStatisticsData;
+import com.amazonaws.services.glue.model.BooleanColumnStatisticsData;
+import com.amazonaws.services.glue.model.ColumnStatistics;
+import com.amazonaws.services.glue.model.ColumnStatisticsType;
+import com.amazonaws.services.glue.model.DateColumnStatisticsData;
+import com.amazonaws.services.glue.model.DecimalColumnStatisticsData;
+import com.amazonaws.services.glue.model.DoubleColumnStatisticsData;
+import com.amazonaws.services.glue.model.ErrorDetail;
+import com.amazonaws.services.glue.model.LongColumnStatisticsData;
+import com.amazonaws.services.glue.model.StringColumnStatisticsData;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import static org.apache.commons.lang3.ObjectUtils.firstNonNull;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DateColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.Decimal;
+import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.FunctionType;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.ResourceType;
+import org.apache.hadoop.hive.metastore.api.ResourceUri;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.SkewedInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+public class BaseCatalogToHiveConverter implements CatalogToHiveConverter {
+
+  private static final Logger logger = Logger.getLogger(BaseCatalogToHiveConverter.class);
+
+  private static final ImmutableMap<String, HiveException> EXCEPTION_MAP = ImmutableMap.<String, HiveException>builder()
+      .put("AlreadyExistsException", new HiveException() {
+        public TException get(String msg) {
+          return new AlreadyExistsException(msg);
+        }
+      })
+      .put("InvalidInputException", new HiveException() {
+        public TException get(String msg) {
+          return new InvalidObjectException(msg);
+        }
+      })
+      .put("InternalServiceException", new HiveException() {
+        public TException get(String msg) {
+          return new MetaException(msg);
+        }
+      })
+      .put("ResourceNumberLimitExceededException", new HiveException() {
+        public TException get(String msg) {
+          return new MetaException(msg);
+        }
+      })
+      .put("OperationTimeoutException", new HiveException() {
+        public TException get(String msg) {
+          return new MetaException(msg);
+        }
+      })
+      .put("EntityNotFoundException", new HiveException() {
+        public TException get(String msg) {
+          return new NoSuchObjectException(msg);
+        }
+      })
+      .build();
+
+  interface HiveException {
+    TException get(String msg);
+  }
+
+  public TException wrapInHiveException(Throwable e) {
+    return getHiveException(e.getClass().getSimpleName(), e.getMessage());
+  }
+
+  public TException errorDetailToHiveException(ErrorDetail errorDetail) {
+    return getHiveException(errorDetail.getErrorCode(), errorDetail.getErrorMessage());
+  }
+
+  private TException getHiveException(String errorName, String errorMsg) {
+    if (EXCEPTION_MAP.containsKey(errorName)) {
+      return EXCEPTION_MAP.get(errorName).get(errorMsg);
+    } else {
+      logger.warn("Hive Exception type not found for " + errorName);
+      return new MetaException(errorMsg);
+    }
+  }
+
+  public Database convertDatabase(com.amazonaws.services.glue.model.Database catalogDatabase) {
+    Database hiveDatabase = new Database();
+    hiveDatabase.setName(catalogDatabase.getName());
+    hiveDatabase.setDescription(catalogDatabase.getDescription());
+    String location = catalogDatabase.getLocationUri();
+    hiveDatabase.setLocationUri(location == null ? "" : location);
+    hiveDatabase.setParameters(firstNonNull(catalogDatabase.getParameters(), Maps.<String, String>newHashMap()));
+    return hiveDatabase;
+  }
+
+  public FieldSchema convertFieldSchema(com.amazonaws.services.glue.model.Column catalogFieldSchema) {
+    FieldSchema hiveFieldSchema = new FieldSchema();
+    hiveFieldSchema.setType(catalogFieldSchema.getType());
+    hiveFieldSchema.setName(catalogFieldSchema.getName());
+    hiveFieldSchema.setComment(catalogFieldSchema.getComment());
+
+    return hiveFieldSchema;
+  }
+
+  public List<FieldSchema> convertFieldSchemaList(List<com.amazonaws.services.glue.model.Column> catalogFieldSchemaList) {
+    List<FieldSchema> hiveFieldSchemaList = new ArrayList<>();
+    if (catalogFieldSchemaList == null) {
+      return hiveFieldSchemaList;
+    }
+    for (com.amazonaws.services.glue.model.Column catalogFieldSchema : catalogFieldSchemaList){
+      hiveFieldSchemaList.add(convertFieldSchema(catalogFieldSchema));
+    }
+
+    return hiveFieldSchemaList;
+  }
+
+  public Table convertTable(com.amazonaws.services.glue.model.Table catalogTable, String dbname) {
+    Table hiveTable = new Table();
+    hiveTable.setDbName(dbname);
+    hiveTable.setTableName(catalogTable.getName());
+    Date createTime = catalogTable.getCreateTime();
+    hiveTable.setCreateTime(createTime == null ? 0 : (int) (createTime.getTime() / 1000));
+    hiveTable.setOwner(catalogTable.getOwner());
+    Date lastAccessedTime = catalogTable.getLastAccessTime();
+    hiveTable.setLastAccessTime(lastAccessedTime == null ? 0 : (int) (lastAccessedTime.getTime() / 1000));
+    hiveTable.setRetention(catalogTable.getRetention());
+    hiveTable.setSd(convertStorageDescriptor(catalogTable.getStorageDescriptor()));
+    hiveTable.setPartitionKeys(convertFieldSchemaList(catalogTable.getPartitionKeys()));
+    // Hive may throw a NPE during dropTable if the parameter map is null.
+    Map<String, String> parameterMap = catalogTable.getParameters();
+    if (parameterMap == null) {
+      parameterMap = Maps.newHashMap();
+    }
+    hiveTable.setParameters(parameterMap);
+    hiveTable.setViewOriginalText(catalogTable.getViewOriginalText());
+    hiveTable.setViewExpandedText(catalogTable.getViewExpandedText());
+    hiveTable.setTableType(catalogTable.getTableType());
+
+    return hiveTable;
+  }
+
+  public TableMeta convertTableMeta(com.amazonaws.services.glue.model.Table catalogTable, String dbName) {
+    TableMeta tableMeta = new TableMeta();
+    tableMeta.setDbName(dbName);
+    tableMeta.setTableName(catalogTable.getName());
+    tableMeta.setTableType(catalogTable.getTableType());
+    if (catalogTable.getParameters().containsKey("comment")) {
+      tableMeta.setComments(catalogTable.getParameters().get("comment"));
+    }
+    return tableMeta;
+  }
+
+  public StorageDescriptor convertStorageDescriptor(com.amazonaws.services.glue.model.StorageDescriptor catalogSd) {
+    StorageDescriptor hiveSd = new StorageDescriptor();
+    hiveSd.setCols(convertFieldSchemaList(catalogSd.getColumns()));
+    hiveSd.setLocation(catalogSd.getLocation());
+    hiveSd.setInputFormat(catalogSd.getInputFormat());
+    hiveSd.setOutputFormat(catalogSd.getOutputFormat());
+    hiveSd.setCompressed(catalogSd.getCompressed());
+    hiveSd.setNumBuckets(catalogSd.getNumberOfBuckets());
+    hiveSd.setSerdeInfo(convertSerDeInfo(catalogSd.getSerdeInfo()));
+    hiveSd.setBucketCols(firstNonNull(catalogSd.getBucketColumns(), Lists.<String>newArrayList()));
+    hiveSd.setSortCols(convertOrderList(catalogSd.getSortColumns()));
+    hiveSd.setParameters(firstNonNull(catalogSd.getParameters(), Maps.<String, String>newHashMap()));
+    hiveSd.setSkewedInfo(convertSkewedInfo(catalogSd.getSkewedInfo()));
+    hiveSd.setStoredAsSubDirectories(catalogSd.getStoredAsSubDirectories());
+
+    return hiveSd;
+  }
+
+  public Order convertOrder(com.amazonaws.services.glue.model.Order catalogOrder) {
+    Order hiveOrder = new Order();
+    hiveOrder.setCol(catalogOrder.getColumn());
+    hiveOrder.setOrder(catalogOrder.getSortOrder());
+
+    return hiveOrder;
+  }
+
+  public List<Order> convertOrderList(List<com.amazonaws.services.glue.model.Order> catalogOrderList) {
+    List<Order> hiveOrderList = new ArrayList<>();
+    if (catalogOrderList == null) {
+      return hiveOrderList;
+    }
+    for (com.amazonaws.services.glue.model.Order catalogOrder : catalogOrderList){
+      hiveOrderList.add(convertOrder(catalogOrder));
+    }
+
+    return hiveOrderList;
+  }
+
+  public SerDeInfo convertSerDeInfo(com.amazonaws.services.glue.model.SerDeInfo catalogSerDeInfo){
+    SerDeInfo hiveSerDeInfo = new SerDeInfo();
+    hiveSerDeInfo.setName(catalogSerDeInfo.getName());
+    hiveSerDeInfo.setParameters(firstNonNull(catalogSerDeInfo.getParameters(), Maps.<String, String>newHashMap()));
+    hiveSerDeInfo.setSerializationLib(catalogSerDeInfo.getSerializationLibrary());
+
+    return hiveSerDeInfo;
+  }
+
+  public SkewedInfo convertSkewedInfo(com.amazonaws.services.glue.model.SkewedInfo catalogSkewedInfo) {
+    if (catalogSkewedInfo == null) {
+      return null;
+    }
+    
+    SkewedInfo hiveSkewedInfo = new SkewedInfo();
+    hiveSkewedInfo.setSkewedColNames(firstNonNull(catalogSkewedInfo.getSkewedColumnNames(), Lists.<String>newArrayList()));
+    hiveSkewedInfo.setSkewedColValues(convertSkewedValue(catalogSkewedInfo.getSkewedColumnValues()));
+    hiveSkewedInfo.setSkewedColValueLocationMaps(convertSkewedMap(catalogSkewedInfo.getSkewedColumnValueLocationMaps()));
+    return hiveSkewedInfo;
+  }
+
+  public Partition convertPartition(com.amazonaws.services.glue.model.Partition src) {
+	  Partition tgt = new Partition();
+	  Date createTime = src.getCreationTime();
+	  if (createTime != null) {
+		  tgt.setCreateTime((int) (createTime.getTime() / 1000)); 
+		  tgt.setCreateTimeIsSet(true);
+	  } else {
+		  tgt.setCreateTimeIsSet(false);
+	  }
+	  String dbName = src.getDatabaseName();
+	  if (dbName != null) {
+		  tgt.setDbName(dbName);
+		  tgt.setDbNameIsSet(true);
+	  } else {
+		  tgt.setDbNameIsSet(false);
+	  }
+	  Date lastAccessTime = src.getLastAccessTime();
+	  if (lastAccessTime != null) {
+		  tgt.setLastAccessTime((int) (lastAccessTime.getTime() / 1000));
+		  tgt.setLastAccessTimeIsSet(true);
+	  } else {
+		  tgt.setLastAccessTimeIsSet(false);
+	  }
+	  Map<String, String> params = src.getParameters();
+	  
+	  // A null parameter map causes Hive to throw a NPE
+	  // so ensure we do not return a Partition object with a null parameter map.
+	  if (params == null) {
+	    params = Maps.newHashMap();
+	  }
+	  
+	  tgt.setParameters(params);
+	  tgt.setParametersIsSet(true);
+	  
+	  String tableName = src.getTableName();
+	  if (tableName != null) {
+		  tgt.setTableName(tableName);
+		  tgt.setTableNameIsSet(true);
+	  } else {
+		  tgt.setTableNameIsSet(false);
+	  }
+	  
+	  List<String> values = src.getValues();
+	  if (values != null) {
+		  tgt.setValues(values);
+		  tgt.setValuesIsSet(true);
+	  } else {
+		  tgt.setValuesIsSet(false);
+	  }
+	  
+	  com.amazonaws.services.glue.model.StorageDescriptor sd = src.getStorageDescriptor();
+	  if (sd != null) {
+		  StorageDescriptor hiveSd = convertStorageDescriptor(sd);
+		  tgt.setSd(hiveSd);
+		  tgt.setSdIsSet(true);
+	  } else {
+		  tgt.setSdIsSet(false);
+	  }
+	  
+	  return tgt;
+  }
+
+  public List<Partition> convertPartitions(List<com.amazonaws.services.glue.model.Partition> src) {
+    if (src == null) {
+      return null;
+    }
+
+    List<Partition> target = Lists.newArrayList();
+    for (com.amazonaws.services.glue.model.Partition partition : src) {
+      target.add(convertPartition(partition));
+    }
+    return target;
+  }
+
+  public List<String> convertStringToList(final String s) {
+    if (s == null) {
+      return null;
+    }
+    List<String> listString = new ArrayList<>();
+    for (int i = 0; i < s.length();) {
+      StringBuilder length = new StringBuilder();
+      for (int j = i; j < s.length(); j++){
+        if (s.charAt(j) != '$') {
+          length.append(s.charAt(j));
+        } else {
+          int lengthOfString = Integer.valueOf(length.toString());
+          listString.add(s.substring(j + 1, j + 1 + lengthOfString));
+          i = j + 1 + lengthOfString;
+          break;
+        }
+      }
+    }
+    return listString;
+  }
+
+  @Nonnull
+  public Map<List<String>, String> convertSkewedMap(final @Nullable Map<String, String> catalogSkewedMap) {
+    Map<List<String>, String> skewedMap = new HashMap<>();
+    if (catalogSkewedMap == null){
+      return skewedMap;
+    }
+
+    for (String coralKey : catalogSkewedMap.keySet()) {
+      skewedMap.put(convertStringToList(coralKey), catalogSkewedMap.get(coralKey));
+    }
+    return skewedMap;
+  }
+
+  @Nonnull
+  public List<List<String>> convertSkewedValue(final @Nullable List<String> catalogSkewedValue) {
+    List<List<String>> skewedValues = new ArrayList<>();
+    if (catalogSkewedValue == null){
+      return skewedValues;
+    }
+
+    for (String skewValue : catalogSkewedValue) {
+      skewedValues.add(convertStringToList(skewValue));
+    }
+    return skewedValues;
+  }
+  
+  public PrincipalType convertPrincipalType(com.amazonaws.services.glue.model.PrincipalType catalogPrincipalType) {
+    if(catalogPrincipalType == null) {
+      return null;
+    }
+    
+    if(catalogPrincipalType == com.amazonaws.services.glue.model.PrincipalType.GROUP) {
+      return PrincipalType.GROUP;
+    } else if(catalogPrincipalType == com.amazonaws.services.glue.model.PrincipalType.USER) {
+      return PrincipalType.USER;
+    } else if(catalogPrincipalType == com.amazonaws.services.glue.model.PrincipalType.ROLE) {
+      return PrincipalType.ROLE;
+    }
+    throw new RuntimeException("Unknown principal type:" + catalogPrincipalType.name());
+  }
+
+  public Function convertFunction(final String dbName,
+                                  final com.amazonaws.services.glue.model.UserDefinedFunction catalogFunction) {
+    if (catalogFunction ==  null) {
+      return null;
+    }
+    Function hiveFunction = new Function();
+    hiveFunction.setClassName(catalogFunction.getClassName());
+    if (catalogFunction.getCreateTime() != null) {
+      //AWS Glue can return function with null create time
+      hiveFunction.setCreateTime((int) (catalogFunction.getCreateTime().getTime() / 1000));
+    }
+    hiveFunction.setDbName(dbName);
+    hiveFunction.setFunctionName(catalogFunction.getFunctionName());
+    hiveFunction.setFunctionType(FunctionType.JAVA);
+    hiveFunction.setOwnerName(catalogFunction.getOwnerName());
+    hiveFunction.setOwnerType(convertPrincipalType(com.amazonaws.services.glue.model.PrincipalType.fromValue(catalogFunction.getOwnerType())));
+    hiveFunction.setResourceUris(convertResourceUriList(catalogFunction.getResourceUris()));
+    return hiveFunction;
+  }
+
+  public List<ResourceUri> convertResourceUriList(
+          final List<com.amazonaws.services.glue.model.ResourceUri> catalogResourceUriList) {
+    if (catalogResourceUriList == null) {
+      return null;
+    }
+    List<ResourceUri> hiveResourceUriList = new ArrayList<>();
+    for (com.amazonaws.services.glue.model.ResourceUri catalogResourceUri : catalogResourceUriList) {
+      ResourceUri hiveResourceUri = new ResourceUri();
+      hiveResourceUri.setUri(catalogResourceUri.getUri());
+      if (catalogResourceUri.getResourceType() != null) {
+        hiveResourceUri.setResourceType(ResourceType.valueOf(catalogResourceUri.getResourceType()));
+      }
+      hiveResourceUriList.add(hiveResourceUri);
+    }
+
+    return hiveResourceUriList;
+  }
+
+  public List<ColumnStatisticsObj> convertColumnStatisticsList(List<ColumnStatistics> catatlogColumnStatisticsList) {
+    List<ColumnStatisticsObj> hiveColumnStatisticsList = new ArrayList<>();
+    for (ColumnStatistics catalogColumnStatistics : catatlogColumnStatisticsList) {
+      ColumnStatisticsObj hiveColumnStatistics = new ColumnStatisticsObj();
+      hiveColumnStatistics.setColName(catalogColumnStatistics.getColumnName());
+      hiveColumnStatistics.setColType(catalogColumnStatistics.getColumnType());
+      hiveColumnStatistics.setStatsData(convertColumnStatisticsData(catalogColumnStatistics.getStatisticsData()));
+      hiveColumnStatisticsList.add(hiveColumnStatistics);
+    }
+
+    return hiveColumnStatisticsList;
+  }
+
+  private ColumnStatisticsData convertColumnStatisticsData(
+      com.amazonaws.services.glue.model.ColumnStatisticsData catalogColumnStatisticsData) {
+    ColumnStatisticsData hiveColumnStatisticsData = new ColumnStatisticsData();
+
+    ColumnStatisticsType type = ColumnStatisticsType.fromValue(catalogColumnStatisticsData.getType());
+    switch (type) {
+      case BINARY:
+        BinaryColumnStatisticsData catalogBinaryData = catalogColumnStatisticsData.getBinaryColumnStatisticsData();
+        BinaryColumnStatsData hiveBinaryData = new BinaryColumnStatsData();
+        hiveBinaryData.setAvgColLen(catalogBinaryData.getAverageLength());
+        hiveBinaryData.setMaxColLen(catalogBinaryData.getMaximumLength());
+        hiveBinaryData.setNumNulls(catalogBinaryData.getNumberOfNulls());
+
+        hiveColumnStatisticsData.setFieldValue(ColumnStatisticsData._Fields.BINARY_STATS, hiveBinaryData);
+        hiveColumnStatisticsData.setBinaryStats(hiveBinaryData);
+        break;
+
+      case BOOLEAN:
+        BooleanColumnStatisticsData catalogBooleanData = catalogColumnStatisticsData.getBooleanColumnStatisticsData();
+        BooleanColumnStatsData hiveBooleanData = new BooleanColumnStatsData();
+        hiveBooleanData.setNumFalses(catalogBooleanData.getNumberOfFalses());
+        hiveBooleanData.setNumTrues(catalogBooleanData.getNumberOfTrues());
+        hiveBooleanData.setNumNulls(catalogBooleanData.getNumberOfNulls());
+
+        hiveColumnStatisticsData.setBooleanStats(hiveBooleanData);
+        break;
+
+      case DATE:
+        DateColumnStatisticsData catalogDateData = catalogColumnStatisticsData.getDateColumnStatisticsData();
+        DateColumnStatsData hiveDateData = new DateColumnStatsData();
+        hiveDateData.setLowValue(ConverterUtils.dateToHiveDate(catalogDateData.getMinimumValue()));
+        hiveDateData.setHighValue(ConverterUtils.dateToHiveDate(catalogDateData.getMaximumValue()));
+        hiveDateData.setNumDVs(catalogDateData.getNumberOfDistinctValues());
+        hiveDateData.setNumNulls(catalogDateData.getNumberOfNulls());
+
+        hiveColumnStatisticsData.setDateStats(hiveDateData);
+        break;
+
+      case DECIMAL:
+        DecimalColumnStatisticsData catalogDecimalData = catalogColumnStatisticsData.getDecimalColumnStatisticsData();
+        DecimalColumnStatsData hiveDecimalData = new DecimalColumnStatsData();
+        hiveDecimalData.setLowValue(convertDecimal(catalogDecimalData.getMinimumValue()));
+        hiveDecimalData.setHighValue(convertDecimal(catalogDecimalData.getMaximumValue()));
+        hiveDecimalData.setNumDVs(catalogDecimalData.getNumberOfDistinctValues());
+        hiveDecimalData.setNumNulls(catalogDecimalData.getNumberOfNulls());
+
+        hiveColumnStatisticsData.setDecimalStats(hiveDecimalData);
+        break;
+
+      case DOUBLE:
+        DoubleColumnStatisticsData catalogDoubleData = catalogColumnStatisticsData.getDoubleColumnStatisticsData();
+        DoubleColumnStatsData hiveDoubleData = new DoubleColumnStatsData();
+        hiveDoubleData.setLowValue(catalogDoubleData.getMinimumValue());
+        hiveDoubleData.setHighValue(catalogDoubleData.getMaximumValue());
+        hiveDoubleData.setNumDVs(catalogDoubleData.getNumberOfDistinctValues());
+        hiveDoubleData.setNumNulls(catalogDoubleData.getNumberOfNulls());
+
+        hiveColumnStatisticsData.setDoubleStats(hiveDoubleData);
+        break;
+
+      case LONG:
+        LongColumnStatisticsData catalogLongData = catalogColumnStatisticsData.getLongColumnStatisticsData();
+        LongColumnStatsData hiveLongData = new LongColumnStatsData();
+        hiveLongData.setLowValue(catalogLongData.getMinimumValue());
+        hiveLongData.setHighValue(catalogLongData.getMaximumValue());
+        hiveLongData.setNumDVs(catalogLongData.getNumberOfDistinctValues());
+        hiveLongData.setNumNulls(catalogLongData.getNumberOfNulls());
+
+        hiveColumnStatisticsData.setLongStats(hiveLongData);
+        break;
+
+      case STRING:
+        StringColumnStatisticsData catalogStringData = catalogColumnStatisticsData.getStringColumnStatisticsData();
+        StringColumnStatsData hiveStringData = new StringColumnStatsData();
+        hiveStringData.setAvgColLen(catalogStringData.getAverageLength());
+        hiveStringData.setMaxColLen(catalogStringData.getMaximumLength());
+        hiveStringData.setNumDVs(catalogStringData.getNumberOfDistinctValues());
+        hiveStringData.setNumNulls(catalogStringData.getNumberOfNulls());
+
+        hiveColumnStatisticsData.setStringStats(hiveStringData);
+        break;
+    }
+
+    return hiveColumnStatisticsData;
+  }
+
+  private Decimal convertDecimal(com.amazonaws.services.glue.model.DecimalNumber catalogDecimal) {
+    Decimal hiveDecimal = new Decimal();
+    hiveDecimal.setUnscaled(catalogDecimal.getUnscaledValue());
+    hiveDecimal.setScale(catalogDecimal.getScale().shortValue());
+    return hiveDecimal;
+  }
+
+}
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverter.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverter.java
new file mode 100644
index 0000000000..7d6a0f4b3a
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverter.java
@@ -0,0 +1,58 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.converters;
+
+import com.amazonaws.services.glue.model.ColumnStatistics;
+import com.amazonaws.services.glue.model.ErrorDetail;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.thrift.TException;
+
+import java.util.List;
+
+public interface CatalogToHiveConverter {
+
+  TException wrapInHiveException(Throwable e);
+
+  TException errorDetailToHiveException(ErrorDetail errorDetail);
+
+  Database convertDatabase(com.amazonaws.services.glue.model.Database catalogDatabase);
+
+  List<FieldSchema> convertFieldSchemaList(List<com.amazonaws.services.glue.model.Column> catalogFieldSchemaList);
+
+  Table convertTable(com.amazonaws.services.glue.model.Table catalogTable, String dbname);
+
+  TableMeta convertTableMeta(com.amazonaws.services.glue.model.Table catalogTable, String dbName);
+
+  Partition convertPartition(com.amazonaws.services.glue.model.Partition src);
+
+  List<Partition> convertPartitions(List<com.amazonaws.services.glue.model.Partition> src);
+
+  Function convertFunction(String dbName, com.amazonaws.services.glue.model.UserDefinedFunction catalogFunction);
+
+  List<ColumnStatisticsObj> convertColumnStatisticsList(List<ColumnStatistics> catatlogColumnStatisticsList);
+}
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverterFactory.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverterFactory.java
new file mode 100644
index 0000000000..d8430ec169
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/CatalogToHiveConverterFactory.java
@@ -0,0 +1,54 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.converters;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hive.common.util.HiveVersionInfo;
+
+public class CatalogToHiveConverterFactory {
+
+  private static final String HIVE_3_VERSION = "3.";
+
+  private static CatalogToHiveConverter catalogToHiveConverter;
+
+  public static CatalogToHiveConverter getCatalogToHiveConverter() {
+    if (catalogToHiveConverter == null) {
+      catalogToHiveConverter = loadConverter();
+    }
+    return catalogToHiveConverter;
+  }
+
+  private static CatalogToHiveConverter loadConverter() {
+    String hiveVersion = HiveVersionInfo.getShortVersion();
+
+    if (hiveVersion.startsWith(HIVE_3_VERSION)) {
+      return new Hive3CatalogToHiveConverter();
+    } else {
+      return new BaseCatalogToHiveConverter();
+    }
+  }
+
+  @VisibleForTesting
+  static void clearConverter() {
+    catalogToHiveConverter = null;
+  }
+}
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/ConverterUtils.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/ConverterUtils.java
new file mode 100644
index 0000000000..b350631931
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/ConverterUtils.java
@@ -0,0 +1,49 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.converters;
+
+import com.amazonaws.services.glue.model.Table;
+import com.google.gson.Gson;
+
+import java.util.Date;
+import java.util.concurrent.TimeUnit;
+
+public class ConverterUtils {
+
+  private static final Gson gson = new Gson();
+
+  public static String catalogTableToString(final Table table) {
+    return gson.toJson(table);
+  }
+
+  public static Table stringToCatalogTable(final String input) {
+    return gson.fromJson(input, Table.class);
+  }
+
+  public static org.apache.hadoop.hive.metastore.api.Date dateToHiveDate(Date date) {
+    return new org.apache.hadoop.hive.metastore.api.Date(TimeUnit.MILLISECONDS.toDays(date.getTime()));
+  }
+
+  public static Date hiveDatetoDate(org.apache.hadoop.hive.metastore.api.Date hiveDate) {
+    return new Date(TimeUnit.DAYS.toMillis(hiveDate.getDaysSinceEpoch()));
+  }
+}
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/GlueInputConverter.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/GlueInputConverter.java
new file mode 100644
index 0000000000..45889e0ae6
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/GlueInputConverter.java
@@ -0,0 +1,116 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.converters;
+
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UserDefinedFunctionInput;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This class provides methods to convert Hive/Catalog objects to Input objects used
+ * for Glue API parameters
+ */
+public final class GlueInputConverter {
+
+  public static DatabaseInput convertToDatabaseInput(Database hiveDatabase) {
+    return convertToDatabaseInput(HiveToCatalogConverter.convertDatabase(hiveDatabase));
+  }
+
+  public static DatabaseInput convertToDatabaseInput(com.amazonaws.services.glue.model.Database database) {
+    DatabaseInput input = new DatabaseInput();
+
+    input.setName(database.getName());
+    input.setDescription(database.getDescription());
+    input.setLocationUri(database.getLocationUri());
+    input.setParameters(database.getParameters());
+
+    return input;
+  }
+
+  public static TableInput convertToTableInput(Table hiveTable) {
+    return convertToTableInput(HiveToCatalogConverter.convertTable(hiveTable));
+  }
+
+  public static TableInput convertToTableInput(com.amazonaws.services.glue.model.Table table) {
+    TableInput tableInput = new TableInput();
+
+    tableInput.setRetention(table.getRetention());
+    tableInput.setPartitionKeys(table.getPartitionKeys());
+    tableInput.setTableType(table.getTableType());
+    tableInput.setName(table.getName());
+    tableInput.setOwner(table.getOwner());
+    tableInput.setLastAccessTime(table.getLastAccessTime());
+    tableInput.setStorageDescriptor(table.getStorageDescriptor());
+    tableInput.setParameters(table.getParameters());
+    tableInput.setViewExpandedText(table.getViewExpandedText());
+    tableInput.setViewOriginalText(table.getViewOriginalText());
+
+    return tableInput;
+  }
+
+  public static PartitionInput convertToPartitionInput(Partition src) {
+    return convertToPartitionInput(HiveToCatalogConverter.convertPartition(src));
+  }
+
+  public static PartitionInput convertToPartitionInput(com.amazonaws.services.glue.model.Partition src) {
+    PartitionInput partitionInput = new PartitionInput();
+
+    partitionInput.setLastAccessTime(src.getLastAccessTime());
+    partitionInput.setParameters(src.getParameters());
+    partitionInput.setStorageDescriptor(src.getStorageDescriptor());
+    partitionInput.setValues(src.getValues());
+
+    return partitionInput;
+  }
+
+  public static List<PartitionInput> convertToPartitionInputs(Collection<com.amazonaws.services.glue.model.Partition> parts) {
+    List<PartitionInput> inputList = new ArrayList<>();
+
+    for (com.amazonaws.services.glue.model.Partition part : parts) {
+      inputList.add(convertToPartitionInput(part));
+    }
+    return inputList;
+  }
+
+  public static UserDefinedFunctionInput convertToUserDefinedFunctionInput(Function hiveFunction) {
+    UserDefinedFunctionInput functionInput = new UserDefinedFunctionInput();
+
+    functionInput.setClassName(hiveFunction.getClassName());
+    functionInput.setFunctionName(hiveFunction.getFunctionName());
+    functionInput.setOwnerName(hiveFunction.getOwnerName());
+    if(hiveFunction.getOwnerType() != null) {
+      functionInput.setOwnerType(hiveFunction.getOwnerType().name());
+    }
+    functionInput.setResourceUris(HiveToCatalogConverter.covertResourceUriList(hiveFunction.getResourceUris()));
+    return functionInput;
+  }
+
+}
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/Hive3CatalogToHiveConverter.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/Hive3CatalogToHiveConverter.java
new file mode 100644
index 0000000000..4252ecd38a
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/Hive3CatalogToHiveConverter.java
@@ -0,0 +1,70 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.converters;
+
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_CATALOG_NAME;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+
+public class Hive3CatalogToHiveConverter extends BaseCatalogToHiveConverter {
+
+  @Override
+  public Database convertDatabase(com.amazonaws.services.glue.model.Database catalogDatabase) {
+    Database hiveDatabase = super.convertDatabase(catalogDatabase);
+    hiveDatabase.setCatalogName(DEFAULT_CATALOG_NAME);
+    return hiveDatabase;
+  }
+
+  @Override
+  public Table convertTable(com.amazonaws.services.glue.model.Table catalogTable, String dbname) {
+    Table hiveTable = super.convertTable(catalogTable, dbname);
+    hiveTable.setCatName(DEFAULT_CATALOG_NAME);
+    return hiveTable;
+  }
+
+  @Override
+  public TableMeta convertTableMeta(com.amazonaws.services.glue.model.Table catalogTable, String dbName) {
+    TableMeta tableMeta = super.convertTableMeta(catalogTable, dbName);
+    tableMeta.setCatName(DEFAULT_CATALOG_NAME);
+    return tableMeta;
+  }
+
+  @Override
+  public Partition convertPartition(com.amazonaws.services.glue.model.Partition src) {
+    Partition hivePartition = super.convertPartition(src);
+    hivePartition.setCatName(DEFAULT_CATALOG_NAME);
+    return hivePartition;
+  }
+
+  @Override
+  public Function convertFunction(String dbName, com.amazonaws.services.glue.model.UserDefinedFunction catalogFunction) {
+    Function hiveFunction = super.convertFunction(dbName, catalogFunction);
+    if (hiveFunction == null) {
+      return null;
+    }
+    hiveFunction.setCatName(DEFAULT_CATALOG_NAME);
+    return hiveFunction;
+  }
+}
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/HiveToCatalogConverter.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/HiveToCatalogConverter.java
new file mode 100644
index 0000000000..48f4ca73df
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/HiveToCatalogConverter.java
@@ -0,0 +1,372 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.converters;
+
+import com.amazonaws.services.glue.model.BinaryColumnStatisticsData;
+import com.amazonaws.services.glue.model.BooleanColumnStatisticsData;
+import com.amazonaws.services.glue.model.ColumnStatisticsType;
+import com.amazonaws.services.glue.model.DateColumnStatisticsData;
+import com.amazonaws.services.glue.model.DecimalColumnStatisticsData;
+import com.amazonaws.services.glue.model.DoubleColumnStatisticsData;
+import com.amazonaws.services.glue.model.LongColumnStatisticsData;
+import com.amazonaws.services.glue.model.StringColumnStatisticsData;
+import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DateColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.Decimal;
+import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.ResourceUri;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.SkewedInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+public class HiveToCatalogConverter {
+
+  public static com.amazonaws.services.glue.model.Database convertDatabase(Database hiveDatabase) {
+    com.amazonaws.services.glue.model.Database catalogDatabase = new com.amazonaws.services.glue.model.Database();
+    catalogDatabase.setName(hiveDatabase.getName());
+    catalogDatabase.setDescription(hiveDatabase.getDescription());
+    catalogDatabase.setLocationUri(hiveDatabase.getLocationUri());
+    catalogDatabase.setParameters(hiveDatabase.getParameters());
+    return catalogDatabase;
+  }
+
+  public static com.amazonaws.services.glue.model.Table convertTable(
+          Table hiveTable) {
+    com.amazonaws.services.glue.model.Table catalogTable = new com.amazonaws.services.glue.model.Table();
+    catalogTable.setRetention(hiveTable.getRetention());
+    catalogTable.setPartitionKeys(convertFieldSchemaList(hiveTable.getPartitionKeys()));
+    catalogTable.setTableType(hiveTable.getTableType());
+    catalogTable.setName(hiveTable.getTableName());
+    catalogTable.setOwner(hiveTable.getOwner());
+    catalogTable.setCreateTime(new Date((long) hiveTable.getCreateTime() * 1000));
+    catalogTable.setLastAccessTime(new Date((long) hiveTable.getLastAccessTime() * 1000));
+    catalogTable.setStorageDescriptor(convertStorageDescriptor(hiveTable.getSd()));
+    catalogTable.setParameters(hiveTable.getParameters());
+    catalogTable.setViewExpandedText(hiveTable.getViewExpandedText());
+    catalogTable.setViewOriginalText(hiveTable.getViewOriginalText());
+
+    return catalogTable;
+  }
+
+  public static com.amazonaws.services.glue.model.StorageDescriptor convertStorageDescriptor(
+          StorageDescriptor hiveSd) {
+    com.amazonaws.services.glue.model.StorageDescriptor catalogSd =
+            new com.amazonaws.services.glue.model.StorageDescriptor();
+    catalogSd.setNumberOfBuckets(hiveSd.getNumBuckets());
+    catalogSd.setCompressed(hiveSd.isCompressed());
+    catalogSd.setParameters(hiveSd.getParameters());
+    catalogSd.setBucketColumns(hiveSd.getBucketCols());
+    catalogSd.setColumns(convertFieldSchemaList(hiveSd.getCols()));
+    catalogSd.setInputFormat(hiveSd.getInputFormat());
+    catalogSd.setLocation(hiveSd.getLocation());
+    catalogSd.setOutputFormat(hiveSd.getOutputFormat());
+    catalogSd.setSerdeInfo(convertSerDeInfo(hiveSd.getSerdeInfo()));
+    catalogSd.setSkewedInfo(convertSkewedInfo(hiveSd.getSkewedInfo()));
+    catalogSd.setSortColumns(convertOrderList(hiveSd.getSortCols()));
+    catalogSd.setStoredAsSubDirectories(hiveSd.isStoredAsSubDirectories());
+
+    return catalogSd;
+  }
+
+  public static com.amazonaws.services.glue.model.Column convertFieldSchema(
+          FieldSchema hiveFieldSchema) {
+    com.amazonaws.services.glue.model.Column catalogFieldSchema =
+            new com.amazonaws.services.glue.model.Column();
+    catalogFieldSchema.setComment(hiveFieldSchema.getComment());
+    catalogFieldSchema.setName(hiveFieldSchema.getName());
+    catalogFieldSchema.setType(hiveFieldSchema.getType());
+
+    return catalogFieldSchema;
+  }
+
+  public static List<com.amazonaws.services.glue.model.Column> convertFieldSchemaList(
+          List<FieldSchema> hiveFieldSchemaList) {
+    List<com.amazonaws.services.glue.model.Column> catalogFieldSchemaList =
+            new ArrayList<com.amazonaws.services.glue.model.Column>();
+    for (FieldSchema hiveFs : hiveFieldSchemaList){
+      catalogFieldSchemaList.add(convertFieldSchema(hiveFs));
+    }
+
+    return catalogFieldSchemaList;
+  }
+
+  public static com.amazonaws.services.glue.model.SerDeInfo convertSerDeInfo(
+          SerDeInfo hiveSerDeInfo) {
+    com.amazonaws.services.glue.model.SerDeInfo catalogSerDeInfo = new com.amazonaws.services.glue.model.SerDeInfo();
+    catalogSerDeInfo.setName(hiveSerDeInfo.getName());
+    catalogSerDeInfo.setParameters(hiveSerDeInfo.getParameters());
+    catalogSerDeInfo.setSerializationLibrary(hiveSerDeInfo.getSerializationLib());
+
+    return catalogSerDeInfo;
+  }
+
+  public static com.amazonaws.services.glue.model.SkewedInfo convertSkewedInfo(SkewedInfo hiveSkewedInfo) {
+    if (hiveSkewedInfo == null)
+      return null;
+    com.amazonaws.services.glue.model.SkewedInfo catalogSkewedInfo = new com.amazonaws.services.glue.model.SkewedInfo()
+            .withSkewedColumnNames(hiveSkewedInfo.getSkewedColNames())
+            .withSkewedColumnValues(convertSkewedValue(hiveSkewedInfo.getSkewedColValues()))
+            .withSkewedColumnValueLocationMaps(convertSkewedMap(hiveSkewedInfo.getSkewedColValueLocationMaps()));
+    return catalogSkewedInfo;
+  }
+
+  public static com.amazonaws.services.glue.model.Order convertOrder(Order hiveOrder) {
+    com.amazonaws.services.glue.model.Order order = new com.amazonaws.services.glue.model.Order();
+    order.setColumn(hiveOrder.getCol());
+    order.setSortOrder(hiveOrder.getOrder());
+
+    return order;
+  }
+
+  public static List<com.amazonaws.services.glue.model.Order> convertOrderList(List<Order> hiveOrderList) {
+    if (hiveOrderList == null) {
+      return null;
+    }
+    List<com.amazonaws.services.glue.model.Order> catalogOrderList = new ArrayList<>();
+    for (Order hiveOrder : hiveOrderList) {
+      catalogOrderList.add(convertOrder(hiveOrder));
+    }
+
+    return catalogOrderList;
+  }
+
+  public static com.amazonaws.services.glue.model.Partition convertPartition(Partition src) {
+    com.amazonaws.services.glue.model.Partition tgt = new com.amazonaws.services.glue.model.Partition();
+
+    tgt.setDatabaseName(src.getDbName());
+    tgt.setTableName(src.getTableName());
+    tgt.setCreationTime(new Date((long) src.getCreateTime() * 1000));
+    tgt.setLastAccessTime(new Date((long) src.getLastAccessTime() * 1000));
+    tgt.setParameters(src.getParameters());
+    tgt.setStorageDescriptor(convertStorageDescriptor(src.getSd()));
+    tgt.setValues(src.getValues());
+
+    return tgt;
+  }
+
+  public static String convertListToString(final List<String> list) {
+    if (list == null) {
+      return null;
+    }
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < list.size(); i++) {
+      String currentString = list.get(i);
+      sb.append(currentString.length() + "$" + currentString);
+    }
+
+    return sb.toString();
+  }
+
+  public static Map<String, String> convertSkewedMap(final Map<List<String>, String> coreSkewedMap){
+    if (coreSkewedMap == null){
+      return null;
+    }
+    Map<String, String> catalogSkewedMap = new HashMap<>();
+    for (List<String> coreKey : coreSkewedMap.keySet()) {
+      catalogSkewedMap.put(convertListToString(coreKey), coreSkewedMap.get(coreKey));
+    }
+    return catalogSkewedMap;
+  }
+
+  public static List<String> convertSkewedValue(final List<List<String>> coreSkewedValue) {
+    if (coreSkewedValue == null) {
+      return null;
+    }
+    List<String> catalogSkewedValue = new ArrayList<>();
+    for (int i = 0; i < coreSkewedValue.size(); i++) {
+      catalogSkewedValue.add(convertListToString(coreSkewedValue.get(i)));
+    }
+
+    return catalogSkewedValue;
+  }
+
+  public static com.amazonaws.services.glue.model.UserDefinedFunction convertFunction(final Function hiveFunction) {
+    if (hiveFunction == null ){
+      return null;
+    }
+    com.amazonaws.services.glue.model.UserDefinedFunction catalogFunction = new com.amazonaws.services.glue.model.UserDefinedFunction();
+    catalogFunction.setClassName(hiveFunction.getClassName());
+    catalogFunction.setFunctionName(hiveFunction.getFunctionName());
+    catalogFunction.setCreateTime(new Date((long) (hiveFunction.getCreateTime()) * 1000));
+    catalogFunction.setOwnerName(hiveFunction.getOwnerName());
+    if(hiveFunction.getOwnerType() != null) {
+      catalogFunction.setOwnerType(hiveFunction.getOwnerType().name());
+    }
+    catalogFunction.setResourceUris(covertResourceUriList(hiveFunction.getResourceUris()));
+    return catalogFunction;
+  }
+
+  public static List<com.amazonaws.services.glue.model.ResourceUri> covertResourceUriList(
+          final List<ResourceUri> hiveResourceUriList) {
+    if (hiveResourceUriList == null) {
+      return null;
+    }
+    List<com.amazonaws.services.glue.model.ResourceUri> catalogResourceUriList = new ArrayList<>();
+    for (ResourceUri hiveResourceUri : hiveResourceUriList) {
+      com.amazonaws.services.glue.model.ResourceUri catalogResourceUri = new com.amazonaws.services.glue.model.ResourceUri();
+      catalogResourceUri.setUri(hiveResourceUri.getUri());
+      if (hiveResourceUri.getResourceType() != null) {
+        catalogResourceUri.setResourceType(hiveResourceUri.getResourceType().name());
+      }
+      catalogResourceUriList.add(catalogResourceUri);
+    }
+    return catalogResourceUriList;
+  }
+
+  public static List<com.amazonaws.services.glue.model.ColumnStatistics> convertColumnStatisticsObjList(
+          ColumnStatistics hiveColumnStatistics) {
+    ColumnStatisticsDesc hiveColumnStatisticsDesc = hiveColumnStatistics.getStatsDesc();
+    List<ColumnStatisticsObj> hiveColumnStatisticsObjs = hiveColumnStatistics.getStatsObj();
+
+    List<com.amazonaws.services.glue.model.ColumnStatistics> catalogColumnStatisticsList = new ArrayList<>();
+    for (ColumnStatisticsObj hiveColumnStatisticsObj : hiveColumnStatisticsObjs) {
+      com.amazonaws.services.glue.model.ColumnStatistics catalogColumnStatistics =
+              new com.amazonaws.services.glue.model.ColumnStatistics();
+      catalogColumnStatistics.setColumnName(hiveColumnStatisticsObj.getColName());
+      catalogColumnStatistics.setColumnType(hiveColumnStatisticsObj.getColType());
+      // Last analyzed time in Hive is in days since Epoch, Java Date is in milliseconds
+      catalogColumnStatistics.setAnalyzedTime(new Date(TimeUnit.DAYS.toMillis(hiveColumnStatisticsDesc.getLastAnalyzed())));
+      catalogColumnStatistics.setStatisticsData(convertColumnStatisticsData(hiveColumnStatisticsObj.getStatsData()));
+      catalogColumnStatisticsList.add(catalogColumnStatistics);
+    }
+
+    return catalogColumnStatisticsList;
+  }
+
+  private static com.amazonaws.services.glue.model.ColumnStatisticsData convertColumnStatisticsData(
+          ColumnStatisticsData hiveColumnStatisticsData) {
+    com.amazonaws.services.glue.model.ColumnStatisticsData catalogColumnStatisticsData =
+            new com.amazonaws.services.glue.model.ColumnStatisticsData();
+
+    // Hive uses the TUnion object to ensure that only one stats object is set at any time, this means that we can
+    // only call the get*() of a stats type if the 'setField' is set to that value
+    ColumnStatisticsData._Fields setField = hiveColumnStatisticsData.getSetField();
+    switch (setField) {
+      case BINARY_STATS:
+        BinaryColumnStatsData hiveBinaryData = hiveColumnStatisticsData.getBinaryStats();
+        BinaryColumnStatisticsData catalogBinaryData = new BinaryColumnStatisticsData();
+        catalogBinaryData.setNumberOfNulls(hiveBinaryData.getNumNulls());
+        catalogBinaryData.setMaximumLength(hiveBinaryData.getMaxColLen());
+        catalogBinaryData.setAverageLength(hiveBinaryData.getAvgColLen());
+        catalogColumnStatisticsData.setType(String.valueOf(ColumnStatisticsType.BINARY));
+        catalogColumnStatisticsData.setBinaryColumnStatisticsData(catalogBinaryData);
+        break;
+
+      case BOOLEAN_STATS:
+        BooleanColumnStatsData hiveBooleanData = hiveColumnStatisticsData.getBooleanStats();
+        BooleanColumnStatisticsData catalogBooleanData = new BooleanColumnStatisticsData();
+        catalogBooleanData.setNumberOfNulls(hiveBooleanData.getNumNulls());
+        catalogBooleanData.setNumberOfFalses(hiveBooleanData.getNumFalses());
+        catalogBooleanData.setNumberOfTrues(hiveBooleanData.getNumTrues());
+        catalogColumnStatisticsData.setType(String.valueOf(ColumnStatisticsType.BOOLEAN));
+        catalogColumnStatisticsData.setBooleanColumnStatisticsData(catalogBooleanData);
+        break;
+
+      case DATE_STATS:
+        DateColumnStatsData hiveDateData = hiveColumnStatisticsData.getDateStats();
+        DateColumnStatisticsData catalogDateData = new DateColumnStatisticsData();
+        catalogDateData.setNumberOfNulls(hiveDateData.getNumNulls());
+        catalogDateData.setNumberOfDistinctValues(hiveDateData.getNumDVs());
+        catalogDateData.setMaximumValue(ConverterUtils.hiveDatetoDate(hiveDateData.getHighValue()));
+        catalogDateData.setMinimumValue(ConverterUtils.hiveDatetoDate(hiveDateData.getLowValue()));
+        catalogColumnStatisticsData.setType(String.valueOf(ColumnStatisticsType.DATE));
+        catalogColumnStatisticsData.setDateColumnStatisticsData(catalogDateData);
+        break;
+
+      case DECIMAL_STATS:
+        DecimalColumnStatsData hiveDecimalData = hiveColumnStatisticsData.getDecimalStats();
+        DecimalColumnStatisticsData catalogDecimalData = new DecimalColumnStatisticsData();
+        catalogDecimalData.setNumberOfNulls(hiveDecimalData.getNumNulls());
+        catalogDecimalData.setNumberOfDistinctValues(hiveDecimalData.getNumDVs());
+        catalogDecimalData.setMaximumValue(convertDecimal(hiveDecimalData.getHighValue()));
+        catalogDecimalData.setMinimumValue(convertDecimal(hiveDecimalData.getLowValue()));
+        catalogColumnStatisticsData.setType(String.valueOf(ColumnStatisticsType.DECIMAL));
+        catalogColumnStatisticsData.setDecimalColumnStatisticsData(catalogDecimalData);
+        break;
+
+      case DOUBLE_STATS:
+        DoubleColumnStatsData hiveDoubleData = hiveColumnStatisticsData.getDoubleStats();
+        DoubleColumnStatisticsData catalogDoubleData = new DoubleColumnStatisticsData();
+        catalogDoubleData.setNumberOfNulls(hiveDoubleData.getNumNulls());
+        catalogDoubleData.setNumberOfDistinctValues(hiveDoubleData.getNumDVs());
+        catalogDoubleData.setMaximumValue(hiveDoubleData.getHighValue());
+        catalogDoubleData.setMinimumValue(hiveDoubleData.getLowValue());
+        catalogColumnStatisticsData.setType(String.valueOf(ColumnStatisticsType.DOUBLE));
+        catalogColumnStatisticsData.setDoubleColumnStatisticsData(catalogDoubleData);
+        break;
+      case LONG_STATS:
+        LongColumnStatsData hiveLongData = hiveColumnStatisticsData.getLongStats();
+        LongColumnStatisticsData catalogLongData = new LongColumnStatisticsData();
+        catalogLongData.setNumberOfNulls(hiveLongData.getNumNulls());
+        catalogLongData.setNumberOfDistinctValues(hiveLongData.getNumDVs());
+        catalogLongData.setMaximumValue(hiveLongData.getHighValue());
+        catalogLongData.setMinimumValue(hiveLongData.getLowValue());
+        catalogColumnStatisticsData.setType(String.valueOf(ColumnStatisticsType.LONG));
+        catalogColumnStatisticsData.setLongColumnStatisticsData(catalogLongData);
+        break;
+
+      case STRING_STATS:
+        StringColumnStatsData hiveStringData = hiveColumnStatisticsData.getStringStats();
+        StringColumnStatisticsData catalogStringData = new StringColumnStatisticsData();
+        catalogStringData.setNumberOfNulls(hiveStringData.getNumNulls());
+        catalogStringData.setNumberOfDistinctValues(hiveStringData.getNumDVs());
+        catalogStringData.setMaximumLength(hiveStringData.getMaxColLen());
+        catalogStringData.setAverageLength(hiveStringData.getAvgColLen());
+        catalogColumnStatisticsData.setType(String.valueOf(ColumnStatisticsType.STRING));
+        catalogColumnStatisticsData.setStringColumnStatisticsData(catalogStringData);
+        break;
+    }
+
+    return catalogColumnStatisticsData;
+  }
+
+  private static com.amazonaws.services.glue.model.DecimalNumber convertDecimal(Decimal hiveDecimal) {
+    com.amazonaws.services.glue.model.DecimalNumber catalogDecimal =
+            new com.amazonaws.services.glue.model.DecimalNumber();
+    catalogDecimal.setUnscaledValue(ByteBuffer.wrap(hiveDecimal.getUnscaled()));
+    catalogDecimal.setScale((int)hiveDecimal.getScale());
+    return catalogDecimal;
+  }
+
+}
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/PartitionNameParser.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/PartitionNameParser.java
new file mode 100644
index 0000000000..1419896a58
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/converters/PartitionNameParser.java
@@ -0,0 +1,143 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.converters;
+
+import com.amazonaws.glue.catalog.exceptions.InvalidPartitionNameException;
+import com.google.common.collect.ImmutableSet;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class PartitionNameParser {
+
+  private static final Pattern PARTITION_NAME_VALUE_PATTERN = Pattern.compile("([^/]+)=([^/]+)");
+  private static final String PARTITION_NAME_DELIMITER = "/";
+
+  private static final char STORE_AS_NUMBER = 'n';
+  private static final char STORE_AS_STRING = 's';
+
+  private static final Set<String> NUMERIC_PARTITION_COLUMN_TYPES = ImmutableSet.of(
+          "tinyint",
+          "smallint",
+          "int",
+          "bigint"
+  );
+
+  public static String getPartitionName(List<String> partitionColumns, List<String> partitionValues) {
+    if (hasInvalidValues(partitionColumns, partitionValues) || hasInvalidSize(partitionColumns, partitionValues)) {
+      throw new IllegalArgumentException("Partition is not well formed. Columns and values do no match.");
+    }
+
+    StringBuilder partitionName = new StringBuilder();
+    partitionName.append(getPartitionColumnName(partitionColumns.get(0), partitionValues.get(0)));
+
+    for (int i = 1; i < partitionColumns.size(); i++) {
+      partitionName.append(PARTITION_NAME_DELIMITER);
+      partitionName.append(getPartitionColumnName(partitionColumns.get(i), partitionValues.get(i)));
+    }
+
+    return partitionName.toString();
+  }
+
+  private static boolean hasInvalidValues(List<String> partitionColumns, List<String> partitionValues) {
+    return partitionColumns == null || partitionValues == null;
+  }
+
+  private static boolean hasInvalidSize(List<String> partitionColumns, List<String> partitionValues) {
+    return partitionColumns.size() != partitionValues.size();
+  }
+
+  private static String getPartitionColumnName(String partitionColumn, String partitionValue) {
+    return partitionColumn + "=" + partitionValue;
+  }
+
+  public static LinkedHashMap<String, String> getPartitionColumns(String partitionName) {
+    LinkedHashMap<String, String> partitionColumns = new LinkedHashMap<>();
+    String[] partitions = partitionName.split(PARTITION_NAME_DELIMITER);
+    for(String partition : partitions) {
+      Entry<String, String> entry = getPartitionColumnValuePair(partition);
+      partitionColumns.put(entry.getKey(), entry.getValue());
+    }
+
+    return partitionColumns;
+  }
+
+  /*
+   * Copied from https://github.com/apache/hive/blob/master/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+   */
+  public static String unescapePathName(String path) {
+    int len = path.length();
+    //pre-allocate sb to have enough buffer size, to avoid realloc
+    StringBuilder sb = new StringBuilder(len);
+    for (int i = 0; i < len; i++) {
+      char c = path.charAt(i);
+      if (c == '%' && i + 2 < len) {
+        int code = -1;
+        try {
+          code = Integer.parseInt(path.substring(i + 1, i + 3), 16);
+        } catch (Exception e) {
+          code = -1;
+        }
+        if (code >= 0) {
+          sb.append((char) code);
+          i += 2;
+          continue;
+        }
+      }
+      sb.append(c);
+    }
+    return sb.toString();
+  }
+
+  private static AbstractMap.SimpleEntry getPartitionColumnValuePair(String partition) {
+    String column = null;
+    String value = null;
+    Matcher partitionMatcher = PARTITION_NAME_VALUE_PATTERN.matcher(partition);
+
+    if (partitionMatcher.matches()) {
+      column = unescapePathName(partitionMatcher.group(1));
+      value = unescapePathName(partitionMatcher.group(2));
+    } else {
+      throw new InvalidPartitionNameException(partition);
+    }
+
+    return new AbstractMap.SimpleEntry(column, value);
+  }
+
+  public static List<String> getPartitionValuesFromName(String partitionName) {
+    List<String> partitionValues = new ArrayList<>();
+    String[] partitions = partitionName.split(PARTITION_NAME_DELIMITER);
+    for(String partition : partitions) {
+      Entry<String, String> entry = getPartitionColumnValuePair(partition);
+      partitionValues.add(entry.getValue());
+    }
+
+    return partitionValues;
+  }
+
+}
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/credentials/ConfigurationAWSCredentialsProvider.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/credentials/ConfigurationAWSCredentialsProvider.java
new file mode 100644
index 0000000000..dd9f480397
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/credentials/ConfigurationAWSCredentialsProvider.java
@@ -0,0 +1,60 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package com.amazonaws.glue.catalog.credentials;
+
+import com.amazonaws.SdkClientException;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.BasicAWSCredentials;
+import com.amazonaws.auth.BasicSessionCredentials;
+import com.amazonaws.glue.catalog.util.AWSGlueConfig;
+import com.amazonaws.util.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+
+public class ConfigurationAWSCredentialsProvider implements AWSCredentialsProvider {
+
+    private Configuration conf;
+
+    public ConfigurationAWSCredentialsProvider(Configuration conf) {
+        this.conf = conf;
+    }
+
+    @Override
+    public AWSCredentials getCredentials() {
+        String accessKey = StringUtils.trim(conf.get(AWSGlueConfig.AWS_GLUE_ACCESS_KEY));
+        String secretKey = StringUtils.trim(conf.get(AWSGlueConfig.AWS_GLUE_SECRET_KEY));
+        String sessionToken = StringUtils.trim(conf.get(AWSGlueConfig.AWS_GLUE_SESSION_TOKEN));
+        if (!StringUtils.isNullOrEmpty(accessKey) && !StringUtils.isNullOrEmpty(secretKey)) {
+            return (StringUtils.isNullOrEmpty(sessionToken) ? new BasicAWSCredentials(accessKey,
+                    secretKey) : new BasicSessionCredentials(accessKey, secretKey, sessionToken));
+        } else {
+            throw new SdkClientException(
+                    "Unable to load AWS credentials from hive conf (aws.glue.access-key and aws.glue.secret-key)");
+        }
+    }
+
+    @Override
+    public void refresh() {
+
+    }
+
+    @Override
+    public String toString() {
+        return this.getClass().getSimpleName();
+    }
+}
diff --git a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/credentials/ConfigurationAWSCredentialsProviderFactory.java
similarity index 62%
copy from fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
copy to fe/fe-core/src/main/java/com/amazonaws/glue/catalog/credentials/ConfigurationAWSCredentialsProviderFactory.java
index 05084c97b5..c1c526b815 100644
--- a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/credentials/ConfigurationAWSCredentialsProviderFactory.java
@@ -15,11 +15,15 @@
 // specific language governing permissions and limitations
 // under the License.
 
-package org.apache.doris.load.loadv2.dpp;
+package com.amazonaws.glue.catalog.credentials;
 
-import org.apache.doris.load.loadv2.etl.EtlJobConfig;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.glue.catalog.metastore.AWSCredentialsProviderFactory;
+import org.apache.hadoop.conf.Configuration;
 
-// RollupTreeBuilder is used to get the RollupTree from the TableMeta
-public abstract interface RollupTreeBuilder {
-    public RollupTreeNode build(EtlJobConfig.EtlTable tableMeta);
+public class ConfigurationAWSCredentialsProviderFactory implements AWSCredentialsProviderFactory {
+    @Override
+    public AWSCredentialsProvider buildAWSCredentialsProvider(Configuration conf) {
+        return new ConfigurationAWSCredentialsProvider(conf);
+    }
 }
diff --git a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/exceptions/InvalidPartitionNameException.java
similarity index 65%
copy from fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
copy to fe/fe-core/src/main/java/com/amazonaws/glue/catalog/exceptions/InvalidPartitionNameException.java
index 05084c97b5..c2870dd2c1 100644
--- a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/exceptions/InvalidPartitionNameException.java
@@ -14,12 +14,20 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.exceptions;
 
-package org.apache.doris.load.loadv2.dpp;
+public class InvalidPartitionNameException extends RuntimeException {
 
-import org.apache.doris.load.loadv2.etl.EtlJobConfig;
+  public InvalidPartitionNameException(String message) {
+    super(message);
+  }
 
-// RollupTreeBuilder is used to get the RollupTree from the TableMeta
-public abstract interface RollupTreeBuilder {
-    public RollupTreeNode build(EtlJobConfig.EtlTable tableMeta);
+  public InvalidPartitionNameException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }
diff --git a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/exceptions/LakeFormationException.java
similarity index 66%
copy from fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
copy to fe/fe-core/src/main/java/com/amazonaws/glue/catalog/exceptions/LakeFormationException.java
index 05084c97b5..25fe259769 100644
--- a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/exceptions/LakeFormationException.java
@@ -14,12 +14,20 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.exceptions;
 
-package org.apache.doris.load.loadv2.dpp;
+public class LakeFormationException extends RuntimeException {
 
-import org.apache.doris.load.loadv2.etl.EtlJobConfig;
+  public LakeFormationException(String message) {
+    super(message);
+  }
 
-// RollupTreeBuilder is used to get the RollupTree from the TableMeta
-public abstract interface RollupTreeBuilder {
-    public RollupTreeNode build(EtlJobConfig.EtlTable tableMeta);
+  public LakeFormationException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCatalogMetastoreClient.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCatalogMetastoreClient.java
new file mode 100644
index 0000000000..23d0ee9aaa
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCatalogMetastoreClient.java
@@ -0,0 +1,2481 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter;
+import com.amazonaws.glue.catalog.converters.GlueInputConverter;
+import com.amazonaws.glue.catalog.converters.Hive3CatalogToHiveConverter;
+import com.amazonaws.glue.catalog.util.BatchDeletePartitionsHelper;
+import com.amazonaws.glue.catalog.util.ExpressionHelper;
+import com.amazonaws.glue.catalog.util.LoggingHelper;
+import com.amazonaws.glue.catalog.util.MetastoreClientUtils;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.isExternalTable;
+import com.amazonaws.services.glue.AWSGlue;
+import com.amazonaws.services.glue.model.AlreadyExistsException;
+import com.amazonaws.services.glue.model.EntityNotFoundException;
+import com.amazonaws.services.glue.model.GetDatabaseRequest;
+import com.amazonaws.services.glue.model.Partition;
+import com.amazonaws.services.glue.model.UpdatePartitionRequest;
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Preconditions;
+import static com.google.common.base.Preconditions.checkNotNull;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.common.ValidWriteIdList;
+import org.apache.hadoop.hive.metastore.HiveMetaHookLoader;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.PartitionDropOptions;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_COMMENT;
+import static org.apache.hadoop.hive.metastore.Warehouse.DEFAULT_DATABASE_NAME;
+import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.Catalog;
+import org.apache.hadoop.hive.metastore.api.CheckConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.CmRecycleRequest;
+import org.apache.hadoop.hive.metastore.api.CmRecycleResponse;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
+import org.apache.hadoop.hive.metastore.api.CreationMetadata;
+import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.DefaultConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.FindSchemasByColsResp;
+import org.apache.hadoop.hive.metastore.api.FindSchemasByColsRqst;
+import org.apache.hadoop.hive.metastore.api.FireEventRequest;
+import org.apache.hadoop.hive.metastore.api.FireEventResponse;
+import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest;
+import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
+import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
+import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
+import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+import org.apache.hadoop.hive.metastore.api.HiveObjectType;
+import org.apache.hadoop.hive.metastore.api.ISchema;
+import org.apache.hadoop.hive.metastore.api.InvalidInputException;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.InvalidPartitionException;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.Materialization;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
+import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest;
+import org.apache.hadoop.hive.metastore.api.NotificationEventsCountResponse;
+import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
+import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+import org.apache.hadoop.hive.metastore.api.PartitionValuesRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
+import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest;
+import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
+import org.apache.hadoop.hive.metastore.api.RuntimeStat;
+import org.apache.hadoop.hive.metastore.api.SQLCheckConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLDefaultConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.SchemaVersion;
+import org.apache.hadoop.hive.metastore.api.SchemaVersionState;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.hadoop.hive.metastore.api.TableValidWriteIds;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hadoop.hive.metastore.api.TxnOpenException;
+import org.apache.hadoop.hive.metastore.api.TxnToWriteId;
+import org.apache.hadoop.hive.metastore.api.UniqueConstraintsRequest;
+import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
+import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.WMFullResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMMapping;
+import org.apache.hadoop.hive.metastore.api.WMNullablePool;
+import org.apache.hadoop.hive.metastore.api.WMNullableResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMPool;
+import org.apache.hadoop.hive.metastore.api.WMResourcePlan;
+import org.apache.hadoop.hive.metastore.api.WMTrigger;
+import org.apache.hadoop.hive.metastore.api.WMValidateResourcePlanResponse;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.utils.ObjectPair;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.regex.Pattern;
+
+public class AWSCatalogMetastoreClient implements IMetaStoreClient {
+
+  // TODO "hook" into Hive logging (hive or hive.metastore)
+  private static final Logger logger = Logger.getLogger(AWSCatalogMetastoreClient.class);
+
+  private final Configuration conf;
+  private final AWSGlue glueClient;
+  private final Warehouse wh;
+  private final GlueMetastoreClientDelegate glueMetastoreClientDelegate;
+  private final String catalogId;
+  private final CatalogToHiveConverter catalogToHiveConverter;
+
+  private static final int BATCH_DELETE_PARTITIONS_PAGE_SIZE = 25;
+  private static final int BATCH_DELETE_PARTITIONS_THREADS_COUNT = 5;
+  static final String BATCH_DELETE_PARTITIONS_THREAD_POOL_NAME_FORMAT = "batch-delete-partitions-%d";
+  private static final ExecutorService BATCH_DELETE_PARTITIONS_THREAD_POOL = Executors.newFixedThreadPool(
+          BATCH_DELETE_PARTITIONS_THREADS_COUNT,
+          new ThreadFactoryBuilder()
+                  .setNameFormat(BATCH_DELETE_PARTITIONS_THREAD_POOL_NAME_FORMAT)
+                  .setDaemon(true).build()
+  );
+
+  private Map<String, String> currentMetaVars;
+  // private final AwsGlueHiveShims hiveShims = ShimsLoader.getHiveShims();
+
+  public AWSCatalogMetastoreClient(Configuration conf, HiveMetaHookLoader hook, Boolean allowEmbedded)
+          throws MetaException {
+    this(conf, hook);
+  }
+
+  public AWSCatalogMetastoreClient(Configuration conf, HiveMetaHookLoader hook) throws MetaException {
+    this.conf = conf;
+    glueClient = new AWSGlueClientFactory(this.conf).newClient();
+    catalogToHiveConverter = new Hive3CatalogToHiveConverter();
+
+    // TODO preserve existing functionality for HiveMetaHook
+    wh = new Warehouse(this.conf);
+
+    AWSGlueMetastore glueMetastore = new AWSGlueMetastoreFactory().newMetastore(conf);
+    glueMetastoreClientDelegate = new GlueMetastoreClientDelegate(this.conf, glueMetastore, wh);
+
+    snapshotActiveConf();
+    if (!doesDefaultDBExist()) {
+      createDefaultDatabase();
+    }
+    catalogId = MetastoreClientUtils.getCatalogId(conf);
+  }
+
+  /**
+   * Currently used for unit tests
+   */
+  public static class Builder {
+
+    private Configuration conf;
+    private Warehouse wh;
+    private GlueClientFactory clientFactory;
+    private AWSGlueMetastoreFactory metastoreFactory;
+    private boolean createDefaults = true;
+    private String catalogId;
+    private GlueMetastoreClientDelegate glueMetastoreClientDelegate;
+
+    public Builder withConf(Configuration conf) {
+      this.conf = conf;
+      return this;
+    }
+
+    public Builder withClientFactory(GlueClientFactory clientFactory) {
+      this.clientFactory = clientFactory;
+      return this;
+    }
+
+    public Builder withMetastoreFactory(AWSGlueMetastoreFactory metastoreFactory) {
+      this.metastoreFactory = metastoreFactory;
+      return this;
+    }
+
+    public Builder withWarehouse(Warehouse wh) {
+      this.wh = wh;
+      return this;
+    }
+
+    public Builder withCatalogId(String catalogId) {
+      this.catalogId = catalogId;
+      return this;
+    }
+
+    public Builder withGlueMetastoreClientDelegate(GlueMetastoreClientDelegate clientDelegate) {
+      this.glueMetastoreClientDelegate = clientDelegate;
+      return this;
+    }
+
+    public AWSCatalogMetastoreClient build() throws MetaException {
+      return new AWSCatalogMetastoreClient(this);
+    }
+
+    public Builder createDefaults(boolean createDefaultDB) {
+      this.createDefaults = createDefaultDB;
+      return this;
+    }
+  }
+
+  private AWSCatalogMetastoreClient(Builder builder) throws MetaException {
+    catalogToHiveConverter = new Hive3CatalogToHiveConverter();
+    conf = MoreObjects.firstNonNull(builder.conf, MetastoreConf.newMetastoreConf());
+
+    if (builder.wh != null) {
+      this.wh = builder.wh;
+    } else {
+      this.wh = new Warehouse(conf);
+    }
+
+    if (builder.catalogId != null) {
+      this.catalogId = builder.catalogId;
+    } else {
+      this.catalogId = null;
+    }
+
+    GlueClientFactory clientFactory = MoreObjects.firstNonNull(builder.clientFactory, new AWSGlueClientFactory(conf));
+    AWSGlueMetastoreFactory metastoreFactory = MoreObjects.firstNonNull(builder.metastoreFactory,
+            new AWSGlueMetastoreFactory());
+
+    glueClient = clientFactory.newClient();
+    AWSGlueMetastore glueMetastore = metastoreFactory.newMetastore(conf);
+    glueMetastoreClientDelegate = new GlueMetastoreClientDelegate(this.conf, glueMetastore, wh);
+
+    /**
+     * It seems weird to create databases as part of client construction. This
+     * part should probably be moved to the section in hive code right after the
+     * metastore client is instantiated. For now, simply copying the
+     * functionality in the thrift server
+     */
+    if(builder.createDefaults && !doesDefaultDBExist()) {
+      createDefaultDatabase();
+    }
+  }
+
+  private boolean doesDefaultDBExist() throws MetaException {
+
+    try {
+      GetDatabaseRequest getDatabaseRequest = new GetDatabaseRequest().withName(DEFAULT_DATABASE_NAME).withCatalogId(
+              catalogId);
+      glueClient.getDatabase(getDatabaseRequest);
+    } catch (EntityNotFoundException e) {
+      return false;
+    } catch (AmazonServiceException e) {
+      String msg = "Unable to verify existence of default database: ";
+      logger.error(msg, e);
+      throw new MetaException(msg + e);
+    }
+    return true;
+  }
+
+  private void createDefaultDatabase() throws MetaException {
+    Database defaultDB = new Database();
+    defaultDB.setName(DEFAULT_DATABASE_NAME);
+    defaultDB.setDescription(DEFAULT_DATABASE_COMMENT);
+    defaultDB.setLocationUri(wh.getDefaultDatabasePath(DEFAULT_DATABASE_NAME).toString());
+
+    org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet principalPrivilegeSet
+            = new org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet();
+    principalPrivilegeSet.setRolePrivileges(Maps.<String, List<org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo>>newHashMap());
+
+    defaultDB.setPrivileges(principalPrivilegeSet);
+
+    /**
+     * TODO: Grant access to role PUBLIC after role support is added
+     */
+    try {
+      createDatabase(defaultDB);
+    } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) {
+      logger.warn("database - default already exists. Ignoring..");
+    } catch (Exception e) {
+      logger.error("Unable to create default database", e);
+    }
+  }
+
+  @Override
+  public void createDatabase(Database database) throws InvalidObjectException,
+          org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException, TException {
+    glueMetastoreClientDelegate.createDatabase(database);
+  }
+
+  @Override
+  public Database getDatabase(String name) throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getDatabase(name);
+  }
+
+  @Override
+  public Database getDatabase(String catalogName, String dbName) throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getDatabase(dbName);
+  }
+
+  @Override
+  public List<String> getDatabases(String pattern) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getDatabases(pattern);
+  }
+
+  @Override
+  public List<String> getDatabases(String catalogName, String dbPattern) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getDatabases(dbPattern);
+  }
+
+  @Override
+  public List<String> getAllDatabases() throws MetaException, TException {
+    return getDatabases(".*");
+  }
+
+  @Override
+  public List<String> getAllDatabases(String catalogName) throws MetaException, TException {
+    return getDatabases(".*");
+  }
+
+  @Override
+  public void alterDatabase(String databaseName, Database database) throws NoSuchObjectException, MetaException,
+          TException {
+    glueMetastoreClientDelegate.alterDatabase(databaseName, database);
+  }
+
+  @Override
+  public void alterDatabase(String catalogName, String databaseName, Database database) throws NoSuchObjectException, MetaException, TException {
+    glueMetastoreClientDelegate.alterDatabase(databaseName, database);
+  }
+
+  @Override
+  public void dropDatabase(String name) throws NoSuchObjectException, InvalidOperationException, MetaException,
+          TException {
+    dropDatabase(name, true, false, false);
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb) throws NoSuchObjectException,
+          InvalidOperationException, MetaException, TException {
+    dropDatabase(name, deleteData, ignoreUnknownDb, false);
+  }
+
+  @Override
+  public void dropDatabase(String name, boolean deleteData, boolean ignoreUnknownDb, boolean cascade)
+          throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.dropDatabase(name, deleteData, ignoreUnknownDb, cascade);
+  }
+
+  @Override
+  public void dropDatabase(String catalogName, String dbName, boolean deleteData, boolean ignoreUnknownDb, boolean cascade) throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.dropDatabase(dbName, deleteData, ignoreUnknownDb, cascade);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition add_partition(org.apache.hadoop.hive.metastore.api.Partition partition)
+          throws InvalidObjectException, org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException,
+          TException {
+    glueMetastoreClientDelegate.addPartitions(Lists.newArrayList(partition), false, true);
+    return partition;
+  }
+
+  @Override
+  public int add_partitions(List<org.apache.hadoop.hive.metastore.api.Partition> partitions)
+          throws InvalidObjectException, org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException,
+          TException {
+    return glueMetastoreClientDelegate.addPartitions(partitions, false, true).size();
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> add_partitions(
+          List<org.apache.hadoop.hive.metastore.api.Partition> partitions,
+          boolean ifNotExists,
+          boolean needResult
+  ) throws TException {
+    return glueMetastoreClientDelegate.addPartitions(partitions, ifNotExists, needResult);
+  }
+
+  @Override
+  public int add_partitions_pspec(
+          PartitionSpecProxy pSpec
+  ) throws InvalidObjectException, org.apache.hadoop.hive.metastore.api.AlreadyExistsException,
+          MetaException, TException {
+    return glueMetastoreClientDelegate.addPartitionsSpecProxy(pSpec);
+  }
+
+  @Override
+  public void alterFunction(String dbName, String functionName, org.apache.hadoop.hive.metastore.api.Function newFunction) throws InvalidObjectException,
+          MetaException, TException {
+    glueMetastoreClientDelegate.alterFunction(dbName, functionName, newFunction);
+  }
+
+  @Override
+  public void alterFunction(String catalogName, String dbName, String functionName, Function newFunction) throws InvalidObjectException, MetaException, TException {
+    glueMetastoreClientDelegate.alterFunction(dbName, functionName, newFunction);
+  }
+
+  @Override
+  public void alter_partition(
+          String dbName,
+          String tblName,
+          org.apache.hadoop.hive.metastore.api.Partition partition
+  ) throws InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.alterPartitions(dbName, tblName, Lists.newArrayList(partition));
+  }
+
+  @Override
+  public void alter_partition(
+          String dbName,
+          String tblName,
+          org.apache.hadoop.hive.metastore.api.Partition partition,
+          EnvironmentContext environmentContext
+  ) throws InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.alterPartitions(dbName, tblName, Lists.newArrayList(partition));
+  }
+
+  @Override
+  public void alter_partition(
+          String catalogName,
+          String dbName,
+          String tblName,
+          org.apache.hadoop.hive.metastore.api.Partition partition,
+          EnvironmentContext environmentContext
+  ) throws InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.alterPartitions(dbName, tblName, Lists.newArrayList(partition));
+  }
+
+  @Override
+  public void alter_partitions(
+          String dbName,
+          String tblName,
+          List<org.apache.hadoop.hive.metastore.api.Partition> partitions
+  ) throws InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.alterPartitions(dbName, tblName, partitions);
+  }
+
+  @Override
+  public void alter_partitions(
+          String dbName,
+          String tblName,
+          List<org.apache.hadoop.hive.metastore.api.Partition> partitions,
+          EnvironmentContext environmentContext
+  ) throws InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.alterPartitions(dbName, tblName, partitions);
+  }
+
+  @Override
+  public void alter_partitions(
+          String catalogName,
+          String dbName,
+          String tblName,
+          List<org.apache.hadoop.hive.metastore.api.Partition> partitions,
+          EnvironmentContext environmentContext
+  ) throws InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.alterPartitions(dbName, tblName, partitions);
+  }
+
+  @Override
+  public void alter_table(String dbName, String tblName, Table table)
+          throws InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.alterTable(dbName, tblName, table, null);
+  }
+
+  @Override
+  public void alter_table(String catalogName, String dbName, String tblName, Table table, EnvironmentContext environmentContext)
+          throws InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.alterTable(dbName, tblName, table, null);
+  }
+
+  @Override
+  public void alter_table(String dbName, String tblName, Table table, boolean cascade)
+          throws InvalidOperationException, MetaException, TException {
+    EnvironmentContext environmentContext = null;
+    if (cascade) {
+      environmentContext = new EnvironmentContext();
+      environmentContext.putToProperties("CASCADE", StatsSetupConst.TRUE);
+    }
+    glueMetastoreClientDelegate.alterTable(dbName, tblName, table, environmentContext);
+  }
+
+  @Override
+  public void alter_table_with_environmentContext(
+          String dbName,
+          String tblName,
+          Table table,
+          EnvironmentContext environmentContext
+  ) throws InvalidOperationException, MetaException, TException {
+    glueMetastoreClientDelegate.alterTable(dbName, tblName, table, environmentContext);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition appendPartition(String dbName, String tblName, List<String> values)
+          throws InvalidObjectException, org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException, TException {
+    return glueMetastoreClientDelegate.appendPartition(dbName, tblName, values);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition appendPartition(String catalogName, String dbName, String tblName, List<String> values)
+          throws InvalidObjectException, org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException, TException {
+    return glueMetastoreClientDelegate.appendPartition(dbName, tblName, values);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition appendPartition(String dbName, String tblName, String partitionName) throws InvalidObjectException,
+          org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException, TException {
+    List<String> partVals = partitionNameToVals(partitionName);
+    return glueMetastoreClientDelegate.appendPartition(dbName, tblName, partVals);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition appendPartition(String catalogName, String dbName, String tblName, String partitionName)
+          throws InvalidObjectException, org.apache.hadoop.hive.metastore.api.AlreadyExistsException, MetaException, TException {
+    List<String> partVals = partitionNameToVals(partitionName);
+    return glueMetastoreClientDelegate.appendPartition(dbName, tblName, partVals);
+  }
+
+  @Override
+  public boolean create_role(org.apache.hadoop.hive.metastore.api.Role role) throws MetaException, TException {
+    return glueMetastoreClientDelegate.createRole(role);
+  }
+
+  @Override
+  public boolean drop_role(String roleName) throws MetaException, TException {
+    return glueMetastoreClientDelegate.dropRole(roleName);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Role> list_roles(
+          String principalName, org.apache.hadoop.hive.metastore.api.PrincipalType principalType
+  ) throws MetaException, TException {
+    return glueMetastoreClientDelegate.listRoles(principalName, principalType);
+  }
+
+  @Override
+  public List<String> listRoleNames() throws MetaException, TException {
+    return glueMetastoreClientDelegate.listRoleNames();
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleResponse get_principals_in_role(
+          org.apache.hadoop.hive.metastore.api.GetPrincipalsInRoleRequest request) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getPrincipalsInRole(request);
+  }
+
+  @Override
+  public GetRoleGrantsForPrincipalResponse get_role_grants_for_principal(
+          GetRoleGrantsForPrincipalRequest request) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getRoleGrantsForPrincipal(request);
+  }
+
+  @Override
+  public boolean grant_role(
+          String roleName,
+          String userName,
+          org.apache.hadoop.hive.metastore.api.PrincipalType principalType,
+          String grantor, org.apache.hadoop.hive.metastore.api.PrincipalType grantorType,
+          boolean grantOption
+  ) throws MetaException, TException {
+    return glueMetastoreClientDelegate.grantRole(roleName, userName, principalType, grantor, grantorType, grantOption);
+  }
+
+  @Override
+  public boolean revoke_role(
+          String roleName,
+          String userName,
+          org.apache.hadoop.hive.metastore.api.PrincipalType principalType,
+          boolean grantOption
+  ) throws MetaException, TException {
+    return glueMetastoreClientDelegate.revokeRole(roleName, userName, principalType, grantOption);
+  }
+
+  @Override
+  public void cancelDelegationToken(String tokenStrForm) throws MetaException, TException {
+    glueMetastoreClientDelegate.cancelDelegationToken(tokenStrForm);
+  }
+
+  @Override
+  public String getTokenStrForm() throws IOException {
+    return glueMetastoreClientDelegate.getTokenStrForm();
+  }
+
+  @Override
+  public boolean addToken(String tokenIdentifier, String delegationToken) throws TException {
+    return glueMetastoreClientDelegate.addToken(tokenIdentifier, delegationToken);
+  }
+
+  @Override
+  public boolean removeToken(String tokenIdentifier) throws TException {
+    return glueMetastoreClientDelegate.removeToken(tokenIdentifier);
+  }
+
+  @Override
+  public String getToken(String tokenIdentifier) throws TException {
+    return glueMetastoreClientDelegate.getToken(tokenIdentifier);
+  }
+
+  @Override
+  public List<String> getAllTokenIdentifiers() throws TException {
+    return glueMetastoreClientDelegate.getAllTokenIdentifiers();
+  }
+
+  @Override
+  public int addMasterKey(String key) throws MetaException, TException {
+    return glueMetastoreClientDelegate.addMasterKey(key);
+  }
+
+  @Override
+  public void updateMasterKey(Integer seqNo, String key) throws NoSuchObjectException, MetaException, TException {
+    glueMetastoreClientDelegate.updateMasterKey(seqNo, key);
+  }
+
+  @Override
+  public boolean removeMasterKey(Integer keySeq) throws TException {
+    return glueMetastoreClientDelegate.removeMasterKey(keySeq);
+  }
+
+  @Override
+  public String[] getMasterKeys() throws TException {
+    return glueMetastoreClientDelegate.getMasterKeys();
+  }
+
+  @Override
+  public LockResponse checkLock(long lockId)
+          throws NoSuchTxnException, TxnAbortedException, NoSuchLockException, TException {
+    return glueMetastoreClientDelegate.checkLock(lockId);
+  }
+
+  @Override
+  public void close() {
+    currentMetaVars = null;
+  }
+
+  @Override
+  public void commitTxn(long txnId) throws NoSuchTxnException, TxnAbortedException, TException {
+    glueMetastoreClientDelegate.commitTxn(txnId);
+  }
+
+  @Override
+  public void replCommitTxn(long srcTxnid, String replPolicy) throws NoSuchTxnException, TxnAbortedException, TException {
+    glueMetastoreClientDelegate.replCommitTxn(srcTxnid, replPolicy);
+  }
+
+  @Override
+  public void abortTxns(List<Long> txnIds) throws TException {
+    glueMetastoreClientDelegate.abortTxns(txnIds);
+  }
+
+  @Override
+  public long allocateTableWriteId(long txnId, String dbName, String tableName) throws TException {
+    throw new UnsupportedOperationException("allocateTableWriteId is not supported.");
+  }
+
+  @Override
+  public void replTableWriteIdState(String validWriteIdList, String dbName, String tableName, List<String> partNames) throws TException {
+    throw new UnsupportedOperationException("replTableWriteIdState is not supported.");
+  }
+
+  @Override
+  public List<TxnToWriteId> allocateTableWriteIdsBatch(List<Long> txnIds, String dbName, String tableName) throws TException {
+    throw new UnsupportedOperationException("allocateTableWriteIdsBatch is not supported.");
+  }
+
+  @Override
+  public List<TxnToWriteId> replAllocateTableWriteIdsBatch(String dbName, String tableName, String replPolicy,
+          List<TxnToWriteId> srcTxnToWriteIdList) throws TException {
+    throw new UnsupportedOperationException("replAllocateTableWriteIdsBatch is not supported.");
+  }
+
+  @Deprecated
+  public void compact(
+          String dbName,
+          String tblName,
+          String partitionName,
+          CompactionType compactionType
+  ) throws TException {
+    glueMetastoreClientDelegate.compact(dbName, tblName, partitionName, compactionType);
+  }
+
+  @Deprecated
+  public void compact(
+          String dbName,
+          String tblName,
+          String partitionName,
+          CompactionType compactionType,
+          Map<String, String> tblProperties
+  ) throws TException {
+    glueMetastoreClientDelegate.compact(dbName, tblName, partitionName, compactionType, tblProperties);
+  }
+
+  @Override
+  public CompactionResponse compact2(
+          String dbName,
+          String tblName,
+          String partitionName,
+          CompactionType compactionType,
+          Map<String, String> tblProperties
+  ) throws TException {
+    return glueMetastoreClientDelegate.compact2(dbName, tblName, partitionName, compactionType, tblProperties);
+  }
+
+  @Override
+  public void createFunction(org.apache.hadoop.hive.metastore.api.Function function) throws InvalidObjectException, MetaException, TException {
+    glueMetastoreClientDelegate.createFunction(function);
+  }
+
+  @Override
+  public void createTable(Table tbl) throws org.apache.hadoop.hive.metastore.api.AlreadyExistsException, InvalidObjectException, MetaException,
+          NoSuchObjectException, TException {
+    glueMetastoreClientDelegate.createTable(tbl);
+  }
+
+  @Override
+  public boolean deletePartitionColumnStatistics(
+          String dbName, String tableName, String partName, String colName
+  ) throws NoSuchObjectException, MetaException, InvalidObjectException,
+          TException, org.apache.hadoop.hive.metastore.api.InvalidInputException {
+    return glueMetastoreClientDelegate.deletePartitionColumnStatistics(dbName, tableName, partName, colName);
+  }
+
+  @Override
+  public boolean deletePartitionColumnStatistics(String catalogName, String dbName, String tableName, String partName, String colName)
+          throws NoSuchObjectException, MetaException, InvalidObjectException, TException, InvalidInputException {
+    return glueMetastoreClientDelegate.deletePartitionColumnStatistics(dbName, tableName, partName, colName);
+  }
+
+  @Override
+  public boolean deleteTableColumnStatistics(
+          String dbName, String tableName, String colName
+  ) throws NoSuchObjectException, MetaException, InvalidObjectException,
+          TException, org.apache.hadoop.hive.metastore.api.InvalidInputException {
+    return glueMetastoreClientDelegate.deleteTableColumnStatistics(dbName, tableName, colName);
+  }
+
+  @Override
+  public boolean deleteTableColumnStatistics(String catalogName, String dbName, String tableName, String colName)
+          throws NoSuchObjectException, MetaException, InvalidObjectException, TException, InvalidInputException {
+    return glueMetastoreClientDelegate.deleteTableColumnStatistics(dbName, tableName, colName);
+  }
+
+  @Override
+  public void dropFunction(String dbName, String functionName) throws MetaException, NoSuchObjectException,
+          InvalidObjectException, org.apache.hadoop.hive.metastore.api.InvalidInputException, TException {
+    glueMetastoreClientDelegate.dropFunction(dbName, functionName);
+  }
+
+  @Override
+  public void dropFunction(String catalogName, String dbName, String functionName)
+          throws MetaException, NoSuchObjectException, InvalidObjectException, InvalidInputException, TException {
+    glueMetastoreClientDelegate.dropFunction(dbName, functionName);
+  }
+
+  private void deleteParentRecursive(Path parent, int depth, boolean mustPurge) throws IOException, MetaException {
+    if (depth > 0 && parent != null && wh.isWritable(parent) && wh.isEmpty(parent)) {
+      wh.deleteDir(parent, true, mustPurge, true);
+      deleteParentRecursive(parent.getParent(), depth - 1, mustPurge);
+    }
+  }
+
+  // This logic is taken from HiveMetaStore#isMustPurge
+  private boolean isMustPurge(Table table, boolean ifPurge) {
+    return (ifPurge || "true".equalsIgnoreCase(table.getParameters().get("auto.purge")));
+  }
+
+  @Override
+  public boolean dropPartition(String dbName, String tblName, List<String> values, boolean deleteData)
+          throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.dropPartition(dbName, tblName, values, false, deleteData, false);
+  }
+
+  @Override
+  public boolean dropPartition(String catalogName, String dbName, String tblName, List<String> values, boolean deleteData)
+          throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.dropPartition(dbName, tblName, values, false, deleteData, false);
+  }
+
+  @Override
+  public boolean dropPartition(String dbName, String tblName, List<String> values, PartitionDropOptions options) throws TException {
+    return glueMetastoreClientDelegate.dropPartition(dbName, tblName, values, options.ifExists, options.deleteData, options.purgeData);
+  }
+
+  @Override
+  public boolean dropPartition(String catalogName, String dbName, String tblName, List<String> values, PartitionDropOptions options)
+          throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.dropPartition(dbName, tblName, values, options.ifExists, options.deleteData, options.purgeData);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> dropPartitions(
+          String dbName,
+          String tblName,
+          List<ObjectPair<Integer, byte[]>> partExprs,
+          boolean deleteData,
+          boolean ifExists
+  ) throws NoSuchObjectException, MetaException, TException {
+    //use defaults from PartitionDropOptions for purgeData
+    return dropPartitions_core(dbName, tblName, partExprs, deleteData, false);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> dropPartitions(
+          String dbName,
+          String tblName,
+          List<ObjectPair<Integer, byte[]>> partExprs,
+          boolean deleteData,
+          boolean ifExists,
+          boolean needResults
+  ) throws NoSuchObjectException, MetaException, TException {
+    return dropPartitions_core(dbName, tblName, partExprs, deleteData, false);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> dropPartitions(
+          String dbName,
+          String tblName,
+          List<ObjectPair<Integer, byte[]>> partExprs,
+          PartitionDropOptions options
+  ) throws NoSuchObjectException, MetaException, TException {
+    return dropPartitions_core(dbName, tblName, partExprs, options.deleteData, options.purgeData);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> dropPartitions(
+          String catalogName,
+          String dbName,
+          String tblName,
+          List<ObjectPair<Integer, byte[]>> partExprs,
+          PartitionDropOptions options
+  ) throws NoSuchObjectException, MetaException, TException {
+    return dropPartitions_core(dbName, tblName, partExprs, options.deleteData, options.purgeData);
+  }
+
+  @Override
+  public boolean dropPartition(String dbName, String tblName, String partitionName, boolean deleteData)
+          throws NoSuchObjectException, MetaException, TException {
+    List<String> values = partitionNameToVals(partitionName);
+    return glueMetastoreClientDelegate.dropPartition(dbName, tblName, values, false, deleteData, false);
+  }
+
+  @Override
+  public boolean dropPartition(String catalogName, String dbName, String tblName, String partitionName, boolean deleteData)
+          throws NoSuchObjectException, MetaException, TException {
+    List<String> values = partitionNameToVals(partitionName);
+    return glueMetastoreClientDelegate.dropPartition(dbName, tblName, values, false, deleteData, false);
+  }
+
+  private List<org.apache.hadoop.hive.metastore.api.Partition> dropPartitions_core(
+          String databaseName,
+          String tableName,
+          List<ObjectPair<Integer, byte[]>> partExprs,
+          boolean deleteData,
+          boolean purgeData
+  ) throws TException {
+    List<org.apache.hadoop.hive.metastore.api.Partition> deleted = Lists.newArrayList();
+    for (ObjectPair<Integer, byte[]> expr : partExprs) {
+      byte[] tmp = expr.getSecond();
+      String exprString = ExpressionHelper.convertHiveExpressionToCatalogExpression(tmp);
+      List<Partition> catalogPartitionsToDelete = glueMetastoreClientDelegate.getCatalogPartitions(databaseName, tableName, exprString, -1);
+      deleted.addAll(batchDeletePartitions(databaseName, tableName, catalogPartitionsToDelete, deleteData, purgeData));
+    }
+    return deleted;
+  }
+
+  /**
+   * Delete all partitions in the list provided with BatchDeletePartitions request. It doesn't use transaction,
+   * so the call may result in partial failure.
+   * @param dbName
+   * @param tableName
+   * @param partitionsToDelete
+   * @return the partitions successfully deleted
+   * @throws TException
+   */
+  private List<org.apache.hadoop.hive.metastore.api.Partition> batchDeletePartitions(
+          final String dbName, final String tableName, final List<Partition> partitionsToDelete,
+          final boolean deleteData, final boolean purgeData) throws TException {
+
+    List<org.apache.hadoop.hive.metastore.api.Partition> deleted = Lists.newArrayList();
+    if (partitionsToDelete == null) {
+      return deleted;
+    }
+
+    validateBatchDeletePartitionsArguments(dbName, tableName, partitionsToDelete);
+
+    List<Future<BatchDeletePartitionsHelper>> batchDeletePartitionsFutures = Lists.newArrayList();
+
+    int numOfPartitionsToDelete = partitionsToDelete.size();
+    for (int i = 0; i < numOfPartitionsToDelete; i += BATCH_DELETE_PARTITIONS_PAGE_SIZE) {
+      int j = Math.min(i + BATCH_DELETE_PARTITIONS_PAGE_SIZE, numOfPartitionsToDelete);
+      final List<Partition> partitionsOnePage = partitionsToDelete.subList(i, j);
+
+      batchDeletePartitionsFutures.add(BATCH_DELETE_PARTITIONS_THREAD_POOL.submit(new Callable<BatchDeletePartitionsHelper>() {
+        @Override
+        public BatchDeletePartitionsHelper call() throws Exception {
+          return new BatchDeletePartitionsHelper(glueClient, dbName, tableName, catalogId, partitionsOnePage).deletePartitions();
+        }
+      }));
+    }
+
+    TException tException = null;
+    for (Future<BatchDeletePartitionsHelper> future : batchDeletePartitionsFutures) {
+      try {
+        BatchDeletePartitionsHelper batchDeletePartitionsHelper = future.get();
+        for (Partition partition : batchDeletePartitionsHelper.getPartitionsDeleted()) {
+          org.apache.hadoop.hive.metastore.api.Partition hivePartition =
+                  catalogToHiveConverter.convertPartition(partition);
+          try {
+            performDropPartitionPostProcessing(dbName, tableName, hivePartition, deleteData, purgeData);
+          } catch (TException e) {
+            logger.error("Drop partition directory failed.", e);
+            tException = tException == null ? e : tException;
+          }
+          deleted.add(hivePartition);
+        }
+        tException = tException == null ? batchDeletePartitionsHelper.getFirstTException() : tException;
+      } catch (Exception e) {
+        logger.error("Exception thrown by BatchDeletePartitions thread pool. ", e);
+      }
+    }
+
+    if (tException != null) {
+      throw tException;
+    }
+    return deleted;
+  }
+
+  private void validateBatchDeletePartitionsArguments(final String dbName, final String tableName,
+          final List<Partition> partitionsToDelete) {
+
+    Preconditions.checkArgument(dbName != null, "Database name cannot be null");
+    Preconditions.checkArgument(tableName != null, "Table name cannot be null");
+    for (Partition partition : partitionsToDelete) {
+      Preconditions.checkArgument(dbName.equals(partition.getDatabaseName()), "Database name cannot be null");
+      Preconditions.checkArgument(tableName.equals(partition.getTableName()), "Table name cannot be null");
+      Preconditions.checkArgument(partition.getValues() != null, "Partition values cannot be null");
+    }
+  }
+
+  // Preserve the logic from Hive metastore
+  private void performDropPartitionPostProcessing(String dbName, String tblName,
+          org.apache.hadoop.hive.metastore.api.Partition partition, boolean deleteData, boolean ifPurge)
+          throws MetaException, NoSuchObjectException, TException {
+    if (deleteData && partition.getSd() != null && partition.getSd().getLocation() != null) {
+      Path partPath = new Path(partition.getSd().getLocation());
+      Table table = getTable(dbName, tblName);
+      if (isExternalTable(table)){
+        //Don't delete external table data
+        return;
+      }
+      boolean mustPurge = isMustPurge(table, ifPurge);
+      wh.deleteDir(partPath, true, mustPurge, true);
+      try {
+        List<String> values = partition.getValues();
+        deleteParentRecursive(partPath.getParent(), values.size() - 1, mustPurge);
+      } catch (IOException e) {
+        throw new MetaException(e.getMessage());
+      }
+    }
+  }
+
+  @Deprecated
+  public void dropTable(String tableName, boolean deleteData) throws MetaException, UnknownTableException, TException,
+          NoSuchObjectException {
+    dropTable(DEFAULT_DATABASE_NAME, tableName, deleteData, false);
+  }
+
+  @Override
+  public void dropTable(String dbname, String tableName) throws MetaException, TException, NoSuchObjectException {
+    dropTable(dbname, tableName, true, true, false);
+  }
+
+  @Override
+  public void dropTable(
+          String catName,
+          String dbName,
+          String tableName,
+          boolean deleteData,
+          boolean ignoreUnknownTable,
+          boolean ifPurge
+  ) throws MetaException, NoSuchObjectException, TException {
+    glueMetastoreClientDelegate.dropTable(dbName, tableName, deleteData, ignoreUnknownTable, ifPurge);
+  }
+
+  @Override
+  public void truncateTable(String dbName, String tableName, List<String> partNames) throws MetaException, TException {
+    throw new UnsupportedOperationException("truncateTable is not supported");
+  }
+
+  @Override
+  public void truncateTable(String catalogName, String dbName, String tableName, List<String> partNames) throws MetaException, TException {
+    throw new UnsupportedOperationException("truncateTable is not supported");
+  }
+
+  @Override
+  public CmRecycleResponse recycleDirToCmPath(CmRecycleRequest cmRecycleRequest) throws MetaException, TException {
+    // Taken from HiveMetaStore#cm_recycle
+    wh.recycleDirToCmPath(new Path(cmRecycleRequest.getDataPath()), cmRecycleRequest.isPurge());
+    return new CmRecycleResponse();
+  }
+
+  @Override
+  public void dropTable(String dbname, String tableName, boolean deleteData, boolean ignoreUnknownTab)
+          throws MetaException, TException, NoSuchObjectException {
+    dropTable(dbname, tableName, deleteData, ignoreUnknownTab, false);
+  }
+
+  @Override
+  public void dropTable(String dbname, String tableName, boolean deleteData, boolean ignoreUnknownTab, boolean ifPurge)
+          throws MetaException, TException, NoSuchObjectException {
+    glueMetastoreClientDelegate.dropTable(dbname, tableName, deleteData, ignoreUnknownTab, ifPurge);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition exchange_partition(
+          Map<String, String> partitionSpecs,
+          String srcDb,
+          String srcTbl,
+          String dstDb,
+          String dstTbl
+  ) throws MetaException, NoSuchObjectException, InvalidObjectException, TException {
+    return glueMetastoreClientDelegate.exchangePartition(partitionSpecs, srcDb, srcTbl, dstDb, dstTbl);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition exchange_partition(
+          Map<String, String> partitionSpecs,
+          String sourceCat,
+          String sourceDb,
+          String sourceTable,
+          String destCat,
+          String destdb,
+          String destTableName
+  ) throws MetaException, NoSuchObjectException, InvalidObjectException, TException {
+    return glueMetastoreClientDelegate.exchangePartition(partitionSpecs, sourceDb, sourceTable, destdb, destTableName);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> exchange_partitions(
+          Map<String, String> partitionSpecs,
+          String sourceDb,
+          String sourceTbl,
+          String destDb,
+          String destTbl
+  ) throws MetaException, NoSuchObjectException, InvalidObjectException, TException {
+    return glueMetastoreClientDelegate.exchangePartitions(partitionSpecs, sourceDb, sourceTbl, destDb, destTbl);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> exchange_partitions(
+          Map<String, String> partitionSpecs,
+          String sourceCat,
+          String sourceDb,
+          String sourceTbl,
+          String destCat,
+          String destDb,
+          String destTbl
+  ) throws MetaException, NoSuchObjectException, InvalidObjectException, TException {
+    return glueMetastoreClientDelegate.exchangePartitions(partitionSpecs, sourceDb, sourceTbl, destDb, destTbl);
+  }
+
+  @Override
+  public AggrStats getAggrColStatsFor(String dbName, String tblName, List<String> colNames, List<String> partName)
+          throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getAggrColStatsFor(dbName, tblName, colNames, partName);
+  }
+
+  @Override
+  public AggrStats getAggrColStatsFor(String catalogName, String dbName, String tblName, List<String> colNames, List<String> partName)
+          throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getAggrColStatsFor(dbName, tblName, colNames, partName);
+  }
+
+  @Override
+  public List<String> getAllTables(String dbname) throws MetaException, TException, UnknownDBException {
+    return getTables(dbname, ".*");
+  }
+
+  @Override
+  public List<String> getAllTables(String catalogName, String dbname) throws MetaException, TException, UnknownDBException {
+    return getTables(dbname, ".*");
+  }
+
+  @Override
+  public String getConfigValue(String name, String defaultValue) throws TException, ConfigValSecurityException {
+    if (name == null) {
+      return defaultValue;
+    }
+
+    if(!Pattern.matches("(hive|hdfs|mapred|metastore).*", name)) {
+      throw new ConfigValSecurityException("For security reasons, the config key " + name + " cannot be accessed");
+    }
+
+    return conf.get(name, defaultValue);
+  }
+
+  @Override
+  public String getDelegationToken(
+          String owner, String renewerKerberosPrincipalName
+  ) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getDelegationToken(owner, renewerKerberosPrincipalName);
+  }
+
+  @Override
+  public List<FieldSchema> getFields(String db, String tableName) throws MetaException, TException,
+          UnknownTableException, UnknownDBException {
+    return glueMetastoreClientDelegate.getFields(db, tableName);
+  }
+
+  @Override
+  public List<FieldSchema> getFields(String catalogName, String db, String tableName) throws MetaException, TException, UnknownTableException, UnknownDBException {
+    return glueMetastoreClientDelegate.getFields(db, tableName);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Function getFunction(String dbName, String functionName) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getFunction(dbName, functionName);
+  }
+
+  @Override
+  public Function getFunction(String catalogName, String dbName, String functionName) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getFunction(dbName, functionName);
+  }
+
+  @Override
+  public List<String> getFunctions(String dbName, String pattern) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getFunctions(dbName, pattern);
+  }
+
+  @Override
+  public List<String> getFunctions(String catalogName, String dbName, String pattern) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getFunctions(dbName, pattern);
+  }
+
+  @Override
+  public GetAllFunctionsResponse getAllFunctions() throws MetaException, TException {
+    return glueMetastoreClientDelegate.getAllFunctions();
+  }
+
+  @Override
+  public String getMetaConf(String key) throws MetaException, TException {
+    MetastoreConf.ConfVars metaConfVar = MetastoreConf.getMetaConf(key);
+    if (metaConfVar == null) {
+      throw new MetaException("Invalid configuration key " + key);
+    }
+    return conf.get(key, metaConfVar.getDefaultVal().toString());
+  }
+
+  @Override
+  public void createCatalog(Catalog catalog) throws org.apache.hadoop.hive.metastore.api.AlreadyExistsException, InvalidObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("createCatalog is not supported");
+  }
+
+  @Override
+  public void alterCatalog(String s, Catalog catalog) throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("alterCatalog is not supported");
+  }
+
+  @Override
+  public Catalog getCatalog(String s) throws NoSuchObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("getCatalog is not supported");
+  }
+
+  @Override
+  public List<String> getCatalogs() throws MetaException, TException {
+    throw new UnsupportedOperationException("getCatalogs is not supported");
+  }
+
+  @Override
+  public void dropCatalog(String s) throws NoSuchObjectException, InvalidOperationException, MetaException, TException {
+    throw new UnsupportedOperationException("dropCatalog is not supported");
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition getPartition(String dbName, String tblName, List<String> values)
+          throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getPartition(dbName, tblName, values);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition getPartition(String catalogName, String dbName, String tblName, List<String> values) throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getPartition(dbName, tblName, values);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition getPartition(String dbName, String tblName, String partitionName)
+          throws MetaException, UnknownTableException, NoSuchObjectException, TException {
+    return glueMetastoreClientDelegate.getPartition(dbName, tblName, partitionName);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition getPartition(String catalogName, String dbName, String tblName, String partitionName) throws MetaException, UnknownTableException, NoSuchObjectException, TException {
+    return glueMetastoreClientDelegate.getPartition(dbName, tblName, partitionName);
+  }
+
+  @Override
+  public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
+          String dbName,
+          String tableName,
+          List<String> partitionNames,
+          List<String> columnNames
+  ) throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getPartitionColumnStatistics(dbName, tableName, partitionNames, columnNames);
+  }
+
+  @Override
+  public Map<String, List<ColumnStatisticsObj>> getPartitionColumnStatistics(
+          String catalogName,
+          String dbName,
+          String tableName,
+          List<String> partitionNames,
+          List<String> columnNames) throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getPartitionColumnStatistics(dbName, tableName, partitionNames, columnNames);
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition getPartitionWithAuthInfo(
+          String databaseName, String tableName, List<String> values,
+          String userName, List<String> groupNames)
+          throws MetaException, UnknownTableException, NoSuchObjectException, TException {
+
+    // TODO move this into the service
+    org.apache.hadoop.hive.metastore.api.Partition partition = getPartition(databaseName, tableName, values);
+    Table table = getTable(databaseName, tableName);
+    if ("TRUE".equalsIgnoreCase(table.getParameters().get("PARTITION_LEVEL_PRIVILEGE"))) {
+      String partName = Warehouse.makePartName(table.getPartitionKeys(), values);
+      HiveObjectRef obj = new HiveObjectRef();
+      obj.setObjectType(HiveObjectType.PARTITION);
+      obj.setDbName(databaseName);
+      obj.setObjectName(tableName);
+      obj.setPartValues(values);
+      org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet privilegeSet =
+              this.get_privilege_set(obj, userName, groupNames);
+      partition.setPrivileges(privilegeSet);
+    }
+
+    return partition;
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.Partition getPartitionWithAuthInfo(
+          String catalogName,
+          String databaseName,
+          String tableName,
+          List<String> values,
+          String userName,
+          List<String> groupNames) throws MetaException, UnknownTableException, NoSuchObjectException, TException {
+    return getPartitionWithAuthInfo(databaseName, tableName, values, userName, groupNames);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(
+          String databaseName, String tableName, List<String> partitionNames)
+          throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getPartitionsByNames(databaseName, tableName, partitionNames);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> getPartitionsByNames(
+          String catalogName,
+          String databaseName,
+          String tableName,
+          List<String> partitionNames
+  ) throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getPartitionsByNames(databaseName, tableName, partitionNames);
+  }
+
+  @Override
+  public List<FieldSchema> getSchema(String db, String tableName) throws MetaException, TException, UnknownTableException,
+          UnknownDBException {
+    return glueMetastoreClientDelegate.getSchema(db, tableName);
+  }
+
+  @Override
+  public List<FieldSchema> getSchema(String catalogName, String db, String tableName) throws MetaException, TException, UnknownTableException, UnknownDBException {
+    return glueMetastoreClientDelegate.getSchema(db, tableName);
+  }
+
+  @Override
+  public Table getTable(String dbName, String tableName)
+          throws MetaException, TException, NoSuchObjectException {
+    return glueMetastoreClientDelegate.getTable(dbName, tableName);
+  }
+
+  @Override
+  public Table getTable(String catalogName, String dbName, String tableName) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getTable(dbName, tableName);
+  }
+
+  @Override
+  public List<ColumnStatisticsObj> getTableColumnStatistics(String dbName, String tableName, List<String> colNames)
+          throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getTableColumnStatistics(dbName, tableName, colNames);
+  }
+
+  @Override
+  public List<ColumnStatisticsObj> getTableColumnStatistics(String catalogName, String dbName, String tableName, List<String> colNames) throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getTableColumnStatistics(dbName, tableName, colNames);
+  }
+
+  @Override
+  public List<Table> getTableObjectsByName(String dbName, List<String> tableNames) throws MetaException,
+          InvalidOperationException, UnknownDBException, TException {
+    List<Table> hiveTables = Lists.newArrayList();
+    for(String tableName : tableNames) {
+      hiveTables.add(getTable(dbName, tableName));
+    }
+
+    return hiveTables;
+  }
+
+  @Override
+  public List<Table> getTableObjectsByName(String catalogName, String dbName, List<String> tableNames) throws MetaException, InvalidOperationException, UnknownDBException, TException {
+    return getTableObjectsByName(dbName, tableNames);
+  }
+
+  @Override
+  public Materialization getMaterializationInvalidationInfo(CreationMetadata creationMetadata, String validTxnList) throws MetaException, InvalidOperationException, UnknownDBException, TException {
+    throw new UnsupportedOperationException("getMaterializationInvalidationInfo is not supported");
+  }
+
+  @Override
+  public void updateCreationMetadata(String dbName, String tableName, CreationMetadata cm) throws MetaException, TException {
+    throw new UnsupportedOperationException("getMaterializationInvalidationInfo is not supported");
+  }
+
+  @Override
+  public void updateCreationMetadata(String catName, String dbName, String tableName, CreationMetadata cm) throws MetaException, TException {
+    throw new UnsupportedOperationException("getMaterializationInvalidationInfo is not supported");
+  }
+
+  @Override
+  public List<String> getTables(String dbname, String tablePattern) throws MetaException, TException, UnknownDBException {
+    return glueMetastoreClientDelegate.getTables(dbname, tablePattern);
+  }
+
+  @Override
+  public List<String> getTables(String catalogName, String dbname, String tablePattern) throws MetaException, TException, UnknownDBException {
+    return glueMetastoreClientDelegate.getTables(dbname, tablePattern);
+  }
+
+  @Override
+  public List<String> getTables(String dbname, String tablePattern, TableType tableType)
+          throws MetaException, TException, UnknownDBException {
+    return glueMetastoreClientDelegate.getTables(dbname, tablePattern, tableType);
+  }
+
+  @Override
+  public List<String> getTables(String catalogName, String dbname, String tablePattern, TableType tableType) throws MetaException, TException, UnknownDBException {
+    return glueMetastoreClientDelegate.getTables(dbname, tablePattern, tableType);
+  }
+
+  @Override
+  public List<String> getMaterializedViewsForRewriting(String dbName) throws MetaException, TException, UnknownDBException {
+    // not supported
+    return Lists.newArrayList();
+  }
+
+  @Override
+  public List<String> getMaterializedViewsForRewriting(String catalogName, String dbName) throws MetaException, TException, UnknownDBException {
+    // not supported
+    return Lists.newArrayList();
+  }
+
+  @Override
+  public List<TableMeta> getTableMeta(String dbPatterns, String tablePatterns, List<String> tableTypes)
+          throws MetaException, TException, UnknownDBException {
+    return glueMetastoreClientDelegate.getTableMeta(dbPatterns, tablePatterns, tableTypes);
+  }
+
+  @Override
+  public List<TableMeta> getTableMeta(String catalogName, String dbPatterns, String tablePatterns, List<String> tableTypes) throws MetaException, TException, UnknownDBException {
+    return glueMetastoreClientDelegate.getTableMeta(dbPatterns, tablePatterns, tableTypes);
+  }
+
+  @Override
+  public ValidTxnList getValidTxns() throws TException {
+    return glueMetastoreClientDelegate.getValidTxns();
+  }
+
+  @Override
+  public ValidTxnList getValidTxns(long currentTxn) throws TException {
+    return glueMetastoreClientDelegate.getValidTxns(currentTxn);
+  }
+
+  @Override
+  public ValidWriteIdList getValidWriteIds(String fullTableName) throws TException {
+    throw new UnsupportedOperationException("getValidWriteIds is not supported");
+  }
+
+  @Override
+  public List<TableValidWriteIds> getValidWriteIds(List<String> tablesList, String validTxnList) throws TException {
+    throw new UnsupportedOperationException("getValidWriteIds is not supported");
+  }
+
+  @Override
+  public org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet get_privilege_set(
+          HiveObjectRef obj,
+          String user, List<String> groups
+  ) throws MetaException, TException {
+    return glueMetastoreClientDelegate.getPrivilegeSet(obj, user, groups);
+  }
+
+  @Override
+  public boolean grant_privileges(org.apache.hadoop.hive.metastore.api.PrivilegeBag privileges)
+          throws MetaException, TException {
+    return glueMetastoreClientDelegate.grantPrivileges(privileges);
+  }
+
+  @Override
+  public boolean revoke_privileges(
+          org.apache.hadoop.hive.metastore.api.PrivilegeBag privileges,
+          boolean grantOption
+  ) throws MetaException, TException {
+    return glueMetastoreClientDelegate.revokePrivileges(privileges, grantOption);
+  }
+
+  @Override
+  public boolean refresh_privileges(HiveObjectRef hiveObjectRef, String s, PrivilegeBag privilegeBag) throws MetaException, TException {
+    throw new UnsupportedOperationException("refresh_privileges is not supported");
+  }
+
+  @Override
+  public void heartbeat(long txnId, long lockId)
+          throws NoSuchLockException, NoSuchTxnException, TxnAbortedException, TException {
+    glueMetastoreClientDelegate.heartbeat(txnId, lockId);
+  }
+
+  @Override
+  public HeartbeatTxnRangeResponse heartbeatTxnRange(long min, long max) throws TException {
+    return glueMetastoreClientDelegate.heartbeatTxnRange(min, max);
+  }
+
+  @Override
+  public boolean isCompatibleWith(Configuration conf) {
+    if (currentMetaVars == null) {
+      return false; // recreate
+    }
+    boolean compatible = true;
+    for (MetastoreConf.ConfVars oneVar : MetastoreConf.metaVars) {
+      // Since metaVars are all of different types, use string for comparison
+      String oldVar = currentMetaVars.get(oneVar.getVarname());
+      String newVar = conf.get(oneVar.getVarname(), "");
+      if (oldVar == null ||
+              (oneVar.isCaseSensitive() ? !oldVar.equals(newVar) : !oldVar.equalsIgnoreCase(newVar))) {
+        logger.info("Mestastore configuration " + oneVar.getVarname() +
+                " changed from " + oldVar + " to " + newVar);
+        compatible = false;
+      }
+    }
+    return compatible;
+  }
+
+  @Override
+  public void setHiveAddedJars(String addedJars) {
+    //taken from HiveMetaStoreClient
+    MetastoreConf.setVar(conf, MetastoreConf.ConfVars.ADDED_JARS, addedJars);
+  }
+
+  @Override
+  public boolean isLocalMetaStore() {
+    return false;
+  }
+
+  private void snapshotActiveConf() {
+    currentMetaVars = new HashMap<String, String>(MetastoreConf.metaVars.length);
+    for (MetastoreConf.ConfVars oneVar : MetastoreConf.metaVars) {
+      currentMetaVars.put(oneVar.getVarname(), conf.get(oneVar.getVarname(), ""));
+    }
+  }
+
+  @Override
+  public boolean isPartitionMarkedForEvent(String dbName, String tblName, Map<String, String> partKVs, PartitionEventType eventType)
+          throws MetaException, NoSuchObjectException, TException, UnknownTableException, UnknownDBException,
+          UnknownPartitionException, InvalidPartitionException {
+    return glueMetastoreClientDelegate.isPartitionMarkedForEvent(dbName, tblName, partKVs, eventType);
+  }
+
+  @Override
+  public boolean isPartitionMarkedForEvent(String catalogName, String dbName, String tblName, Map<String, String> partKVs, PartitionEventType eventType)
+          throws MetaException, NoSuchObjectException, TException, UnknownTableException, UnknownDBException, UnknownPartitionException, InvalidPartitionException {
+    return glueMetastoreClientDelegate.isPartitionMarkedForEvent(dbName, tblName, partKVs, eventType);
+  }
+
+  @Override
+  public List<String> listPartitionNames(String dbName, String tblName, short max)
+          throws MetaException, TException {
+    try {
+      return glueMetastoreClientDelegate.listPartitionNames(dbName, tblName, null, max);
+    } catch (NoSuchObjectException e) {
+      // For compatibility with Hive 1.0.0
+      return Collections.emptyList();
+    }
+  }
+
+  @Override
+  public List<String> listPartitionNames(String catalogName, String dbName, String tblName, int maxParts)
+          throws NoSuchObjectException, MetaException, TException {
+    return listPartitionNames(dbName, tblName, (short) maxParts);
+  }
+
+  @Override
+  public List<String> listPartitionNames(String databaseName, String tableName,
+          List<String> values, short max)
+          throws MetaException, TException, NoSuchObjectException {
+    return glueMetastoreClientDelegate.listPartitionNames(databaseName, tableName, values, max);
+  }
+
+  @Override
+  public List<String> listPartitionNames(String catalogName, String databaseName, String tableName, List<String> values, int max)
+          throws MetaException, TException, NoSuchObjectException {
+    return listPartitionNames(databaseName, tableName, values, (short) max);
+  }
+
+  @Override
+  public PartitionValuesResponse listPartitionValues(PartitionValuesRequest partitionValuesRequest) throws TException {
+    return glueMetastoreClientDelegate.listPartitionValues(partitionValuesRequest);
+  }
+
+  @Override
+  public int getNumPartitionsByFilter(String dbName, String tableName, String filter)
+          throws MetaException, NoSuchObjectException, TException {
+    return glueMetastoreClientDelegate.getNumPartitionsByFilter(dbName, tableName, filter);
+  }
+
+  @Override
+  public int getNumPartitionsByFilter(String catalogName, String dbName, String tableName, String filter)
+          throws MetaException, NoSuchObjectException, TException {
+    return glueMetastoreClientDelegate.getNumPartitionsByFilter(dbName, tableName, filter);
+  }
+
+  @Override
+  public PartitionSpecProxy listPartitionSpecs(String dbName, String tblName, int max) throws TException {
+    return glueMetastoreClientDelegate.listPartitionSpecs(dbName, tblName, max);
+  }
+
+  @Override
+  public PartitionSpecProxy listPartitionSpecs(String catalogName, String dbName, String tblName, int max) throws TException {
+    return glueMetastoreClientDelegate.listPartitionSpecs(dbName, tblName, max);
+  }
+
+  @Override
+  public PartitionSpecProxy listPartitionSpecsByFilter(String dbName, String tblName, String filter, int max)
+          throws MetaException, NoSuchObjectException, TException {
+    return glueMetastoreClientDelegate.listPartitionSpecsByFilter(dbName, tblName, filter, max);
+  }
+
+  @Override
+  public PartitionSpecProxy listPartitionSpecsByFilter(String catalogName, String dbName, String tblName, String filter, int max)
+          throws MetaException, NoSuchObjectException, TException {
+    return glueMetastoreClientDelegate.listPartitionSpecsByFilter(dbName, tblName, filter, max);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> listPartitions(String dbName, String tblName, short max)
+          throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getPartitions(dbName, tblName, null, max);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> listPartitions(String catalogName, String dbName, String tblName, int max)
+          throws NoSuchObjectException, MetaException, TException {
+    return glueMetastoreClientDelegate.getPartitions(dbName, tblName, null, max);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> listPartitions(
+          String databaseName,
+          String tableName,
+          List<String> values,
+          short max
+  ) throws NoSuchObjectException, MetaException, TException {
+    String expression = null;
+    if (values != null) {
+      Table table = getTable(databaseName, tableName);
+      expression = ExpressionHelper.buildExpressionFromPartialSpecification(table, values);
+    }
+    return glueMetastoreClientDelegate.getPartitions(databaseName, tableName, expression, (long) max);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> listPartitions(
+          String catalogName,
+          String databaseName,
+          String tableName,
+          List<String> values,
+          int max) throws NoSuchObjectException, MetaException, TException {
+    return listPartitions(databaseName, tableName, values, (short) max);
+  }
+
+  @Override
+  public boolean listPartitionsByExpr(
+          String databaseName,
+          String tableName,
+          byte[] expr,
+          String defaultPartitionName,
+          short max,
+          List<org.apache.hadoop.hive.metastore.api.Partition> result
+  ) throws TException {
+    checkNotNull(result, "The result argument cannot be null.");
+
+    String catalogExpression =  ExpressionHelper.convertHiveExpressionToCatalogExpression(expr);
+    List<org.apache.hadoop.hive.metastore.api.Partition> partitions =
+            glueMetastoreClientDelegate.getPartitions(databaseName, tableName, catalogExpression, (long) max);
+    result.addAll(partitions);
+
+    return false;
+  }
+
+  @Override
+  public boolean listPartitionsByExpr(
+          String catalogName,
+          String databaseName,
+          String tableName,
+          byte[] expr,
+          String defaultPartitionName,
+          int max,
+          List<org.apache.hadoop.hive.metastore.api.Partition> result) throws TException {
+    return listPartitionsByExpr(databaseName, tableName, expr, defaultPartitionName, (short) max, result);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> listPartitionsByFilter(
+          String databaseName,
+          String tableName,
+          String filter,
+          short max
+  ) throws MetaException, NoSuchObjectException, TException {
+    // we need to replace double quotes with single quotes in the filter expression
+    // since server side does not accept double quote expressions.
+    if (StringUtils.isNotBlank(filter)) {
+      filter = ExpressionHelper.replaceDoubleQuoteWithSingleQuotes(filter);
+    }
+    return glueMetastoreClientDelegate.getPartitions(databaseName, tableName, filter, (long) max);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> listPartitionsByFilter(
+          String catalogName,
+          String databaseName,
+          String tableName,
+          String filter,
+          int max) throws MetaException, NoSuchObjectException, TException {
+    return listPartitionsByFilter(databaseName, tableName, filter, (short) max);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> listPartitionsWithAuthInfo(String database, String table, short maxParts,
+          String user, List<String> groups)
+          throws MetaException, TException, NoSuchObjectException {
+    List<org.apache.hadoop.hive.metastore.api.Partition> partitions = listPartitions(database, table, maxParts);
+
+    for (org.apache.hadoop.hive.metastore.api.Partition p : partitions) {
+      HiveObjectRef obj = new HiveObjectRef();
+      obj.setObjectType(HiveObjectType.PARTITION);
+      obj.setDbName(database);
+      obj.setObjectName(table);
+      obj.setPartValues(p.getValues());
+      org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet set = this.get_privilege_set(obj, user, groups);
+      p.setPrivileges(set);
+    }
+
+    return partitions;
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> listPartitionsWithAuthInfo(
+          String catalogName,
+          String database,
+          String table,
+          int maxParts,
+          String user,
+          List<String> groups
+  ) throws MetaException, TException, NoSuchObjectException {
+    return listPartitionsWithAuthInfo(database, table, (short) maxParts, user, groups);
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> listPartitionsWithAuthInfo(String database, String table,
+          List<String> partVals, short maxParts,
+          String user, List<String> groups) throws MetaException, TException, NoSuchObjectException {
+    List<org.apache.hadoop.hive.metastore.api.Partition> partitions = listPartitions(database, table, partVals, maxParts);
+
+    for (org.apache.hadoop.hive.metastore.api.Partition p : partitions) {
+      HiveObjectRef obj = new HiveObjectRef();
+      obj.setObjectType(HiveObjectType.PARTITION);
+      obj.setDbName(database);
+      obj.setObjectName(table);
+      obj.setPartValues(p.getValues());
+      org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet set;
+      try {
+        set = get_privilege_set(obj, user, groups);
+      } catch (MetaException e) {
+        logger.info(String.format("No privileges found for user: %s, "
+                + "groups: [%s]", user, LoggingHelper.concatCollectionToStringForLogging(groups, ",")));
+        set = new org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet();
+      }
+      p.setPrivileges(set);
+    }
+
+    return partitions;
+  }
+
+  @Override
+  public List<org.apache.hadoop.hive.metastore.api.Partition> listPartitionsWithAuthInfo(
+          String catalogName,
+          String database,
+          String table,
+          List<String> partVals,
+          int maxParts,
+          String user,
+          List<String> groups) throws MetaException, TException, NoSuchObjectException {
+    return listPartitionsWithAuthInfo(database, table, partVals, (short) maxParts, user, groups);
+  }
+
+  @Override
+  public List<String> listTableNamesByFilter(String dbName, String filter, short maxTables) throws MetaException,
+          TException, InvalidOperationException, UnknownDBException {
+    return glueMetastoreClientDelegate.listTableNamesByFilter(dbName, filter, maxTables);
+  }
+
+  @Override
+  public List<String> listTableNamesByFilter(String catalogName, String dbName, String filter, int maxTables) throws TException, InvalidOperationException, UnknownDBException {
+    return glueMetastoreClientDelegate.listTableNamesByFilter(dbName, filter, (short) maxTables);
+  }
+
+  @Override
+  public List<HiveObjectPrivilege> list_privileges(
+          String principal,
+          org.apache.hadoop.hive.metastore.api.PrincipalType principalType,
+          HiveObjectRef objectRef
+  ) throws MetaException, TException {
+    return glueMetastoreClientDelegate.listPrivileges(principal, principalType, objectRef);
+  }
+
+  @Override
+  public LockResponse lock(LockRequest lockRequest) throws NoSuchTxnException, TxnAbortedException, TException {
+    return glueMetastoreClientDelegate.lock(lockRequest);
+  }
+
+  @Override
+  public void markPartitionForEvent(
+          String dbName,
+          String tblName,
+          Map<String, String> partKVs,
+          PartitionEventType eventType
+  ) throws MetaException, NoSuchObjectException, TException, UnknownTableException, UnknownDBException,
+          UnknownPartitionException, InvalidPartitionException {
+    glueMetastoreClientDelegate.markPartitionForEvent(dbName, tblName, partKVs, eventType);
+  }
+
+  @Override
+  public void markPartitionForEvent(
+          String catalogName,
+          String dbName,
+          String tblName,
+          Map<String, String> partKVs,
+          PartitionEventType eventType
+  ) throws MetaException, NoSuchObjectException, TException, UnknownTableException, UnknownDBException, UnknownPartitionException, InvalidPartitionException {
+    glueMetastoreClientDelegate.markPartitionForEvent(dbName, tblName, partKVs, eventType);
+  }
+
+  @Override
+  public long openTxn(String user) throws TException {
+    return glueMetastoreClientDelegate.openTxn(user);
+  }
+
+  @Override
+  public List<Long> replOpenTxn(String replPolicy, List<Long> srcTxnIds, String user) throws TException {
+    throw new UnsupportedOperationException("replOpenTxn is not supported");
+  }
+
+  @Override
+  public OpenTxnsResponse openTxns(String user, int numTxns) throws TException {
+    return glueMetastoreClientDelegate.openTxns(user, numTxns);
+  }
+
+  @Override
+  public Map<String, String> partitionNameToSpec(String name) throws MetaException, TException {
+    // Lifted from HiveMetaStore
+    if (name.length() == 0) {
+      return new HashMap<String, String>();
+    }
+    return Warehouse.makeSpecFromName(name);
+  }
+
+  @Override
+  public List<String> partitionNameToVals(String name) throws MetaException, TException {
+    return glueMetastoreClientDelegate.partitionNameToVals(name);
+  }
+
+  @Override
+  public void reconnect() throws MetaException {
+    // TODO reset active Hive confs for metastore glueClient
+    logger.debug("reconnect() was called.");
+  }
+
+  @Override
+  public void renamePartition(String dbName, String tblName, List<String> partitionValues,
+          org.apache.hadoop.hive.metastore.api.Partition newPartition)
+          throws InvalidOperationException, MetaException, TException {
+    throw new TException("Not implement yet");
+    // Commend out to avoid using shim
+    //// Set DDL time to now if not specified
+    //setDDLTime(newPartition);
+    //Table tbl;
+    //org.apache.hadoop.hive.metastore.api.Partition oldPart;
+    //
+    //try {
+    //  tbl = getTable(dbName, tblName);
+    //  oldPart = getPartition(dbName, tblName, partitionValues);
+    //} catch(NoSuchObjectException e) {
+    //  throw new InvalidOperationException(e.getMessage());
+    //}
+    //
+    //if(newPartition.getSd() == null || oldPart.getSd() == null ) {
+    //  throw new InvalidOperationException("Storage descriptor cannot be null");
+    //}
+    //
+    //// if an external partition is renamed, the location should not change
+    //if (!Strings.isNullOrEmpty(tbl.getTableType()) && tbl.getTableType().equals(TableType.EXTERNAL_TABLE.toString())) {
+    //  newPartition.getSd().setLocation(oldPart.getSd().getLocation());
+    //  renamePartitionInCatalog(dbName, tblName, partitionValues, newPartition);
+    //} else {
+    //
+    //  Path destPath = getDestinationPathForRename(dbName, tbl, newPartition);
+    //  Path srcPath = new Path(oldPart.getSd().getLocation());
+    //  FileSystem srcFs = wh.getFs(srcPath);
+    //  FileSystem destFs = wh.getFs(destPath);
+    //
+    //  verifyDestinationLocation(srcFs, destFs, srcPath, destPath, tbl, newPartition);
+    //  newPartition.getSd().setLocation(destPath.toString());
+    //
+    //  renamePartitionInCatalog(dbName, tblName, partitionValues, newPartition);
+    //  boolean success = true;
+    //  try{
+    //    if (srcFs.exists(srcPath)) {
+    //      //if destPath's parent path doesn't exist, we should mkdir it
+    //      Path destParentPath = destPath.getParent();
+    //      if (!hiveShims.mkdirs(wh, destParentPath)) {
+    //        throw new IOException("Unable to create path " + destParentPath);
+    //      }
+    //      wh.renameDir(srcPath, destPath, true);
+    //    }
+    //  } catch (IOException e) {
+    //    success = false;
+    //    throw new InvalidOperationException("Unable to access old location "
+    //          + srcPath + " for partition " + tbl.getDbName() + "."
+    //          + tbl.getTableName() + " " + partitionValues);
+    //  } finally {
+    //    if(!success) {
+    //      // revert metastore operation
+    //      renamePartitionInCatalog(dbName, tblName, newPartition.getValues(), oldPart);
+    //    }
+    //  }
+    //}
+  }
+
+  @Override
+  public void renamePartition(
+          String catalogName,
+          String dbName,
+          String tblName,
+          List<String> partitionValues,
+          org.apache.hadoop.hive.metastore.api.Partition newPartition
+  ) throws InvalidOperationException, MetaException, TException {
+    renamePartition(dbName, tblName, partitionValues, newPartition);
+  }
+
+  private void verifyDestinationLocation(FileSystem srcFs, FileSystem destFs, Path srcPath, Path destPath, Table tbl, org.apache.hadoop.hive.metastore.api.Partition newPartition)
+          throws InvalidOperationException {
+    String oldPartLoc = srcPath.toString();
+    String newPartLoc = destPath.toString();
+
+    // check that src and dest are on the same file system
+    if (!FileUtils.equalsFileSystem(srcFs, destFs)) {
+      throw new InvalidOperationException("table new location " + destPath
+              + " is on a different file system than the old location "
+              + srcPath + ". This operation is not supported");
+    }
+    try {
+      srcFs.exists(srcPath); // check that src exists and also checks
+      if (newPartLoc.compareTo(oldPartLoc) != 0 && destFs.exists(destPath)) {
+        throw new InvalidOperationException("New location for this partition "
+                + tbl.getDbName() + "." + tbl.getTableName() + "." + newPartition.getValues()
+                + " already exists : " + destPath);
+      }
+    } catch (IOException e) {
+      throw new InvalidOperationException("Unable to access new location "
+              + destPath + " for partition " + tbl.getDbName() + "."
+              + tbl.getTableName() + " " + newPartition.getValues());
+    }
+  }
+
+  private Path getDestinationPathForRename(String dbName, Table tbl, org.apache.hadoop.hive.metastore.api.Partition newPartition)
+          throws InvalidOperationException, MetaException, TException {
+    throw new TException("Not implement yet");
+    // Commend out to avoid using shim
+    // try {
+    //   Path destPath = new Path(hiveShims.getDefaultTablePath(getDatabase(dbName), tbl.getTableName(), wh),
+    //         Warehouse.makePartName(tbl.getPartitionKeys(), newPartition.getValues()));
+    //   return constructRenamedPath(destPath, new Path(newPartition.getSd().getLocation()));
+    // } catch (NoSuchObjectException e) {
+    //   throw new InvalidOperationException(
+    //         "Unable to change partition or table. Database " + dbName + " does not exist"
+    //               + " Check metastore logs for detailed stack." + e.getMessage());
+    // }
+  }
+
+  private void setDDLTime(org.apache.hadoop.hive.metastore.api.Partition partition) {
+    if (partition.getParameters() == null ||
+            partition.getParameters().get(hive_metastoreConstants.DDL_TIME) == null ||
+            Integer.parseInt(partition.getParameters().get(hive_metastoreConstants.DDL_TIME)) == 0) {
+      partition.putToParameters(hive_metastoreConstants.DDL_TIME, Long.toString(System
+              .currentTimeMillis() / 1000));
+    }
+  }
+
+  private void renamePartitionInCatalog(String databaseName, String tableName,
+          List<String> partitionValues, org.apache.hadoop.hive.metastore.api.Partition newPartition)
+          throws InvalidOperationException, MetaException, TException {
+    try {
+      glueClient.updatePartition(
+              new UpdatePartitionRequest()
+                      .withDatabaseName(databaseName)
+                      .withTableName(tableName)
+                      .withPartitionValueList(partitionValues)
+                      .withPartitionInput(GlueInputConverter.convertToPartitionInput(newPartition)));
+    } catch (AmazonServiceException e) {
+      throw catalogToHiveConverter.wrapInHiveException(e);
+    }
+  }
+
+  @Override
+  public long renewDelegationToken(String tokenStrForm) throws MetaException, TException {
+    return glueMetastoreClientDelegate.renewDelegationToken(tokenStrForm);
+  }
+
+  @Override
+  public void rollbackTxn(long txnId) throws NoSuchTxnException, TException {
+    glueMetastoreClientDelegate.rollbackTxn(txnId);
+  }
+
+  @Override
+  public void replRollbackTxn(long l, String s) throws NoSuchTxnException, TException {
+    throw new UnsupportedOperationException("replRollbackTxn is not supported");
+  }
+
+  @Override
+  public void setMetaConf(String key, String value) throws MetaException, TException {
+    MetastoreConf.ConfVars confVar = MetastoreConf.getMetaConf(key);
+    if (confVar == null) {
+      throw new MetaException("Invalid configuration key " + key);
+    }
+    try {
+      confVar.validate(value);
+    } catch (IllegalArgumentException e) {
+      throw new MetaException("Invalid configuration value " + value + " for key " + key +
+              " by " + e.getMessage());
+    }
+    conf.set(key, value);
+  }
+
+  @Override
+  public boolean setPartitionColumnStatistics(org.apache.hadoop.hive.metastore.api.SetPartitionsStatsRequest request)
+          throws NoSuchObjectException, InvalidObjectException,
+          MetaException, TException, org.apache.hadoop.hive.metastore.api.InvalidInputException {
+    return glueMetastoreClientDelegate.setPartitionColumnStatistics(request);
+  }
+
+  @Override
+  public void flushCache() {
+    //no op
+  }
+
+  @Override
+  public Iterable<Map.Entry<Long, ByteBuffer>> getFileMetadata(List<Long> fileIds) throws TException {
+    return glueMetastoreClientDelegate.getFileMetadata(fileIds);
+  }
+
+  @Override
+  public Iterable<Map.Entry<Long, MetadataPpdResult>> getFileMetadataBySarg(
+          List<Long> fileIds,
+          ByteBuffer sarg,
+          boolean doGetFooters
+  ) throws TException {
+    return glueMetastoreClientDelegate.getFileMetadataBySarg(fileIds, sarg, doGetFooters);
+  }
+
+  @Override
+  public void clearFileMetadata(List<Long> fileIds) throws TException {
+    glueMetastoreClientDelegate.clearFileMetadata(fileIds);
+  }
+
+  @Override
+  public void putFileMetadata(List<Long> fileIds, List<ByteBuffer> metadata) throws TException {
+    glueMetastoreClientDelegate.putFileMetadata(fileIds, metadata);
+  }
+
+  @Override
+  public boolean isSameConfObj(Configuration conf) {
+    //taken from HiveMetaStoreClient
+    return this.conf == conf;
+  }
+
+  @Override
+  public boolean cacheFileMetadata(String dbName, String tblName, String partName, boolean allParts) throws TException {
+    return glueMetastoreClientDelegate.cacheFileMetadata(dbName, tblName, partName, allParts);
+  }
+
+  @Override
+  public List<SQLPrimaryKey> getPrimaryKeys(PrimaryKeysRequest primaryKeysRequest) throws TException {
+    // PrimaryKeys are currently unsupported
+    return Lists.newArrayList();
+  }
+
+  @Override
+  public List<SQLForeignKey> getForeignKeys(ForeignKeysRequest foreignKeysRequest) throws TException {
+    // PrimaryKeys are currently unsupported
+    // return empty list to not break DESCRIBE (FORMATTED | EXTENDED)
+    return Lists.newArrayList();
+  }
+
+  @Override
+  public List<SQLUniqueConstraint> getUniqueConstraints(UniqueConstraintsRequest uniqueConstraintsRequest) throws MetaException, NoSuchObjectException, TException {
+    // Not supported, called by DESCRIBE (FORMATTED | EXTENDED)
+    return Lists.newArrayList();
+  }
+
+  @Override
+  public List<SQLNotNullConstraint> getNotNullConstraints(NotNullConstraintsRequest notNullConstraintsRequest) throws MetaException, NoSuchObjectException, TException {
+    // Not supported, called by DESCRIBE (FORMATTED | EXTENDED)
+    return Lists.newArrayList();
+  }
+
+  @Override
+  public List<SQLDefaultConstraint> getDefaultConstraints(DefaultConstraintsRequest defaultConstraintsRequest) throws MetaException, NoSuchObjectException, TException {
+    // Not supported, called by DESCRIBE (FORMATTED | EXTENDED)
+    return Lists.newArrayList();
+  }
+
+  @Override
+  public List<SQLCheckConstraint> getCheckConstraints(CheckConstraintsRequest checkConstraintsRequest) throws MetaException, NoSuchObjectException, TException {
+    // Not supported, called by DESCRIBE (FORMATTED | EXTENDED)
+    return Lists.newArrayList();
+  }
+
+  @Override
+  public void createTableWithConstraints(
+          Table table,
+          List<SQLPrimaryKey> primaryKeys,
+          List<SQLForeignKey> foreignKeys,
+          List<SQLUniqueConstraint> uniqueConstraints,
+          List<SQLNotNullConstraint> notNullConstraints,
+          List<SQLDefaultConstraint> defaultConstraints,
+          List<SQLCheckConstraint> checkConstraints
+  ) throws AlreadyExistsException, InvalidObjectException, MetaException, NoSuchObjectException, TException {
+    glueMetastoreClientDelegate.createTableWithConstraints(table, primaryKeys, foreignKeys);
+  }
+
+  @Override
+  public void dropConstraint(
+          String dbName,
+          String tblName,
+          String constraintName
+  ) throws MetaException, NoSuchObjectException, TException {
+    glueMetastoreClientDelegate.dropConstraint(dbName, tblName, constraintName);
+  }
+
+  @Override
+  public void dropConstraint(String catalogName, String dbName, String tblName, String constraintName)
+          throws MetaException, NoSuchObjectException, TException {
+    glueMetastoreClientDelegate.dropConstraint(dbName, tblName, constraintName);
+  }
+
+  @Override
+  public void addPrimaryKey(List<SQLPrimaryKey> primaryKeyCols)
+          throws MetaException, NoSuchObjectException, TException {
+    glueMetastoreClientDelegate.addPrimaryKey(primaryKeyCols);
+  }
+
+  @Override
+  public void addForeignKey(List<SQLForeignKey> foreignKeyCols)
+          throws MetaException, NoSuchObjectException, TException {
+    glueMetastoreClientDelegate.addForeignKey(foreignKeyCols);
+  }
+
+  @Override
+  public void addUniqueConstraint(List<SQLUniqueConstraint> uniqueConstraintCols) throws MetaException, NoSuchObjectException, TException {
+    throw new UnsupportedOperationException("addUniqueConstraint is not supported");
+  }
+
+  @Override
+  public void addNotNullConstraint(List<SQLNotNullConstraint> notNullConstraintCols) throws MetaException, NoSuchObjectException, TException {
+    throw new UnsupportedOperationException("addNotNullConstraint is not supported");
+  }
+
+  @Override
+  public void addDefaultConstraint(List<SQLDefaultConstraint> defaultConstraints) throws MetaException, NoSuchObjectException, TException {
+    throw new UnsupportedOperationException("addDefaultConstraint is not supported");
+  }
+
+  @Override
+  public void addCheckConstraint(List<SQLCheckConstraint> checkConstraints) throws MetaException, NoSuchObjectException, TException {
+    throw new UnsupportedOperationException("addCheckConstraint is not supported");
+  }
+
+  @Override
+  public String getMetastoreDbUuid() throws MetaException, TException {
+    throw new UnsupportedOperationException("getMetastoreDbUuid is not supported");
+  }
+
+  @Override
+  public void createResourcePlan(WMResourcePlan resourcePlan, String copyFromName) throws InvalidObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("createResourcePlan is not supported");
+  }
+
+  @Override
+  public WMFullResourcePlan getResourcePlan(String resourcePlanName) throws NoSuchObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("getResourcePlan is not supported");
+  }
+
+  @Override
+  public List<WMResourcePlan> getAllResourcePlans() throws NoSuchObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("getAllResourcePlans is not supported");
+  }
+
+  @Override
+  public void dropResourcePlan(String resourcePlanName) throws NoSuchObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("dropResourcePlan is not supported");
+  }
+
+  @Override
+  public WMFullResourcePlan alterResourcePlan(
+          String resourcePlanName,
+          WMNullableResourcePlan wmNullableResourcePlan,
+          boolean canActivateDisabled,
+          boolean isForceDeactivate,
+          boolean isReplace
+  ) throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("alterResourcePlan is not supported");
+  }
+
+  @Override
+  public WMFullResourcePlan getActiveResourcePlan() throws MetaException, TException {
+    throw new UnsupportedOperationException("getActiveResourcePlan is not supported");
+  }
+
+  @Override
+  public WMValidateResourcePlanResponse validateResourcePlan(String resourcePlanName) throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("validateResourcePlan is not supported");
+  }
+
+  @Override
+  public void createWMTrigger(WMTrigger wmTrigger) throws InvalidObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("createWMTrigger is not supported");
+  }
+
+  @Override
+  public void alterWMTrigger(WMTrigger wmTrigger) throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("alterWMTrigger is not supported");
+  }
+
+  @Override
+  public void dropWMTrigger(String resourcePlanName, String triggerName) throws NoSuchObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("dropWMTrigger is not supported");
+  }
+
+  @Override
+  public List<WMTrigger> getTriggersForResourcePlan(String resourcePlan) throws NoSuchObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("getTriggersForResourcePlan is not supported");
+  }
+
+  @Override
+  public void createWMPool(WMPool wmPool) throws NoSuchObjectException, InvalidObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("createWMPool is not supported");
+  }
+
+  @Override
+  public void alterWMPool(WMNullablePool wmNullablePool, String poolPath) throws NoSuchObjectException, InvalidObjectException, TException {
+    throw new UnsupportedOperationException("alterWMPool is not supported");
+  }
+
+  @Override
+  public void dropWMPool(String resourcePlanName, String poolPath) throws TException {
+    throw new UnsupportedOperationException("dropWMPool is not supported");
+  }
+
+  @Override
+  public void createOrUpdateWMMapping(WMMapping wmMapping, boolean isUpdate) throws TException {
+    throw new UnsupportedOperationException("createOrUpdateWMMapping is not supported");
+  }
+
+  @Override
+  public void dropWMMapping(WMMapping wmMapping) throws TException {
+    throw new UnsupportedOperationException("dropWMMapping is not supported");
+  }
+
+  @Override
+  public void createOrDropTriggerToPoolMapping(String resourcePlanName, String triggerName, String poolPath, boolean shouldDrop)
+          throws org.apache.hadoop.hive.metastore.api.AlreadyExistsException, NoSuchObjectException, InvalidObjectException, MetaException, TException {
+    throw new UnsupportedOperationException("createOrDropTriggerToPoolMapping is not supported");
+  }
+
+  @Override
+  public void createISchema(ISchema iSchema) throws TException {
+    throw new UnsupportedOperationException("createISchema is not supported");
+  }
+
+  @Override
+  public void alterISchema(String catName, String dbName, String schemaName, ISchema newSchema) throws TException {
+    throw new UnsupportedOperationException("alterISchema is not supported");
+  }
+
+  @Override
+  public ISchema getISchema(String catName, String dbName, String name) throws TException {
+    throw new UnsupportedOperationException("getISchema is not supported");
+  }
+
+  @Override
+  public void dropISchema(String catName, String dbName, String name) throws TException {
+    throw new UnsupportedOperationException("dropISchema is not supported");
+  }
+
+  @Override
+  public void addSchemaVersion(SchemaVersion schemaVersion) throws TException {
+    throw new UnsupportedOperationException("addSchemaVersion is not supported");
+  }
+
+  @Override
+  public SchemaVersion getSchemaVersion(String catName, String dbName, String schemaName, int version) throws TException {
+    throw new UnsupportedOperationException("getSchemaVersion is not supported");
+  }
+
+  @Override
+  public SchemaVersion getSchemaLatestVersion(String catName, String dbName, String schemaName) throws TException {
+    throw new UnsupportedOperationException("getSchemaLatestVersion is not supported");
+  }
+
+  @Override
+  public List<SchemaVersion> getSchemaAllVersions(String catName, String dbName, String schemaName) throws TException {
+    throw new UnsupportedOperationException("getSchemaAllVersions is not supported");
+  }
+
+  @Override
+  public void dropSchemaVersion(String catName, String dbName, String schemaName, int version) throws TException {
+    throw new UnsupportedOperationException("dropSchemaVersion is not supported");
+  }
+
+  @Override
+  public FindSchemasByColsResp getSchemaByCols(FindSchemasByColsRqst findSchemasByColsRqst) throws TException {
+    throw new UnsupportedOperationException("getSchemaByCols is not supported");
+  }
+
+  @Override
+  public void mapSchemaVersionToSerde(String catName, String dbName, String schemaName, int version, String serdeName) throws TException {
+    throw new UnsupportedOperationException("mapSchemaVersionToSerde is not supported");
+  }
+
+  @Override
+  public void setSchemaVersionState(String catName, String dbName, String schemaName, int version, SchemaVersionState state) throws TException {
+    throw new UnsupportedOperationException("setSchemaVersionState is not supported");
+  }
+
+  @Override
+  public void addSerDe(SerDeInfo serDeInfo) throws TException {
+    throw new UnsupportedOperationException("addSerDe is not supported");
+  }
+
+  @Override
+  public SerDeInfo getSerDe(String serDeName) throws TException {
+    throw new UnsupportedOperationException("getSerDe is not supported");
+  }
+
+  @Override
+  public LockResponse lockMaterializationRebuild(String dbName, String tableName, long txnId) throws TException {
+    throw new UnsupportedOperationException("lockMaterializationRebuild is not supported");
+  }
+
+  @Override
+  public boolean heartbeatLockMaterializationRebuild(String dbName, String tableName, long txnId) throws TException {
+    throw new UnsupportedOperationException("heartbeatLockMaterializationRebuild is not supported");
+  }
+
+  @Override
+  public void addRuntimeStat(RuntimeStat runtimeStat) throws TException {
+    throw new UnsupportedOperationException("addRuntimeStat is not supported");
+  }
+
+  @Override
+  public List<RuntimeStat> getRuntimeStats(int maxWeight, int maxCreateTime) throws TException {
+    throw new UnsupportedOperationException("getRuntimeStats is not supported");
+  }
+
+  @Override
+  public ShowCompactResponse showCompactions() throws TException {
+    return glueMetastoreClientDelegate.showCompactions();
+  }
+
+  @Override
+  public void addDynamicPartitions(
+          long txnId,
+          long writeId,
+          String dbName,
+          String tblName,
+          List<String> partNames
+  ) throws TException {
+    glueMetastoreClientDelegate.addDynamicPartitions(txnId, dbName, tblName, partNames);
+  }
+
+  @Override
+  public void addDynamicPartitions(
+          long txnId,
+          long writeId,
+          String dbName,
+          String tblName,
+          List<String> partNames,
+          DataOperationType operationType
+  ) throws TException {
+    glueMetastoreClientDelegate.addDynamicPartitions(txnId, dbName, tblName, partNames, operationType);
+  }
+
+  @Override
+  public void insertTable(Table table, boolean overwrite) throws MetaException {
+    glueMetastoreClientDelegate.insertTable(table, overwrite);
+  }
+
+  @Override
+  public NotificationEventResponse getNextNotification(
+          long lastEventId, int maxEvents, NotificationFilter notificationFilter) throws TException {
+    // Unsupported, workaround for HS2's notification poll.
+    return new NotificationEventResponse();
+  }
+
+  @Override
+  public CurrentNotificationEventId getCurrentNotificationEventId() throws TException {
+    // Unsupported, workaround for HS2's notification poll.
+    return new CurrentNotificationEventId(0);
+  }
+
+  @Override
+  public NotificationEventsCountResponse getNotificationEventsCount(NotificationEventsCountRequest notificationEventsCountRequest) throws TException {
+    throw new UnsupportedOperationException("getNotificationEventsCount is not supported");
+  }
+
+  @Override
+  public FireEventResponse fireListenerEvent(FireEventRequest fireEventRequest) throws TException {
+    return glueMetastoreClientDelegate.fireListenerEvent(fireEventRequest);
+  }
+
+  @Override
+  public ShowLocksResponse showLocks() throws TException {
+    return glueMetastoreClientDelegate.showLocks();
+  }
+
+  @Override
+  public ShowLocksResponse showLocks(ShowLocksRequest showLocksRequest) throws TException {
+    return glueMetastoreClientDelegate.showLocks(showLocksRequest);
+  }
+
+  @Override
+  public GetOpenTxnsInfoResponse showTxns() throws TException {
+    return glueMetastoreClientDelegate.showTxns();
+  }
+
+  @Deprecated
+  public boolean tableExists(String tableName) throws MetaException, TException, UnknownDBException {
+    //this method has been deprecated;
+    return tableExists(DEFAULT_DATABASE_NAME, tableName);
+  }
+
+  @Override
+  public boolean tableExists(String databaseName, String tableName) throws MetaException, TException,
+          UnknownDBException {
+    return glueMetastoreClientDelegate.tableExists(databaseName, tableName);
+  }
+
+  @Override
+  public boolean tableExists(String catalogName, String databaseName, String tableName)
+          throws MetaException, TException, UnknownDBException {
+    return glueMetastoreClientDelegate.tableExists(databaseName, tableName);
+  }
+
+  @Override
+  public void unlock(long lockId) throws NoSuchLockException, TxnOpenException, TException {
+    glueMetastoreClientDelegate.unlock(lockId);
+  }
+
+  @Override
+  public boolean updatePartitionColumnStatistics(org.apache.hadoop.hive.metastore.api.ColumnStatistics columnStatistics)
+          throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
+          org.apache.hadoop.hive.metastore.api.InvalidInputException {
+    return glueMetastoreClientDelegate.updatePartitionColumnStatistics(columnStatistics);
+  }
+
+  @Override
+  public boolean updateTableColumnStatistics(org.apache.hadoop.hive.metastore.api.ColumnStatistics columnStatistics)
+          throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
+          org.apache.hadoop.hive.metastore.api.InvalidInputException {
+    return glueMetastoreClientDelegate.updateTableColumnStatistics(columnStatistics);
+  }
+
+  @Override
+  public void validatePartitionNameCharacters(List<String> part_vals) throws TException, MetaException {
+    try {
+      String partitionValidationRegex =
+              MetastoreConf.getVar(conf, MetastoreConf.ConfVars.PARTITION_NAME_WHITELIST_PATTERN);
+      Pattern partitionValidationPattern = Strings.isNullOrEmpty(partitionValidationRegex) ? null
+              : Pattern.compile(partitionValidationRegex);
+      MetaStoreUtils.validatePartitionNameCharacters(part_vals, partitionValidationPattern);
+    } catch (Exception e){
+      if (e instanceof MetaException) {
+        throw (MetaException) e;
+      } else {
+        throw new MetaException(e.getMessage());
+      }
+    }
+  }
+
+  private Path constructRenamedPath(Path defaultNewPath, Path currentPath) {
+    URI currentUri = currentPath.toUri();
+
+    return new Path(currentUri.getScheme(), currentUri.getAuthority(),
+            defaultNewPath.toUri().getPath());
+  }
+
+}
diff --git a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCredentialsProviderFactory.java
similarity index 67%
copy from fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
copy to fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCredentialsProviderFactory.java
index 05084c97b5..41444078e4 100644
--- a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSCredentialsProviderFactory.java
@@ -14,12 +14,18 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import org.apache.hadoop.conf.Configuration;
 
-package org.apache.doris.load.loadv2.dpp;
+import com.amazonaws.auth.AWSCredentialsProvider;
 
-import org.apache.doris.load.loadv2.etl.EtlJobConfig;
+public interface AWSCredentialsProviderFactory {
 
-// RollupTreeBuilder is used to get the RollupTree from the TableMeta
-public abstract interface RollupTreeBuilder {
-    public RollupTreeNode build(EtlJobConfig.EtlTable tableMeta);
+  AWSCredentialsProvider buildAWSCredentialsProvider(Configuration conf);
 }
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueClientFactory.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueClientFactory.java
new file mode 100644
index 0000000000..72e75a891f
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueClientFactory.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.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.client.builder.AwsClientBuilder;
+import com.amazonaws.regions.Region;
+import com.amazonaws.regions.Regions;
+import com.amazonaws.services.glue.AWSGlue;
+import com.amazonaws.services.glue.AWSGlueClientBuilder;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_CATALOG_CREDENTIALS_PROVIDER_FACTORY_CLASS;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_CATALOG_SEPARATOR;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_CONNECTION_TIMEOUT;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_ENDPOINT;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_MAX_CONNECTIONS;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_MAX_RETRY;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_SOCKET_TIMEOUT;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_REGION;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.DEFAULT_CONNECTION_TIMEOUT;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.DEFAULT_MAX_CONNECTIONS;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.DEFAULT_MAX_RETRY;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.DEFAULT_SOCKET_TIMEOUT;
+
+public final class AWSGlueClientFactory implements GlueClientFactory {
+
+  private static final Logger logger = Logger.getLogger(AWSGlueClientFactory.class);
+
+  private final Configuration conf;
+
+  public AWSGlueClientFactory(Configuration conf) {
+    Preconditions.checkNotNull(conf, "Configuration cannot be null");
+    this.conf = conf;
+  }
+
+  @Override
+  public AWSGlue newClient() throws MetaException {
+    try {
+      AWSGlueClientBuilder glueClientBuilder = AWSGlueClientBuilder.standard()
+              .withCredentials(getAWSCredentialsProvider(conf));
+
+      String regionStr = getProperty(AWS_REGION, conf);
+      String glueEndpoint = getProperty(AWS_GLUE_ENDPOINT, conf);
+
+      // ClientBuilder only allows one of EndpointConfiguration or Region to be set
+      if (StringUtils.isNotBlank(glueEndpoint)) {
+        logger.info("Setting glue service endpoint to " + glueEndpoint);
+        glueClientBuilder.setEndpointConfiguration(new AwsClientBuilder.EndpointConfiguration(glueEndpoint, null));
+      } else if (StringUtils.isNotBlank(regionStr)) {
+        logger.info("Setting region to : " + regionStr);
+        glueClientBuilder.setRegion(regionStr);
+      } else {
+        Region currentRegion = Regions.getCurrentRegion();
+        if (currentRegion != null) {
+          logger.info("Using region from ec2 metadata : " + currentRegion.getName());
+          glueClientBuilder.setRegion(currentRegion.getName());
+        } else {
+          logger.info("No region info found, using SDK default region: us-east-1");
+        }
+      }
+
+      glueClientBuilder.setClientConfiguration(buildClientConfiguration(conf));
+      return decorateGlueClient(glueClientBuilder.build());
+    } catch (Exception e) {
+      String message = "Unable to build AWSGlueClient: " + e;
+      logger.error(message);
+      throw new MetaException(message);
+    }
+  }
+
+  private AWSGlue decorateGlueClient(AWSGlue originalGlueClient) {
+    if (Strings.isNullOrEmpty(getProperty(AWS_GLUE_CATALOG_SEPARATOR, conf))) {
+      return originalGlueClient;
+    }
+    return new AWSGlueMultipleCatalogDecorator(
+            originalGlueClient,
+            getProperty(AWS_GLUE_CATALOG_SEPARATOR, conf));
+  }
+
+  @VisibleForTesting
+  AWSCredentialsProvider getAWSCredentialsProvider(Configuration conf) {
+
+    Class<? extends AWSCredentialsProviderFactory> providerFactoryClass = conf
+            .getClass(AWS_CATALOG_CREDENTIALS_PROVIDER_FACTORY_CLASS,
+                    DefaultAWSCredentialsProviderFactory.class).asSubclass(
+                    AWSCredentialsProviderFactory.class);
+    AWSCredentialsProviderFactory provider = ReflectionUtils.newInstance(
+            providerFactoryClass, conf);
+    return provider.buildAWSCredentialsProvider(conf);
+  }
+
+  private String createUserAgent() {
+    try {
+      String ugi = UserGroupInformation.getCurrentUser().getUserName();
+      return "ugi=" + ugi;
+    } catch (IOException e) {
+      /*
+       * IOException here means that the login failed according
+       * to UserGroupInformation.getCurrentUser(). In this case,
+       * we will throw a RuntimeException the same way as
+       * HiveMetaStoreClient.java
+       * If not catching IOException, the build will fail with
+       * unreported exception IOExcetion.
+       */
+      logger.error("Unable to resolve current user name " + e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  private ClientConfiguration buildClientConfiguration(Configuration conf) {
+    // Pass UserAgent to client configuration, which enable CloudTrail to audit UGI info
+    // when using Glue Catalog as metastore
+    ClientConfiguration clientConfiguration = new ClientConfiguration()
+            .withUserAgent(createUserAgent())
+            .withMaxErrorRetry(conf.getInt(AWS_GLUE_MAX_RETRY, DEFAULT_MAX_RETRY))
+            .withMaxConnections(conf.getInt(AWS_GLUE_MAX_CONNECTIONS, DEFAULT_MAX_CONNECTIONS))
+            .withConnectionTimeout(conf.getInt(AWS_GLUE_CONNECTION_TIMEOUT, DEFAULT_CONNECTION_TIMEOUT))
+            .withSocketTimeout(conf.getInt(AWS_GLUE_SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT));
+    return clientConfiguration;
+  }
+
+  private static String getProperty(String propertyName, Configuration conf) {
+    return Strings.isNullOrEmpty(System.getProperty(propertyName)) ?
+            conf.get(propertyName) : System.getProperty(propertyName);
+  }
+}
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueDecoratorBase.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueDecoratorBase.java
new file mode 100644
index 0000000000..e707516188
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueDecoratorBase.java
@@ -0,0 +1,1153 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.regions.Region;
+import com.amazonaws.services.glue.AWSGlue;
+import com.amazonaws.services.glue.model.BatchCreatePartitionRequest;
+import com.amazonaws.services.glue.model.BatchCreatePartitionResult;
+import com.amazonaws.services.glue.model.BatchDeleteConnectionRequest;
+import com.amazonaws.services.glue.model.BatchDeleteConnectionResult;
+import com.amazonaws.services.glue.model.BatchDeletePartitionRequest;
+import com.amazonaws.services.glue.model.BatchDeletePartitionResult;
+import com.amazonaws.services.glue.model.BatchDeleteTableRequest;
+import com.amazonaws.services.glue.model.BatchDeleteTableResult;
+import com.amazonaws.services.glue.model.BatchDeleteTableVersionRequest;
+import com.amazonaws.services.glue.model.BatchDeleteTableVersionResult;
+import com.amazonaws.services.glue.model.BatchGetCrawlersRequest;
+import com.amazonaws.services.glue.model.BatchGetCrawlersResult;
+import com.amazonaws.services.glue.model.BatchGetDevEndpointsRequest;
+import com.amazonaws.services.glue.model.BatchGetDevEndpointsResult;
+import com.amazonaws.services.glue.model.BatchGetJobsRequest;
+import com.amazonaws.services.glue.model.BatchGetJobsResult;
+import com.amazonaws.services.glue.model.BatchGetPartitionRequest;
+import com.amazonaws.services.glue.model.BatchGetPartitionResult;
+import com.amazonaws.services.glue.model.BatchGetTriggersRequest;
+import com.amazonaws.services.glue.model.BatchGetTriggersResult;
+import com.amazonaws.services.glue.model.BatchGetWorkflowsRequest;
+import com.amazonaws.services.glue.model.BatchGetWorkflowsResult;
+import com.amazonaws.services.glue.model.BatchStopJobRunRequest;
+import com.amazonaws.services.glue.model.BatchStopJobRunResult;
+import com.amazonaws.services.glue.model.BatchUpdatePartitionRequest;
+import com.amazonaws.services.glue.model.BatchUpdatePartitionResult;
+import com.amazonaws.services.glue.model.CancelMLTaskRunRequest;
+import com.amazonaws.services.glue.model.CancelMLTaskRunResult;
+import com.amazonaws.services.glue.model.CheckSchemaVersionValidityRequest;
+import com.amazonaws.services.glue.model.CheckSchemaVersionValidityResult;
+import com.amazonaws.services.glue.model.CreateClassifierRequest;
+import com.amazonaws.services.glue.model.CreateClassifierResult;
+import com.amazonaws.services.glue.model.CreateConnectionRequest;
+import com.amazonaws.services.glue.model.CreateConnectionResult;
+import com.amazonaws.services.glue.model.CreateCrawlerRequest;
+import com.amazonaws.services.glue.model.CreateCrawlerResult;
+import com.amazonaws.services.glue.model.CreateDatabaseRequest;
+import com.amazonaws.services.glue.model.CreateDatabaseResult;
+import com.amazonaws.services.glue.model.CreateDevEndpointRequest;
+import com.amazonaws.services.glue.model.CreateDevEndpointResult;
+import com.amazonaws.services.glue.model.CreateJobRequest;
+import com.amazonaws.services.glue.model.CreateJobResult;
+import com.amazonaws.services.glue.model.CreateMLTransformRequest;
+import com.amazonaws.services.glue.model.CreateMLTransformResult;
+import com.amazonaws.services.glue.model.CreatePartitionIndexRequest;
+import com.amazonaws.services.glue.model.CreatePartitionIndexResult;
+import com.amazonaws.services.glue.model.CreatePartitionRequest;
+import com.amazonaws.services.glue.model.CreatePartitionResult;
+import com.amazonaws.services.glue.model.CreateRegistryRequest;
+import com.amazonaws.services.glue.model.CreateRegistryResult;
+import com.amazonaws.services.glue.model.CreateSchemaRequest;
+import com.amazonaws.services.glue.model.CreateSchemaResult;
+import com.amazonaws.services.glue.model.CreateScriptRequest;
+import com.amazonaws.services.glue.model.CreateScriptResult;
+import com.amazonaws.services.glue.model.CreateSecurityConfigurationRequest;
+import com.amazonaws.services.glue.model.CreateSecurityConfigurationResult;
+import com.amazonaws.services.glue.model.CreateTableRequest;
+import com.amazonaws.services.glue.model.CreateTableResult;
+import com.amazonaws.services.glue.model.CreateTriggerRequest;
+import com.amazonaws.services.glue.model.CreateTriggerResult;
+import com.amazonaws.services.glue.model.CreateUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.CreateUserDefinedFunctionResult;
+import com.amazonaws.services.glue.model.CreateWorkflowRequest;
+import com.amazonaws.services.glue.model.CreateWorkflowResult;
+import com.amazonaws.services.glue.model.DeleteClassifierRequest;
+import com.amazonaws.services.glue.model.DeleteClassifierResult;
+import com.amazonaws.services.glue.model.DeleteColumnStatisticsForPartitionRequest;
+import com.amazonaws.services.glue.model.DeleteColumnStatisticsForPartitionResult;
+import com.amazonaws.services.glue.model.DeleteColumnStatisticsForTableRequest;
+import com.amazonaws.services.glue.model.DeleteColumnStatisticsForTableResult;
+import com.amazonaws.services.glue.model.DeleteConnectionRequest;
+import com.amazonaws.services.glue.model.DeleteConnectionResult;
+import com.amazonaws.services.glue.model.DeleteCrawlerRequest;
+import com.amazonaws.services.glue.model.DeleteCrawlerResult;
+import com.amazonaws.services.glue.model.DeleteDatabaseRequest;
+import com.amazonaws.services.glue.model.DeleteDatabaseResult;
+import com.amazonaws.services.glue.model.DeleteDevEndpointRequest;
+import com.amazonaws.services.glue.model.DeleteDevEndpointResult;
+import com.amazonaws.services.glue.model.DeleteJobRequest;
+import com.amazonaws.services.glue.model.DeleteJobResult;
+import com.amazonaws.services.glue.model.DeleteMLTransformRequest;
+import com.amazonaws.services.glue.model.DeleteMLTransformResult;
+import com.amazonaws.services.glue.model.DeletePartitionIndexRequest;
+import com.amazonaws.services.glue.model.DeletePartitionIndexResult;
+import com.amazonaws.services.glue.model.DeletePartitionRequest;
+import com.amazonaws.services.glue.model.DeletePartitionResult;
+import com.amazonaws.services.glue.model.DeleteRegistryRequest;
+import com.amazonaws.services.glue.model.DeleteRegistryResult;
+import com.amazonaws.services.glue.model.DeleteResourcePolicyRequest;
+import com.amazonaws.services.glue.model.DeleteResourcePolicyResult;
+import com.amazonaws.services.glue.model.DeleteSchemaRequest;
+import com.amazonaws.services.glue.model.DeleteSchemaResult;
+import com.amazonaws.services.glue.model.DeleteSchemaVersionsRequest;
+import com.amazonaws.services.glue.model.DeleteSchemaVersionsResult;
+import com.amazonaws.services.glue.model.DeleteSecurityConfigurationRequest;
+import com.amazonaws.services.glue.model.DeleteSecurityConfigurationResult;
+import com.amazonaws.services.glue.model.DeleteTableRequest;
+import com.amazonaws.services.glue.model.DeleteTableResult;
+import com.amazonaws.services.glue.model.DeleteTableVersionRequest;
+import com.amazonaws.services.glue.model.DeleteTableVersionResult;
+import com.amazonaws.services.glue.model.DeleteTriggerRequest;
+import com.amazonaws.services.glue.model.DeleteTriggerResult;
+import com.amazonaws.services.glue.model.DeleteUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.DeleteUserDefinedFunctionResult;
+import com.amazonaws.services.glue.model.DeleteWorkflowRequest;
+import com.amazonaws.services.glue.model.DeleteWorkflowResult;
+import com.amazonaws.services.glue.model.GetCatalogImportStatusRequest;
+import com.amazonaws.services.glue.model.GetCatalogImportStatusResult;
+import com.amazonaws.services.glue.model.GetClassifierRequest;
+import com.amazonaws.services.glue.model.GetClassifierResult;
+import com.amazonaws.services.glue.model.GetClassifiersRequest;
+import com.amazonaws.services.glue.model.GetClassifiersResult;
+import com.amazonaws.services.glue.model.GetConnectionRequest;
+import com.amazonaws.services.glue.model.GetColumnStatisticsForPartitionResult;
+import com.amazonaws.services.glue.model.GetColumnStatisticsForPartitionRequest;
+import com.amazonaws.services.glue.model.GetColumnStatisticsForTableResult;
+import com.amazonaws.services.glue.model.GetColumnStatisticsForTableRequest;
+import com.amazonaws.services.glue.model.GetConnectionResult;
+import com.amazonaws.services.glue.model.GetConnectionsRequest;
+import com.amazonaws.services.glue.model.GetConnectionsResult;
+import com.amazonaws.services.glue.model.GetCrawlerMetricsRequest;
+import com.amazonaws.services.glue.model.GetCrawlerMetricsResult;
+import com.amazonaws.services.glue.model.GetCrawlerRequest;
+import com.amazonaws.services.glue.model.GetCrawlerResult;
+import com.amazonaws.services.glue.model.GetCrawlersRequest;
+import com.amazonaws.services.glue.model.GetCrawlersResult;
+import com.amazonaws.services.glue.model.GetDataCatalogEncryptionSettingsRequest;
+import com.amazonaws.services.glue.model.GetDataCatalogEncryptionSettingsResult;
+import com.amazonaws.services.glue.model.GetDatabaseRequest;
+import com.amazonaws.services.glue.model.GetDatabaseResult;
+import com.amazonaws.services.glue.model.GetDatabasesRequest;
+import com.amazonaws.services.glue.model.GetDatabasesResult;
+import com.amazonaws.services.glue.model.GetDataflowGraphRequest;
+import com.amazonaws.services.glue.model.GetDataflowGraphResult;
+import com.amazonaws.services.glue.model.GetDevEndpointRequest;
+import com.amazonaws.services.glue.model.GetDevEndpointResult;
+import com.amazonaws.services.glue.model.GetDevEndpointsRequest;
+import com.amazonaws.services.glue.model.GetDevEndpointsResult;
+import com.amazonaws.services.glue.model.GetJobBookmarkRequest;
+import com.amazonaws.services.glue.model.GetJobBookmarkResult;
+import com.amazonaws.services.glue.model.GetJobRequest;
+import com.amazonaws.services.glue.model.GetJobResult;
+import com.amazonaws.services.glue.model.GetJobRunRequest;
+import com.amazonaws.services.glue.model.GetJobRunResult;
+import com.amazonaws.services.glue.model.GetJobRunsRequest;
+import com.amazonaws.services.glue.model.GetJobRunsResult;
+import com.amazonaws.services.glue.model.GetJobsRequest;
+import com.amazonaws.services.glue.model.GetJobsResult;
+import com.amazonaws.services.glue.model.GetMLTaskRunRequest;
+import com.amazonaws.services.glue.model.GetMLTaskRunResult;
+import com.amazonaws.services.glue.model.GetMLTaskRunsRequest;
+import com.amazonaws.services.glue.model.GetMLTaskRunsResult;
+import com.amazonaws.services.glue.model.GetMLTransformRequest;
+import com.amazonaws.services.glue.model.GetMLTransformResult;
+import com.amazonaws.services.glue.model.GetMLTransformsRequest;
+import com.amazonaws.services.glue.model.GetMLTransformsResult;
+import com.amazonaws.services.glue.model.GetMappingRequest;
+import com.amazonaws.services.glue.model.GetMappingResult;
+import com.amazonaws.services.glue.model.GetPartitionIndexesRequest;
+import com.amazonaws.services.glue.model.GetPartitionIndexesResult;
+import com.amazonaws.services.glue.model.GetPartitionRequest;
+import com.amazonaws.services.glue.model.GetPartitionResult;
+import com.amazonaws.services.glue.model.GetPartitionsRequest;
+import com.amazonaws.services.glue.model.GetPartitionsResult;
+import com.amazonaws.services.glue.model.GetPlanRequest;
+import com.amazonaws.services.glue.model.GetPlanResult;
+import com.amazonaws.services.glue.model.GetRegistryRequest;
+import com.amazonaws.services.glue.model.GetRegistryResult;
+import com.amazonaws.services.glue.model.GetResourcePoliciesRequest;
+import com.amazonaws.services.glue.model.GetResourcePoliciesResult;
+import com.amazonaws.services.glue.model.GetResourcePolicyRequest;
+import com.amazonaws.services.glue.model.GetResourcePolicyResult;
+import com.amazonaws.services.glue.model.GetSchemaByDefinitionRequest;
+import com.amazonaws.services.glue.model.GetSchemaByDefinitionResult;
+import com.amazonaws.services.glue.model.GetSchemaRequest;
+import com.amazonaws.services.glue.model.GetSchemaResult;
+import com.amazonaws.services.glue.model.GetSchemaVersionRequest;
+import com.amazonaws.services.glue.model.GetSchemaVersionResult;
+import com.amazonaws.services.glue.model.GetSchemaVersionsDiffRequest;
+import com.amazonaws.services.glue.model.GetSchemaVersionsDiffResult;
+import com.amazonaws.services.glue.model.GetSecurityConfigurationRequest;
+import com.amazonaws.services.glue.model.GetSecurityConfigurationResult;
+import com.amazonaws.services.glue.model.GetSecurityConfigurationsRequest;
+import com.amazonaws.services.glue.model.GetSecurityConfigurationsResult;
+import com.amazonaws.services.glue.model.GetTableRequest;
+import com.amazonaws.services.glue.model.GetTableResult;
+import com.amazonaws.services.glue.model.GetTableVersionRequest;
+import com.amazonaws.services.glue.model.GetTableVersionResult;
+import com.amazonaws.services.glue.model.GetTableVersionsRequest;
+import com.amazonaws.services.glue.model.GetTableVersionsResult;
+import com.amazonaws.services.glue.model.GetTablesRequest;
+import com.amazonaws.services.glue.model.GetTablesResult;
+import com.amazonaws.services.glue.model.GetTagsRequest;
+import com.amazonaws.services.glue.model.GetTagsResult;
+import com.amazonaws.services.glue.model.GetTriggerRequest;
+import com.amazonaws.services.glue.model.GetTriggerResult;
+import com.amazonaws.services.glue.model.GetTriggersRequest;
+import com.amazonaws.services.glue.model.GetTriggersResult;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionResult;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionsRequest;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionsResult;
+import com.amazonaws.services.glue.model.GetWorkflowRequest;
+import com.amazonaws.services.glue.model.GetWorkflowResult;
+import com.amazonaws.services.glue.model.GetWorkflowRunPropertiesRequest;
+import com.amazonaws.services.glue.model.GetWorkflowRunPropertiesResult;
+import com.amazonaws.services.glue.model.GetWorkflowRunRequest;
+import com.amazonaws.services.glue.model.GetWorkflowRunResult;
+import com.amazonaws.services.glue.model.GetWorkflowRunsRequest;
+import com.amazonaws.services.glue.model.GetWorkflowRunsResult;
+import com.amazonaws.services.glue.model.ImportCatalogToGlueRequest;
+import com.amazonaws.services.glue.model.ImportCatalogToGlueResult;
+import com.amazonaws.services.glue.model.ListCrawlersRequest;
+import com.amazonaws.services.glue.model.ListCrawlersResult;
+import com.amazonaws.services.glue.model.ListDevEndpointsRequest;
+import com.amazonaws.services.glue.model.ListDevEndpointsResult;
+import com.amazonaws.services.glue.model.ListJobsRequest;
+import com.amazonaws.services.glue.model.ListJobsResult;
+import com.amazonaws.services.glue.model.ListMLTransformsRequest;
+import com.amazonaws.services.glue.model.ListMLTransformsResult;
+import com.amazonaws.services.glue.model.ListRegistriesRequest;
+import com.amazonaws.services.glue.model.ListRegistriesResult;
+import com.amazonaws.services.glue.model.ListSchemaVersionsRequest;
+import com.amazonaws.services.glue.model.ListSchemaVersionsResult;
+import com.amazonaws.services.glue.model.ListSchemasRequest;
+import com.amazonaws.services.glue.model.ListSchemasResult;
+import com.amazonaws.services.glue.model.ListTriggersRequest;
+import com.amazonaws.services.glue.model.ListTriggersResult;
+import com.amazonaws.services.glue.model.ListWorkflowsRequest;
+import com.amazonaws.services.glue.model.ListWorkflowsResult;
+import com.amazonaws.services.glue.model.PutDataCatalogEncryptionSettingsRequest;
+import com.amazonaws.services.glue.model.PutDataCatalogEncryptionSettingsResult;
+import com.amazonaws.services.glue.model.PutResourcePolicyRequest;
+import com.amazonaws.services.glue.model.PutResourcePolicyResult;
+import com.amazonaws.services.glue.model.PutSchemaVersionMetadataRequest;
+import com.amazonaws.services.glue.model.PutSchemaVersionMetadataResult;
+import com.amazonaws.services.glue.model.PutWorkflowRunPropertiesRequest;
+import com.amazonaws.services.glue.model.PutWorkflowRunPropertiesResult;
+import com.amazonaws.services.glue.model.QuerySchemaVersionMetadataRequest;
+import com.amazonaws.services.glue.model.QuerySchemaVersionMetadataResult;
+import com.amazonaws.services.glue.model.RegisterSchemaVersionRequest;
+import com.amazonaws.services.glue.model.RegisterSchemaVersionResult;
+import com.amazonaws.services.glue.model.RemoveSchemaVersionMetadataRequest;
+import com.amazonaws.services.glue.model.RemoveSchemaVersionMetadataResult;
+import com.amazonaws.services.glue.model.ResetJobBookmarkRequest;
+import com.amazonaws.services.glue.model.ResetJobBookmarkResult;
+import com.amazonaws.services.glue.model.ResumeWorkflowRunRequest;
+import com.amazonaws.services.glue.model.ResumeWorkflowRunResult;
+import com.amazonaws.services.glue.model.SearchTablesRequest;
+import com.amazonaws.services.glue.model.SearchTablesResult;
+import com.amazonaws.services.glue.model.StartCrawlerRequest;
+import com.amazonaws.services.glue.model.StartCrawlerResult;
+import com.amazonaws.services.glue.model.StartCrawlerScheduleRequest;
+import com.amazonaws.services.glue.model.StartCrawlerScheduleResult;
+import com.amazonaws.services.glue.model.StartExportLabelsTaskRunRequest;
+import com.amazonaws.services.glue.model.StartExportLabelsTaskRunResult;
+import com.amazonaws.services.glue.model.StartImportLabelsTaskRunRequest;
+import com.amazonaws.services.glue.model.StartImportLabelsTaskRunResult;
+import com.amazonaws.services.glue.model.StartJobRunRequest;
+import com.amazonaws.services.glue.model.StartJobRunResult;
+import com.amazonaws.services.glue.model.StartMLEvaluationTaskRunRequest;
+import com.amazonaws.services.glue.model.StartMLEvaluationTaskRunResult;
+import com.amazonaws.services.glue.model.StartMLLabelingSetGenerationTaskRunRequest;
+import com.amazonaws.services.glue.model.StartMLLabelingSetGenerationTaskRunResult;
+import com.amazonaws.services.glue.model.StartTriggerRequest;
+import com.amazonaws.services.glue.model.StartTriggerResult;
+import com.amazonaws.services.glue.model.StartWorkflowRunRequest;
+import com.amazonaws.services.glue.model.StartWorkflowRunResult;
+import com.amazonaws.services.glue.model.StopCrawlerRequest;
+import com.amazonaws.services.glue.model.StopCrawlerResult;
+import com.amazonaws.services.glue.model.StopCrawlerScheduleRequest;
+import com.amazonaws.services.glue.model.StopCrawlerScheduleResult;
+import com.amazonaws.services.glue.model.StopTriggerRequest;
+import com.amazonaws.services.glue.model.StopTriggerResult;
+import com.amazonaws.services.glue.model.StopWorkflowRunRequest;
+import com.amazonaws.services.glue.model.StopWorkflowRunResult;
+import com.amazonaws.services.glue.model.TagResourceRequest;
+import com.amazonaws.services.glue.model.TagResourceResult;
+import com.amazonaws.services.glue.model.UntagResourceRequest;
+import com.amazonaws.services.glue.model.UntagResourceResult;
+import com.amazonaws.services.glue.model.UpdateClassifierRequest;
+import com.amazonaws.services.glue.model.UpdateClassifierResult;
+import com.amazonaws.services.glue.model.UpdateColumnStatisticsForPartitionRequest;
+import com.amazonaws.services.glue.model.UpdateColumnStatisticsForPartitionResult;
+import com.amazonaws.services.glue.model.UpdateColumnStatisticsForTableRequest;
+import com.amazonaws.services.glue.model.UpdateColumnStatisticsForTableResult;
+import com.amazonaws.services.glue.model.UpdateConnectionRequest;
+import com.amazonaws.services.glue.model.UpdateConnectionResult;
+import com.amazonaws.services.glue.model.UpdateCrawlerRequest;
+import com.amazonaws.services.glue.model.UpdateCrawlerResult;
+import com.amazonaws.services.glue.model.UpdateCrawlerScheduleRequest;
+import com.amazonaws.services.glue.model.UpdateCrawlerScheduleResult;
+import com.amazonaws.services.glue.model.UpdateDatabaseRequest;
+import com.amazonaws.services.glue.model.UpdateDatabaseResult;
+import com.amazonaws.services.glue.model.UpdateDevEndpointRequest;
+import com.amazonaws.services.glue.model.UpdateDevEndpointResult;
+import com.amazonaws.services.glue.model.UpdateJobRequest;
+import com.amazonaws.services.glue.model.UpdateJobResult;
+import com.amazonaws.services.glue.model.UpdateMLTransformRequest;
+import com.amazonaws.services.glue.model.UpdateMLTransformResult;
+import com.amazonaws.services.glue.model.UpdatePartitionRequest;
+import com.amazonaws.services.glue.model.UpdatePartitionResult;
+import com.amazonaws.services.glue.model.UpdateRegistryRequest;
+import com.amazonaws.services.glue.model.UpdateRegistryResult;
+import com.amazonaws.services.glue.model.UpdateSchemaRequest;
+import com.amazonaws.services.glue.model.UpdateSchemaResult;
+import com.amazonaws.services.glue.model.UpdateTableRequest;
+import com.amazonaws.services.glue.model.UpdateTableResult;
+import com.amazonaws.services.glue.model.UpdateTriggerRequest;
+import com.amazonaws.services.glue.model.UpdateTriggerResult;
+import com.amazonaws.services.glue.model.UpdateUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.UpdateUserDefinedFunctionResult;
+import com.amazonaws.services.glue.model.UpdateWorkflowRequest;
+import com.amazonaws.services.glue.model.UpdateWorkflowResult;
+
+/**
+ * Base decorator for AWSGlue interface. It doesn't decorate any functionality but just proxy all methods to
+ * decoratedAwsGlue. It should be used as a parent for specific decorators where only necessary methods are overwritten
+ * and decorated.
+ * All @Override methods are generated by IntelliJ IDEA.
+ */
+public class AWSGlueDecoratorBase implements AWSGlue {
+    private AWSGlue decoratedAwsGlue;
+
+    public AWSGlueDecoratorBase(AWSGlue awsGlueToBeDecorated) {
+        this.decoratedAwsGlue = awsGlueToBeDecorated;
+    }
+
+    @Override
+    public BatchCreatePartitionResult batchCreatePartition(BatchCreatePartitionRequest batchCreatePartitionRequest) {
+        return decoratedAwsGlue.batchCreatePartition(batchCreatePartitionRequest);
+    }
+
+    @Override
+    public BatchDeleteConnectionResult batchDeleteConnection(BatchDeleteConnectionRequest batchDeleteConnectionRequest) {
+        return decoratedAwsGlue.batchDeleteConnection(batchDeleteConnectionRequest);
+    }
+
+    @Override
+    public BatchDeletePartitionResult batchDeletePartition(BatchDeletePartitionRequest batchDeletePartitionRequest) {
+        return decoratedAwsGlue.batchDeletePartition(batchDeletePartitionRequest);
+    }
+
+    @Override
+    public BatchDeleteTableResult batchDeleteTable(BatchDeleteTableRequest batchDeleteTableRequest) {
+        return decoratedAwsGlue.batchDeleteTable(batchDeleteTableRequest);
+    }
+
+    @Override
+    public BatchDeleteTableVersionResult batchDeleteTableVersion(BatchDeleteTableVersionRequest batchDeleteTableVersionRequest) {
+        return decoratedAwsGlue.batchDeleteTableVersion(batchDeleteTableVersionRequest);
+    }
+
+    @Override
+    public BatchGetCrawlersResult batchGetCrawlers(BatchGetCrawlersRequest batchGetCrawlersRequest) {
+        return decoratedAwsGlue.batchGetCrawlers(batchGetCrawlersRequest);
+    }
+
+    @Override
+    public BatchGetDevEndpointsResult batchGetDevEndpoints(BatchGetDevEndpointsRequest batchGetDevEndpointsRequest) {
+        return decoratedAwsGlue.batchGetDevEndpoints(batchGetDevEndpointsRequest);
+    }
+
+    @Override
+    public BatchGetJobsResult batchGetJobs(BatchGetJobsRequest batchGetJobsRequest) {
+        return decoratedAwsGlue.batchGetJobs(batchGetJobsRequest);
+    }
+
+    @Override
+    public BatchGetPartitionResult batchGetPartition(BatchGetPartitionRequest batchGetPartitionRequest) {
+        return decoratedAwsGlue.batchGetPartition(batchGetPartitionRequest);
+    }
+
+    @Override
+    public BatchGetTriggersResult batchGetTriggers(BatchGetTriggersRequest batchGetTriggersRequest) {
+        return decoratedAwsGlue.batchGetTriggers(batchGetTriggersRequest);
+    }
+
+    @Override
+    public BatchGetWorkflowsResult batchGetWorkflows(BatchGetWorkflowsRequest batchGetWorkflowsRequest) {
+        return decoratedAwsGlue.batchGetWorkflows(batchGetWorkflowsRequest);
+    }
+
+    @Override
+    public BatchStopJobRunResult batchStopJobRun(BatchStopJobRunRequest batchStopJobRunRequest) {
+        return decoratedAwsGlue.batchStopJobRun(batchStopJobRunRequest);
+    }
+
+    @Override
+    public BatchUpdatePartitionResult batchUpdatePartition(BatchUpdatePartitionRequest batchUpdatePartitionRequest) {
+        return decoratedAwsGlue.batchUpdatePartition(batchUpdatePartitionRequest);
+    }
+
+    @Override
+    public CancelMLTaskRunResult cancelMLTaskRun(CancelMLTaskRunRequest cancelMLTaskRunRequest) {
+        return decoratedAwsGlue.cancelMLTaskRun(cancelMLTaskRunRequest);
+    }
+
+    @Override
+    public CheckSchemaVersionValidityResult checkSchemaVersionValidity(CheckSchemaVersionValidityRequest checkSchemaVersionValidityRequest) {
+        return null;
+    }
+
+    @Override
+    public CreateClassifierResult createClassifier(CreateClassifierRequest createClassifierRequest) {
+        return decoratedAwsGlue.createClassifier(createClassifierRequest);
+    }
+
+    @Override
+    public CreateConnectionResult createConnection(CreateConnectionRequest createConnectionRequest) {
+        return decoratedAwsGlue.createConnection(createConnectionRequest);
+    }
+
+    @Override
+    public CreateCrawlerResult createCrawler(CreateCrawlerRequest createCrawlerRequest) {
+        return decoratedAwsGlue.createCrawler(createCrawlerRequest);
+    }
+
+    @Override
+    public CreateDatabaseResult createDatabase(CreateDatabaseRequest createDatabaseRequest) {
+        return decoratedAwsGlue.createDatabase(createDatabaseRequest);
+    }
+
+    @Override
+    public CreateDevEndpointResult createDevEndpoint(CreateDevEndpointRequest createDevEndpointRequest) {
+        return decoratedAwsGlue.createDevEndpoint(createDevEndpointRequest);
+    }
+
+    @Override
+    public CreateJobResult createJob(CreateJobRequest createJobRequest) {
+        return decoratedAwsGlue.createJob(createJobRequest);
+    }
+
+    @Override
+    public CreateMLTransformResult createMLTransform(CreateMLTransformRequest createMLTransformRequest) {
+        return decoratedAwsGlue.createMLTransform(createMLTransformRequest);
+    }
+
+    @Override
+    public CreatePartitionResult createPartition(CreatePartitionRequest createPartitionRequest) {
+        return decoratedAwsGlue.createPartition(createPartitionRequest);
+    }
+
+    @Override
+    public CreatePartitionIndexResult createPartitionIndex(CreatePartitionIndexRequest createPartitionIndexRequest) {
+        return null;
+    }
+
+    @Override
+    public CreateRegistryResult createRegistry(CreateRegistryRequest createRegistryRequest) {
+        return null;
+    }
+
+    @Override
+    public CreateSchemaResult createSchema(CreateSchemaRequest createSchemaRequest) {
+        return null;
+    }
+
+    @Override
+    public CreateScriptResult createScript(CreateScriptRequest createScriptRequest) {
+        return decoratedAwsGlue.createScript(createScriptRequest);
+    }
+
+    @Override
+    public CreateSecurityConfigurationResult createSecurityConfiguration(CreateSecurityConfigurationRequest createSecurityConfigurationRequest) {
+        return decoratedAwsGlue.createSecurityConfiguration(createSecurityConfigurationRequest);
+    }
+
+    @Override
+    public CreateTableResult createTable(CreateTableRequest createTableRequest) {
+        return decoratedAwsGlue.createTable(createTableRequest);
+    }
+
+    @Override
+    public CreateTriggerResult createTrigger(CreateTriggerRequest createTriggerRequest) {
+        return decoratedAwsGlue.createTrigger(createTriggerRequest);
+    }
+
+    @Override
+    public CreateUserDefinedFunctionResult createUserDefinedFunction(CreateUserDefinedFunctionRequest createUserDefinedFunctionRequest) {
+        return decoratedAwsGlue.createUserDefinedFunction(createUserDefinedFunctionRequest);
+    }
+
+    @Override
+    public CreateWorkflowResult createWorkflow(CreateWorkflowRequest createWorkflowRequest) {
+        return decoratedAwsGlue.createWorkflow(createWorkflowRequest);
+    }
+
+    @Override
+    public DeleteClassifierResult deleteClassifier(DeleteClassifierRequest deleteClassifierRequest) {
+        return decoratedAwsGlue.deleteClassifier(deleteClassifierRequest);
+    }
+
+    @Override
+    public DeleteConnectionResult deleteConnection(DeleteConnectionRequest deleteConnectionRequest) {
+        return decoratedAwsGlue.deleteConnection(deleteConnectionRequest);
+    }
+
+    @Override
+    public DeleteCrawlerResult deleteCrawler(DeleteCrawlerRequest deleteCrawlerRequest) {
+        return decoratedAwsGlue.deleteCrawler(deleteCrawlerRequest);
+    }
+
+    @Override
+    public DeleteDatabaseResult deleteDatabase(DeleteDatabaseRequest deleteDatabaseRequest) {
+        return decoratedAwsGlue.deleteDatabase(deleteDatabaseRequest);
+    }
+
+    @Override
+    public DeleteDevEndpointResult deleteDevEndpoint(DeleteDevEndpointRequest deleteDevEndpointRequest) {
+        return decoratedAwsGlue.deleteDevEndpoint(deleteDevEndpointRequest);
+    }
+
+    @Override
+    public DeleteJobResult deleteJob(DeleteJobRequest deleteJobRequest) {
+        return decoratedAwsGlue.deleteJob(deleteJobRequest);
+    }
+
+    @Override
+    public DeleteMLTransformResult deleteMLTransform(DeleteMLTransformRequest deleteMLTransformRequest) {
+        return decoratedAwsGlue.deleteMLTransform(deleteMLTransformRequest);
+    }
+
+    @Override
+    public DeletePartitionResult deletePartition(DeletePartitionRequest deletePartitionRequest) {
+        return decoratedAwsGlue.deletePartition(deletePartitionRequest);
+    }
+
+    @Override
+    public DeletePartitionIndexResult deletePartitionIndex(DeletePartitionIndexRequest deletePartitionIndexRequest) {
+        return null;
+    }
+
+    @Override
+    public DeleteRegistryResult deleteRegistry(DeleteRegistryRequest deleteRegistryRequest) {
+        return null;
+    }
+
+    @Override
+    public DeleteResourcePolicyResult deleteResourcePolicy(DeleteResourcePolicyRequest deleteResourcePolicyRequest) {
+        return decoratedAwsGlue.deleteResourcePolicy(deleteResourcePolicyRequest);
+    }
+
+    @Override
+    public DeleteSchemaResult deleteSchema(DeleteSchemaRequest deleteSchemaRequest) {
+        return null;
+    }
+
+    @Override
+    public DeleteSchemaVersionsResult deleteSchemaVersions(DeleteSchemaVersionsRequest deleteSchemaVersionsRequest) {
+        return null;
+    }
+
+    @Override
+    public DeleteSecurityConfigurationResult deleteSecurityConfiguration(DeleteSecurityConfigurationRequest deleteSecurityConfigurationRequest) {
+        return decoratedAwsGlue.deleteSecurityConfiguration(deleteSecurityConfigurationRequest);
+    }
+
+    @Override
+    public DeleteTableResult deleteTable(DeleteTableRequest deleteTableRequest) {
+        return decoratedAwsGlue.deleteTable(deleteTableRequest);
+    }
+
+    @Override
+    public DeleteTableVersionResult deleteTableVersion(DeleteTableVersionRequest deleteTableVersionRequest) {
+        return decoratedAwsGlue.deleteTableVersion(deleteTableVersionRequest);
+    }
+
+    @Override
+    public DeleteTriggerResult deleteTrigger(DeleteTriggerRequest deleteTriggerRequest) {
+        return decoratedAwsGlue.deleteTrigger(deleteTriggerRequest);
+    }
+
+    @Override
+    public DeleteUserDefinedFunctionResult deleteUserDefinedFunction(DeleteUserDefinedFunctionRequest deleteUserDefinedFunctionRequest) {
+        return decoratedAwsGlue.deleteUserDefinedFunction(deleteUserDefinedFunctionRequest);
+    }
+
+    @Override
+    public DeleteWorkflowResult deleteWorkflow(DeleteWorkflowRequest deleteWorkflowRequest) {
+        return decoratedAwsGlue.deleteWorkflow(deleteWorkflowRequest);
+    }
+
+    @Override
+    public GetCatalogImportStatusResult getCatalogImportStatus(GetCatalogImportStatusRequest getCatalogImportStatusRequest) {
+        return decoratedAwsGlue.getCatalogImportStatus(getCatalogImportStatusRequest);
+    }
+
+    @Override
+    public GetClassifierResult getClassifier(GetClassifierRequest getClassifierRequest) {
+        return decoratedAwsGlue.getClassifier(getClassifierRequest);
+    }
+
+    @Override
+    public GetClassifiersResult getClassifiers(GetClassifiersRequest getClassifiersRequest) {
+        return decoratedAwsGlue.getClassifiers(getClassifiersRequest);
+    }
+
+    @Override
+    public GetConnectionResult getConnection(GetConnectionRequest getConnectionRequest) {
+        return decoratedAwsGlue.getConnection(getConnectionRequest);
+    }
+
+    @Override
+    public GetConnectionsResult getConnections(GetConnectionsRequest getConnectionsRequest) {
+        return decoratedAwsGlue.getConnections(getConnectionsRequest);
+    }
+
+    @Override
+    public GetCrawlerResult getCrawler(GetCrawlerRequest getCrawlerRequest) {
+        return decoratedAwsGlue.getCrawler(getCrawlerRequest);
+    }
+
+    @Override
+    public GetCrawlerMetricsResult getCrawlerMetrics(GetCrawlerMetricsRequest getCrawlerMetricsRequest) {
+        return decoratedAwsGlue.getCrawlerMetrics(getCrawlerMetricsRequest);
+    }
+
+    @Override
+    public GetCrawlersResult getCrawlers(GetCrawlersRequest getCrawlersRequest) {
+        return decoratedAwsGlue.getCrawlers(getCrawlersRequest);
+    }
+
+    @Override
+    public GetDataCatalogEncryptionSettingsResult getDataCatalogEncryptionSettings(GetDataCatalogEncryptionSettingsRequest getDataCatalogEncryptionSettingsRequest) {
+        return decoratedAwsGlue.getDataCatalogEncryptionSettings(getDataCatalogEncryptionSettingsRequest);
+    }
+
+    @Override
+    public GetDatabaseResult getDatabase(GetDatabaseRequest getDatabaseRequest) {
+        return decoratedAwsGlue.getDatabase(getDatabaseRequest);
+    }
+
+    @Override
+    public GetDatabasesResult getDatabases(GetDatabasesRequest getDatabasesRequest) {
+        return decoratedAwsGlue.getDatabases(getDatabasesRequest);
+    }
+
+    @Override
+    public GetDataflowGraphResult getDataflowGraph(GetDataflowGraphRequest getDataflowGraphRequest) {
+        return decoratedAwsGlue.getDataflowGraph(getDataflowGraphRequest);
+    }
+
+    @Override
+    public GetDevEndpointResult getDevEndpoint(GetDevEndpointRequest getDevEndpointRequest) {
+        return decoratedAwsGlue.getDevEndpoint(getDevEndpointRequest);
+    }
+
+    @Override
+    public GetDevEndpointsResult getDevEndpoints(GetDevEndpointsRequest getDevEndpointsRequest) {
+        return decoratedAwsGlue.getDevEndpoints(getDevEndpointsRequest);
+    }
+
+    @Override
+    public GetJobResult getJob(GetJobRequest getJobRequest) {
+        return decoratedAwsGlue.getJob(getJobRequest);
+    }
+
+    @Override
+    public GetJobBookmarkResult getJobBookmark(GetJobBookmarkRequest getJobBookmarkRequest) {
+        return decoratedAwsGlue.getJobBookmark(getJobBookmarkRequest);
+    }
+
+    @Override
+    public GetJobRunResult getJobRun(GetJobRunRequest getJobRunRequest) {
+        return decoratedAwsGlue.getJobRun(getJobRunRequest);
+    }
+
+    @Override
+    public GetJobRunsResult getJobRuns(GetJobRunsRequest getJobRunsRequest) {
+        return decoratedAwsGlue.getJobRuns(getJobRunsRequest);
+    }
+
+    @Override
+    public GetJobsResult getJobs(GetJobsRequest getJobsRequest) {
+        return decoratedAwsGlue.getJobs(getJobsRequest);
+    }
+
+    @Override
+    public GetMLTaskRunResult getMLTaskRun(GetMLTaskRunRequest getMLTaskRunRequest) {
+        return decoratedAwsGlue.getMLTaskRun(getMLTaskRunRequest);
+    }
+
+    @Override
+    public GetMLTaskRunsResult getMLTaskRuns(GetMLTaskRunsRequest getMLTaskRunsRequest) {
+        return decoratedAwsGlue.getMLTaskRuns(getMLTaskRunsRequest);
+    }
+
+    @Override
+    public GetMLTransformResult getMLTransform(GetMLTransformRequest getMLTransformRequest) {
+        return decoratedAwsGlue.getMLTransform(getMLTransformRequest);
+    }
+
+    @Override
+    public GetMLTransformsResult getMLTransforms(GetMLTransformsRequest getMLTransformsRequest) {
+        return decoratedAwsGlue.getMLTransforms(getMLTransformsRequest);
+    }
+
+    @Override
+    public GetMappingResult getMapping(GetMappingRequest getMappingRequest) {
+        return decoratedAwsGlue.getMapping(getMappingRequest);
+    }
+
+    @Override
+    public GetPartitionIndexesResult getPartitionIndexes(GetPartitionIndexesRequest getPartitionIndexesRequest) {
+        return decoratedAwsGlue.getPartitionIndexes(getPartitionIndexesRequest);
+    }
+
+    @Override
+    public GetPartitionResult getPartition(GetPartitionRequest getPartitionRequest) {
+        return decoratedAwsGlue.getPartition(getPartitionRequest);
+    }
+
+    @Override
+    public GetPartitionsResult getPartitions(GetPartitionsRequest getPartitionsRequest) {
+        return decoratedAwsGlue.getPartitions(getPartitionsRequest);
+    }
+
+    @Override
+    public GetPlanResult getPlan(GetPlanRequest getPlanRequest) {
+        return decoratedAwsGlue.getPlan(getPlanRequest);
+    }
+
+    @Override
+    public GetRegistryResult getRegistry(GetRegistryRequest getRegistryRequest) {
+        return null;
+    }
+
+    @Override
+    public GetResourcePolicyResult getResourcePolicy(GetResourcePolicyRequest getResourcePolicyRequest) {
+        return decoratedAwsGlue.getResourcePolicy(getResourcePolicyRequest);
+    }
+
+    @Override
+    public GetSchemaResult getSchema(GetSchemaRequest getSchemaRequest) {
+        return null;
+    }
+
+    @Override
+    public GetSchemaByDefinitionResult getSchemaByDefinition(GetSchemaByDefinitionRequest getSchemaByDefinitionRequest) {
+        return null;
+    }
+
+    @Override
+    public GetSchemaVersionResult getSchemaVersion(GetSchemaVersionRequest getSchemaVersionRequest) {
+        return null;
+    }
+
+    @Override
+    public GetSchemaVersionsDiffResult getSchemaVersionsDiff(GetSchemaVersionsDiffRequest getSchemaVersionsDiffRequest) {
+        return null;
+    }
+
+    @Override
+    public GetSecurityConfigurationResult getSecurityConfiguration(GetSecurityConfigurationRequest getSecurityConfigurationRequest) {
+        return decoratedAwsGlue.getSecurityConfiguration(getSecurityConfigurationRequest);
+    }
+
+    @Override
+    public GetSecurityConfigurationsResult getSecurityConfigurations(GetSecurityConfigurationsRequest getSecurityConfigurationsRequest) {
+        return decoratedAwsGlue.getSecurityConfigurations(getSecurityConfigurationsRequest);
+    }
+
+    @Override
+    public GetTableResult getTable(GetTableRequest getTableRequest) {
+        return decoratedAwsGlue.getTable(getTableRequest);
+    }
+
+    @Override
+    public GetTableVersionResult getTableVersion(GetTableVersionRequest getTableVersionRequest) {
+        return decoratedAwsGlue.getTableVersion(getTableVersionRequest);
+    }
+
+    @Override
+    public GetTableVersionsResult getTableVersions(GetTableVersionsRequest getTableVersionsRequest) {
+        return decoratedAwsGlue.getTableVersions(getTableVersionsRequest);
+    }
+
+    @Override
+    public GetTablesResult getTables(GetTablesRequest getTablesRequest) {
+        return decoratedAwsGlue.getTables(getTablesRequest);
+    }
+
+    @Override
+    public GetTagsResult getTags(GetTagsRequest getTagsRequest) {
+        return decoratedAwsGlue.getTags(getTagsRequest);
+    }
+
+    @Override
+    public GetTriggerResult getTrigger(GetTriggerRequest getTriggerRequest) {
+        return decoratedAwsGlue.getTrigger(getTriggerRequest);
+    }
+
+    @Override
+    public GetTriggersResult getTriggers(GetTriggersRequest getTriggersRequest) {
+        return decoratedAwsGlue.getTriggers(getTriggersRequest);
+    }
+
+    @Override
+    public GetUserDefinedFunctionResult getUserDefinedFunction(GetUserDefinedFunctionRequest getUserDefinedFunctionRequest) {
+        return decoratedAwsGlue.getUserDefinedFunction(getUserDefinedFunctionRequest);
+    }
+
+    @Override
+    public GetUserDefinedFunctionsResult getUserDefinedFunctions(GetUserDefinedFunctionsRequest getUserDefinedFunctionsRequest) {
+        return decoratedAwsGlue.getUserDefinedFunctions(getUserDefinedFunctionsRequest);
+    }
+
+    @Override
+    public GetWorkflowResult getWorkflow(GetWorkflowRequest getWorkflowRequest) {
+        return decoratedAwsGlue.getWorkflow(getWorkflowRequest);
+    }
+
+    @Override
+    public GetWorkflowRunResult getWorkflowRun(GetWorkflowRunRequest getWorkflowRunRequest) {
+        return decoratedAwsGlue.getWorkflowRun(getWorkflowRunRequest);
+    }
+
+    @Override
+    public GetWorkflowRunPropertiesResult getWorkflowRunProperties(GetWorkflowRunPropertiesRequest getWorkflowRunPropertiesRequest) {
+        return decoratedAwsGlue.getWorkflowRunProperties(getWorkflowRunPropertiesRequest);
+    }
+
+    @Override
+    public GetWorkflowRunsResult getWorkflowRuns(GetWorkflowRunsRequest getWorkflowRunsRequest) {
+        return decoratedAwsGlue.getWorkflowRuns(getWorkflowRunsRequest);
+    }
+
+    @Override
+    public ImportCatalogToGlueResult importCatalogToGlue(ImportCatalogToGlueRequest importCatalogToGlueRequest) {
+        return decoratedAwsGlue.importCatalogToGlue(importCatalogToGlueRequest);
+    }
+
+    @Override
+    public ListCrawlersResult listCrawlers(ListCrawlersRequest listCrawlersRequest) {
+        return decoratedAwsGlue.listCrawlers(listCrawlersRequest);
+    }
+
+    @Override
+    public ListDevEndpointsResult listDevEndpoints(ListDevEndpointsRequest listDevEndpointsRequest) {
+        return decoratedAwsGlue.listDevEndpoints(listDevEndpointsRequest);
+    }
+
+    @Override
+    public ListJobsResult listJobs(ListJobsRequest listJobsRequest) {
+        return decoratedAwsGlue.listJobs(listJobsRequest);
+    }
+
+    @Override
+    public ListMLTransformsResult listMLTransforms(ListMLTransformsRequest listMLTransformsRequest) {
+        return decoratedAwsGlue.listMLTransforms(listMLTransformsRequest);
+    }
+
+    @Override
+    public ListRegistriesResult listRegistries(ListRegistriesRequest listRegistriesRequest) {
+        return null;
+    }
+
+    @Override
+    public ListSchemaVersionsResult listSchemaVersions(ListSchemaVersionsRequest listSchemaVersionsRequest) {
+        return null;
+    }
+
+    @Override
+    public ListSchemasResult listSchemas(ListSchemasRequest listSchemasRequest) {
+        return null;
+    }
+
+    @Override
+    public ListTriggersResult listTriggers(ListTriggersRequest listTriggersRequest) {
+        return decoratedAwsGlue.listTriggers(listTriggersRequest);
+    }
+
+    @Override
+    public ListWorkflowsResult listWorkflows(ListWorkflowsRequest listWorkflowsRequest) {
+        return decoratedAwsGlue.listWorkflows(listWorkflowsRequest);
+    }
+
+    @Override
+    public PutDataCatalogEncryptionSettingsResult putDataCatalogEncryptionSettings(PutDataCatalogEncryptionSettingsRequest putDataCatalogEncryptionSettingsRequest) {
+        return decoratedAwsGlue.putDataCatalogEncryptionSettings(putDataCatalogEncryptionSettingsRequest);
+    }
+
+    @Override
+    public PutResourcePolicyResult putResourcePolicy(PutResourcePolicyRequest putResourcePolicyRequest) {
+        return decoratedAwsGlue.putResourcePolicy(putResourcePolicyRequest);
+    }
+
+    @Override
+    public PutSchemaVersionMetadataResult putSchemaVersionMetadata(PutSchemaVersionMetadataRequest putSchemaVersionMetadataRequest) {
+        return null;
+    }
+
+    @Override
+    public PutWorkflowRunPropertiesResult putWorkflowRunProperties(PutWorkflowRunPropertiesRequest putWorkflowRunPropertiesRequest) {
+        return decoratedAwsGlue.putWorkflowRunProperties(putWorkflowRunPropertiesRequest);
+    }
+
+    @Override
+    public QuerySchemaVersionMetadataResult querySchemaVersionMetadata(QuerySchemaVersionMetadataRequest querySchemaVersionMetadataRequest) {
+        return null;
+    }
+
+    @Override
+    public RegisterSchemaVersionResult registerSchemaVersion(RegisterSchemaVersionRequest registerSchemaVersionRequest) {
+        return null;
+    }
+
+    @Override
+    public RemoveSchemaVersionMetadataResult removeSchemaVersionMetadata(RemoveSchemaVersionMetadataRequest removeSchemaVersionMetadataRequest) {
+        return null;
+    }
+
+    @Override
+    public ResetJobBookmarkResult resetJobBookmark(ResetJobBookmarkRequest resetJobBookmarkRequest) {
+        return decoratedAwsGlue.resetJobBookmark(resetJobBookmarkRequest);
+    }
+
+    @Override
+    public SearchTablesResult searchTables(SearchTablesRequest searchTablesRequest) {
+        return decoratedAwsGlue.searchTables(searchTablesRequest);
+    }
+
+    @Override
+    public StartCrawlerResult startCrawler(StartCrawlerRequest startCrawlerRequest) {
+        return decoratedAwsGlue.startCrawler(startCrawlerRequest);
+    }
+
+    @Override
+    public StartCrawlerScheduleResult startCrawlerSchedule(StartCrawlerScheduleRequest startCrawlerScheduleRequest) {
+        return decoratedAwsGlue.startCrawlerSchedule(startCrawlerScheduleRequest);
+    }
+
+    @Override
+    public StartExportLabelsTaskRunResult startExportLabelsTaskRun(StartExportLabelsTaskRunRequest startExportLabelsTaskRunRequest) {
+        return decoratedAwsGlue.startExportLabelsTaskRun(startExportLabelsTaskRunRequest);
+    }
+
+    @Override
+    public StartImportLabelsTaskRunResult startImportLabelsTaskRun(StartImportLabelsTaskRunRequest startImportLabelsTaskRunRequest) {
+        return decoratedAwsGlue.startImportLabelsTaskRun(startImportLabelsTaskRunRequest);
+    }
+
+    @Override
+    public StartJobRunResult startJobRun(StartJobRunRequest startJobRunRequest) {
+        return decoratedAwsGlue.startJobRun(startJobRunRequest);
+    }
+
+    @Override
+    public StartMLEvaluationTaskRunResult startMLEvaluationTaskRun(StartMLEvaluationTaskRunRequest startMLEvaluationTaskRunRequest) {
+        return decoratedAwsGlue.startMLEvaluationTaskRun(startMLEvaluationTaskRunRequest);
+    }
+
+    @Override
+    public StartMLLabelingSetGenerationTaskRunResult startMLLabelingSetGenerationTaskRun(StartMLLabelingSetGenerationTaskRunRequest startMLLabelingSetGenerationTaskRunRequest) {
+        return decoratedAwsGlue.startMLLabelingSetGenerationTaskRun(startMLLabelingSetGenerationTaskRunRequest);
+    }
+
+    @Override
+    public StartTriggerResult startTrigger(StartTriggerRequest startTriggerRequest) {
+        return decoratedAwsGlue.startTrigger(startTriggerRequest);
+    }
+
+    @Override
+    public StartWorkflowRunResult startWorkflowRun(StartWorkflowRunRequest startWorkflowRunRequest) {
+        return decoratedAwsGlue.startWorkflowRun(startWorkflowRunRequest);
+    }
+
+    @Override
+    public StopCrawlerResult stopCrawler(StopCrawlerRequest stopCrawlerRequest) {
+        return decoratedAwsGlue.stopCrawler(stopCrawlerRequest);
+    }
+
+    @Override
+    public StopCrawlerScheduleResult stopCrawlerSchedule(StopCrawlerScheduleRequest stopCrawlerScheduleRequest) {
+        return decoratedAwsGlue.stopCrawlerSchedule(stopCrawlerScheduleRequest);
+    }
+
+    @Override
+    public StopTriggerResult stopTrigger(StopTriggerRequest stopTriggerRequest) {
+        return decoratedAwsGlue.stopTrigger(stopTriggerRequest);
+    }
+
+    @Override
+    public StopWorkflowRunResult stopWorkflowRun(StopWorkflowRunRequest stopWorkflowRunRequest) {
+        return decoratedAwsGlue.stopWorkflowRun(stopWorkflowRunRequest);
+    }
+
+    @Override
+    public TagResourceResult tagResource(TagResourceRequest tagResourceRequest) {
+        return decoratedAwsGlue.tagResource(tagResourceRequest);
+    }
+
+    @Override
+    public UntagResourceResult untagResource(UntagResourceRequest untagResourceRequest) {
+        return decoratedAwsGlue.untagResource(untagResourceRequest);
+    }
+
+    @Override
+    public UpdateClassifierResult updateClassifier(UpdateClassifierRequest updateClassifierRequest) {
+        return decoratedAwsGlue.updateClassifier(updateClassifierRequest);
+    }
+
+    @Override
+    public UpdateConnectionResult updateConnection(UpdateConnectionRequest updateConnectionRequest) {
+        return decoratedAwsGlue.updateConnection(updateConnectionRequest);
+    }
+
+    @Override
+    public UpdateCrawlerResult updateCrawler(UpdateCrawlerRequest updateCrawlerRequest) {
+        return decoratedAwsGlue.updateCrawler(updateCrawlerRequest);
+    }
+
+    @Override
+    public UpdateCrawlerScheduleResult updateCrawlerSchedule(UpdateCrawlerScheduleRequest updateCrawlerScheduleRequest) {
+        return decoratedAwsGlue.updateCrawlerSchedule(updateCrawlerScheduleRequest);
+    }
+
+    @Override
+    public UpdateDatabaseResult updateDatabase(UpdateDatabaseRequest updateDatabaseRequest) {
+        return decoratedAwsGlue.updateDatabase(updateDatabaseRequest);
+    }
+
+    @Override
+    public UpdateDevEndpointResult updateDevEndpoint(UpdateDevEndpointRequest updateDevEndpointRequest) {
+        return decoratedAwsGlue.updateDevEndpoint(updateDevEndpointRequest);
+    }
+
+    @Override
+    public UpdateJobResult updateJob(UpdateJobRequest updateJobRequest) {
+        return decoratedAwsGlue.updateJob(updateJobRequest);
+    }
+
+    @Override
+    public UpdateMLTransformResult updateMLTransform(UpdateMLTransformRequest updateMLTransformRequest) {
+        return decoratedAwsGlue.updateMLTransform(updateMLTransformRequest);
+    }
+
+    @Override
+    public UpdatePartitionResult updatePartition(UpdatePartitionRequest updatePartitionRequest) {
+        return decoratedAwsGlue.updatePartition(updatePartitionRequest);
+    }
+
+    @Override
+    public UpdateRegistryResult updateRegistry(UpdateRegistryRequest updateRegistryRequest) {
+        return null;
+    }
+
+    @Override
+    public UpdateSchemaResult updateSchema(UpdateSchemaRequest updateSchemaRequest) {
+        return null;
+    }
+
+    @Override
+    public UpdateTableResult updateTable(UpdateTableRequest updateTableRequest) {
+        return decoratedAwsGlue.updateTable(updateTableRequest);
+    }
+
+    @Override
+    public UpdateTriggerResult updateTrigger(UpdateTriggerRequest updateTriggerRequest) {
+        return decoratedAwsGlue.updateTrigger(updateTriggerRequest);
+    }
+
+    @Override
+    public UpdateUserDefinedFunctionResult updateUserDefinedFunction(UpdateUserDefinedFunctionRequest updateUserDefinedFunctionRequest) {
+        return decoratedAwsGlue.updateUserDefinedFunction(updateUserDefinedFunctionRequest);
+    }
+
+    @Override
+    public UpdateWorkflowResult updateWorkflow(UpdateWorkflowRequest updateWorkflowRequest) {
+        return decoratedAwsGlue.updateWorkflow(updateWorkflowRequest);
+    }
+
+    @Override
+    public void shutdown() {
+        decoratedAwsGlue.shutdown();
+    }
+
+    @Override
+    public ResponseMetadata getCachedResponseMetadata(AmazonWebServiceRequest amazonWebServiceRequest) {
+        return decoratedAwsGlue.getCachedResponseMetadata(amazonWebServiceRequest);
+    }
+
+
+    @Override
+    public UpdateColumnStatisticsForTableResult updateColumnStatisticsForTable(UpdateColumnStatisticsForTableRequest updateColumnStatisticsForTableRequest) {
+        return decoratedAwsGlue.updateColumnStatisticsForTable(updateColumnStatisticsForTableRequest);
+    }
+
+    @Override
+    public UpdateColumnStatisticsForPartitionResult updateColumnStatisticsForPartition(UpdateColumnStatisticsForPartitionRequest updateColumnStatisticsForPartitionRequest) {
+        return decoratedAwsGlue.updateColumnStatisticsForPartition(updateColumnStatisticsForPartitionRequest);
+    }
+
+    @Override
+    public ResumeWorkflowRunResult resumeWorkflowRun(ResumeWorkflowRunRequest resumeWorkflowRunRequest) {
+        return decoratedAwsGlue.resumeWorkflowRun(resumeWorkflowRunRequest);
+    }
+
+    @Override
+    public GetResourcePoliciesResult getResourcePolicies(GetResourcePoliciesRequest getResourcePoliciesRequest) {
+        return decoratedAwsGlue.getResourcePolicies(getResourcePoliciesRequest);
+    }
+
+    @Override
+    public GetColumnStatisticsForTableResult getColumnStatisticsForTable(GetColumnStatisticsForTableRequest getColumnStatisticsForTableRequest) {
+        return decoratedAwsGlue.getColumnStatisticsForTable(getColumnStatisticsForTableRequest);
+    }
+
+    @Override
+    public GetColumnStatisticsForPartitionResult getColumnStatisticsForPartition(GetColumnStatisticsForPartitionRequest getColumnStatisticsForPartitionRequest) {
+        return decoratedAwsGlue.getColumnStatisticsForPartition(getColumnStatisticsForPartitionRequest);
+    }
+
+    @Override
+    public DeleteColumnStatisticsForTableResult deleteColumnStatisticsForTable(DeleteColumnStatisticsForTableRequest deleteColumnStatisticsForTableRequest) {
+        return decoratedAwsGlue.deleteColumnStatisticsForTable(deleteColumnStatisticsForTableRequest);
+    }
+
+    @Override
+    public DeleteColumnStatisticsForPartitionResult deleteColumnStatisticsForPartition(DeleteColumnStatisticsForPartitionRequest deleteColumnStatisticsForPartitionRequest) {
+        return decoratedAwsGlue.deleteColumnStatisticsForPartition(deleteColumnStatisticsForPartitionRequest);
+    }
+
+}
+
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastore.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastore.java
new file mode 100644
index 0000000000..ec74797bfa
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastore.java
@@ -0,0 +1,133 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.model.ColumnStatistics;
+import com.amazonaws.services.glue.model.ColumnStatisticsError;
+import com.amazonaws.services.glue.model.Database;
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.Partition;
+import com.amazonaws.services.glue.model.PartitionError;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.PartitionValueList;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UserDefinedFunction;
+import com.amazonaws.services.glue.model.UserDefinedFunctionInput;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.thrift.TException;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This is the accessor interface for using AWS Glue as a metastore.
+ * The generic AWSGlue interface{@link com.amazonaws.services.glue.AWSGlue}
+ * has a number of methods that are irrelevant for clients using Glue only
+ * as a metastore.
+ * Think of this interface as a wrapper over AWSGlue. This additional layer
+ * of abstraction achieves the following -
+ * a) Hides the non-metastore related operations present in AWSGlue
+ * b) Hides away the batching and pagination related limitations of AWSGlue
+ */
+public interface AWSGlueMetastore {
+
+    void createDatabase(DatabaseInput databaseInput);
+
+    Database getDatabase(String dbName);
+
+    List<Database> getAllDatabases();
+
+    void updateDatabase(String databaseName, DatabaseInput databaseInput);
+
+    void deleteDatabase(String dbName);
+
+    void createTable(String dbName, TableInput tableInput);
+
+    Table getTable(String dbName, String tableName);
+
+    List<Table> getTables(String dbname, String tablePattern);
+
+    void updateTable(String dbName, TableInput tableInput);
+
+    void updateTable(String dbName, TableInput tableInput, EnvironmentContext environmentContext);
+
+    void deleteTable(String dbName, String tableName);
+
+    Partition getPartition(String dbName, String tableName, List<String> partitionValues);
+
+    List<Partition> getPartitionsByNames(String dbName, String tableName,
+            List<PartitionValueList> partitionsToGet);
+
+    List<Partition> getPartitions(String dbName, String tableName, String expression,
+            long max) throws TException;
+
+    void updatePartition(String dbName, String tableName, List<String> partitionValues,
+            PartitionInput partitionInput);
+
+    void deletePartition(String dbName, String tableName, List<String> partitionValues);
+
+    List<PartitionError> createPartitions(String dbName, String tableName,
+            List<PartitionInput> partitionInputs);
+
+    void createUserDefinedFunction(String dbName, UserDefinedFunctionInput functionInput);
+
+    UserDefinedFunction getUserDefinedFunction(String dbName, String functionName);
+
+    List<UserDefinedFunction> getUserDefinedFunctions(String dbName, String pattern);
+
+    List<UserDefinedFunction> getUserDefinedFunctions(String pattern);
+
+    void deleteUserDefinedFunction(String dbName, String functionName);
+
+    void updateUserDefinedFunction(String dbName, String functionName, UserDefinedFunctionInput functionInput);
+
+    void deletePartitionColumnStatistics(String dbName, String tableName, List<String> partitionValues, String colName);
+
+    void deleteTableColumnStatistics(String dbName, String tableName, String colName);
+
+    Map<String, List<ColumnStatistics>> getPartitionColumnStatistics(
+            String dbName,
+            String tableName,
+            List<String> partitionValues,
+            List<String> columnNames
+    );
+
+    List<ColumnStatistics> getTableColumnStatistics(
+            String dbName,
+            String tableName,
+            List<String> colNames
+    );
+
+    List<ColumnStatisticsError> updatePartitionColumnStatistics(
+            String dbName,
+            String tableName,
+            List<String> partitionValues,
+            List<ColumnStatistics> columnStatistics
+    );
+
+    List<ColumnStatisticsError> updateTableColumnStatistics(
+            String dbName,
+            String tableName,
+            List<ColumnStatistics> columnStatistics
+    );
+}
\ No newline at end of file
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreBaseDecorator.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreBaseDecorator.java
new file mode 100644
index 0000000000..e8da0056b2
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreBaseDecorator.java
@@ -0,0 +1,198 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.model.ColumnStatistics;
+import com.amazonaws.services.glue.model.ColumnStatisticsError;
+import com.amazonaws.services.glue.model.Database;
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.Partition;
+import com.amazonaws.services.glue.model.PartitionError;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.PartitionValueList;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UserDefinedFunction;
+import com.amazonaws.services.glue.model.UserDefinedFunctionInput;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.thrift.TException;
+
+import java.util.List;
+import java.util.Map;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class AWSGlueMetastoreBaseDecorator implements AWSGlueMetastore {
+
+    private final AWSGlueMetastore awsGlueMetastore;
+
+    public AWSGlueMetastoreBaseDecorator(AWSGlueMetastore awsGlueMetastore) {
+        checkNotNull(awsGlueMetastore, "awsGlueMetastore can not be null");
+        this.awsGlueMetastore = awsGlueMetastore;
+    }
+
+    @Override
+    public void createDatabase(DatabaseInput databaseInput) {
+        awsGlueMetastore.createDatabase(databaseInput);
+    }
+
+    @Override
+    public Database getDatabase(String dbName) {
+        return awsGlueMetastore.getDatabase(dbName);
+    }
+
+    @Override
+    public List<Database> getAllDatabases() {
+        return awsGlueMetastore.getAllDatabases();
+    }
+
+    @Override
+    public void updateDatabase(String databaseName, DatabaseInput databaseInput) {
+        awsGlueMetastore.updateDatabase(databaseName, databaseInput);
+    }
+
+    @Override
+    public void deleteDatabase(String dbName) {
+        awsGlueMetastore.deleteDatabase(dbName);
+    }
+
+    @Override
+    public void createTable(String dbName, TableInput tableInput) {
+        awsGlueMetastore.createTable(dbName, tableInput);
+    }
+
+    @Override
+    public Table getTable(String dbName, String tableName) {
+        return awsGlueMetastore.getTable(dbName, tableName);
+    }
+
+    @Override
+    public List<Table> getTables(String dbname, String tablePattern) {
+        return awsGlueMetastore.getTables(dbname, tablePattern);
+    }
+
+    @Override
+    public void updateTable(String dbName, TableInput tableInput) {
+        awsGlueMetastore.updateTable(dbName, tableInput);
+    }
+
+    @Override
+    public void updateTable(String dbName, TableInput tableInput, EnvironmentContext environmentContext) {
+        awsGlueMetastore.updateTable(dbName, tableInput, environmentContext);
+    }
+
+    @Override
+    public void deleteTable(String dbName, String tableName) {
+        awsGlueMetastore.deleteTable(dbName, tableName);
+    }
+
+    @Override
+    public Partition getPartition(String dbName, String tableName, List<String> partitionValues) {
+        return awsGlueMetastore.getPartition(dbName, tableName, partitionValues);
+    }
+
+    @Override
+    public List<Partition> getPartitionsByNames(String dbName, String tableName, List<PartitionValueList> partitionsToGet) {
+        return awsGlueMetastore.getPartitionsByNames(dbName, tableName, partitionsToGet);
+    }
+
+    @Override
+    public List<Partition> getPartitions(String dbName, String tableName, String expression, long max) throws TException {
+        return awsGlueMetastore.getPartitions(dbName, tableName, expression, max);
+    }
+
+    @Override
+    public void updatePartition(String dbName, String tableName, List<String> partitionValues, PartitionInput partitionInput) {
+        awsGlueMetastore.updatePartition(dbName, tableName, partitionValues, partitionInput);
+    }
+
+    @Override
+    public void deletePartition(String dbName, String tableName, List<String> partitionValues) {
+        awsGlueMetastore.deletePartition(dbName, tableName, partitionValues);
+    }
+
+    @Override
+    public List<PartitionError> createPartitions(String dbName, String tableName, List<PartitionInput> partitionInputs) {
+        return awsGlueMetastore.createPartitions(dbName, tableName, partitionInputs);
+    }
+
+    @Override
+    public void createUserDefinedFunction(String dbName, UserDefinedFunctionInput functionInput) {
+        awsGlueMetastore.createUserDefinedFunction(dbName, functionInput);
+    }
+
+    @Override
+    public UserDefinedFunction getUserDefinedFunction(String dbName, String functionName) {
+        return awsGlueMetastore.getUserDefinedFunction(dbName, functionName);
+    }
+
+    @Override
+    public List<UserDefinedFunction> getUserDefinedFunctions(String dbName, String pattern) {
+        return awsGlueMetastore.getUserDefinedFunctions(dbName, pattern);
+    }
+
+    @Override
+    public List<UserDefinedFunction> getUserDefinedFunctions(String pattern) {
+        return awsGlueMetastore.getUserDefinedFunctions(pattern);
+    }
+
+    @Override
+    public void deleteUserDefinedFunction(String dbName, String functionName) {
+        awsGlueMetastore.deleteUserDefinedFunction(dbName, functionName);
+    }
+
+    @Override
+    public void updateUserDefinedFunction(String dbName, String functionName, UserDefinedFunctionInput functionInput) {
+        awsGlueMetastore.updateUserDefinedFunction(dbName, functionName, functionInput);
+    }
+
+    @Override
+    public void deletePartitionColumnStatistics(String dbName, String tableName, List<String> partitionValues, String colName) {
+        awsGlueMetastore.deletePartitionColumnStatistics(dbName, tableName, partitionValues, colName);
+    }
+
+    @Override
+    public void deleteTableColumnStatistics(String dbName, String tableName, String colName) {
+        awsGlueMetastore.deleteTableColumnStatistics(dbName, tableName, colName);
+    }
+
+    @Override
+    public Map<String, List<ColumnStatistics>> getPartitionColumnStatistics(String dbName, String tableName, List<String> partitionValues, List<String> columnNames) {
+        return awsGlueMetastore.getPartitionColumnStatistics(dbName, tableName, partitionValues, columnNames);
+    }
+
+    @Override
+    public List<ColumnStatistics> getTableColumnStatistics(String dbName, String tableName, List<String> colNames) {
+        return awsGlueMetastore.getTableColumnStatistics(dbName, tableName, colNames);
+    }
+
+    @Override
+    public List<ColumnStatisticsError> updatePartitionColumnStatistics(String dbName, String tableName, List<String> partitionValues, List<ColumnStatistics> columnStatistics) {
+        return awsGlueMetastore.updatePartitionColumnStatistics(dbName, tableName, partitionValues, columnStatistics);
+    }
+
+    @Override
+    public List<ColumnStatisticsError> updateTableColumnStatistics(String dbName, String tableName, List<ColumnStatistics> columnStatistics) {
+        return awsGlueMetastore.updateTableColumnStatistics(dbName, tableName, columnStatistics);
+    }
+
+}
\ No newline at end of file
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreCacheDecorator.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreCacheDecorator.java
new file mode 100644
index 0000000000..7ef0280e15
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreCacheDecorator.java
@@ -0,0 +1,185 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.model.Database;
+import com.amazonaws.services.glue.model.Table;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.Logger;
+
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_DB_CACHE_ENABLE;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_DB_CACHE_SIZE;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_DB_CACHE_TTL_MINS;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_TABLE_CACHE_ENABLE;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_TABLE_CACHE_SIZE;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_TABLE_CACHE_TTL_MINS;
+
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class AWSGlueMetastoreCacheDecorator extends AWSGlueMetastoreBaseDecorator {
+
+    private static final Logger logger = Logger.getLogger(AWSGlueMetastoreCacheDecorator.class);
+
+    private final Configuration conf;
+
+    private final boolean databaseCacheEnabled;
+
+    private final boolean tableCacheEnabled;
+
+    @VisibleForTesting
+    protected Cache<String, Database> databaseCache;
+    @VisibleForTesting
+    protected Cache<TableIdentifier, Table> tableCache;
+
+    public AWSGlueMetastoreCacheDecorator(Configuration conf, AWSGlueMetastore awsGlueMetastore) {
+        super(awsGlueMetastore);
+
+        checkNotNull(conf, "conf can not be null");
+        this.conf = conf;
+
+        databaseCacheEnabled = conf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false);
+        if(databaseCacheEnabled) {
+            int dbCacheSize = conf.getInt(AWS_GLUE_DB_CACHE_SIZE, 0);
+            int dbCacheTtlMins = conf.getInt(AWS_GLUE_DB_CACHE_TTL_MINS, 0);
+
+            //validate config values for size and ttl
+            validateConfigValueIsGreaterThanZero(AWS_GLUE_DB_CACHE_SIZE, dbCacheSize);
+            validateConfigValueIsGreaterThanZero(AWS_GLUE_DB_CACHE_TTL_MINS, dbCacheTtlMins);
+
+            //initialize database cache
+            databaseCache = CacheBuilder.newBuilder().maximumSize(dbCacheSize)
+                    .expireAfterWrite(dbCacheTtlMins, TimeUnit.MINUTES).build();
+        } else {
+            databaseCache = null;
+        }
+
+        tableCacheEnabled = conf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false);
+        if(tableCacheEnabled) {
+            int tableCacheSize = conf.getInt(AWS_GLUE_TABLE_CACHE_SIZE, 0);
+            int tableCacheTtlMins = conf.getInt(AWS_GLUE_TABLE_CACHE_TTL_MINS, 0);
+
+            //validate config values for size and ttl
+            validateConfigValueIsGreaterThanZero(AWS_GLUE_TABLE_CACHE_SIZE, tableCacheSize);
+            validateConfigValueIsGreaterThanZero(AWS_GLUE_TABLE_CACHE_TTL_MINS, tableCacheTtlMins);
+
+            //initialize table cache
+            tableCache = CacheBuilder.newBuilder().maximumSize(tableCacheSize)
+                    .expireAfterWrite(tableCacheTtlMins, TimeUnit.MINUTES).build();
+        } else {
+            tableCache = null;
+        }
+
+        logger.info("Constructed");
+    }
+
+    private void validateConfigValueIsGreaterThanZero(String configName, int value) {
+        checkArgument(value > 0, String.format("Invalid value for Hive Config %s. " +
+                "Provide a value greater than zero", configName));
+
+    }
+
+    @Override
+    public Database getDatabase(String dbName) {
+        Database result;
+        if(databaseCacheEnabled) {
+            Database valueFromCache = databaseCache.getIfPresent(dbName);
+            if(valueFromCache != null) {
+                logger.info("Cache hit for operation [getDatabase] on key [" + dbName + "]");
+                result = valueFromCache;
+            } else {
+                logger.info("Cache miss for operation [getDatabase] on key [" + dbName + "]");
+                result = super.getDatabase(dbName);
+                databaseCache.put(dbName, result);
+            }
+        } else {
+            result = super.getDatabase(dbName);
+        }
+        return result;
+    }
+
+    @Override
+    public Table getTable(String dbName, String tableName) {
+        Table result;
+        if(tableCacheEnabled) {
+            TableIdentifier key = new TableIdentifier(dbName, tableName);
+            Table valueFromCache = tableCache.getIfPresent(key);
+            if(valueFromCache != null) {
+                logger.info("Cache hit for operation [getTable] on key [" + key + "]");
+                result = valueFromCache;
+            } else {
+                logger.info("Cache miss for operation [getTable] on key [" + key + "]");
+                result = super.getTable(dbName, tableName);
+                tableCache.put(key, result);
+            }
+        } else {
+            result = super.getTable(dbName, tableName);
+        }
+        return result;
+    }
+
+    static class TableIdentifier {
+        private final String dbName;
+        private final String tableName;
+
+        public TableIdentifier(String dbName, String tableName) {
+            this.dbName = dbName;
+            this.tableName = tableName;
+        }
+
+        public String getDbName() {
+            return dbName;
+        }
+
+        public String getTableName() {
+            return tableName;
+        }
+
+        @Override
+        public String toString() {
+            return "TableIdentifier{" +
+                    "dbName='" + dbName + '\'' +
+                    ", tableName='" + tableName + '\'' +
+                    '}';
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+            TableIdentifier that = (TableIdentifier) o;
+            return Objects.equals(dbName, that.dbName) &&
+                    Objects.equals(tableName, that.tableName);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(dbName, tableName);
+        }
+    }
+}
\ No newline at end of file
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreFactory.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreFactory.java
new file mode 100644
index 0000000000..ad0353d096
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMetastoreFactory.java
@@ -0,0 +1,47 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.AWSGlue;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_DB_CACHE_ENABLE;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_TABLE_CACHE_ENABLE;
+
+public class AWSGlueMetastoreFactory {
+
+    public AWSGlueMetastore newMetastore(Configuration conf) throws MetaException {
+        AWSGlue glueClient = new AWSGlueClientFactory(conf).newClient();
+        AWSGlueMetastore defaultMetastore = new DefaultAWSGlueMetastore(conf, glueClient);
+        if(isCacheEnabled(conf)) {
+            return new AWSGlueMetastoreCacheDecorator(conf, defaultMetastore);
+        }
+        return defaultMetastore;
+    }
+
+    private boolean isCacheEnabled(Configuration conf) {
+        boolean databaseCacheEnabled = conf.getBoolean(AWS_GLUE_DB_CACHE_ENABLE, false);
+        boolean tableCacheEnabled = conf.getBoolean(AWS_GLUE_TABLE_CACHE_ENABLE, false);
+        return (databaseCacheEnabled || tableCacheEnabled);
+    }
+}
\ No newline at end of file
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMultipleCatalogDecorator.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMultipleCatalogDecorator.java
new file mode 100644
index 0000000000..c94472260d
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/AWSGlueMultipleCatalogDecorator.java
@@ -0,0 +1,370 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.AWSGlue;
+import com.amazonaws.services.glue.model.BatchCreatePartitionRequest;
+import com.amazonaws.services.glue.model.BatchCreatePartitionResult;
+import com.amazonaws.services.glue.model.BatchDeletePartitionRequest;
+import com.amazonaws.services.glue.model.BatchDeletePartitionResult;
+import com.amazonaws.services.glue.model.BatchDeleteTableRequest;
+import com.amazonaws.services.glue.model.BatchDeleteTableResult;
+import com.amazonaws.services.glue.model.BatchGetPartitionRequest;
+import com.amazonaws.services.glue.model.BatchGetPartitionResult;
+import com.amazonaws.services.glue.model.CreateDatabaseRequest;
+import com.amazonaws.services.glue.model.CreateDatabaseResult;
+import com.amazonaws.services.glue.model.CreatePartitionRequest;
+import com.amazonaws.services.glue.model.CreatePartitionResult;
+import com.amazonaws.services.glue.model.CreateTableRequest;
+import com.amazonaws.services.glue.model.CreateTableResult;
+import com.amazonaws.services.glue.model.CreateUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.CreateUserDefinedFunctionResult;
+import com.amazonaws.services.glue.model.DeleteDatabaseRequest;
+import com.amazonaws.services.glue.model.DeleteDatabaseResult;
+import com.amazonaws.services.glue.model.DeletePartitionRequest;
+import com.amazonaws.services.glue.model.DeletePartitionResult;
+import com.amazonaws.services.glue.model.DeleteTableRequest;
+import com.amazonaws.services.glue.model.DeleteTableResult;
+import com.amazonaws.services.glue.model.DeleteUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.DeleteUserDefinedFunctionResult;
+import com.amazonaws.services.glue.model.GetDatabaseRequest;
+import com.amazonaws.services.glue.model.GetDatabaseResult;
+import com.amazonaws.services.glue.model.GetPartitionRequest;
+import com.amazonaws.services.glue.model.GetPartitionResult;
+import com.amazonaws.services.glue.model.GetPartitionsRequest;
+import com.amazonaws.services.glue.model.GetPartitionsResult;
+import com.amazonaws.services.glue.model.GetTableRequest;
+import com.amazonaws.services.glue.model.GetTableResult;
+import com.amazonaws.services.glue.model.GetTableVersionsRequest;
+import com.amazonaws.services.glue.model.GetTableVersionsResult;
+import com.amazonaws.services.glue.model.GetTablesRequest;
+import com.amazonaws.services.glue.model.GetTablesResult;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionResult;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionsRequest;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionsResult;
+import com.amazonaws.services.glue.model.UpdateDatabaseRequest;
+import com.amazonaws.services.glue.model.UpdateDatabaseResult;
+import com.amazonaws.services.glue.model.UpdatePartitionRequest;
+import com.amazonaws.services.glue.model.UpdatePartitionResult;
+import com.amazonaws.services.glue.model.UpdateTableRequest;
+import com.amazonaws.services.glue.model.UpdateTableResult;
+import com.amazonaws.services.glue.model.UpdateUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.UpdateUserDefinedFunctionResult;
+import com.google.common.base.Strings;
+
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+
+public class AWSGlueMultipleCatalogDecorator extends AWSGlueDecoratorBase {
+
+    // We're not importing this from Hive's Warehouse class as the package name is changed between Hive 1.x and Hive 3.x
+    private static final String DEFAULT_DATABASE_NAME = "default";
+
+    private String catalogSeparator;
+
+    public AWSGlueMultipleCatalogDecorator(AWSGlue awsGlueToBeDecorated, String catalogSeparator) {
+        super(awsGlueToBeDecorated);
+        this.catalogSeparator = catalogSeparator;
+    }
+
+    private void configureRequest(Supplier<String> getDatabaseFunc,
+            Consumer<String> setDatabaseFunc,
+            Consumer<String> setCatalogFunc) {
+        if (!Strings.isNullOrEmpty(this.catalogSeparator) && (getDatabaseFunc.get() != null)
+                && !getDatabaseFunc.get().equals(DEFAULT_DATABASE_NAME)) {
+            String databaseName = getDatabaseFunc.get();
+            int idx = databaseName.indexOf(this.catalogSeparator);
+            if (idx >= 0) {
+                setCatalogFunc.accept(databaseName.substring(0, idx));
+                setDatabaseFunc.accept(databaseName.substring(idx + this.catalogSeparator.length()));
+            }
+        }
+    }
+
+    @Override
+    public BatchCreatePartitionResult batchCreatePartition(BatchCreatePartitionRequest batchCreatePartitionRequest) {
+        configureRequest(
+                batchCreatePartitionRequest::getDatabaseName,
+                batchCreatePartitionRequest::setDatabaseName,
+                batchCreatePartitionRequest::setCatalogId
+        );
+        return super.batchCreatePartition(batchCreatePartitionRequest);
+    }
+
+    @Override
+    public BatchDeletePartitionResult batchDeletePartition(BatchDeletePartitionRequest batchDeletePartitionRequest) {
+        configureRequest(
+                batchDeletePartitionRequest::getDatabaseName,
+                batchDeletePartitionRequest::setDatabaseName,
+                batchDeletePartitionRequest::setCatalogId
+        );
+        return super.batchDeletePartition(batchDeletePartitionRequest);
+    }
+
+    @Override
+    public BatchDeleteTableResult batchDeleteTable(BatchDeleteTableRequest batchDeleteTableRequest) {
+        configureRequest(
+                batchDeleteTableRequest::getDatabaseName,
+                batchDeleteTableRequest::setDatabaseName,
+                batchDeleteTableRequest::setCatalogId
+        );
+        return super.batchDeleteTable(batchDeleteTableRequest);
+    }
+
+    @Override
+    public BatchGetPartitionResult batchGetPartition(BatchGetPartitionRequest batchGetPartitionRequest) {
+        String originalDatabaseName = batchGetPartitionRequest.getDatabaseName();
+        configureRequest(
+                batchGetPartitionRequest::getDatabaseName,
+                batchGetPartitionRequest::setDatabaseName,
+                batchGetPartitionRequest::setCatalogId
+        );
+        BatchGetPartitionResult result = super.batchGetPartition(batchGetPartitionRequest);
+        result.getPartitions().forEach(partition -> partition.setDatabaseName(originalDatabaseName));
+        return result;
+    }
+
+    @Override
+    public CreateDatabaseResult createDatabase(CreateDatabaseRequest createDatabaseRequest) {
+        configureRequest(
+                () -> createDatabaseRequest.getDatabaseInput().getName(),
+                name -> createDatabaseRequest.getDatabaseInput().setName(name),
+                createDatabaseRequest::setCatalogId
+        );
+        return super.createDatabase(createDatabaseRequest);
+    }
+
+    @Override
+    public CreatePartitionResult createPartition(CreatePartitionRequest createPartitionRequest) {
+        configureRequest(
+                createPartitionRequest::getDatabaseName,
+                createPartitionRequest::setDatabaseName,
+                createPartitionRequest::setCatalogId
+        );
+        return super.createPartition(createPartitionRequest);
+    }
+
+    @Override
+    public CreateTableResult createTable(CreateTableRequest createTableRequest) {
+        configureRequest(
+                createTableRequest::getDatabaseName,
+                createTableRequest::setDatabaseName,
+                createTableRequest::setCatalogId
+        );
+        return super.createTable(createTableRequest);
+    }
+
+    @Override
+    public CreateUserDefinedFunctionResult createUserDefinedFunction(CreateUserDefinedFunctionRequest createUserDefinedFunctionRequest) {
+        configureRequest(
+                createUserDefinedFunctionRequest::getDatabaseName,
+                createUserDefinedFunctionRequest::setDatabaseName,
+                createUserDefinedFunctionRequest::setCatalogId
+        );
+        return super.createUserDefinedFunction(createUserDefinedFunctionRequest);
+    }
+
+    @Override
+    public DeleteDatabaseResult deleteDatabase(DeleteDatabaseRequest deleteDatabaseRequest) {
+        configureRequest(
+                deleteDatabaseRequest::getName,
+                deleteDatabaseRequest::setName,
+                deleteDatabaseRequest::setCatalogId
+        );
+        return super.deleteDatabase(deleteDatabaseRequest);
+    }
+
+    @Override
+    public DeletePartitionResult deletePartition(DeletePartitionRequest deletePartitionRequest) {
+        configureRequest(
+                deletePartitionRequest::getDatabaseName,
+                deletePartitionRequest::setDatabaseName,
+                deletePartitionRequest::setCatalogId
+        );
+        return super.deletePartition(deletePartitionRequest);
+    }
+
+    @Override
+    public DeleteTableResult deleteTable(DeleteTableRequest deleteTableRequest) {
+        configureRequest(
+                deleteTableRequest::getDatabaseName,
+                deleteTableRequest::setDatabaseName,
+                deleteTableRequest::setCatalogId
+        );
+        return super.deleteTable(deleteTableRequest);
+    }
+
+    @Override
+    public DeleteUserDefinedFunctionResult deleteUserDefinedFunction(DeleteUserDefinedFunctionRequest deleteUserDefinedFunctionRequest) {
+        configureRequest(
+                deleteUserDefinedFunctionRequest::getDatabaseName,
+                deleteUserDefinedFunctionRequest::setDatabaseName,
+                deleteUserDefinedFunctionRequest::setCatalogId
+        );
+        return super.deleteUserDefinedFunction(deleteUserDefinedFunctionRequest);
+    }
+
+    @Override
+    public GetDatabaseResult getDatabase(GetDatabaseRequest getDatabaseRequest) {
+        String originalDatabaseName = getDatabaseRequest.getName();
+        configureRequest(
+                getDatabaseRequest::getName,
+                getDatabaseRequest::setName,
+                getDatabaseRequest::setCatalogId
+        );
+        GetDatabaseResult result = super.getDatabase(getDatabaseRequest);
+        result.getDatabase().setName(originalDatabaseName);
+        return result;
+    }
+
+    @Override
+    public GetPartitionResult getPartition(GetPartitionRequest getPartitionRequest) {
+        String originalDatabaseName = getPartitionRequest.getDatabaseName();
+        configureRequest(
+                getPartitionRequest::getDatabaseName,
+                getPartitionRequest::setDatabaseName,
+                getPartitionRequest::setCatalogId
+        );
+        GetPartitionResult result = super.getPartition(getPartitionRequest);
+        result.getPartition().setDatabaseName(originalDatabaseName);
+        return result;
+    }
+
+    @Override
+    public GetPartitionsResult getPartitions(GetPartitionsRequest getPartitionsRequest) {
+        String originalDatabaseName = getPartitionsRequest.getDatabaseName();
+        configureRequest(
+                getPartitionsRequest::getDatabaseName,
+                getPartitionsRequest::setDatabaseName,
+                getPartitionsRequest::setCatalogId
+        );
+        GetPartitionsResult result = super.getPartitions(getPartitionsRequest);
+        result.getPartitions().forEach(partition -> partition.setDatabaseName(originalDatabaseName));
+        return result;
+    }
+
+    @Override
+    public GetTableResult getTable(GetTableRequest getTableRequest) {
+        String originalDatabaseName = getTableRequest.getDatabaseName();
+        configureRequest(
+                getTableRequest::getDatabaseName,
+                getTableRequest::setDatabaseName,
+                getTableRequest::setCatalogId
+        );
+        GetTableResult result = super.getTable(getTableRequest);
+        result.getTable().setDatabaseName(originalDatabaseName);
+        return result;
+    }
+
+    @Override
+    public GetTableVersionsResult getTableVersions(GetTableVersionsRequest getTableVersionsRequest) {
+        String originalDatabaseName = getTableVersionsRequest.getDatabaseName();
+        configureRequest(
+                getTableVersionsRequest::getDatabaseName,
+                getTableVersionsRequest::setDatabaseName,
+                getTableVersionsRequest::setCatalogId
+        );
+        GetTableVersionsResult result = super.getTableVersions(getTableVersionsRequest);
+        result.getTableVersions().forEach(tableVersion -> tableVersion.getTable().setDatabaseName(originalDatabaseName));
+        return result;
+    }
+
+    @Override
+    public GetTablesResult getTables(GetTablesRequest getTablesRequest) {
+        String originalDatabaseName = getTablesRequest.getDatabaseName();
+        configureRequest(
+                getTablesRequest::getDatabaseName,
+                getTablesRequest::setDatabaseName,
+                getTablesRequest::setCatalogId
+        );
+        GetTablesResult result = super.getTables(getTablesRequest);
+        result.getTableList().forEach(table -> table.setDatabaseName(originalDatabaseName));
+        return result;
+    }
+
+    @Override
+    public GetUserDefinedFunctionResult getUserDefinedFunction(GetUserDefinedFunctionRequest getUserDefinedFunctionRequest) {
+        configureRequest(
+                getUserDefinedFunctionRequest::getDatabaseName,
+                getUserDefinedFunctionRequest::setDatabaseName,
+                getUserDefinedFunctionRequest::setCatalogId
+        );
+        return super.getUserDefinedFunction(getUserDefinedFunctionRequest);
+    }
+
+    @Override
+    public GetUserDefinedFunctionsResult getUserDefinedFunctions(GetUserDefinedFunctionsRequest getUserDefinedFunctionsRequest) {
+        configureRequest(
+                getUserDefinedFunctionsRequest::getDatabaseName,
+                getUserDefinedFunctionsRequest::setDatabaseName,
+                getUserDefinedFunctionsRequest::setCatalogId
+        );
+        return super.getUserDefinedFunctions(getUserDefinedFunctionsRequest);
+    }
+
+    @Override
+    public UpdateDatabaseResult updateDatabase(UpdateDatabaseRequest updateDatabaseRequest) {
+        configureRequest(
+                updateDatabaseRequest::getName,
+                updateDatabaseRequest::setName,
+                updateDatabaseRequest::setCatalogId
+        );
+        configureRequest(
+                () -> updateDatabaseRequest.getDatabaseInput().getName(),
+                name -> updateDatabaseRequest.getDatabaseInput().setName(name),
+                catalogId -> {}
+        );
+        return super.updateDatabase(updateDatabaseRequest);
+    }
+
+    @Override
+    public UpdatePartitionResult updatePartition(UpdatePartitionRequest updatePartitionRequest) {
+        configureRequest(
+                updatePartitionRequest::getDatabaseName,
+                updatePartitionRequest::setDatabaseName,
+                updatePartitionRequest::setCatalogId
+        );
+        return super.updatePartition(updatePartitionRequest);
+    }
+
+    @Override
+    public UpdateTableResult updateTable(UpdateTableRequest updateTableRequest) {
+        configureRequest(
+                updateTableRequest::getDatabaseName,
+                updateTableRequest::setDatabaseName,
+                updateTableRequest::setCatalogId
+        );
+        return super.updateTable(updateTableRequest);
+    }
+
+    @Override
+    public UpdateUserDefinedFunctionResult updateUserDefinedFunction(UpdateUserDefinedFunctionRequest updateUserDefinedFunctionRequest) {
+        configureRequest(
+                updateUserDefinedFunctionRequest::getDatabaseName,
+                updateUserDefinedFunctionRequest::setDatabaseName,
+                updateUserDefinedFunctionRequest::setCatalogId
+        );
+        return super.updateUserDefinedFunction(updateUserDefinedFunctionRequest);
+    }
+}
diff --git a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSCredentialsProviderFactory.java
similarity index 57%
copy from fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
copy to fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSCredentialsProviderFactory.java
index 05084c97b5..2f87efa38a 100644
--- a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSCredentialsProviderFactory.java
@@ -14,12 +14,24 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import org.apache.hadoop.conf.Configuration;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
 
-package org.apache.doris.load.loadv2.dpp;
+public class DefaultAWSCredentialsProviderFactory implements
+        AWSCredentialsProviderFactory {
 
-import org.apache.doris.load.loadv2.etl.EtlJobConfig;
+    @Override
+    public AWSCredentialsProvider buildAWSCredentialsProvider(Configuration conf) {
+        return new DefaultAWSCredentialsProviderChain();
+    }
 
-// RollupTreeBuilder is used to get the RollupTree from the TableMeta
-public abstract interface RollupTreeBuilder {
-    public RollupTreeNode build(EtlJobConfig.EtlTable tableMeta);
 }
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSGlueMetastore.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSGlueMetastore.java
new file mode 100644
index 0000000000..7569139251
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultAWSGlueMetastore.java
@@ -0,0 +1,662 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.glue.catalog.converters.PartitionNameParser;
+import com.amazonaws.glue.catalog.util.MetastoreClientUtils;
+import com.amazonaws.services.glue.AWSGlue;
+import com.amazonaws.services.glue.model.BatchCreatePartitionRequest;
+import com.amazonaws.services.glue.model.BatchGetPartitionRequest;
+import com.amazonaws.services.glue.model.BatchGetPartitionResult;
+import com.amazonaws.services.glue.model.ColumnStatistics;
+import com.amazonaws.services.glue.model.ColumnStatisticsError;
+import com.amazonaws.services.glue.model.CreateDatabaseRequest;
+import com.amazonaws.services.glue.model.CreateTableRequest;
+import com.amazonaws.services.glue.model.CreateUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.Database;
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.DeleteColumnStatisticsForPartitionRequest;
+import com.amazonaws.services.glue.model.DeleteColumnStatisticsForTableRequest;
+import com.amazonaws.services.glue.model.DeleteDatabaseRequest;
+import com.amazonaws.services.glue.model.DeletePartitionRequest;
+import com.amazonaws.services.glue.model.DeleteTableRequest;
+import com.amazonaws.services.glue.model.DeleteUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.GetColumnStatisticsForPartitionRequest;
+import com.amazonaws.services.glue.model.GetColumnStatisticsForPartitionResult;
+import com.amazonaws.services.glue.model.GetColumnStatisticsForTableRequest;
+import com.amazonaws.services.glue.model.GetColumnStatisticsForTableResult;
+import com.amazonaws.services.glue.model.GetDatabaseRequest;
+import com.amazonaws.services.glue.model.GetDatabaseResult;
+import com.amazonaws.services.glue.model.GetDatabasesRequest;
+import com.amazonaws.services.glue.model.GetDatabasesResult;
+import com.amazonaws.services.glue.model.GetPartitionRequest;
+import com.amazonaws.services.glue.model.GetPartitionsRequest;
+import com.amazonaws.services.glue.model.GetPartitionsResult;
+import com.amazonaws.services.glue.model.GetTableRequest;
+import com.amazonaws.services.glue.model.GetTableResult;
+import com.amazonaws.services.glue.model.GetTablesRequest;
+import com.amazonaws.services.glue.model.GetTablesResult;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionsRequest;
+import com.amazonaws.services.glue.model.GetUserDefinedFunctionsResult;
+import com.amazonaws.services.glue.model.Partition;
+import com.amazonaws.services.glue.model.PartitionError;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.PartitionValueList;
+import com.amazonaws.services.glue.model.Segment;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UpdateColumnStatisticsForPartitionRequest;
+import com.amazonaws.services.glue.model.UpdateColumnStatisticsForPartitionResult;
+import com.amazonaws.services.glue.model.UpdateColumnStatisticsForTableRequest;
+import com.amazonaws.services.glue.model.UpdateColumnStatisticsForTableResult;
+import com.amazonaws.services.glue.model.UpdateDatabaseRequest;
+import com.amazonaws.services.glue.model.UpdatePartitionRequest;
+import com.amazonaws.services.glue.model.UpdateTableRequest;
+import com.amazonaws.services.glue.model.UpdateUserDefinedFunctionRequest;
+import com.amazonaws.services.glue.model.UserDefinedFunction;
+import com.amazonaws.services.glue.model.UserDefinedFunctionInput;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.thrift.TException;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+public class DefaultAWSGlueMetastore implements AWSGlueMetastore {
+
+    public static final int BATCH_GET_PARTITIONS_MAX_REQUEST_SIZE = 1000;
+    /**
+     * Based on the maxResults parameter at https://docs.aws.amazon.com/glue/latest/webapi/API_GetPartitions.html
+     */
+    public static final int GET_PARTITIONS_MAX_SIZE = 1000;
+    /**
+     * Maximum number of Glue Segments. A segment defines a non-overlapping region of a table's partitions,
+     * allowing multiple requests to be executed in parallel.
+     */
+    public static final int DEFAULT_NUM_PARTITION_SEGMENTS = 5;
+    /**
+     * Currently the upper limit allowed by Glue is 10.
+     * https://docs.aws.amazon.com/glue/latest/webapi/API_Segment.html
+     */
+    public static final int MAX_NUM_PARTITION_SEGMENTS = 10;
+    public static final String NUM_PARTITION_SEGMENTS_CONF = "aws.glue.partition.num.segments";
+    public static final String CUSTOM_EXECUTOR_FACTORY_CONF = "hive.metastore.executorservice.factory.class";
+
+    /**
+     * Based on the ColumnNames parameter at https://docs.aws.amazon.com/glue/latest/webapi/API_GetColumnStatisticsForPartition.html
+     */
+    public static final int GET_COLUMNS_STAT_MAX_SIZE = 100;
+    public static final int UPDATE_COLUMNS_STAT_MAX_SIZE = 25;
+
+    /**
+     * To be used with UpdateTable
+     */
+    public static final String SKIP_AWS_GLUE_ARCHIVE = "skipAWSGlueArchive";
+
+    private static final int NUM_EXECUTOR_THREADS = 5;
+    static final String GLUE_METASTORE_DELEGATE_THREADPOOL_NAME_FORMAT = "glue-metastore-delegate-%d";
+    private static final ExecutorService GLUE_METASTORE_DELEGATE_THREAD_POOL = Executors.newFixedThreadPool(
+            NUM_EXECUTOR_THREADS,
+            new ThreadFactoryBuilder()
+                    .setNameFormat(GLUE_METASTORE_DELEGATE_THREADPOOL_NAME_FORMAT)
+                    .setDaemon(true).build()
+    );
+
+    private final Configuration conf;
+    private final AWSGlue glueClient;
+    private final String catalogId;
+    private final ExecutorService executorService;
+    private final int numPartitionSegments;
+
+    protected ExecutorService getExecutorService(Configuration conf) {
+        Class<? extends ExecutorServiceFactory> executorFactoryClass = conf
+                .getClass(CUSTOM_EXECUTOR_FACTORY_CONF,
+                        DefaultExecutorServiceFactory.class).asSubclass(
+                        ExecutorServiceFactory.class);
+        ExecutorServiceFactory factory = ReflectionUtils.newInstance(
+                executorFactoryClass, conf);
+        return factory.getExecutorService(conf);
+    }
+
+    public DefaultAWSGlueMetastore(Configuration conf, AWSGlue glueClient) {
+        checkNotNull(conf, "Hive Config cannot be null");
+        checkNotNull(glueClient, "glueClient cannot be null");
+        this.numPartitionSegments = conf.getInt(NUM_PARTITION_SEGMENTS_CONF, DEFAULT_NUM_PARTITION_SEGMENTS);
+        checkArgument(numPartitionSegments <= MAX_NUM_PARTITION_SEGMENTS,
+                String.format("Hive Config [%s] can't exceed %d", NUM_PARTITION_SEGMENTS_CONF, MAX_NUM_PARTITION_SEGMENTS));
+        this.conf = conf;
+        this.glueClient = glueClient;
+        this.catalogId = MetastoreClientUtils.getCatalogId(conf);
+        this.executorService = getExecutorService(conf);
+    }
+
+    // ======================= Database =======================
+
+    @Override
+    public void createDatabase(DatabaseInput databaseInput) {
+        CreateDatabaseRequest createDatabaseRequest = new CreateDatabaseRequest().withDatabaseInput(databaseInput)
+                .withCatalogId(catalogId);
+        glueClient.createDatabase(createDatabaseRequest);
+    }
+
+    @Override
+    public Database getDatabase(String dbName) {
+        GetDatabaseRequest getDatabaseRequest = new GetDatabaseRequest().withCatalogId(catalogId).withName(dbName);
+        GetDatabaseResult result = glueClient.getDatabase(getDatabaseRequest);
+        return result.getDatabase();
+    }
+
+    @Override
+    public List<Database> getAllDatabases() {
+        List<Database> ret = Lists.newArrayList();
+        String nextToken = null;
+        do {
+            GetDatabasesRequest getDatabasesRequest = new GetDatabasesRequest().withNextToken(nextToken).withCatalogId(
+                    catalogId);
+            GetDatabasesResult result = glueClient.getDatabases(getDatabasesRequest);
+            nextToken = result.getNextToken();
+            ret.addAll(result.getDatabaseList());
+        } while (nextToken != null);
+        return ret;
+    }
+
+    @Override
+    public void updateDatabase(String databaseName, DatabaseInput databaseInput) {
+        UpdateDatabaseRequest updateDatabaseRequest = new UpdateDatabaseRequest().withName(databaseName)
+                .withDatabaseInput(databaseInput).withCatalogId(catalogId);
+        glueClient.updateDatabase(updateDatabaseRequest);
+    }
+
+    @Override
+    public void deleteDatabase(String dbName) {
+        DeleteDatabaseRequest deleteDatabaseRequest = new DeleteDatabaseRequest().withName(dbName).withCatalogId(
+                catalogId);
+        glueClient.deleteDatabase(deleteDatabaseRequest);
+    }
+
+    // ======================== Table ========================
+
+    @Override
+    public void createTable(String dbName, TableInput tableInput) {
+        CreateTableRequest createTableRequest = new CreateTableRequest().withTableInput(tableInput)
+                .withDatabaseName(dbName).withCatalogId(catalogId);
+        glueClient.createTable(createTableRequest);
+    }
+
+    @Override
+    public Table getTable(String dbName, String tableName) {
+        GetTableRequest getTableRequest = new GetTableRequest().withDatabaseName(dbName).withName(tableName)
+                .withCatalogId(catalogId);
+        GetTableResult result = glueClient.getTable(getTableRequest);
+        return result.getTable();
+    }
+
+    @Override
+    public List<Table> getTables(String dbname, String tablePattern) {
+        List<Table> ret = new ArrayList<>();
+        String nextToken = null;
+        do {
+            GetTablesRequest getTablesRequest = new GetTablesRequest().withDatabaseName(dbname)
+                    .withExpression(tablePattern).withNextToken(nextToken).withCatalogId(catalogId);
+            GetTablesResult result = glueClient.getTables(getTablesRequest);
+            ret.addAll(result.getTableList());
+            nextToken = result.getNextToken();
+        } while (nextToken != null);
+        return ret;
+    }
+
+    @Override
+    public void updateTable(String dbName, TableInput tableInput) {
+        UpdateTableRequest updateTableRequest = new UpdateTableRequest().withDatabaseName(dbName)
+                .withTableInput(tableInput).withCatalogId(catalogId);
+        glueClient.updateTable(updateTableRequest);
+    }
+
+    @Override
+    public void updateTable(String dbName, TableInput tableInput, EnvironmentContext environmentContext) {
+        UpdateTableRequest updateTableRequest = new UpdateTableRequest().withDatabaseName(dbName)
+                .withTableInput(tableInput).withCatalogId(catalogId).withSkipArchive(skipArchive(environmentContext));
+        glueClient.updateTable(updateTableRequest);
+    }
+
+    private boolean skipArchive(EnvironmentContext environmentContext) {
+        return environmentContext != null &&
+                environmentContext.isSetProperties() &&
+                StatsSetupConst.TRUE.equals(environmentContext.getProperties().get(SKIP_AWS_GLUE_ARCHIVE));
+    }
+
+    @Override
+    public void deleteTable(String dbName, String tableName) {
+        DeleteTableRequest deleteTableRequest = new DeleteTableRequest().withDatabaseName(dbName).withName(tableName)
+                .withCatalogId(catalogId);
+        glueClient.deleteTable(deleteTableRequest);
+    }
+
+    // =========================== Partition ===========================
+
+    @Override
+    public Partition getPartition(String dbName, String tableName, List<String> partitionValues) {
+        GetPartitionRequest request = new GetPartitionRequest()
+                .withDatabaseName(dbName)
+                .withTableName(tableName)
+                .withPartitionValues(partitionValues)
+                .withCatalogId(catalogId);
+        return glueClient.getPartition(request).getPartition();
+    }
+
+    @Override
+    public List<Partition> getPartitionsByNames(String dbName, String tableName,
+            List<PartitionValueList> partitionsToGet) {
+
+        List<List<PartitionValueList>> batchedPartitionsToGet = Lists.partition(partitionsToGet,
+                BATCH_GET_PARTITIONS_MAX_REQUEST_SIZE);
+        List<Future<BatchGetPartitionResult>> batchGetPartitionFutures = Lists.newArrayList();
+
+        for (List<PartitionValueList> batch : batchedPartitionsToGet) {
+            final BatchGetPartitionRequest request = new BatchGetPartitionRequest()
+                    .withDatabaseName(dbName)
+                    .withTableName(tableName)
+                    .withPartitionsToGet(batch)
+                    .withCatalogId(catalogId);
+            batchGetPartitionFutures.add(this.executorService.submit(new Callable<BatchGetPartitionResult>() {
+                @Override
+                public BatchGetPartitionResult call() throws Exception {
+                    return glueClient.batchGetPartition(request);
+                }
+            }));
+        }
+
+        List<Partition> result = Lists.newArrayList();
+        try {
+            for (Future<BatchGetPartitionResult> future : batchGetPartitionFutures) {
+                result.addAll(future.get().getPartitions());
+            }
+        } catch (ExecutionException e) {
+            Throwables.propagateIfInstanceOf(e.getCause(), AmazonServiceException.class);
+            Throwables.propagate(e.getCause());
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+        return result;
+    }
+
+    @Override
+    public List<Partition> getPartitions(String dbName, String tableName, String expression,
+            long max) throws TException {
+        if (max == 0) {
+            return Collections.emptyList();
+        }
+        if (max < 0 || max > GET_PARTITIONS_MAX_SIZE) {
+            return getPartitionsParallel(dbName, tableName, expression, max);
+        } else {
+            // We don't need to get too many partitions, so just do it serially.
+            return getCatalogPartitions(dbName, tableName, expression, max, null);
+        }
+    }
+
+    private List<Partition> getPartitionsParallel(
+            final String databaseName,
+            final String tableName,
+            final String expression,
+            final long max) throws TException {
+        // Prepare the segments
+        List<Segment> segments = Lists.newArrayList();
+        for (int i = 0; i < numPartitionSegments; i++) {
+            segments.add(new Segment()
+                    .withSegmentNumber(i)
+                    .withTotalSegments(numPartitionSegments));
+        }
+        // Submit Glue API calls in parallel using the thread pool.
+        // We could convert this into a parallelStream after upgrading to JDK 8 compiler base.
+        List<Future<List<Partition>>> futures = Lists.newArrayList();
+        for (final Segment segment : segments) {
+            futures.add(this.executorService.submit(new Callable<List<Partition>>() {
+                @Override
+                public List<Partition> call() throws Exception {
+                    return getCatalogPartitions(databaseName, tableName, expression, max, segment);
+                }
+            }));
+        }
+
+        // Get the results
+        List<Partition> partitions = Lists.newArrayList();
+        try {
+            for (Future<List<Partition>> future : futures) {
+                List<Partition> segmentPartitions = future.get();
+                if (partitions.size() + segmentPartitions.size() >= max && max > 0) {
+                    // Extract the required number of partitions from the segment and we're done.
+                    long remaining = max - partitions.size();
+                    partitions.addAll(segmentPartitions.subList(0, (int) remaining));
+                    break;
+                } else {
+                    partitions.addAll(segmentPartitions);
+                }
+            }
+        } catch (ExecutionException e) {
+            Throwables.propagateIfInstanceOf(e.getCause(), AmazonServiceException.class);
+            Throwables.propagate(e.getCause());
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+        return partitions;
+    }
+
+
+    private List<Partition> getCatalogPartitions(String databaseName, String tableName, String expression,
+            long max, Segment segment) {
+        List<Partition> partitions = Lists.newArrayList();
+        String nextToken = null;
+        do {
+            GetPartitionsRequest request = new GetPartitionsRequest()
+                    .withDatabaseName(databaseName)
+                    .withTableName(tableName)
+                    .withExpression(expression)
+                    .withNextToken(nextToken)
+                    .withCatalogId(catalogId)
+                    .withSegment(segment);
+            GetPartitionsResult res = glueClient.getPartitions(request);
+            List<Partition> list = res.getPartitions();
+            if ((partitions.size() + list.size()) >= max && max > 0) {
+                long remaining = max - partitions.size();
+                partitions.addAll(list.subList(0, (int) remaining));
+                break;
+            }
+            partitions.addAll(list);
+            nextToken = res.getNextToken();
+        } while (nextToken != null);
+        return partitions;
+    }
+
+    @Override
+    public void updatePartition(String dbName, String tableName, List<String> partitionValues,
+            PartitionInput partitionInput) {
+        UpdatePartitionRequest updatePartitionRequest = new UpdatePartitionRequest().withDatabaseName(dbName)
+                .withTableName(tableName).withPartitionValueList(partitionValues)
+                .withPartitionInput(partitionInput).withCatalogId(catalogId);
+        glueClient.updatePartition(updatePartitionRequest);
+    }
+
+    @Override
+    public void deletePartition(String dbName, String tableName, List<String> partitionValues) {
+        DeletePartitionRequest request = new DeletePartitionRequest()
+                .withDatabaseName(dbName)
+                .withTableName(tableName)
+                .withPartitionValues(partitionValues)
+                .withCatalogId(catalogId);
+        glueClient.deletePartition(request);
+    }
+
+    @Override
+    public List<PartitionError> createPartitions(String dbName, String tableName,
+            List<PartitionInput> partitionInputs) {
+        BatchCreatePartitionRequest request =
+                new BatchCreatePartitionRequest().withDatabaseName(dbName)
+                        .withTableName(tableName).withCatalogId(catalogId)
+                        .withPartitionInputList(partitionInputs);
+        return glueClient.batchCreatePartition(request).getErrors();
+    }
+
+    // ====================== User Defined Function ======================
+
+    @Override
+    public void createUserDefinedFunction(String dbName, UserDefinedFunctionInput functionInput) {
+        CreateUserDefinedFunctionRequest createUserDefinedFunctionRequest = new CreateUserDefinedFunctionRequest()
+                .withDatabaseName(dbName).withFunctionInput(functionInput).withCatalogId(catalogId);
+        glueClient.createUserDefinedFunction(createUserDefinedFunctionRequest);
+    }
+
+    @Override
+    public UserDefinedFunction getUserDefinedFunction(String dbName, String functionName) {
+        GetUserDefinedFunctionRequest getUserDefinedFunctionRequest = new GetUserDefinedFunctionRequest()
+                .withDatabaseName(dbName).withFunctionName(functionName).withCatalogId(catalogId);
+        return glueClient.getUserDefinedFunction(getUserDefinedFunctionRequest).getUserDefinedFunction();
+    }
+
+    @Override
+    public List<UserDefinedFunction> getUserDefinedFunctions(String dbName, String pattern) {
+        List<UserDefinedFunction> ret = Lists.newArrayList();
+        String nextToken = null;
+        do {
+            GetUserDefinedFunctionsRequest getUserDefinedFunctionsRequest = new GetUserDefinedFunctionsRequest()
+                    .withDatabaseName(dbName).withPattern(pattern).withNextToken(nextToken).withCatalogId(catalogId);
+            GetUserDefinedFunctionsResult result = glueClient.getUserDefinedFunctions(getUserDefinedFunctionsRequest);
+            nextToken = result.getNextToken();
+            ret.addAll(result.getUserDefinedFunctions());
+        } while (nextToken != null);
+        return ret;
+    }
+
+    @Override
+    public List<UserDefinedFunction> getUserDefinedFunctions(String pattern) {
+        List<UserDefinedFunction> ret = Lists.newArrayList();
+        String nextToken = null;
+        do {
+            GetUserDefinedFunctionsRequest getUserDefinedFunctionsRequest = new GetUserDefinedFunctionsRequest()
+                    .withPattern(pattern).withNextToken(nextToken).withCatalogId(catalogId);
+            GetUserDefinedFunctionsResult result = glueClient.getUserDefinedFunctions(getUserDefinedFunctionsRequest);
+            nextToken = result.getNextToken();
+            ret.addAll(result.getUserDefinedFunctions());
+        } while (nextToken != null);
+        return ret;
+    }
+
+    @Override
+    public void deleteUserDefinedFunction(String dbName, String functionName) {
+        DeleteUserDefinedFunctionRequest deleteUserDefinedFunctionRequest = new DeleteUserDefinedFunctionRequest()
+                .withDatabaseName(dbName).withFunctionName(functionName).withCatalogId(catalogId);
+        glueClient.deleteUserDefinedFunction(deleteUserDefinedFunctionRequest);
+    }
+
+    @Override
+    public void updateUserDefinedFunction(String dbName, String functionName, UserDefinedFunctionInput functionInput) {
+        UpdateUserDefinedFunctionRequest updateUserDefinedFunctionRequest = new UpdateUserDefinedFunctionRequest()
+                .withDatabaseName(dbName).withFunctionName(functionName).withFunctionInput(functionInput)
+                .withCatalogId(catalogId);
+        glueClient.updateUserDefinedFunction(updateUserDefinedFunctionRequest);
+    }
+
+    @Override
+    public void deletePartitionColumnStatistics(String dbName, String tableName, List<String> partitionValues, String colName) {
+        DeleteColumnStatisticsForPartitionRequest request = new DeleteColumnStatisticsForPartitionRequest()
+                .withCatalogId(catalogId)
+                .withDatabaseName(dbName)
+                .withTableName(tableName)
+                .withPartitionValues(partitionValues)
+                .withColumnName(colName);
+        glueClient.deleteColumnStatisticsForPartition(request);
+    }
+
+    @Override
+    public void deleteTableColumnStatistics(String dbName, String tableName, String colName) {
+        DeleteColumnStatisticsForTableRequest request = new DeleteColumnStatisticsForTableRequest()
+                .withCatalogId(catalogId)
+                .withDatabaseName(dbName)
+                .withTableName(tableName)
+                .withColumnName(colName);
+        glueClient.deleteColumnStatisticsForTable(request);
+    }
+
+    @Override
+    public Map<String, List<ColumnStatistics>> getPartitionColumnStatistics(String dbName, String tableName, List<String> partitionValues, List<String> columnNames) {
+        Map<String, List<ColumnStatistics>> partitionStatistics = new HashMap<>();
+        List<List<String>> pagedColNames = Lists.partition(columnNames, GET_COLUMNS_STAT_MAX_SIZE);
+        List<String> partValues;
+        for (String partName : partitionValues) {
+            partValues = PartitionNameParser.getPartitionValuesFromName(partName);
+            List<Future<GetColumnStatisticsForPartitionResult>> pagedResult = new ArrayList<>();
+            for (List<String> cols : pagedColNames) {
+                GetColumnStatisticsForPartitionRequest request = new GetColumnStatisticsForPartitionRequest()
+                        .withCatalogId(catalogId)
+                        .withDatabaseName(dbName)
+                        .withTableName(tableName)
+                        .withPartitionValues(partValues)
+                        .withColumnNames(cols);
+                pagedResult.add(GLUE_METASTORE_DELEGATE_THREAD_POOL.submit(new Callable<GetColumnStatisticsForPartitionResult>() {
+                    @Override
+                    public GetColumnStatisticsForPartitionResult call() throws Exception {
+                        return glueClient.getColumnStatisticsForPartition(request);
+                    }
+                }));
+            }
+
+            List<ColumnStatistics> result = new ArrayList<>();
+            for (Future<GetColumnStatisticsForPartitionResult> page : pagedResult) {
+                try {
+                    result.addAll(page.get().getColumnStatisticsList());
+                } catch (ExecutionException e) {
+                    Throwables.propagateIfInstanceOf(e.getCause(), AmazonServiceException.class);
+                    Throwables.propagate(e.getCause());
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                }
+            }
+            partitionStatistics.put(partName, result);
+        }
+        return partitionStatistics;
+    }
+
+    @Override
+    public List<ColumnStatistics> getTableColumnStatistics(String dbName, String tableName, List<String> colNames) {
+        List<List<String>> pagedColNames = Lists.partition(colNames, GET_COLUMNS_STAT_MAX_SIZE);
+        List<Future<GetColumnStatisticsForTableResult>> pagedResult = new ArrayList<>();
+
+        for (List<String> cols : pagedColNames) {
+            GetColumnStatisticsForTableRequest request = new GetColumnStatisticsForTableRequest()
+                    .withCatalogId(catalogId)
+                    .withDatabaseName(dbName)
+                    .withTableName(tableName)
+                    .withColumnNames(cols);
+            pagedResult.add(GLUE_METASTORE_DELEGATE_THREAD_POOL.submit(new Callable<GetColumnStatisticsForTableResult>() {
+                @Override
+                public GetColumnStatisticsForTableResult call() throws Exception {
+                    return glueClient.getColumnStatisticsForTable(request);
+                }
+            }));
+        }
+        List<ColumnStatistics> results = new ArrayList<>();
+
+        for (Future<GetColumnStatisticsForTableResult> page : pagedResult) {
+            try {
+                results.addAll(page.get().getColumnStatisticsList());
+            } catch (ExecutionException e) {
+                Throwables.propagateIfInstanceOf(e.getCause(), AmazonServiceException.class);
+                Throwables.propagate(e.getCause());
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+        }
+        return results;
+    }
+
+    @Override
+    public List<ColumnStatisticsError> updatePartitionColumnStatistics(
+            String dbName,
+            String tableName,
+            List<String> partitionValues,
+            List<ColumnStatistics> columnStatistics) {
+
+        List<List<ColumnStatistics>> statisticsListPaged = Lists.partition(columnStatistics, UPDATE_COLUMNS_STAT_MAX_SIZE);
+        List<Future<UpdateColumnStatisticsForPartitionResult>> pagedResult = new ArrayList<>();
+        for (List<ColumnStatistics> statList : statisticsListPaged) {
+            UpdateColumnStatisticsForPartitionRequest request = new UpdateColumnStatisticsForPartitionRequest()
+                    .withCatalogId(catalogId)
+                    .withDatabaseName(dbName)
+                    .withTableName(tableName)
+                    .withPartitionValues(partitionValues)
+                    .withColumnStatisticsList(statList);
+            pagedResult.add(GLUE_METASTORE_DELEGATE_THREAD_POOL.submit(new Callable<UpdateColumnStatisticsForPartitionResult>() {
+                @Override
+                public UpdateColumnStatisticsForPartitionResult call() throws Exception {
+                    return glueClient.updateColumnStatisticsForPartition(request);
+                }
+            }));
+        }
+        // Waiting for calls to finish. Will fail the call if one of the future task fails
+        List<ColumnStatisticsError> columnStatisticsErrors = new ArrayList<>();
+        try {
+            for (Future<UpdateColumnStatisticsForPartitionResult> page : pagedResult) {
+                Optional.ofNullable(page.get().getErrors()).ifPresent(error -> columnStatisticsErrors.addAll(error));
+            }
+        } catch (ExecutionException e) {
+            Throwables.propagateIfInstanceOf(e.getCause(), AmazonServiceException.class);
+            Throwables.propagate(e.getCause());
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+        return columnStatisticsErrors;
+    }
+
+    @Override
+    public List<ColumnStatisticsError> updateTableColumnStatistics(
+            String dbName,
+            String tableName,
+            List<ColumnStatistics> columnStatistics) {
+
+        List<List<ColumnStatistics>> statisticsListPaged = Lists.partition(columnStatistics, UPDATE_COLUMNS_STAT_MAX_SIZE);
+        List<Future<UpdateColumnStatisticsForTableResult>> pagedResult = new ArrayList<>();
+        for (List<ColumnStatistics> statList : statisticsListPaged) {
+            UpdateColumnStatisticsForTableRequest request = new UpdateColumnStatisticsForTableRequest()
+                    .withCatalogId(catalogId)
+                    .withDatabaseName(dbName)
+                    .withTableName(tableName)
+                    .withColumnStatisticsList(statList);
+            pagedResult.add(GLUE_METASTORE_DELEGATE_THREAD_POOL.submit(new Callable<UpdateColumnStatisticsForTableResult>() {
+                @Override
+                public UpdateColumnStatisticsForTableResult call() throws Exception {
+                    return glueClient.updateColumnStatisticsForTable(request);
+                }
+            }));
+        }
+
+        // Waiting for calls to finish. Will fail the call if one of the future task fails
+        List<ColumnStatisticsError> columnStatisticsErrors = new ArrayList<>();
+        try {
+            for (Future<UpdateColumnStatisticsForTableResult> page : pagedResult) {
+                Optional.ofNullable(page.get().getErrors()).ifPresent(error -> columnStatisticsErrors.addAll(error));
+            }
+        } catch (ExecutionException e) {
+            Throwables.propagateIfInstanceOf(e.getCause(), AmazonServiceException.class);
+            Throwables.propagate(e.getCause());
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+        return columnStatisticsErrors;
+    }
+}
\ No newline at end of file
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultExecutorServiceFactory.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultExecutorServiceFactory.java
new file mode 100644
index 0000000000..326587f161
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/DefaultExecutorServiceFactory.java
@@ -0,0 +1,43 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.conf.Configuration;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+public class DefaultExecutorServiceFactory implements ExecutorServiceFactory {
+    private static final int NUM_EXECUTOR_THREADS = 5;
+
+    private static final ExecutorService GLUE_METASTORE_DELEGATE_THREAD_POOL = Executors.newFixedThreadPool(
+            NUM_EXECUTOR_THREADS, new ThreadFactoryBuilder()
+                    .setNameFormat(GlueMetastoreClientDelegate.GLUE_METASTORE_DELEGATE_THREADPOOL_NAME_FORMAT)
+                    .setDaemon(true).build()
+    );
+
+    @Override
+    public ExecutorService getExecutorService(Configuration conf) {
+        return GLUE_METASTORE_DELEGATE_THREAD_POOL;
+    }
+}
diff --git a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/ExecutorServiceFactory.java
similarity index 65%
copy from fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
copy to fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/ExecutorServiceFactory.java
index 05084c97b5..a9b53f55ad 100644
--- a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/ExecutorServiceFactory.java
@@ -14,12 +14,20 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
 
-package org.apache.doris.load.loadv2.dpp;
+import org.apache.hadoop.conf.Configuration;
 
-import org.apache.doris.load.loadv2.etl.EtlJobConfig;
+import java.util.concurrent.ExecutorService;
 
-// RollupTreeBuilder is used to get the RollupTree from the TableMeta
-public abstract interface RollupTreeBuilder {
-    public RollupTreeNode build(EtlJobConfig.EtlTable tableMeta);
+/*
+ * Interface for creating an ExecutorService
+ */
+public interface ExecutorServiceFactory {
+    public ExecutorService getExecutorService(Configuration conf);
 }
diff --git a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/GlueClientFactory.java
similarity index 66%
copy from fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
copy to fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/GlueClientFactory.java
index 05084c97b5..409d8863c3 100644
--- a/fe/spark-dpp/src/main/java/org/apache/doris/load/loadv2/dpp/RollupTreeBuilder.java
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/GlueClientFactory.java
@@ -14,12 +14,21 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.services.glue.AWSGlue;
+import org.apache.hadoop.hive.metastore.api.MetaException;
 
-package org.apache.doris.load.loadv2.dpp;
+/***
+ * Interface for creating Glue AWS Client
+ */
+public interface GlueClientFactory {
 
-import org.apache.doris.load.loadv2.etl.EtlJobConfig;
+    AWSGlue newClient() throws MetaException;
 
-// RollupTreeBuilder is used to get the RollupTree from the TableMeta
-public abstract interface RollupTreeBuilder {
-    public RollupTreeNode build(EtlJobConfig.EtlTable tableMeta);
 }
diff --git a/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/GlueMetastoreClientDelegate.java b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/GlueMetastoreClientDelegate.java
new file mode 100644
index 0000000000..a6eaab123e
--- /dev/null
+++ b/fe/fe-core/src/main/java/com/amazonaws/glue/catalog/metastore/GlueMetastoreClientDelegate.java
@@ -0,0 +1,1843 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+//
+// Copied from
+// https://github.com/awslabs/aws-glue-data-catalog-client-for-apache-hive-metastore/blob/branch-3.4.0/
+//
+
+package com.amazonaws.glue.catalog.metastore;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.glue.catalog.converters.CatalogToHiveConverter;
+import com.amazonaws.glue.catalog.converters.CatalogToHiveConverterFactory;
+import com.amazonaws.glue.catalog.converters.GlueInputConverter;
+import com.amazonaws.glue.catalog.converters.HiveToCatalogConverter;
+import com.amazonaws.glue.catalog.converters.PartitionNameParser;
+import static com.amazonaws.glue.catalog.util.AWSGlueConfig.AWS_GLUE_DISABLE_UDF;
+import com.amazonaws.glue.catalog.util.BatchCreatePartitionsHelper;
+import com.amazonaws.glue.catalog.util.ExpressionHelper;
+import com.amazonaws.glue.catalog.util.MetastoreClientUtils;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.deepCopyMap;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.isExternalTable;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.makeDirs;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.validateGlueTable;
+import static com.amazonaws.glue.catalog.util.MetastoreClientUtils.validateTableObject;
+import com.amazonaws.glue.catalog.util.PartitionKey;
+import com.amazonaws.services.glue.model.Column;
+import com.amazonaws.services.glue.model.ColumnStatistics;
+import com.amazonaws.services.glue.model.ColumnStatisticsError;
+import com.amazonaws.services.glue.model.Database;
+import com.amazonaws.services.glue.model.DatabaseInput;
+import com.amazonaws.services.glue.model.EntityNotFoundException;
+import com.amazonaws.services.glue.model.Partition;
+import com.amazonaws.services.glue.model.PartitionInput;
+import com.amazonaws.services.glue.model.PartitionValueList;
+import com.amazonaws.services.glue.model.Table;
+import com.amazonaws.services.glue.model.TableInput;
+import com.amazonaws.services.glue.model.UserDefinedFunction;
+import com.amazonaws.services.glue.model.UserDefinedFunctionInput;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import static org.apache.hadoop.hive.metastore.HiveMetaStore.PUBLIC;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.TableType;
+import static org.apache.hadoop.hive.metastore.TableType.EXTERNAL_TABLE;
+import static org.apache.hadoop.hive.metastore.TableType.MANAGED_TABLE;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.AggrStats;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CompactionResponse;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
+import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
+import org.apache.hadoop.hive.metastore.api.DataOperationType;
+import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.FireEventRequest;
+import org.apache.hadoop.hive.metastore.api.FireEventResponse;
+import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse;
+import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalRequest;
+import org.apache.hadoop.hive.metastore.api.GetRoleGrantsForPrincipalResponse;
+import org.apache.hadoop.hive.metastore.api.HeartbeatTxnRangeResponse;
+import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
+import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+import org.apache.hadoop.hive.metastore.api.InvalidObjectException;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.MetadataPpdResult;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
+import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
+import org.apache.hadoop.hive.metastore.api.PartitionEventType;
+import org.apache.hadoop.hive.metastore.api.PartitionValuesRequest;
+import org.apache.hadoop.hive.metastore.api.PartitionValuesResponse;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.Role;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.ShowLocksRequest;
+import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
+import org.apache.hadoop.hive.metastore.api.TableMeta;
+import org.apache.hadoop.hive.metastore.api.UnknownDBException;
+import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
+import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/***
+ * Delegate Class to provide all common functionality
+ * between Spark-hive version, Hive and Presto clients
+ *
+ */
+public class GlueMetastoreClientDelegate {
+
+  private static final Logger logger = Logger.getLogger(GlueMetastoreClientDelegate.class);
+
+  private static final List<Role> implicitRoles = Lists.newArrayList(new Role(PUBLIC, 0, PUBLIC));
+  public static final int MILLISECOND_TO_SECOND_FACTOR = 1000;
+  public static final Long NO_MAX = -1L;
+  public static final String MATCH_ALL = ".*";
+  private static final int BATCH_CREATE_PARTITIONS_MAX_REQUEST_SIZE = 100;
+
+  private static final int NUM_EXECUTOR_THREADS = 5;
+  static final String GLUE_METASTORE_DELEGATE_THREADPOOL_NAME_FORMAT = "glue-metastore-delegate-%d";
+  private static final ExecutorService GLUE_METASTORE_DELEGATE_THREAD_POOL = Executors.newFixedThreadPool(
+          NUM_EXECUTOR_THREADS,
+          new ThreadFactoryBuilder()
+                  .setNameFormat(GLUE_METASTORE_DELEGATE_THREADPOOL_NAME_FORMAT)
+                  .setDaemon(true).build()
+  );
+
+  private final AWSGlueMetastore glueMetastore;
+  private final Configuration conf;
+  private final Warehouse wh;
+  // private final AwsGlueHiveShims hiveShims = ShimsLoader.getHiveShims();
+  private final CatalogToHiveConverter catalogToHiveConverter;
+  private final String catalogId;
+
+  public static final String CATALOG_ID_CONF = "hive.metastore.glue.catalogid";
+  public static final String NUM_PARTITION_SEGMENTS_CONF = "aws.glue.partition.num.segments";
+
+  public GlueMetastoreClientDelegate(Configuration conf, AWSGlueMetastore glueMetastore,
+          Warehouse wh) throws MetaException {
+    checkNotNull(conf, "Hive Config cannot be null");
+    checkNotNull(glueMetastore, "glueMetastore cannot be null");
+    checkNotNull(wh, "Warehouse cannot be null");
+
+    catalogToHiveConverter = CatalogToHiveConverterFactory.getCatalogToHiveConverter();
+    this.conf = conf;
+    this.glueMetastore = glueMetastore;
+    this.wh = wh;
+    // TODO - May be validate catalogId confirms to AWS AccountId too.
+    catalogId = MetastoreClientUtils.getCatalogId(conf);
+  }
+
+  // ======================= Database =======================
+
+  public void createDatabase(org.apache.hadoop.hive.metastore.api.Database database) throws TException {
+    checkNotNull(database, "database cannot be null");
+
+    if (StringUtils.isEmpty(database.getLocationUri())) {
+      database.setLocationUri(wh.getDefaultDatabasePath(database.getName()).toString());
+    } else {
+      database.setLocationUri(wh.getDnsPath(new Path(database.getLocationUri())).toString());
+    }
+    Path dbPath = new Path(database.getLocationUri());
+    boolean madeDir = makeDirs(wh, dbPath);
+
+    try {
+      DatabaseInput catalogDatabase = GlueInputConverter.convertToDatabaseInput(database);
+      glueMetastore.createDatabase(catalogDatabase);
+    } catch (AmazonServiceException e) {
+      if (madeDir) {
+        // hiveShims.deleteDir(wh, dbPath, true, false);
+      }
+      throw catalogToHiveConverter.wrapInHiveException(e);
+    } catch (Exception e) {
+      String msg = "Unable to create database: ";
+      logger.error(msg, e);
+      throw new MetaException(msg + e);
+    }
+  }
+
+  public org.apache.hadoop.hive.metastore.api.Database getDatabase(String name) throws TException {
+    checkArgument(StringUtils.isNotEmpty(name), "name cannot be null or empty");
+
+    try {
+      Database catalogDatabase = glueMetastore.getDatabase(name);
... 10736 lines suppressed ...


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 09/11: [feature](jdbc catalog) support SQLServer jdbc catalog (#16093)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit c7d1535dbad4db8b2f522c3423117b062e58b458
Author: Tiewei Fang <43...@users.noreply.github.com>
AuthorDate: Fri Jan 20 12:37:38 2023 +0800

    [feature](jdbc catalog) support SQLServer jdbc catalog (#16093)
---
 .../sqlserver/init/01-drop-database.sql            | 18 +++++
 .../sqlserver/init/02-create-database.sql          | 18 +++++
 .../sqlserver/init/03-create-table.sql             | 89 ++++++++++++++++++++++
 .../docker-compose/sqlserver/init/04-insert.sql    | 52 +++++++++++++
 .../docker-compose/sqlserver/sqlserver.env         | 19 +++++
 .../docker-compose/sqlserver/sqlserver.yaml        | 47 ++++++++++++
 docker/thirdparties/start-thirdparties-docker.sh   |  7 ++
 docker/thirdparties/stop-thirdparties-docker.sh    |  3 +
 .../Create/CREATE-CATALOG.md                       | 24 ++++++
 .../Create/CREATE-CATALOG.md                       | 24 ++++++
 .../org/apache/doris/catalog/JdbcResource.java     |  2 +-
 .../java/org/apache/doris/catalog/OdbcTable.java   | 12 ++-
 .../org/apache/doris/external/jdbc/JdbcClient.java | 78 ++++++++++++++-----
 regression-test/conf/regression-conf.groovy        |  1 +
 .../test_sqlserver_jdbc_catalog.out                | 43 +++++++++++
 .../jdbc_catalog_p0/test_pg_jdbc_catalog.groovy    | 10 +--
 .../test_sqlserver_jdbc_catalog.groovy             | 72 +++++++++++++++++
 17 files changed, 494 insertions(+), 25 deletions(-)

diff --git a/docker/thirdparties/docker-compose/sqlserver/init/01-drop-database.sql b/docker/thirdparties/docker-compose/sqlserver/init/01-drop-database.sql
new file mode 100644
index 0000000000..60e0db7773
--- /dev/null
+++ b/docker/thirdparties/docker-compose/sqlserver/init/01-drop-database.sql
@@ -0,0 +1,18 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--   http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing,
+-- software distributed under the License is distributed on an
+-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+-- KIND, either express or implied.  See the License for the
+-- specific language governing permissions and limitations
+-- under the License.
+
+drop database if exists doris_test;
\ No newline at end of file
diff --git a/docker/thirdparties/docker-compose/sqlserver/init/02-create-database.sql b/docker/thirdparties/docker-compose/sqlserver/init/02-create-database.sql
new file mode 100644
index 0000000000..c2e3414541
--- /dev/null
+++ b/docker/thirdparties/docker-compose/sqlserver/init/02-create-database.sql
@@ -0,0 +1,18 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--   http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing,
+-- software distributed under the License is distributed on an
+-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+-- KIND, either express or implied.  See the License for the
+-- specific language governing permissions and limitations
+-- under the License.
+
+create database doris_test;
\ No newline at end of file
diff --git a/docker/thirdparties/docker-compose/sqlserver/init/03-create-table.sql b/docker/thirdparties/docker-compose/sqlserver/init/03-create-table.sql
new file mode 100644
index 0000000000..af5bcb483b
--- /dev/null
+++ b/docker/thirdparties/docker-compose/sqlserver/init/03-create-table.sql
@@ -0,0 +1,89 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--   http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing,
+-- software distributed under the License is distributed on an
+-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+-- KIND, either express or implied.  See the License for the
+-- specific language governing permissions and limitations
+-- under the License.
+
+use doris_test;
+
+CREATE TABLE dbo.student (
+    id int PRIMARY KEY NOT NULL,
+    name varchar(10) NOT NULL, 
+    age int NULL
+);
+
+CREATE TABLE dbo.test_int (
+    id int PRIMARY KEY NOT NULL,
+    tinyint_value tinyint NOT NULL,
+    smallint_value smallint NULL,
+    bigint_value bigint NULL
+);
+
+CREATE TABLE dbo.test_float (
+    id int PRIMARY KEY NOT NULL,
+    real_value real NOT NULL,
+    float_value float NULL,
+    floatn_value float(5) NULL,
+    decimal_value decimal(10,5) NULL,
+    numeric_value numeric(10,5) NULL
+);
+
+CREATE TABLE dbo.test_decimal (
+    id int PRIMARY KEY NOT NULL,
+    decimal_value decimal(38,0) NULL,
+    numeric_value numeric(38,0) NULL,
+    decimal_value2 decimal(38,10) NULL,
+    numeric_value2 numeric(38,10) NULL
+);
+
+CREATE TABLE dbo.test_char (
+    id int PRIMARY KEY NOT NULL,
+    char_value char(20) NOT NULL,
+    varchar_value varchar(20) NULL,
+    varcharmax_value varchar(max) NULL,
+    nchar_value nchar(20) NULL,
+    nvarchar_value nvarchar(20) NULL,
+    nvarcharmax_value nvarchar(max) NULL
+);
+
+CREATE TABLE dbo.test_time (
+    id int PRIMARY KEY NOT NULL,
+    date_value date NOT NULL,
+    time_value time NULL,
+    datetime_value datetime NULL,
+    datetime2_value datetime2 NULL,
+    smalldatetime_value smalldatetime NULL,
+    datetimeoffset_value datetimeoffset NULL
+);
+
+CREATE TABLE dbo.test_text (
+    id int PRIMARY KEY NOT NULL,
+    text_value text NOT NULL,
+    ntext_value ntext NULL
+);
+
+CREATE TABLE dbo.test_money (
+    id int PRIMARY KEY NOT NULL,
+    money_value money NOT NULL,
+    smallmoney_value smallmoney NULL
+);
+
+CREATE TABLE dbo.test_binary (
+    id int PRIMARY KEY NOT NULL,
+    bit_value bit NOT NULL,
+    binary_value binary(20) NULL,
+    varbinary_value varbinary(20) NULL
+);
+
+
diff --git a/docker/thirdparties/docker-compose/sqlserver/init/04-insert.sql b/docker/thirdparties/docker-compose/sqlserver/init/04-insert.sql
new file mode 100644
index 0000000000..f1d8849897
--- /dev/null
+++ b/docker/thirdparties/docker-compose/sqlserver/init/04-insert.sql
@@ -0,0 +1,52 @@
+-- Licensed to the Apache Software Foundation (ASF) under one
+-- or more contributor license agreements.  See the NOTICE file
+-- distributed with this work for additional information
+-- regarding copyright ownership.  The ASF licenses this file
+-- to you under the Apache License, Version 2.0 (the
+-- "License"); you may not use this file except in compliance
+-- with the License.  You may obtain a copy of the License at
+--
+--   http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing,
+-- software distributed under the License is distributed on an
+-- "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+-- KIND, either express or implied.  See the License for the
+-- specific language governing permissions and limitations
+-- under the License.
+
+Insert into dbo.student values (1, 'doris', 18), (2, 'alice', 19), (3, 'bob', 20);
+
+Insert into dbo.test_int values
+(1, 0, 1, 1), (2, 1, -1, -1),
+(3, 255, 32767, 9223372036854775807), (4, 128, -32768, -9223372036854775808);
+
+Insert into dbo.test_float values
+(1, 123.123, 123.123, 123.123, 123.123, 123.123),
+(2, 12345.12345, 12345.12345, 12345.12345, 12345.12345,12345.12345),
+(3, -123.123, -123.123, -123.123, -123.123, -123.123);
+
+Insert into dbo.test_decimal values
+(1, 12345678901234567890123456789012345678, 12345678901234567890123456789012345678, 1234567890123456789012345678.0123456789, 1234567890123456789012345678.0123456789),
+(2, -12345678901234567890123456789012345678, -12345678901234567890123456789012345678, -1234567890123456789012345678.0123456789, -1234567890123456789012345678.0123456789);
+
+Insert into dbo.test_char values
+(1, 'Make Doris Great!', 'Make Doris Great!', 'Make Doris Great!', 'Make Doris Great!', 'Make Doris Great!', 'Make Doris Great!');
+
+Insert into dbo.test_time values (1, '2023-01-17', '16:49:05.1234567', '2023-01-17 16:49:05', '2023-01-17 16:49:05.1234567', '2023-01-17 16:49:05', '2023-01-17 16:49:05+08:00'),
+(2, '2023-01-17', '16:49:05', '2023-01-17 16:49:05', '2023-01-17 16:49:05', '2023-01-17 16:49:05', '2023-01-17 16:49:05+08:00');
+Insert into dbo.test_time values (3, '2023-01-17', '16:49:05.1234567', '2023-01-17 16:49:05', '2023-01-17 16:49:05.1234567', '2023-01-17 16:49:05', '2023-01-17 16:49:05.1234567+08:00');
+Insert into dbo.test_time values (4, '2023-01-17', '16:49:05', '2023-01-17 16:49:05', '2023-01-17 16:49:05', '2023-01-17 16:49:05', '2023-01-17 16:49:05+08:00');
+
+Insert into dbo.test_text values (1, 'Make Doris Great!', 'Make Doris Great!');
+
+Insert into dbo.test_money values (1, 922337203685477.5807, 214748.3647);
+Insert into dbo.test_money values (2, -922337203685477.5808, -214748.3648);
+Insert into dbo.test_money values (3, 123.123, 123.123);
+
+insert into dbo.test_binary values (1, 0, 0x4D616B6520446F72697320477265617421, 0x4D616B6520446F72697320477265617421);
+insert into dbo.test_binary values (2, 1, 0x4D616B6520446F72697320477265617421, 0x4D616B6520446F72697320477265617421);
+insert into dbo.test_binary values (3, -1, 0x4D616B6520446F72697320477265617421, 0x4D616B6520446F72697320477265617421);
+
+
+
diff --git a/docker/thirdparties/docker-compose/sqlserver/sqlserver.env b/docker/thirdparties/docker-compose/sqlserver/sqlserver.env
new file mode 100644
index 0000000000..0aa395abdf
--- /dev/null
+++ b/docker/thirdparties/docker-compose/sqlserver/sqlserver.env
@@ -0,0 +1,19 @@
+#!/usr/bin/env bash
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+DOCKER_SQLSERVER_EXTERNAL_PORT=1433
\ No newline at end of file
diff --git a/docker/thirdparties/docker-compose/sqlserver/sqlserver.yaml b/docker/thirdparties/docker-compose/sqlserver/sqlserver.yaml
new file mode 100644
index 0000000000..02f909f5d5
--- /dev/null
+++ b/docker/thirdparties/docker-compose/sqlserver/sqlserver.yaml
@@ -0,0 +1,47 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+version: '3'
+services:
+  doris--sqlserver_2022:
+    image: "mcr.microsoft.com/mssql/server:2022-latest"
+    container_name: "doris--sqlserver_2022"
+    ports:
+      - ${DOCKER_SQLSERVER_EXTERNAL_PORT}:1433
+    healthcheck:
+      test: ["CMD", "/opt/mssql-tools/bin/sqlcmd", "-Usa", "-PDoris123456", "-Q", "select 1"]
+      interval: 5s
+      timeout: 30s
+      retries: 120
+    volumes:
+        - ./init:/docker-entrypoint-initdb.d
+    restart: always
+    environment:
+      # Accept the end user license Agreement
+      - ACCEPT_EULA=Y
+      # password of SA
+      - SA_PASSWORD=Doris123456
+    networks:
+      - doris--sqlserver_2022
+  hello-world:
+      image: hello-world
+      depends_on:
+        doris--sqlserver_2022:
+          condition: service_healthy
+
+networks:
+  doris--sqlserver_2022:
\ No newline at end of file
diff --git a/docker/thirdparties/start-thirdparties-docker.sh b/docker/thirdparties/start-thirdparties-docker.sh
index 61e0df9713..e25be54711 100755
--- a/docker/thirdparties/start-thirdparties-docker.sh
+++ b/docker/thirdparties/start-thirdparties-docker.sh
@@ -63,6 +63,13 @@ sudo mkdir -p "${ROOT}"/docker-compose/oracle/data/
 sudo rm "${ROOT}"/docker-compose/oracle/data/* -rf
 sudo docker compose -f "${ROOT}"/docker-compose/oracle/oracle-11.yaml --env-file "${ROOT}"/docker-compose/oracle/oracle-11.env up -d
 
+# sqlserver
+sed -i "s/doris--/${CONTAINER_UID}/g" "${ROOT}"/docker-compose/sqlserver/sqlserver.yaml
+sudo docker compose -f "${ROOT}"/docker-compose/sqlserver/sqlserver.yaml --env-file "${ROOT}"/docker-compose/sqlserver/sqlserver.env down
+sudo mkdir -p "${ROOT}"/docker-compose/sqlserver/data/
+sudo rm "${ROOT}"/docker-compose/sqlserver/data/* -rf
+sudo docker compose -f "${ROOT}"/docker-compose/sqlserver/sqlserver.yaml --env-file "${ROOT}"/docker-compose/sqlserver/sqlserver.env up -d
+
 # hive
 # before start it, you need to download parquet file package, see "README" in "docker-compose/hive/scripts/"
 sed -i "s/doris--/${CONTAINER_UID}/g" "${ROOT}"/docker-compose/hive/hive-2x.yaml
diff --git a/docker/thirdparties/stop-thirdparties-docker.sh b/docker/thirdparties/stop-thirdparties-docker.sh
index dc2c5773d8..963bd1e85d 100755
--- a/docker/thirdparties/stop-thirdparties-docker.sh
+++ b/docker/thirdparties/stop-thirdparties-docker.sh
@@ -36,5 +36,8 @@ sudo docker compose -f "${ROOT}"/docker-compose/postgresql/postgresql-14.yaml --
 # oracle 11
 sudo docker compose -f "${ROOT}"/docker-compose/oracle/oracle-11.yaml --env-file "${ROOT}"/docker-compose/oracle/oracle-11.env down
 
+# sqlserver
+sudo docker compose -f "${ROOT}"/docker-compose/sqlserver/sqlserver.yaml --env-file "${ROOT}"/docker-compose/sqlserver/sqlserver.env down
+
 # hive
 sudo docker compose -f "${ROOT}"/docker-compose/hive/hive-2x.yaml --env-file "${ROOT}"/docker-compose/hive/hadoop-hive.env down
diff --git a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-CATALOG.md b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-CATALOG.md
index b659e3a7fc..d530c2ced2 100644
--- a/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-CATALOG.md
+++ b/docs/en/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-CATALOG.md
@@ -216,6 +216,30 @@ CREATE CATALOG catalog_name PROPERTIES (
 	);	
 	```
 
+	**SQLServer**
+	```sql
+	-- The first way
+	CREATE RESOURCE sqlserver_resource PROPERTIES (
+		"type"="jdbc",
+		"user"="SA",
+		"password"="Doris123456",
+		"jdbc_url" = "jdbc:sqlserver://localhost:1433;DataBaseName=doris_test",
+		"driver_url" = "file:/path/to/mssql-jdbc-11.2.3.jre8.jar",
+		"driver_class" = "com.microsoft.sqlserver.jdbc.SQLServerDriver"
+	);
+	CREATE CATALOG sqlserver_catlog WITH RESOURCE sqlserver_resource;
+
+	-- The second way, note: keys have 'jdbc' prefix in front.
+	CREATE CATALOG sqlserver_catlog PROPERTIES (
+		"type"="jdbc",
+		"jdbc.user"="SA",
+		"jdbc.password"="Doris123456",
+		"jdbc.jdbc_url" = "jdbc:sqlserver://localhost:1433;DataBaseName=doris_test",
+		"jdbc.driver_url" = "file:/path/to/mssql-jdbc-11.2.3.jre8.jar",
+		"jdbc.driver_class" = "com.microsoft.sqlserver.jdbc.SQLServerDriver"
+	);	
+	```
+
 ### Keywords
 
 CREATE, CATALOG
diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-CATALOG.md b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-CATALOG.md
index 9c71ee73a8..65b3417302 100644
--- a/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-CATALOG.md
+++ b/docs/zh-CN/docs/sql-manual/sql-reference/Data-Definition-Statements/Create/CREATE-CATALOG.md
@@ -222,6 +222,30 @@ CREATE CATALOG catalog_name PROPERTIES (
 	);	
 	```
 
+	**SQLServer**
+	```sql
+	-- 方式一
+	CREATE RESOURCE sqlserver_resource PROPERTIES (
+		"type"="jdbc",
+		"user"="SA",
+		"password"="Doris123456",
+		"jdbc_url" = "jdbc:sqlserver://localhost:1433;DataBaseName=doris_test",
+		"driver_url" = "file:/path/to/mssql-jdbc-11.2.3.jre8.jar",
+		"driver_class" = "com.microsoft.sqlserver.jdbc.SQLServerDriver"
+	);
+	CREATE CATALOG sqlserver_catlog WITH RESOURCE sqlserver_resource;
+
+	-- 方式二,注意有jdbc前缀
+	CREATE CATALOG sqlserver_catlog PROPERTIES (
+		"type"="jdbc",
+		"jdbc.user"="SA",
+		"jdbc.password"="Doris123456",
+		"jdbc.jdbc_url" = "jdbc:sqlserver://localhost:1433;DataBaseName=doris_test",
+		"jdbc.driver_url" = "file:/path/to/mssql-jdbc-11.2.3.jre8.jar",
+		"jdbc.driver_class" = "com.microsoft.sqlserver.jdbc.SQLServerDriver"
+	);	
+	```
+
 ### Keywords
 
 CREATE, CATALOG
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java
index afdd099702..c02666d928 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/JdbcResource.java
@@ -70,7 +70,7 @@ public class JdbcResource extends Resource {
     public static final String MYSQL = "MYSQL";
     public static final String POSTGRESQL = "POSTGRESQL";
     public static final String ORACLE = "ORACLE";
-    private static final String SQLSERVER = "SQLSERVER";
+    public static final String SQLSERVER = "SQLSERVER";
     public static final String CLICKHOUSE = "CLICKHOUSE";
 
     public static final String JDBC_PROPERTIES_PREFIX = "jdbc.";
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcTable.java
index 562f4d7807..5372c56312 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcTable.java
@@ -88,7 +88,17 @@ public class OdbcTable extends Table {
     }
 
     private static String mssqlProperName(String name) {
-        return "[" + name + "]";
+        // In JdbcExternalTable, the name contains databaseName, like: db.table
+        // So, we should split db and table, then switch to [db].[table].
+        String[] fields = name.split("\\.");
+        String result = "";
+        for (int i = 0; i < fields.length; ++i) {
+            if (i != 0) {
+                result += ".";
+            }
+            result += ("[" + fields[i] + "]");
+        }
+        return result;
     }
 
     private static String psqlProperName(String name) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
index 50476d5a64..0f75b4755a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
@@ -23,6 +23,7 @@ import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
 
 import com.google.common.collect.Lists;
 import com.zaxxer.hikari.HikariConfig;
@@ -57,8 +58,11 @@ public class JdbcClient {
 
     public JdbcClient(String user, String password, String jdbcUrl, String driverUrl, String driverClass) {
         this.jdbcUser = user;
-        this.dbType = parseDbType(jdbcUrl);
-
+        try {
+            this.dbType = JdbcResource.parseDbType(jdbcUrl);
+        } catch (DdlException e) {
+            throw new JdbcClientException("Failed to parse db type from jdbcUrl: " + jdbcUrl, e);
+        }
         ClassLoader oldClassLoader = Thread.currentThread().getContextClassLoader();
         try {
             // TODO(ftw): The problem here is that the jar package is handled by FE
@@ -158,6 +162,9 @@ public class JdbcClient {
                 case JdbcResource.ORACLE:
                     rs = stmt.executeQuery("SELECT DISTINCT OWNER FROM all_tables");
                     break;
+                case JdbcResource.SQLSERVER:
+                    rs = stmt.executeQuery("SELECT name FROM sys.schemas");
+                    break;
                 default:
                     throw new JdbcClientException("Not supported jdbc type");
             }
@@ -190,6 +197,7 @@ public class JdbcClient {
                 case JdbcResource.POSTGRESQL:
                 case JdbcResource.ORACLE:
                 case JdbcResource.CLICKHOUSE:
+                case JdbcResource.SQLSERVER:
                     rs = databaseMetaData.getTables(null, dbName, null, types);
                     break;
                 default:
@@ -219,6 +227,7 @@ public class JdbcClient {
                 case JdbcResource.POSTGRESQL:
                 case JdbcResource.ORACLE:
                 case JdbcResource.CLICKHOUSE:
+                case JdbcResource.SQLSERVER:
                     rs = databaseMetaData.getTables(null, dbName, null, types);
                     break;
                 default:
@@ -289,6 +298,7 @@ public class JdbcClient {
                 case JdbcResource.POSTGRESQL:
                 case JdbcResource.ORACLE:
                 case JdbcResource.CLICKHOUSE:
+                case JdbcResource.SQLSERVER:
                     rs = databaseMetaData.getColumns(null, dbName, tableName, null);
                     break;
                 default:
@@ -325,6 +335,8 @@ public class JdbcClient {
                 return clickhouseTypeToDoris(fieldSchema);
             case JdbcResource.ORACLE:
                 return oracleTypeToDoris(fieldSchema);
+            case JdbcResource.SQLSERVER:
+                return sqlserverTypeToDoris(fieldSchema);
             default:
                 throw new JdbcClientException("Unknown database type");
         }
@@ -614,6 +626,52 @@ public class JdbcClient {
         }
     }
 
+    public Type sqlserverTypeToDoris(JdbcFieldSchema fieldSchema) {
+        String sqlserverType = fieldSchema.getDataTypeName();
+        switch (sqlserverType) {
+            case "bit":
+                return Type.BOOLEAN;
+            case "tinyint":
+            case "smallint":
+                return Type.SMALLINT;
+            case "int":
+                return Type.INT;
+            case "bigint":
+                return Type.BIGINT;
+            case "real":
+                return Type.FLOAT;
+            case "float":
+            case "money":
+            case "smallmoney":
+                return Type.DOUBLE;
+            case "decimal":
+            case "numeric":
+                int precision = fieldSchema.getColumnSize();
+                int scale = fieldSchema.getDecimalDigits();
+                return ScalarType.createDecimalV3Type(precision, scale);
+            case "date":
+                return ScalarType.getDefaultDateType(Type.DATE);
+            case "datetime":
+            case "datetime2":
+            case "smalldatetime":
+                return ScalarType.createDatetimeV2Type(6);
+            case "char":
+            case "varchar":
+            case "nchar":
+            case "nvarchar":
+            case "text":
+            case "ntext":
+            case "time":
+            case "datetimeoffset":
+                return ScalarType.createStringType();
+            case "image":
+            case "binary":
+            case "varbinary":
+            default:
+                return Type.UNSUPPORTED;
+        }
+    }
+
     public List<Column> getColumnsFromJdbc(String dbName, String tableName) {
         List<JdbcFieldSchema> jdbcTableSchema = getJdbcColumnsInfo(dbName, tableName);
         List<Column> dorisTableSchema = Lists.newArrayListWithCapacity(jdbcTableSchema.size());
@@ -625,20 +683,4 @@ public class JdbcClient {
         }
         return dorisTableSchema;
     }
-
-    private String parseDbType(String url) {
-        if (url.startsWith(JdbcResource.JDBC_MYSQL) || url.startsWith(JdbcResource.JDBC_MARIADB)) {
-            return JdbcResource.MYSQL;
-        } else if (url.startsWith(JdbcResource.JDBC_POSTGRESQL)) {
-            return JdbcResource.POSTGRESQL;
-        } else if (url.startsWith(JdbcResource.JDBC_ORACLE)) {
-            return JdbcResource.ORACLE;
-        } else if (url.startsWith(JdbcResource.JDBC_CLICKHOUSE)) {
-            return JdbcResource.CLICKHOUSE;
-        }
-        // else if (url.startsWith("jdbc:sqlserver")) {
-        //     return SQLSERVER;
-        // }
-        throw new JdbcClientException("Unsupported jdbc database type, please check jdbcUrl: " + url);
-    }
 }
diff --git a/regression-test/conf/regression-conf.groovy b/regression-test/conf/regression-conf.groovy
index 0b743fe9dd..544d4b12c1 100644
--- a/regression-test/conf/regression-conf.groovy
+++ b/regression-test/conf/regression-conf.groovy
@@ -76,6 +76,7 @@ enableJdbcTest=false
 mysql_57_port=3316
 pg_14_port=5442
 oracle_11_port=1521
+sqlserver_2022_port=1433
 
 // hive catalog test config
 // To enable jdbc test, you need first start hive container.
diff --git a/regression-test/data/jdbc_catalog_p0/test_sqlserver_jdbc_catalog.out b/regression-test/data/jdbc_catalog_p0/test_sqlserver_jdbc_catalog.out
new file mode 100644
index 0000000000..3768e8b9a6
--- /dev/null
+++ b/regression-test/data/jdbc_catalog_p0/test_sqlserver_jdbc_catalog.out
@@ -0,0 +1,43 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !test0 --
+1	doris	18
+2	alice	19
+3	bob	20
+
+-- !in_tb --
+1	doris	18
+2	alice	19
+3	bob	20
+
+-- !test1 --
+1	0	1	1
+2	1	-1	-1
+3	255	32767	9223372036854775807
+4	128	-32768	-9223372036854775808
+
+-- !test2 --
+1	123.123	123.123	123.123	123.12300	123.12300
+2	12345.123	12345.12345	12345.123	12345.12345	12345.12345
+3	-123.123	-123.123	-123.123	-123.12300	-123.12300
+
+-- !test3 --
+1	Make Doris Great!   	Make Doris Great!	Make Doris Great!	Make Doris Great!   	Make Doris Great!	Make Doris Great!
+
+-- !test5 --
+1	2023-01-17	16:49:05	2023-01-17T16:49:05	2023-01-17T16:49:05	2023-01-17T16:49	2023-01-17 16:49:05 +08:00
+2	2023-01-17	16:49:05	2023-01-17T16:49:05	2023-01-17T16:49:05	2023-01-17T16:49	2023-01-17 16:49:05 +08:00
+3	2023-01-17	16:49:05	2023-01-17T16:49:05	2023-01-17T16:49:05	2023-01-17T16:49	2023-01-17 16:49:05.1234567 +08:00
+4	2023-01-17	16:49:05	2023-01-17T16:49:05	2023-01-17T16:49:05	2023-01-17T16:49	2023-01-17 16:49:05 +08:00
+
+-- !test6 --
+1	9.223372036854776E14	214748.3647
+2	-9.223372036854776E14	-214748.3648
+3	123.123	123.123
+
+-- !test7 --
+1	12345678901234567890123456789012345678	12345678901234567890123456789012345678	1234567890123456789012345678.0123456789	1234567890123456789012345678.0123456789
+2	-12345678901234567890123456789012345678	-12345678901234567890123456789012345678	-1234567890123456789012345678.0123456789	-1234567890123456789012345678.0123456789
+
+-- !test8 --
+1	Make Doris Great!	Make Doris Great!
+
diff --git a/regression-test/suites/jdbc_catalog_p0/test_pg_jdbc_catalog.groovy b/regression-test/suites/jdbc_catalog_p0/test_pg_jdbc_catalog.groovy
index 629761bfb3..71462cc307 100644
--- a/regression-test/suites/jdbc_catalog_p0/test_pg_jdbc_catalog.groovy
+++ b/regression-test/suites/jdbc_catalog_p0/test_pg_jdbc_catalog.groovy
@@ -38,7 +38,7 @@ suite("test_pg_jdbc_catalog", "p0") {
             "driver_class" = "org.postgresql.Driver"
         );"""
 
-        sql """CREATE CATALOG ${catalog_name} WITH RESOURCE jdbc_resource_catalog_pg"""
+        sql """CREATE CATALOG ${catalog_name} WITH RESOURCE ${resource_name}"""
 
         sql  """ drop table if exists ${inDorisTable} """
         sql  """
@@ -74,7 +74,7 @@ suite("test_pg_jdbc_catalog", "p0") {
         order_qt_test14  """ select * from test12 order by id; """
 
         sql """drop catalog if exists ${catalog_name} """
-        sql """drop resource if exists jdbc_resource_catalog_pg"""
+        sql """drop resource if exists ${resource_name}"""
 
         // test old create-catalog syntax for compatibility
         sql """ CREATE CATALOG ${catalog_name} PROPERTIES (
@@ -86,9 +86,9 @@ suite("test_pg_jdbc_catalog", "p0") {
             "jdbc.driver_class" = "org.postgresql.Driver");
         """
 
-        sql """switch ${catalog_name}"""
-        sql """use ${ex_schema_name}"""
+        sql """ switch ${catalog_name} """
+        sql """ use ${ex_schema_name} """
         order_qt_test_old  """ select * from test3 order by id; """
-        sql """drop resource if exists jdbc_resource_catalog_pg"""
+        sql """ drop catalog if exists ${catalog_name} """
     }
 }
diff --git a/regression-test/suites/jdbc_catalog_p0/test_sqlserver_jdbc_catalog.groovy b/regression-test/suites/jdbc_catalog_p0/test_sqlserver_jdbc_catalog.groovy
new file mode 100644
index 0000000000..4301a80b29
--- /dev/null
+++ b/regression-test/suites/jdbc_catalog_p0/test_sqlserver_jdbc_catalog.groovy
@@ -0,0 +1,72 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_sqlserver_jdbc_catalog", "p0") {
+    String enabled = context.config.otherConfigs.get("enableJdbcTest");
+    if (enabled != null && enabled.equalsIgnoreCase("true")) {
+        String resource_name = "sqlserver_catalog_resource";
+        String catalog_name = "sqlserver_catalog";
+        String internal_db_name = "regression_test_jdbc_catalog_p0";
+        String ex_db_name = "dbo";
+        String sqlserver_port = context.config.otherConfigs.get("sqlserver_2022_port");
+
+        String inDorisTable = "doris_in_tb";
+
+        sql """ drop catalog if exists ${catalog_name} """
+        sql """ drop resource if exists ${resource_name} """
+
+        sql """ create resource if not exists ${resource_name} properties(
+                    "type"="jdbc",
+                    "user"="SA",
+                    "password"="Doris123456",
+                    "jdbc_url" = "jdbc:sqlserver://127.0.0.1:${sqlserver_port};DataBaseName=doris_test",
+                    "driver_url" = "https://doris-community-test-1308700295.cos.ap-hongkong.myqcloud.com/jdbc_driver/mssql-jdbc-11.2.3.jre8.jar",
+                    "driver_class" = "com.microsoft.sqlserver.jdbc.SQLServerDriver"
+        );"""
+
+        sql """ CREATE CATALOG ${catalog_name} WITH RESOURCE ${resource_name} """
+
+        sql  """ drop table if exists ${inDorisTable} """
+        sql  """
+              CREATE TABLE ${inDorisTable} (
+                `id` INT NULL COMMENT "主键id",
+                `name` string NULL COMMENT "名字",
+                `age` INT NULL COMMENT "年龄"
+                ) DISTRIBUTED BY HASH(id) BUCKETS 10
+                PROPERTIES("replication_num" = "1");
+        """
+
+        sql """ switch ${catalog_name} """
+        sql """ use ${ex_db_name} """
+
+        order_qt_test0  """ select * from student order by id; """
+        sql  """ insert into internal.${internal_db_name}.${inDorisTable} select * from student; """
+        order_qt_in_tb  """ select id, name, age from internal.${internal_db_name}.${inDorisTable} order by id; """
+
+        order_qt_test1  """ select * from test_int order by id; """
+        order_qt_test2  """ select * from test_float order by id; """
+        order_qt_test3  """ select * from test_char order by id; """
+        order_qt_test5  """ select * from test_time order by id; """
+        order_qt_test6  """ select * from test_money order by id; """
+        order_qt_test7  """ select * from test_decimal order by id; """
+        order_qt_test8  """ select * from test_text order by id; """
+
+
+        sql """ drop catalog if exists ${catalog_name} """
+        sql """ drop resource if exists ${resource_name} """
+    }
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 02/11: [Feature](multi-catalog)Add support for JuiceFS (#15969)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit c9e68b7481f26e28f8077ebc7fd35fa95db53019
Author: Liqf <10...@users.noreply.github.com>
AuthorDate: Thu Jan 19 08:54:16 2023 +0800

    [Feature](multi-catalog)Add support for JuiceFS (#15969)
    
    The broker implements the interface to juicefs,It supports loading data from juicefs to doris through broker.
    At the same time, it also implements the multi catalog to read the hive data stored in juicefs
---
 docs/en/docs/advanced/broker.md                    |  15 ++-
 docs/zh-CN/docs/advanced/broker.md                 |  13 ++
 .../java/org/apache/doris/analysis/BrokerDesc.java |   1 +
 .../java/org/apache/doris/analysis/ExportStmt.java |   5 +-
 .../org/apache/doris/analysis/StorageBackend.java  |   3 +
 .../java/org/apache/doris/backup/BlobStorage.java  |   6 +-
 .../doris/catalog/HiveMetaStoreClientHelper.java   |   3 +-
 .../java/org/apache/doris/common/FeConstants.java  |   1 +
 .../org/apache/doris/planner/BrokerScanNode.java   |   3 +-
 .../org/apache/doris/planner/HiveScanNode.java     |   2 +
 .../doris/planner/external/HiveScanProvider.java   |   4 +-
 .../doris/broker/hdfs/FileSystemManager.java       | 132 ++++++++++++++++++++-
 gensrc/thrift/Types.thrift                         |   1 +
 13 files changed, 179 insertions(+), 10 deletions(-)

diff --git a/docs/en/docs/advanced/broker.md b/docs/en/docs/advanced/broker.md
index dd7ac2cf86..69ebe7af65 100644
--- a/docs/en/docs/advanced/broker.md
+++ b/docs/en/docs/advanced/broker.md
@@ -32,7 +32,8 @@ Broker is an optional process in the Doris cluster. It is mainly used to support
 - Aliyun OSS
 - Tencent Cloud CHDFS
 - Huawei Cloud OBS (since 1.2.0)
-- Amazon S3
+- Amazon S3 
+- JuiceFS (since master)
 
 Broker provides services through an RPC service port. It is a stateless JVM process that is responsible for encapsulating some POSIX-like file operations for read and write operations on remote storage, such as open, pred, pwrite, and so on.
 In addition, the Broker does not record any other information, so the connection information, file information, permission information, and so on stored remotely need to be passed to the Broker process in the RPC call through parameters in order for the Broker to read and write files correctly .
@@ -234,3 +235,15 @@ Same as Apache HDFS
     "fs.s3a.endpoint" = "xx"
 )
 ```
+
+#### JuiceFS
+
+```
+(
+    "fs.defaultFS" = "jfs://xxx/",
+    "fs.jfs.impl" = "io.juicefs.JuiceFileSystem",
+    "fs.AbstractFileSystem.jfs.impl" = "io.juicefs.JuiceFS",
+    "juicefs.meta" = "xxx",
+    "juicefs.access-log" = "xxx"
+)
+```
\ No newline at end of file
diff --git a/docs/zh-CN/docs/advanced/broker.md b/docs/zh-CN/docs/advanced/broker.md
index 8b95a5f497..59a7f2b2ca 100644
--- a/docs/zh-CN/docs/advanced/broker.md
+++ b/docs/zh-CN/docs/advanced/broker.md
@@ -33,6 +33,7 @@ Broker 是 Doris 集群中一种可选进程,主要用于支持 Doris 读写
 - 腾讯云 CHDFS
 - 华为云 OBS (1.2.0 版本后支持)
 - 亚马逊 S3
+- JuiceFS (master 版本支持)
 
 Broker 通过提供一个 RPC 服务端口来提供服务,是一个无状态的 Java 进程,负责为远端存储的读写操作封装一些类 POSIX 的文件操作,如 open,pread,pwrite 等等。除此之外,Broker 不记录任何其他信息,所以包括远端存储的连接信息、文件信息、权限信息等等,都需要通过参数在 RPC 调用中传递给 Broker 进程,才能使得 Broker 能够正确读写文件。
 
@@ -227,3 +228,15 @@ WITH BROKER "broker_name"
     "fs.s3a.endpoint" = "xx"
 )
 ```
+
+#### JuiceFS
+
+```
+(
+    "fs.defaultFS" = "jfs://xxx/",
+    "fs.jfs.impl" = "io.juicefs.JuiceFileSystem",
+    "fs.AbstractFileSystem.jfs.impl" = "io.juicefs.JuiceFS",
+    "juicefs.meta" = "xxx",
+    "juicefs.access-log" = "xxx"
+)
+```
\ No newline at end of file
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerDesc.java
index bac9e80b08..8f1fc5dfe4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BrokerDesc.java
@@ -118,6 +118,7 @@ public class BrokerDesc extends StorageDesc implements Writable {
                 return TFileType.FILE_STREAM;
             case BROKER:
             case OFS:
+            case JFS:
             default:
                 return TFileType.FILE_BROKER;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java
index 62fb2c9977..eb35fe4c0d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ExportStmt.java
@@ -243,9 +243,10 @@ public class ExportStmt extends StatementBase {
                     && !schema.equalsIgnoreCase("obs")
                     && !schema.equalsIgnoreCase("oss")
                     && !schema.equalsIgnoreCase("s3a")
-                    && !schema.equalsIgnoreCase("cosn"))) {
+                    && !schema.equalsIgnoreCase("cosn")
+                    && !schema.equalsIgnoreCase("jfs"))) {
                 throw new AnalysisException("Invalid broker path. please use valid 'hdfs://', 'afs://' , 'bos://',"
-                        + " 'ofs://', 'obs://', 'oss://', 's3a://' or 'cosn://' path.");
+                        + " 'ofs://', 'obs://', 'oss://', 's3a://', 'cosn://' or 'jfs://' path.");
             }
         } else if (type == StorageBackend.StorageType.S3) {
             if (schema == null || !schema.equalsIgnoreCase("s3")) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageBackend.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageBackend.java
index dfdbd4f8a2..dc212a481b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageBackend.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/StorageBackend.java
@@ -115,6 +115,7 @@ public class StorageBackend extends StorageDesc implements ParseNode {
         HDFS("Hadoop Distributed File System"),
         LOCAL("Local file system"),
         OFS("Tencent CHDFS"),
+        JFS("Juicefs"),
         STREAM("Stream load pipe");
 
         private final String description;
@@ -136,6 +137,8 @@ public class StorageBackend extends StorageDesc implements ParseNode {
                     return TStorageBackendType.HDFS;
                 case OFS:
                     return TStorageBackendType.OFS;
+                case JFS:
+                    return TStorageBackendType.JFS;
                 case LOCAL:
                     return TStorageBackendType.LOCAL;
                 default:
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BlobStorage.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BlobStorage.java
index e4a3362490..be02be0690 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/BlobStorage.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BlobStorage.java
@@ -50,10 +50,12 @@ public abstract class BlobStorage implements Writable {
     public static BlobStorage create(String name, StorageBackend.StorageType type, Map<String, String> properties) {
         if (type == StorageBackend.StorageType.S3) {
             return new S3Storage(properties);
-        } else if (type == StorageBackend.StorageType.HDFS || type == StorageBackend.StorageType.OFS) {
+        } else if (type == StorageBackend.StorageType.HDFS
+                || type == StorageBackend.StorageType.OFS
+                || type == StorageBackend.StorageType.JFS) {
             BlobStorage storage = new HdfsStorage(properties);
             // as of ofs files, use hdfs storage, but it's type should be ofs
-            if (type == StorageBackend.StorageType.OFS) {
+            if (type == StorageBackend.StorageType.OFS || type == StorageBackend.StorageType.JFS) {
                 storage.setType(type);
                 storage.setName(type.name());
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveMetaStoreClientHelper.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveMetaStoreClientHelper.java
index fd7c27760b..adf5dccc0a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveMetaStoreClientHelper.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/HiveMetaStoreClientHelper.java
@@ -213,7 +213,8 @@ public class HiveMetaStoreClientHelper {
     private static String getAllFileStatus(List<TBrokerFileStatus> fileStatuses,
             List<RemoteIterator<LocatedFileStatus>> remoteIterators, BlobStorage storage) throws UserException {
         boolean needFullPath = storage.getStorageType() == StorageBackend.StorageType.S3
-                || storage.getStorageType() == StorageBackend.StorageType.OFS;
+                || storage.getStorageType() == StorageBackend.StorageType.OFS
+                || storage.getStorageType() == StorageBackend.StorageType.JFS;
         String hdfsUrl = "";
         Queue<RemoteIterator<LocatedFileStatus>> queue = Queues.newArrayDeque(remoteIterators);
         while (queue.peek() != null) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java b/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java
index bff471f039..dfde23e0a0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java
@@ -73,6 +73,7 @@ public class FeConstants {
     public static String FS_PREFIX_COS = "cos";
     public static String FS_PREFIX_OBS = "obs";
     public static String FS_PREFIX_OFS = "ofs";
+    public static String FS_PREFIX_JFS = "jfs";
     public static String FS_PREFIX_HDFS = "hdfs";
     public static String FS_PREFIX_FILE = "file";
     public static final String INTERNAL_DB_NAME = "__internal_schema";
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java
index 38284b097e..1f9ddee238 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java
@@ -326,7 +326,8 @@ public class BrokerScanNode extends LoadScanNode {
                 throw new UserException(e.getMessage());
             }
             brokerScanRange.addToBrokerAddresses(new TNetworkAddress(broker.ip, broker.port));
-        } else if (brokerDesc.getStorageType() == StorageBackend.StorageType.OFS) {
+        } else if (brokerDesc.getStorageType() == StorageBackend.StorageType.OFS
+                || brokerDesc.getStorageType() == StorageBackend.StorageType.JFS) {
             FsBroker broker = Env.getCurrentEnv().getBrokerMgr().getAnyAliveBroker();
             if (broker == null) {
                 throw new UserException("No alive broker.");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/HiveScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/HiveScanNode.java
index 121e2df54d..0b932eb976 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/HiveScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/HiveScanNode.java
@@ -135,6 +135,8 @@ public class HiveScanNode extends BrokerScanNode {
             this.storageType = StorageBackend.StorageType.HDFS;
         } else if (storagePrefix.equalsIgnoreCase(FeConstants.FS_PREFIX_OFS)) {
             this.storageType = StorageBackend.StorageType.OFS;
+        } else if (storagePrefix.equalsIgnoreCase(FeConstants.FS_PREFIX_JFS)) {
+            this.storageType = StorageBackend.StorageType.JFS;
         } else {
             throw new UserException("Not supported storage type: " + storagePrefix);
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanProvider.java b/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanProvider.java
index 340597f6da..daf238ee8c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanProvider.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/external/HiveScanProvider.java
@@ -124,7 +124,9 @@ public class HiveScanProvider extends HMSTableScanProvider {
                 return TFileType.FILE_HDFS;
             } else if (location.startsWith(FeConstants.FS_PREFIX_FILE)) {
                 return TFileType.FILE_LOCAL;
-            }  else if (location.startsWith(FeConstants.FS_PREFIX_OFS)) {
+            } else if (location.startsWith(FeConstants.FS_PREFIX_OFS)) {
+                return TFileType.FILE_BROKER;
+            } else if (location.startsWith(FeConstants.FS_PREFIX_JFS)) {
                 return TFileType.FILE_BROKER;
             }
         }
diff --git a/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java b/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java
index 27b548af8c..f66872a10a 100644
--- a/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java
+++ b/fs_brokers/apache_hdfs_broker/src/main/java/org/apache/doris/broker/hdfs/FileSystemManager.java
@@ -73,6 +73,7 @@ public class FileSystemManager {
     private static final String OSS_SCHEME = "oss";
     private static final String COS_SCHEME = "cosn";
     private static final String BOS_SCHEME = "bos";
+    private static final String JFS_SCHEME = "jfs";
     private static final String AFS_SCHEME = "afs";
 
     private static final String USER_NAME_KEY = "username";
@@ -142,7 +143,6 @@ public class FileSystemManager {
     private static final String FS_BOS_IMPL = "fs.bos.impl";
     private static final String FS_BOS_MULTIPART_UPLOADS_BLOCK_SIZE = "fs.bos.multipart.uploads.block.size";
 
-
     // arguments for afs
     private static final String HADOOP_JOB_GROUP_NAME = "hadoop.job.group.name";
     private static final String HADOOP_JOB_UGI = "hadoop.job.ugi";
@@ -219,7 +219,9 @@ public class FileSystemManager {
             brokerFileSystem = getCOSFileSystem(path, properties);
         } else if (scheme.equals(BOS_SCHEME)) {
             brokerFileSystem = getBOSFileSystem(path, properties);
-        } else {
+        } else if (scheme.equals(JFS_SCHEME)) {
+            brokerFileSystem = getJuiceFileSystem(path, properties);
+        }else {
             throw new BrokerException(TBrokerOperationStatusCode.INVALID_INPUT_FILE_PATH,
                 "invalid path. scheme is not supported");
         }
@@ -805,6 +807,132 @@ public class FileSystemManager {
         }
     }
 
+    /**
+     * visible for test
+     *
+     * file system handle is cached, the identity is for all juicefs.
+     * @param path
+     * @param properties
+     * @return
+     * @throws URISyntaxException
+     * @throws Exception
+     */
+    public BrokerFileSystem getJuiceFileSystem(String path, Map<String, String> properties) {
+        WildcardURI pathUri = new WildcardURI(path);
+        String host = JFS_SCHEME;
+        if (Strings.isNullOrEmpty(pathUri.getAuthority())) {
+            if (properties.containsKey(FS_DEFAULTFS_KEY)) {
+                host = properties.get(FS_DEFAULTFS_KEY);
+                logger.info("no schema and authority in path. use fs.defaultFs");
+            } else {
+                logger.warn("invalid jfs path. authority is null,path:" + path);
+                throw  new BrokerException(TBrokerOperationStatusCode.INVALID_ARGUMENT,
+                    "invalid jfs path. authority is null");
+            }
+        }
+        String authentication = properties.getOrDefault(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+            AUTHENTICATION_SIMPLE);
+        if (Strings.isNullOrEmpty(authentication) || (!authentication.equals(AUTHENTICATION_SIMPLE)
+            && !authentication.equals(AUTHENTICATION_KERBEROS))) {
+            logger.warn("invalid authentication:" + authentication);
+            throw new BrokerException(TBrokerOperationStatusCode.INVALID_ARGUMENT,
+                "invalid authentication:" + authentication);
+        }
+
+        FileSystemIdentity fileSystemIdentity = null;
+        if (authentication.equals(AUTHENTICATION_SIMPLE)) {
+            fileSystemIdentity = new FileSystemIdentity(host, "");
+        } else {
+            // for kerberos, use host + principal + keytab as filesystemindentity
+            String kerberosContent = "";
+            if (properties.containsKey(KERBEROS_KEYTAB)) {
+                kerberosContent = properties.get(KERBEROS_KEYTAB);
+            } else if (properties.containsKey(KERBEROS_KEYTAB_CONTENT)) {
+                kerberosContent = properties.get(KERBEROS_KEYTAB_CONTENT);
+            } else {
+                throw new BrokerException(TBrokerOperationStatusCode.INVALID_ARGUMENT,
+                    "keytab is required for kerberos authentication");
+            }
+            if (!properties.containsKey(KERBEROS_PRINCIPAL)) {
+                throw new BrokerException(TBrokerOperationStatusCode.INVALID_ARGUMENT,
+                    "principal is required for kerberos authentication");
+            } else {
+                kerberosContent = kerberosContent + properties.get(KERBEROS_PRINCIPAL);
+            }
+            try {
+                MessageDigest digest = MessageDigest.getInstance("md5");
+                byte[] result = digest.digest(kerberosContent.getBytes());
+                String kerberosUgi = new String(result);
+                fileSystemIdentity = new FileSystemIdentity(host, kerberosUgi);
+            } catch (NoSuchAlgorithmException e) {
+                throw  new BrokerException(TBrokerOperationStatusCode.INVALID_ARGUMENT,
+                    e.getMessage());
+            }
+        }
+        BrokerFileSystem fileSystem = updateCachedFileSystem(fileSystemIdentity, properties);
+        fileSystem.getLock().lock();
+        try {
+            // create a new filesystem
+            Configuration conf = new Configuration();
+            for (Map.Entry<String, String> propElement : properties.entrySet()) {
+                conf.set(propElement.getKey(), propElement.getValue());
+            }
+
+            if (fileSystem.getDFSFileSystem() == null) {
+                logger.info("create file system for new path " + path);
+                String tmpFilePath = null;
+                if (authentication.equals(AUTHENTICATION_KERBEROS)){
+                    conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+                        AUTHENTICATION_KERBEROS);
+
+                    String principal = preparePrincipal(properties.get(KERBEROS_PRINCIPAL));
+                    String keytab = "";
+                    if (properties.containsKey(KERBEROS_KEYTAB)) {
+                        keytab = properties.get(KERBEROS_KEYTAB);
+                    } else if (properties.containsKey(KERBEROS_KEYTAB_CONTENT)) {
+                        // pass kerberos keytab content use base64 encoding
+                        // so decode it and write it to tmp path under /tmp
+                        // because ugi api only accept a local path as argument
+                        String keytabContent = properties.get(KERBEROS_KEYTAB_CONTENT);
+                        byte[] base64decodedBytes = Base64.getDecoder().decode(keytabContent);
+                        long currentTime = System.currentTimeMillis();
+                        Random random = new Random(currentTime);
+                        int randNumber = random.nextInt(10000);
+                        tmpFilePath = "/tmp/." + Long.toString(currentTime) + "_" + Integer.toString(randNumber);
+                        FileOutputStream fileOutputStream = new FileOutputStream(tmpFilePath);
+                        fileOutputStream.write(base64decodedBytes);
+                        fileOutputStream.close();
+                        keytab = tmpFilePath;
+                    } else {
+                        throw  new BrokerException(TBrokerOperationStatusCode.INVALID_ARGUMENT,
+                            "keytab is required for kerberos authentication");
+                    }
+                    UserGroupInformation.setConfiguration(conf);
+                    UserGroupInformation.loginUserFromKeytab(principal, keytab);
+                    if (properties.containsKey(KERBEROS_KEYTAB_CONTENT)) {
+                        try {
+                            File file = new File(tmpFilePath);
+                            if(!file.delete()){
+                                logger.warn("delete tmp file:" +  tmpFilePath + " failed");
+                            }
+                        } catch (Exception e) {
+                            throw new  BrokerException(TBrokerOperationStatusCode.FILE_NOT_FOUND,
+                                e.getMessage());
+                        }
+                    }
+                }
+                FileSystem jfsFileSystem = FileSystem.get(pathUri.getUri(), conf);
+                fileSystem.setFileSystem(jfsFileSystem);
+            }
+            return fileSystem;
+        } catch (Exception e) {
+            logger.error("errors while connect to " + path, e);
+            throw new BrokerException(TBrokerOperationStatusCode.NOT_AUTHORIZED, e);
+        } finally {
+            fileSystem.getLock().unlock();
+        }
+    }
+
     private BrokerFileSystem getAfsFileSystem(String path, Map<String, String> properties) {
         URI pathUri = new WildcardURI(path).getUri();
         String host = pathUri.getScheme() + "://" + pathUri.getAuthority();
diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift
index 589b95f77c..ba3d28e1a2 100644
--- a/gensrc/thrift/Types.thrift
+++ b/gensrc/thrift/Types.thrift
@@ -105,6 +105,7 @@ enum TStorageBackendType {
     BROKER,
     S3,
     HDFS,
+    JFS,
     LOCAL,
     OFS
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 07/11: [fix](multi-catalog)Make ES catalog and resource compatible (#16096)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 88096e09932a471994b3627114d5f2f2d814d02f
Author: qiye <ji...@gmail.com>
AuthorDate: Fri Jan 20 09:31:57 2023 +0800

    [fix](multi-catalog)Make ES catalog and resource compatible (#16096)
    
    close #16099
    
    1. Make ES resource compatible with `username` property. Keep the same behavior with ES catalog.
    2. Change ES catalog `username` to `user` to avoid confusion.
    3. Add log in ESRestClient and make debug easier.
---
 .../main/java/org/apache/doris/catalog/EsResource.java    | 15 ++++++++++++---
 .../apache/doris/external/elasticsearch/EsRestClient.java |  2 ++
 2 files changed, 14 insertions(+), 3 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/EsResource.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/EsResource.java
index 513093d0ae..012ad119e3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/EsResource.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/EsResource.java
@@ -87,7 +87,7 @@ public class EsResource extends Resource {
     @Override
     protected void setProperties(Map<String, String> properties) throws DdlException {
         valid(properties, false);
-        this.properties = properties;
+        this.properties = processCompatibleProperties(properties);
     }
 
     public static void valid(Map<String, String> properties, boolean isAlter) throws DdlException {
@@ -129,15 +129,24 @@ public class EsResource extends Resource {
         }
     }
 
+    private Map<String, String> processCompatibleProperties(Map<String, String> props) {
+        // Compatible with ES catalog properties
+        Map<String, String> properties = Maps.newHashMap(props);
+        if (properties.containsKey("username")) {
+            properties.put(EsResource.USER, properties.remove("username"));
+        }
+        return properties;
+    }
+
     @Override
     public Map<String, String> getCopiedProperties() {
-        return Maps.newHashMap(properties);
+        return Maps.newHashMap(processCompatibleProperties(properties));
     }
 
     @Override
     protected void getProcNodeData(BaseProcResult result) {
         String lowerCaseType = type.name().toLowerCase();
-        for (Map.Entry<String, String> entry : properties.entrySet()) {
+        for (Map.Entry<String, String> entry : processCompatibleProperties(properties).entrySet()) {
             result.addRow(Lists.newArrayList(name, lowerCaseType, entry.getKey(), entry.getValue()));
         }
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRestClient.java b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRestClient.java
index 1627895936..c5618b8732 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRestClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsRestClient.java
@@ -258,6 +258,8 @@ public class EsRestClient {
             try (Response response = executeResponse(httpClient, path)) {
                 if (response.isSuccessful()) {
                     return response.body().string();
+                } else {
+                    LOG.warn("request response code: {}, body: {}", response.code(), response.body().string());
                 }
             } catch (IOException e) {
                 LOG.warn("request node [{}] [{}] failures {}, try next nodes", currentNode, path, e);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 10/11: [Fix](Oracle External Table) fix that oracle external table can not insert batch values (#16117)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit c32a372b3cb8ddfb0481133b51f7ef19ceab4d28
Author: Tiewei Fang <43...@users.noreply.github.com>
AuthorDate: Sat Jan 21 07:57:12 2023 +0800

    [Fix](Oracle External Table) fix that oracle external table can not insert batch values (#16117)
    
    Issue Number: close #xxx
    
    This pr fix two bugs:
    
    _jdbc_scanner may be nullptr in vjdbc_connector.cpp, so we use another method to count jdbc statistic. close [Enhencement](jdbc scanner) add profile for jdbc scanner #15914
    In the batch insertion scenario, oracle database does not support syntax insert into tables values (...),(...); , what it supports is:
    insert all
    into table(col1,col2) values(c1v1, c2v1)
    into table(col1,col2) values(c1v2, c2v2)
    SELECT 1 FROM DUAL;
---
 be/src/exec/table_connector.cpp                    | 41 +++++++++++++++++++++-
 be/src/exec/table_connector.h                      |  9 +++++
 be/src/vec/exec/scan/new_jdbc_scanner.cpp          | 15 +++++++-
 be/src/vec/exec/scan/new_jdbc_scanner.h            |  2 ++
 be/src/vec/exec/vjdbc_connector.cpp                | 20 ++++-------
 be/src/vec/exec/vjdbc_connector.h                  | 16 +++++++--
 .../java/org/apache/doris/udf/JdbcExecutor.java    |  1 -
 7 files changed, 85 insertions(+), 19 deletions(-)

diff --git a/be/src/exec/table_connector.cpp b/be/src/exec/table_connector.cpp
index 0e07edd48d..f2c3ff8101 100644
--- a/be/src/exec/table_connector.cpp
+++ b/be/src/exec/table_connector.cpp
@@ -166,7 +166,14 @@ Status TableConnector::append(const std::string& table_name, vectorized::Block*
                               TOdbcTableType::type table_type) {
     _insert_stmt_buffer.clear();
     std::u16string insert_stmt;
-    {
+    if (table_type == TOdbcTableType::ORACLE) {
+        SCOPED_TIMER(_convert_tuple_timer);
+        oracle_type_append(table_name, block, output_vexpr_ctxs, start_send_row, num_rows_sent,
+                           table_type);
+        // Translate utf8 string to utf16 to use unicode encoding
+        insert_stmt = utf8_to_u16string(_insert_stmt_buffer.data(),
+                                        _insert_stmt_buffer.data() + _insert_stmt_buffer.size());
+    } else {
         SCOPED_TIMER(_convert_tuple_timer);
         fmt::format_to(_insert_stmt_buffer, "INSERT INTO {} VALUES (", table_name);
 
@@ -203,6 +210,38 @@ Status TableConnector::append(const std::string& table_name, vectorized::Block*
     return Status::OK();
 }
 
+Status TableConnector::oracle_type_append(
+        const std::string& table_name, vectorized::Block* block,
+        const std::vector<vectorized::VExprContext*>& output_vexpr_ctxs, uint32_t start_send_row,
+        uint32_t* num_rows_sent, TOdbcTableType::type table_type) {
+    fmt::format_to(_insert_stmt_buffer, "INSERT ALL ");
+    int num_rows = block->rows();
+    int num_columns = block->columns();
+    for (int i = start_send_row; i < num_rows; ++i) {
+        (*num_rows_sent)++;
+        fmt::format_to(_insert_stmt_buffer, "INTO {} VALUES (", table_name);
+        // Construct insert statement of odbc/jdbc table
+        for (int j = 0; j < num_columns; ++j) {
+            if (j != 0) {
+                fmt::format_to(_insert_stmt_buffer, "{}", ", ");
+            }
+            auto& column_ptr = block->get_by_position(j).column;
+            auto& type_ptr = block->get_by_position(j).type;
+            RETURN_IF_ERROR(convert_column_data(
+                    column_ptr, type_ptr, output_vexpr_ctxs[j]->root()->type(), i, table_type));
+        }
+
+        if (i < num_rows - 1 && _insert_stmt_buffer.size() < INSERT_BUFFER_SIZE) {
+            fmt::format_to(_insert_stmt_buffer, "{}", ") ");
+        } else {
+            // batch exhausted or _insert_stmt_buffer is full, need to do real insert stmt
+            fmt::format_to(_insert_stmt_buffer, "{}", ") SELECT 1 FROM DUAL");
+            break;
+        }
+    }
+    return Status::OK();
+}
+
 Status TableConnector::convert_column_data(const vectorized::ColumnPtr& column_ptr,
                                            const vectorized::DataTypePtr& type_ptr,
                                            const TypeDescriptor& type, int row,
diff --git a/be/src/exec/table_connector.h b/be/src/exec/table_connector.h
index f051776b49..3fe0ec5721 100644
--- a/be/src/exec/table_connector.h
+++ b/be/src/exec/table_connector.h
@@ -87,6 +87,15 @@ protected:
     RuntimeProfile::Counter* _result_send_timer = nullptr;
     // number of sent rows
     RuntimeProfile::Counter* _sent_rows_counter = nullptr;
+
+private:
+    // Because Oracle database do not support
+    // insert into tables values (...),(...);
+    // Here we do something special for Oracle.
+    Status oracle_type_append(const std::string& table_name, vectorized::Block* block,
+                              const std::vector<vectorized::VExprContext*>& output_vexpr_ctxs,
+                              uint32_t start_send_row, uint32_t* num_rows_sent,
+                              TOdbcTableType::type table_type);
 };
 
 } // namespace doris
diff --git a/be/src/vec/exec/scan/new_jdbc_scanner.cpp b/be/src/vec/exec/scan/new_jdbc_scanner.cpp
index 80fc3669c8..0091efc7cd 100644
--- a/be/src/vec/exec/scan/new_jdbc_scanner.cpp
+++ b/be/src/vec/exec/scan/new_jdbc_scanner.cpp
@@ -18,6 +18,7 @@
 #include "new_jdbc_scanner.h"
 
 #include "util/runtime_profile.h"
+#include "vec/exec/vjdbc_connector.h"
 
 namespace doris::vectorized {
 NewJdbcScanner::NewJdbcScanner(RuntimeState* state, NewJdbcScanNode* parent, int64_t limit,
@@ -76,7 +77,7 @@ Status NewJdbcScanner::prepare(RuntimeState* state, VExprContext** vconjunct_ctx
     _jdbc_param.query_string = std::move(_query_string);
     _jdbc_param.table_type = _table_type;
 
-    _jdbc_connector.reset(new (std::nothrow) JdbcConnector(this, _jdbc_param));
+    _jdbc_connector.reset(new (std::nothrow) JdbcConnector(_jdbc_param));
     if (_jdbc_connector == nullptr) {
         return Status::InternalError("new a jdbc scanner failed.");
     }
@@ -113,6 +114,7 @@ Status NewJdbcScanner::_get_block_impl(RuntimeState* state, Block* block, bool*
 
     if (_jdbc_eos == true) {
         *eof = true;
+        _update_profile();
         return Status::OK();
     }
 
@@ -138,6 +140,7 @@ Status NewJdbcScanner::_get_block_impl(RuntimeState* state, Block* block, bool*
 
         if (_jdbc_eos == true) {
             if (block->rows() == 0) {
+                _update_profile();
                 *eof = true;
             }
             break;
@@ -160,6 +163,16 @@ Status NewJdbcScanner::_get_block_impl(RuntimeState* state, Block* block, bool*
     return Status::OK();
 }
 
+void NewJdbcScanner::_update_profile() {
+    JdbcConnector::JdbcStatistic& jdbc_statistic = _jdbc_connector->get_jdbc_statistic();
+    COUNTER_UPDATE(_load_jar_timer, jdbc_statistic._load_jar_timer);
+    COUNTER_UPDATE(_init_connector_timer, jdbc_statistic._init_connector_timer);
+    COUNTER_UPDATE(_check_type_timer, jdbc_statistic._check_type_timer);
+    COUNTER_UPDATE(_get_data_timer, jdbc_statistic._get_data_timer);
+    COUNTER_UPDATE(_execte_read_timer, jdbc_statistic._execte_read_timer);
+    COUNTER_UPDATE(_connector_close_timer, jdbc_statistic._connector_close_timer);
+}
+
 Status NewJdbcScanner::close(RuntimeState* state) {
     RETURN_IF_ERROR(VScanner::close(state));
     RETURN_IF_ERROR(_jdbc_connector->close());
diff --git a/be/src/vec/exec/scan/new_jdbc_scanner.h b/be/src/vec/exec/scan/new_jdbc_scanner.h
index 4f869d0f41..da5a3e7faf 100644
--- a/be/src/vec/exec/scan/new_jdbc_scanner.h
+++ b/be/src/vec/exec/scan/new_jdbc_scanner.h
@@ -48,6 +48,8 @@ protected:
     RuntimeProfile::Counter* _connector_close_timer = nullptr;
 
 private:
+    void _update_profile();
+
     bool _is_init;
 
     bool _jdbc_eos;
diff --git a/be/src/vec/exec/vjdbc_connector.cpp b/be/src/vec/exec/vjdbc_connector.cpp
index 5752e7ab19..519ccbbc77 100644
--- a/be/src/vec/exec/vjdbc_connector.cpp
+++ b/be/src/vec/exec/vjdbc_connector.cpp
@@ -53,12 +53,6 @@ JdbcConnector::JdbcConnector(const JdbcConnectorParam& param)
           _conn_param(param),
           _closed(false) {}
 
-JdbcConnector::JdbcConnector(NewJdbcScanner* jdbc_scanner, const JdbcConnectorParam& param)
-        : TableConnector(param.tuple_desc, param.query_string),
-          _jdbc_scanner(jdbc_scanner),
-          _conn_param(param),
-          _closed(false) {}
-
 JdbcConnector::~JdbcConnector() {
     if (!_closed) {
         close();
@@ -71,7 +65,7 @@ JdbcConnector::~JdbcConnector() {
 #define DELETE_BASIC_JAVA_CLAZZ_REF(CPP_TYPE) env->DeleteGlobalRef(_executor_##CPP_TYPE##_clazz);
 
 Status JdbcConnector::close() {
-    SCOPED_TIMER(_jdbc_scanner->_connector_close_timer);
+    SCOPED_RAW_TIMER(&_jdbc_statistic._connector_close_timer);
     _closed = true;
     if (!_is_open) {
         return Status::OK();
@@ -132,12 +126,12 @@ Status JdbcConnector::open(RuntimeState* state, bool read) {
         if (_conn_param.resource_name.empty()) {
             // for jdbcExternalTable, _conn_param.resource_name == ""
             // so, we use _conn_param.driver_path as key of jarpath
-            SCOPED_TIMER(_jdbc_scanner->_load_jar_timer);
+            SCOPED_RAW_TIMER(&_jdbc_statistic._load_jar_timer);
             RETURN_IF_ERROR(function_cache->get_jarpath(
                     std::abs((int64_t)hash_str(_conn_param.driver_path)), _conn_param.driver_path,
                     _conn_param.driver_checksum, &local_location));
         } else {
-            SCOPED_TIMER(_jdbc_scanner->_load_jar_timer);
+            SCOPED_RAW_TIMER(&_jdbc_statistic._load_jar_timer);
             RETURN_IF_ERROR(function_cache->get_jarpath(
                     std::abs((int64_t)hash_str(_conn_param.resource_name)), _conn_param.driver_path,
                     _conn_param.driver_checksum, &local_location));
@@ -158,7 +152,7 @@ Status JdbcConnector::open(RuntimeState* state, bool read) {
         RETURN_IF_ERROR(jni_frame.push(env));
         RETURN_IF_ERROR(SerializeThriftMsg(env, &ctor_params, &ctor_params_bytes));
         {
-            SCOPED_TIMER(_jdbc_scanner->_init_connector_timer);
+            SCOPED_RAW_TIMER(&_jdbc_statistic._init_connector_timer);
             _executor_obj = env->NewObject(_executor_clazz, _executor_ctor_id, ctor_params_bytes);
         }
         jbyte* pBytes = env->GetByteArrayElements(ctor_params_bytes, nullptr);
@@ -186,7 +180,7 @@ Status JdbcConnector::query() {
     JNIEnv* env = nullptr;
     RETURN_IF_ERROR(JniUtil::GetJNIEnv(&env));
     {
-        SCOPED_TIMER(_jdbc_scanner->_execte_read_timer);
+        SCOPED_RAW_TIMER(&_jdbc_statistic._execte_read_timer);
         jint colunm_count =
                 env->CallNonvirtualIntMethod(_executor_obj, _executor_clazz, _executor_read_id);
         RETURN_IF_ERROR(JniUtil::GetJniExceptionMsg(env));
@@ -201,7 +195,7 @@ Status JdbcConnector::query() {
 }
 
 Status JdbcConnector::_check_column_type() {
-    SCOPED_TIMER(_jdbc_scanner->_check_type_timer);
+    SCOPED_RAW_TIMER(&_jdbc_statistic._check_type_timer);
     JNIEnv* env = nullptr;
     RETURN_IF_ERROR(JniUtil::GetJNIEnv(&env));
     jobject type_lists =
@@ -350,7 +344,7 @@ Status JdbcConnector::get_next(bool* eos, std::vector<MutableColumnPtr>& columns
     if (!_is_open) {
         return Status::InternalError("get_next before open of jdbc connector.");
     }
-    SCOPED_TIMER(_jdbc_scanner->_get_data_timer);
+    SCOPED_RAW_TIMER(&_jdbc_statistic._get_data_timer);
     JNIEnv* env = nullptr;
     RETURN_IF_ERROR(JniUtil::GetJNIEnv(&env));
     jboolean has_next =
diff --git a/be/src/vec/exec/vjdbc_connector.h b/be/src/vec/exec/vjdbc_connector.h
index ee99be8ec5..c1d416783c 100644
--- a/be/src/vec/exec/vjdbc_connector.h
+++ b/be/src/vec/exec/vjdbc_connector.h
@@ -47,9 +47,16 @@ struct JdbcConnectorParam {
 
 class JdbcConnector : public TableConnector {
 public:
-    JdbcConnector(const JdbcConnectorParam& param);
+    struct JdbcStatistic {
+        int64_t _load_jar_timer = 0;
+        int64_t _init_connector_timer = 0;
+        int64_t _get_data_timer = 0;
+        int64_t _check_type_timer = 0;
+        int64_t _execte_read_timer = 0;
+        int64_t _connector_close_timer = 0;
+    };
 
-    JdbcConnector(NewJdbcScanner* jdbc_scanner, const JdbcConnectorParam& param);
+    JdbcConnector(const JdbcConnectorParam& param);
 
     ~JdbcConnector() override;
 
@@ -68,6 +75,8 @@ public:
     Status abort_trans() override; // should be call after transaction abort
     Status finish_trans() override; // should be call after transaction commit
 
+    JdbcStatistic& get_jdbc_statistic() { return _jdbc_statistic; }
+
     Status close() override;
 
 private:
@@ -89,7 +98,6 @@ private:
     Status _cast_string_to_array(const SlotDescriptor* slot_desc, Block* block, int column_index,
                                  int rows);
 
-    NewJdbcScanner* _jdbc_scanner;
     const JdbcConnectorParam& _conn_param;
     //java.sql.Types: https://docs.oracle.com/javase/7/docs/api/constant-values.html#java.sql.Types.INTEGER
     std::map<int, PrimitiveType> _arr_jdbc_map {
@@ -126,6 +134,8 @@ private:
     std::vector<MutableColumnPtr>
             str_array_cols; // for array type to save data like big string [1,2,3]
 
+    JdbcStatistic _jdbc_statistic;
+
 #define FUNC_VARI_DECLARE(RETURN_TYPE)                                \
     RETURN_TYPE _jobject_to_##RETURN_TYPE(JNIEnv* env, jobject jobj); \
     jclass _executor_##RETURN_TYPE##_clazz;
diff --git a/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java b/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
index 0c9894ad18..d90aa4055a 100644
--- a/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
+++ b/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
@@ -272,4 +272,3 @@ public class JdbcExecutor {
         }
     }
 }
-


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 08/11: [fix](oracle catalog) oracle catalog support `TIMESTAMP` dateType of oracle (#16113)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit e35ef74f72b0a1fbbcb8267f2859157b50b36808
Author: Tiewei Fang <43...@users.noreply.github.com>
AuthorDate: Fri Jan 20 14:47:58 2023 +0800

    [fix](oracle catalog) oracle catalog support `TIMESTAMP` dateType of oracle (#16113)
    
    `TIMESTAMP` dateType of Oracle will map to `DateTime` dateType of Doris
---
 be/src/vec/exec/vjdbc_connector.cpp                      |  5 ++++-
 .../docker-compose/oracle/init/03-create-table.sql       | 10 ++++++++++
 .../docker-compose/oracle/init/04-insert.sql             |  8 ++++++++
 .../java/org/apache/doris/external/jdbc/JdbcClient.java  |  5 +++++
 fe/java-udf/pom.xml                                      |  6 ++++++
 .../src/main/java/org/apache/doris/udf/JdbcExecutor.java | 16 +++++++++++-----
 .../data/jdbc_catalog_p0/test_oracle_jdbc_catalog.out    | 12 +++++++++++-
 .../jdbc_catalog_p0/test_oracle_jdbc_catalog.groovy      |  1 +
 8 files changed, 56 insertions(+), 7 deletions(-)

diff --git a/be/src/vec/exec/vjdbc_connector.cpp b/be/src/vec/exec/vjdbc_connector.cpp
index b7c2b388be..5752e7ab19 100644
--- a/be/src/vec/exec/vjdbc_connector.cpp
+++ b/be/src/vec/exec/vjdbc_connector.cpp
@@ -299,7 +299,8 @@ Status JdbcConnector::_check_type(SlotDescriptor* slot_desc, const std::string&
     case TYPE_DATETIME:
     case TYPE_DATETIMEV2: {
         if (type_str != "java.sql.Timestamp" && type_str != "java.time.LocalDateTime" &&
-            type_str != "java.sql.Date" && type_str != "java.time.LocalDate") {
+            type_str != "java.sql.Date" && type_str != "java.time.LocalDate" &&
+            type_str != "oracle.sql.TIMESTAMP") {
             return Status::InternalError(error_msg);
         }
         break;
@@ -525,11 +526,13 @@ Status JdbcConnector::_insert_column_data(JNIEnv* env, jobject jobj, const TypeD
     }
     case TYPE_DATETIME: {
         int64_t num = _jobject_to_datetime(env, jobj, false);
+        RETURN_IF_ERROR(JniUtil::GetJniExceptionMsg(env));
         reinterpret_cast<vectorized::ColumnVector<vectorized::Int64>*>(col_ptr)->insert_value(num);
         break;
     }
     case TYPE_DATETIMEV2: {
         int64_t num = _jobject_to_datetime(env, jobj, true);
+        RETURN_IF_ERROR(JniUtil::GetJniExceptionMsg(env));
         uint64_t num2 = static_cast<uint64_t>(num);
         reinterpret_cast<vectorized::ColumnVector<vectorized::UInt64>*>(col_ptr)->insert_value(
                 num2);
diff --git a/docker/thirdparties/docker-compose/oracle/init/03-create-table.sql b/docker/thirdparties/docker-compose/oracle/init/03-create-table.sql
index efd648dff6..d5dd8cf1c6 100644
--- a/docker/thirdparties/docker-compose/oracle/init/03-create-table.sql
+++ b/docker/thirdparties/docker-compose/oracle/init/03-create-table.sql
@@ -68,3 +68,13 @@ t1 date,
 t2 interval year(3) to month,
 t3 interval day(3) to second(6)
 );
+
+create table doris_test.test_timestamp(
+id int,
+t1 date,
+t2 timestamp(6),
+t3 timestamp(9),
+t4 timestamp,
+t5 interval year(3) to month,
+t6 interval day(3) to second(6)
+);
diff --git a/docker/thirdparties/docker-compose/oracle/init/04-insert.sql b/docker/thirdparties/docker-compose/oracle/init/04-insert.sql
index fd6ea2a57c..0c474e8f20 100644
--- a/docker/thirdparties/docker-compose/oracle/init/04-insert.sql
+++ b/docker/thirdparties/docker-compose/oracle/init/04-insert.sql
@@ -45,4 +45,12 @@ insert into doris_test.test_date (id, t2) values (3, interval '11' year);
 insert into doris_test.test_date (id, t2) values (4, interval '223-9' year(3) to month);
 insert into doris_test.test_date (id, t3) values (5, interval '12 10:23:01.1234568' day to second);
 
+insert into doris_test.test_timestamp (id, t1) values (1, to_date('2013-1-21 5:23:01','yyyy-mm-dd hh24:mi:ss'));
+insert into doris_test.test_timestamp (id, t1) values (2, to_date('20131112203256', 'yyyymmddhh24miss'));
+insert into doris_test.test_timestamp (id, t2) values (3, to_timestamp('20191112203357.999997623', 'yyyymmddhh24miss.ff'));
+insert into doris_test.test_timestamp (id, t3) values (4, to_timestamp_tz('20191112203357.999996623', 'yyyymmddhh24miss.ff'));
+insert into doris_test.test_timestamp (id, t4) values (5, to_timestamp_tz('20191112203357.999996623', 'yyyymmddhh24miss.ff'));
+insert into doris_test.test_timestamp (id, t5) values (6, interval '11' year);
+insert into doris_test.test_timestamp (id, t5) values (7, interval '223-9' year(3) to month);
+insert into doris_test.test_timestamp (id, t6) values (8, interval '12 10:23:01.1234568' day to second);
 commit;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
index 6f420246c5..50476d5a64 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/jdbc/JdbcClient.java
@@ -558,6 +558,11 @@ public class JdbcClient {
         String oracleType = fieldSchema.getDataTypeName();
         if (oracleType.startsWith("INTERVAL")) {
             oracleType = oracleType.substring(0, 8);
+        } else if (oracleType.startsWith("TIMESTAMP")) {
+            if (oracleType.equals("TIMESTAMPTZ") || oracleType.equals("TIMESTAMPLTZ")) {
+                return Type.UNSUPPORTED;
+            }
+            return ScalarType.getDefaultDateType(Type.DATETIME);
         }
         switch (oracleType) {
             case "NUMBER":
diff --git a/fe/java-udf/pom.xml b/fe/java-udf/pom.xml
index 46e63ec96b..493015d49c 100644
--- a/fe/java-udf/pom.xml
+++ b/fe/java-udf/pom.xml
@@ -84,6 +84,12 @@ under the License.
             <artifactId>httpclient</artifactId>
             <version>4.5.13</version>
         </dependency>
+        <!-- https://mvnrepository.com/artifact/com.oracle.database.jdbc/ojdbc6 -->
+        <dependency>
+            <groupId>com.oracle.database.jdbc</groupId>
+            <artifactId>ojdbc6</artifactId>
+            <version>11.2.0.4</version>
+        </dependency>
         <!-- https://mvnrepository.com/artifact/org.junit.jupiter/junit-jupiter-engine -->
         <dependency>
             <groupId>org.junit.jupiter</groupId>
diff --git a/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java b/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
index 7f2366b948..0c9894ad18 100644
--- a/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
+++ b/fe/java-udf/src/main/java/org/apache/doris/udf/JdbcExecutor.java
@@ -37,7 +37,6 @@ import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.sql.Timestamp;
 import java.time.LocalDate;
 import java.time.LocalDateTime;
 import java.util.ArrayList;
@@ -217,13 +216,20 @@ public class JdbcExecutor {
                 0, 0, 0, true);
     }
 
-    public long convertDateTimeToLong(Object obj, boolean isDateTimeV2) {
-        LocalDateTime date;
+    public long convertDateTimeToLong(Object obj, boolean isDateTimeV2) throws UdfRuntimeException {
+        LocalDateTime date = null;
         // TODO: not for sure: https://bugs.mysql.com/bug.php?id=101413
         if (obj instanceof LocalDateTime) {
             date = (LocalDateTime) obj;
-        } else {
-            date = ((Timestamp) obj).toLocalDateTime();
+        } else if (obj instanceof java.sql.Timestamp) {
+            date = ((java.sql.Timestamp) obj).toLocalDateTime();
+        } else if (obj instanceof oracle.sql.TIMESTAMP) {
+            try {
+                date = ((oracle.sql.TIMESTAMP) obj).timestampValue().toLocalDateTime();
+            } catch (SQLException e) {
+                throw new UdfRuntimeException("Convert oracle.sql.TIMESTAMP"
+                        + " to LocalDateTime failed: ", e);
+            }
         }
         if (isDateTimeV2) {
             return UdfUtils.convertToDateTimeV2(date.getYear(), date.getMonthValue(), date.getDayOfMonth(),
diff --git a/regression-test/data/jdbc_catalog_p0/test_oracle_jdbc_catalog.out b/regression-test/data/jdbc_catalog_p0/test_oracle_jdbc_catalog.out
index 0fe8aa929c..9f45b5584c 100644
--- a/regression-test/data/jdbc_catalog_p0/test_oracle_jdbc_catalog.out
+++ b/regression-test/data/jdbc_catalog_p0/test_oracle_jdbc_catalog.out
@@ -29,5 +29,15 @@
 2	2022-11-12T20:32:56	\N	\N
 3	\N	11-0	\N
 4	\N	223-9	\N
-5	\N	\N	12 10:23:1.123457000
+5	\N	\N	12 10:23:1.123457
+
+-- !test6 --
+1	2013-01-21T05:23:01	\N	\N	\N	\N	\N
+2	2013-11-12T20:32:56	\N	\N	\N	\N	\N
+3	\N	2019-11-12T20:33:57	\N	\N	\N	\N
+4	\N	\N	2019-11-12T20:33:57	\N	\N	\N
+5	\N	\N	\N	2019-11-12T20:33:57	\N	\N
+6	\N	\N	\N	\N	11-0	\N
+7	\N	\N	\N	\N	223-9	\N
+8	\N	\N	\N	\N	\N	12 10:23:1.123457
 
diff --git a/regression-test/suites/jdbc_catalog_p0/test_oracle_jdbc_catalog.groovy b/regression-test/suites/jdbc_catalog_p0/test_oracle_jdbc_catalog.groovy
index 9d473e7598..51a7b174f7 100644
--- a/regression-test/suites/jdbc_catalog_p0/test_oracle_jdbc_catalog.groovy
+++ b/regression-test/suites/jdbc_catalog_p0/test_oracle_jdbc_catalog.groovy
@@ -62,6 +62,7 @@ suite("test_oracle_jdbc_catalog", "p0") {
         order_qt_test2  """ select * from TEST_CHAR order by ID; """
         order_qt_test3  """ select * from TEST_INT order by ID; """
         order_qt_test5  """ select * from TEST_DATE order by ID; """
+        order_qt_test6  """ select * from TEST_TIMESTAMP order by ID; """
 
         // The result of TEST_RAW will change
         // So instead of qt, we're using sql here.


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[doris] 03/11: [fix](DOE) only return first batch data in ES 8.x (#16025)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 1db3e975cda4bb9f9a33277457f422ad417eb303
Author: qiye <ji...@gmail.com>
AuthorDate: Wed Jan 18 09:28:34 2023 +0800

    [fix](DOE) only return first batch data in ES 8.x (#16025)
    
    Do not use terminate_after and size together in scroll request of ES 8.x.
---
 be/src/exec/es/es_scan_reader.cpp | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/be/src/exec/es/es_scan_reader.cpp b/be/src/exec/es/es_scan_reader.cpp
index 8470350dd0..fab99c85ec 100644
--- a/be/src/exec/es/es_scan_reader.cpp
+++ b/be/src/exec/es/es_scan_reader.cpp
@@ -76,8 +76,8 @@ ESScanReader::ESScanReader(const std::string& target,
         std::stringstream scratch;
         // just send a normal search  against the elasticsearch with additional terminate_after param to achieve terminate early effect when limit take effect
         if (_type.empty()) {
+            // `terminate_after` and `size` can not be used together in scroll request of ES 8.x
             scratch << _target << REQUEST_SEPARATOR << _index << "/_search?"
-                    << "terminate_after=" << props.at(KEY_TERMINATE_AFTER)
                     << REQUEST_PREFERENCE_PREFIX << _shards << "&" << filter_path;
         } else {
             scratch << _target << REQUEST_SEPARATOR << _index << REQUEST_SEPARATOR << _type
@@ -92,9 +92,10 @@ ESScanReader::ESScanReader(const std::string& target,
         // scroll request for scanning
         // add terminate_after for the first scroll to avoid decompress all postings list
         if (_type.empty()) {
+            // `terminate_after` and `size` can not be used together in scroll request of ES 8.x
             scratch << _target << REQUEST_SEPARATOR << _index << "/_search?"
                     << "scroll=" << _scroll_keep_alive << REQUEST_PREFERENCE_PREFIX << _shards
-                    << "&" << filter_path << "&terminate_after=" << batch_size_str;
+                    << "&" << filter_path;
         } else {
             scratch << _target << REQUEST_SEPARATOR << _index << REQUEST_SEPARATOR << _type
                     << "/_search?"


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org